From 87caba923cb5bf8f3a6ec4c0c064852a78d868a5 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 24 Jun 2015 09:40:46 -0700 Subject: [PATCH 001/275] First commit --- .gitignore | 2 +- README.md | 2 +- pom.xml | 35 +++++ .../streaming/CopartitioningGroup.java | 19 +++ .../streaming/CopartitioningGroupFactory.java | 10 ++ .../DefaultCopartitioningGroupFactory.java | 18 +++ .../java/io/confluent/streaming/KStream.java | 33 ++++ .../io/confluent/streaming/KStreamConfig.java | 14 ++ .../confluent/streaming/KStreamContext.java | 12 ++ .../io/confluent/streaming/KStreamJob.java | 12 ++ .../confluent/streaming/KStreamWindowed.java | 11 ++ .../java/io/confluent/streaming/KeyValue.java | 20 +++ .../confluent/streaming/KeyValueMapper.java | 10 ++ .../streaming/NotCopartitionedException.java | 8 + .../io/confluent/streaming/Predicate.java | 10 ++ .../io/confluent/streaming/Processor.java | 14 ++ .../confluent/streaming/RecordCollector.java | 12 ++ .../streaming/StreamSynchronizer.java | 18 +++ .../TimeBasedStreamSynchronizer.java | 48 ++++++ .../streaming/TimestampExtractor.java | 10 ++ .../io/confluent/streaming/ValueJoiner.java | 10 ++ .../io/confluent/streaming/ValueMapper.java | 10 ++ .../java/io/confluent/streaming/Window.java | 18 +++ .../io/confluent/streaming/WindowByCount.java | 57 +++++++ .../io/confluent/streaming/WindowByTime.java | 96 ++++++++++++ .../streaming/internal/KStreamBranch.java | 49 ++++++ .../streaming/internal/KStreamConfigImpl.java | 76 ++++++++++ .../internal/KStreamContextImpl.java | 78 ++++++++++ .../streaming/internal/KStreamFilter.java | 25 +++ .../streaming/internal/KStreamFlatMap.java | 27 ++++ .../internal/KStreamFlatMapValues.java | 26 ++++ .../streaming/internal/KStreamImpl.java | 143 ++++++++++++++++++ .../streaming/internal/KStreamJoin.java | 74 +++++++++ .../streaming/internal/KStreamMap.java | 25 +++ .../streaming/internal/KStreamMapValues.java | 24 +++ .../streaming/internal/KStreamNestedLoop.java | 31 ++++ .../streaming/internal/KStreamSource.java | 18 +++ .../internal/KStreamWindowedImpl.java | 47 ++++++ .../streaming/internal/PartitioningInfo.java | 35 +++++ .../streaming/internal/Receiver.java | 14 ++ .../io/confluent/streaming/util/Stamped.java | 33 ++++ .../io/confluent/streaming/util/Util.java | 16 ++ 42 files changed, 1248 insertions(+), 2 deletions(-) create mode 100644 pom.xml create mode 100644 src/main/java/io/confluent/streaming/CopartitioningGroup.java create mode 100644 src/main/java/io/confluent/streaming/CopartitioningGroupFactory.java create mode 100644 src/main/java/io/confluent/streaming/DefaultCopartitioningGroupFactory.java create mode 100644 src/main/java/io/confluent/streaming/KStream.java create mode 100644 src/main/java/io/confluent/streaming/KStreamConfig.java create mode 100644 src/main/java/io/confluent/streaming/KStreamContext.java create mode 100644 src/main/java/io/confluent/streaming/KStreamJob.java create mode 100644 src/main/java/io/confluent/streaming/KStreamWindowed.java create mode 100644 src/main/java/io/confluent/streaming/KeyValue.java create mode 100644 src/main/java/io/confluent/streaming/KeyValueMapper.java create mode 100644 src/main/java/io/confluent/streaming/NotCopartitionedException.java create mode 100644 src/main/java/io/confluent/streaming/Predicate.java create mode 100644 src/main/java/io/confluent/streaming/Processor.java create mode 100644 src/main/java/io/confluent/streaming/RecordCollector.java create mode 100644 src/main/java/io/confluent/streaming/StreamSynchronizer.java create mode 100644 src/main/java/io/confluent/streaming/TimeBasedStreamSynchronizer.java create mode 100644 src/main/java/io/confluent/streaming/TimestampExtractor.java create mode 100644 src/main/java/io/confluent/streaming/ValueJoiner.java create mode 100644 src/main/java/io/confluent/streaming/ValueMapper.java create mode 100644 src/main/java/io/confluent/streaming/Window.java create mode 100644 src/main/java/io/confluent/streaming/WindowByCount.java create mode 100644 src/main/java/io/confluent/streaming/WindowByTime.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamBranch.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamFilter.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamJoin.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamMap.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamMapValues.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamSource.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/PartitioningInfo.java create mode 100644 src/main/java/io/confluent/streaming/internal/Receiver.java create mode 100644 src/main/java/io/confluent/streaming/util/Stamped.java create mode 100644 src/main/java/io/confluent/streaming/util/Util.java diff --git a/.gitignore b/.gitignore index dbc05075b8d27..37f242bc2e8de 100644 --- a/.gitignore +++ b/.gitignore @@ -34,4 +34,4 @@ gradle/wrapper/* results tests/results .ducktape -tests/.ducktape +tests/.ducktape \ No newline at end of file diff --git a/README.md b/README.md index a4a3f7876472c..eb1538c27c455 100644 --- a/README.md +++ b/README.md @@ -117,4 +117,4 @@ To contribute follow the instructions here: * http://kafka.apache.org/contributing.html We also welcome patches for the website and documentation which can be found here: - * https://svn.apache.org/repos/asf/kafka/site + * https://svn.apache.org/repos/asf/kafka/site \ No newline at end of file diff --git a/pom.xml b/pom.xml new file mode 100644 index 0000000000000..108039a64bba6 --- /dev/null +++ b/pom.xml @@ -0,0 +1,35 @@ + + 4.0.0 + iio.confluent.streaming + streaming + jar + 1.0-SNAPSHOT + streaming + + + Confluent, Inc. + http://confluent.io + + http://confluent.io + + + 0.8.3-SNAPSHOT + 2.10 + UTF-8 + + + + + org.apache.kafka + kafka_${kafka.scala.version} + ${kafka.version} + + + junit + junit + 3.8.1 + test + + + diff --git a/src/main/java/io/confluent/streaming/CopartitioningGroup.java b/src/main/java/io/confluent/streaming/CopartitioningGroup.java new file mode 100644 index 0000000000000..ea364c198f9a2 --- /dev/null +++ b/src/main/java/io/confluent/streaming/CopartitioningGroup.java @@ -0,0 +1,19 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/22/15. + */ +public final class CopartitioningGroup { + + public final String name; + public final int numPartitions; + + public final StreamSynchronizer streamSynchronizer; + + public CopartitioningGroup(String name, int numPartitions, StreamSynchronizer streamSynchronizer) { + this.name = name; + this.numPartitions = numPartitions; + this.streamSynchronizer = streamSynchronizer; + } + +} diff --git a/src/main/java/io/confluent/streaming/CopartitioningGroupFactory.java b/src/main/java/io/confluent/streaming/CopartitioningGroupFactory.java new file mode 100644 index 0000000000000..c4c06660f2aed --- /dev/null +++ b/src/main/java/io/confluent/streaming/CopartitioningGroupFactory.java @@ -0,0 +1,10 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/22/15. + */ +public interface CopartitioningGroupFactory { + + CopartitioningGroup create(String name, int numPartitions); + +} diff --git a/src/main/java/io/confluent/streaming/DefaultCopartitioningGroupFactory.java b/src/main/java/io/confluent/streaming/DefaultCopartitioningGroupFactory.java new file mode 100644 index 0000000000000..508a1684c96a3 --- /dev/null +++ b/src/main/java/io/confluent/streaming/DefaultCopartitioningGroupFactory.java @@ -0,0 +1,18 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/22/15. + */ +public class DefaultCopartitioningGroupFactory implements CopartitioningGroupFactory { + + private final TimestampExtractor timestampExtractor; + + public DefaultCopartitioningGroupFactory(TimestampExtractor timestampExtractor) { + this.timestampExtractor = timestampExtractor; + } + + public CopartitioningGroup create(String name, int numPartitions) { + return new CopartitioningGroup(name, numPartitions, new TimeBasedStreamSynchronizer(timestampExtractor)); + } + +} diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java new file mode 100644 index 0000000000000..bf9f7c2fe83b8 --- /dev/null +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -0,0 +1,33 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/17/15. + */ +public interface KStream { + + KStream filter(Predicate predicate); + + KStream filterOut(Predicate predicate); + + KStream map(KeyValueMapper mapper); + + KStream mapValues(ValueMapper mapper); + + KStream flatMap(KeyValueMapper, K, V> mapper); + + KStream flatMapValues(ValueMapper, V> processor); + + KStreamWindowed with(Window window); + + KStream nestedLoop(KStreamWindowed other, ValueJoiner processor) + throws NotCopartitionedException; + + KStream[] branch(Predicate... predicates); + + KStream through(String topic); + + void sendTo(String topic); + + void process(Processor processor); + +} diff --git a/src/main/java/io/confluent/streaming/KStreamConfig.java b/src/main/java/io/confluent/streaming/KStreamConfig.java new file mode 100644 index 0000000000000..d7fa390bb42c7 --- /dev/null +++ b/src/main/java/io/confluent/streaming/KStreamConfig.java @@ -0,0 +1,14 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/22/15. + */ +public interface KStreamConfig { + + void setDefaultStreamSynchronizer(StreamSynchronizer streamSynchronizer); + + void setCopartitioningGroupFactory(CopartitioningGroupFactory copartitioningGroupFactory); + + void addTopicToCopartitioningGroup(String topic, String groupName); + +} diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java new file mode 100644 index 0000000000000..d9da4593b6d42 --- /dev/null +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -0,0 +1,12 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/19/15. + */ +public interface KStreamContext { + + KStream from(String topic); + + RecordCollector getRecordCollector(); + +} diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java new file mode 100644 index 0000000000000..bb3781f5d950f --- /dev/null +++ b/src/main/java/io/confluent/streaming/KStreamJob.java @@ -0,0 +1,12 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/22/15. + */ +public interface KStreamJob { + + void configure(KStreamConfig config); + + void build(KStreamContext ksc); + +} diff --git a/src/main/java/io/confluent/streaming/KStreamWindowed.java b/src/main/java/io/confluent/streaming/KStreamWindowed.java new file mode 100644 index 0000000000000..83b0886df7709 --- /dev/null +++ b/src/main/java/io/confluent/streaming/KStreamWindowed.java @@ -0,0 +1,11 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/18/15. + */ +public interface KStreamWindowed extends KStream { + + KStream join(KStreamWindowed other, ValueJoiner processor) + throws NotCopartitionedException; + +} diff --git a/src/main/java/io/confluent/streaming/KeyValue.java b/src/main/java/io/confluent/streaming/KeyValue.java new file mode 100644 index 0000000000000..b9b26d7c576aa --- /dev/null +++ b/src/main/java/io/confluent/streaming/KeyValue.java @@ -0,0 +1,20 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/17/15. + */ +public class KeyValue { + + public final K key; + public final V value; + + private KeyValue(K key, V value) { + this.key = key; + this.value = value; + } + + public static KeyValue pair(K key, V value) { + return new KeyValue(key, value); + } + +} diff --git a/src/main/java/io/confluent/streaming/KeyValueMapper.java b/src/main/java/io/confluent/streaming/KeyValueMapper.java new file mode 100644 index 0000000000000..f501ae15ea60f --- /dev/null +++ b/src/main/java/io/confluent/streaming/KeyValueMapper.java @@ -0,0 +1,10 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/17/15. + */ +public interface KeyValueMapper { + + KeyValue apply(K key, V value); + +} diff --git a/src/main/java/io/confluent/streaming/NotCopartitionedException.java b/src/main/java/io/confluent/streaming/NotCopartitionedException.java new file mode 100644 index 0000000000000..849f614506888 --- /dev/null +++ b/src/main/java/io/confluent/streaming/NotCopartitionedException.java @@ -0,0 +1,8 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/19/15. + */ +public class NotCopartitionedException extends Exception { + +} diff --git a/src/main/java/io/confluent/streaming/Predicate.java b/src/main/java/io/confluent/streaming/Predicate.java new file mode 100644 index 0000000000000..469d1e30636d3 --- /dev/null +++ b/src/main/java/io/confluent/streaming/Predicate.java @@ -0,0 +1,10 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/17/15. + */ +public interface Predicate { + + boolean apply(K key, V value); + +} diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java new file mode 100644 index 0000000000000..373d86ca2413f --- /dev/null +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -0,0 +1,14 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/17/15. + */ +public interface Processor { + + void apply(K key, V value); + + void punctuate(long timestamp); + + void flush(); + +} diff --git a/src/main/java/io/confluent/streaming/RecordCollector.java b/src/main/java/io/confluent/streaming/RecordCollector.java new file mode 100644 index 0000000000000..7341395a4dd54 --- /dev/null +++ b/src/main/java/io/confluent/streaming/RecordCollector.java @@ -0,0 +1,12 @@ +package io.confluent.streaming; + +import org.apache.kafka.clients.producer.ProducerRecord; + +/** + * Created by yasuhiro on 6/19/15. + */ +public interface RecordCollector { + + void send(ProducerRecord record); + +} diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/StreamSynchronizer.java new file mode 100644 index 0000000000000..1d64b2fd5b550 --- /dev/null +++ b/src/main/java/io/confluent/streaming/StreamSynchronizer.java @@ -0,0 +1,18 @@ +package io.confluent.streaming; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +/** + * Created by yasuhiro on 6/23/15. + */ +public interface StreamSynchronizer { + + void add(ConsumerRecord record); + + ConsumerRecord next(); + + long currentStreamTime(); + + void close(); + +} diff --git a/src/main/java/io/confluent/streaming/TimeBasedStreamSynchronizer.java b/src/main/java/io/confluent/streaming/TimeBasedStreamSynchronizer.java new file mode 100644 index 0000000000000..f0702be93f12b --- /dev/null +++ b/src/main/java/io/confluent/streaming/TimeBasedStreamSynchronizer.java @@ -0,0 +1,48 @@ +package io.confluent.streaming; + +import io.confluent.streaming.util.Stamped; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import java.util.PriorityQueue; + +/** + * Created by yasuhiro on 6/23/15. + */ +public class TimeBasedStreamSynchronizer implements StreamSynchronizer { + + private final TimestampExtractor timestampExtractor; + private final PriorityQueue>> pq = new PriorityQueue(); + private long streamTime = -1; + + public TimeBasedStreamSynchronizer(TimestampExtractor timestampExtractor) { + this.timestampExtractor = timestampExtractor; + } + + public void add(ConsumerRecord record) { + long timestamp = timestampExtractor.extract(record.topic(), record.key(), record.value()); + + if (timestamp < 0) timestamp = System.currentTimeMillis(); + + Stamped> stamped = new Stamped(record, timestamp); + pq.offer(stamped); + } + + public ConsumerRecord next() { + Stamped> stamped = pq.poll(); + + if (stamped == null) return null; + + if (streamTime < stamped.timestamp) streamTime = stamped.timestamp; + + return stamped.value; + } + + public long currentStreamTime() { + return streamTime; + } + + public void close() { + pq.clear(); + } + +} diff --git a/src/main/java/io/confluent/streaming/TimestampExtractor.java b/src/main/java/io/confluent/streaming/TimestampExtractor.java new file mode 100644 index 0000000000000..b1e1cdab5ffa7 --- /dev/null +++ b/src/main/java/io/confluent/streaming/TimestampExtractor.java @@ -0,0 +1,10 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/22/15. + */ +public interface TimestampExtractor { + + long extract(String topic, K key, V value); + +} diff --git a/src/main/java/io/confluent/streaming/ValueJoiner.java b/src/main/java/io/confluent/streaming/ValueJoiner.java new file mode 100644 index 0000000000000..aeae1139a94f6 --- /dev/null +++ b/src/main/java/io/confluent/streaming/ValueJoiner.java @@ -0,0 +1,10 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/17/15. + */ +public interface ValueJoiner { + + R apply(V1 value1, V2 value2); + +} diff --git a/src/main/java/io/confluent/streaming/ValueMapper.java b/src/main/java/io/confluent/streaming/ValueMapper.java new file mode 100644 index 0000000000000..ad71546c5a74c --- /dev/null +++ b/src/main/java/io/confluent/streaming/ValueMapper.java @@ -0,0 +1,10 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/17/15. + */ +public interface ValueMapper { + + R apply(V value); + +} diff --git a/src/main/java/io/confluent/streaming/Window.java b/src/main/java/io/confluent/streaming/Window.java new file mode 100644 index 0000000000000..f213df00a6878 --- /dev/null +++ b/src/main/java/io/confluent/streaming/Window.java @@ -0,0 +1,18 @@ +package io.confluent.streaming; + +import java.util.Iterator; + +/** + * Created by yasuhiro on 6/17/15. + */ +public interface Window { + + Iterator find(K key, long timestamp); + + void put(K key, V value, long timestamp); + + void punctuate(long timestamp); + + void flush(); + +} diff --git a/src/main/java/io/confluent/streaming/WindowByCount.java b/src/main/java/io/confluent/streaming/WindowByCount.java new file mode 100644 index 0000000000000..18e2ae74807fd --- /dev/null +++ b/src/main/java/io/confluent/streaming/WindowByCount.java @@ -0,0 +1,57 @@ +package io.confluent.streaming; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; + +/** + * Created by yasuhiro on 6/18/15. + */ +public class WindowByCount implements Window { + + private final int maxCount; + private LinkedList list = new LinkedList(); + private HashMap> map = new HashMap>(); + private int count = 0; + + public WindowByCount(int maxCount) { + this.maxCount = maxCount; + } + + public Iterator find(K key, long timestamp) { + LinkedList values = map.get(key); + if (values == null) + return null; + else + return values.iterator(); + } + + public void put(K key, V value, long timestamp) { + list.offerLast(key); + + LinkedList values = map.get(key); + if (values == null) { + values = new LinkedList(); + map.put(key, values); + } + values.offerLast(value); + count++; + + mayEvict(); + } + + private void mayEvict() { + while (count > maxCount) { + K oldestKey = list.poll(); + LinkedList values = map.get(oldestKey); + values.removeFirst(); + if (values.isEmpty()) map.remove(oldestKey); + count--; + } + } + + public void punctuate(long timestamp) {} + + public void flush() {} + +} diff --git a/src/main/java/io/confluent/streaming/WindowByTime.java b/src/main/java/io/confluent/streaming/WindowByTime.java new file mode 100644 index 0000000000000..8215d235193d1 --- /dev/null +++ b/src/main/java/io/confluent/streaming/WindowByTime.java @@ -0,0 +1,96 @@ +package io.confluent.streaming; + +import io.confluent.streaming.util.Stamped; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; + +/** + * Created by yasuhiro on 6/18/15. + */ +public class WindowByTime implements Window { + + private final long duration; + private final int maxCount; + private LinkedList list = new LinkedList(); + private HashMap>> map = new HashMap>>(); + + public WindowByTime(long duration, int maxCount) { + this.duration = duration; + this.maxCount = maxCount; + } + + public Iterator find(K key, long timestamp) { + final LinkedList> values = map.get(key); + + if (values == null) { + return null; + } + else { + final Iterator> inner = values.iterator(); + + return new Iterator() { + public boolean hasNext() { + return inner.hasNext(); + } + public V next() { + return inner.next().value; + } + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + } + + public void put(K key, V value, long timestamp) { + list.offerLast(key); + + LinkedList> values = map.get(key); + if (values == null) { + values = new LinkedList>(); + map.put(key, values); + } + + values.offerLast(new Stamped(value, timestamp)); + + evictExcess(); + evictExpired(timestamp - duration); + } + + private void evictExcess() { + while (list.size() > maxCount) { + K oldestKey = list.pollFirst(); + + LinkedList> values = map.get(oldestKey); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } + } + + private void evictExpired(long cutoffTime) { + while (true) { + K oldestKey = list.peekFirst(); + + LinkedList> values = map.get(oldestKey); + Stamped oldestValue = values.peekFirst(); + + if (oldestValue.timestamp < cutoffTime) { + list.pollFirst(); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } + else { + break; + } + } + } + + public void punctuate(long timestamp) {} + + public void flush() {} + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java new file mode 100644 index 0000000000000..1755eec768441 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -0,0 +1,49 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.Predicate; + +import java.util.Arrays; + +/** + * Created by yasuhiro on 6/18/15. + */ +class KStreamBranch implements Receiver { + + private final Predicate[] predicates; + final KStreamSource[] branches; + + KStreamBranch(Predicate[] predicates, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + this.predicates = Arrays.copyOf(predicates, predicates.length); + branches = (KStreamSource[]) new Object[predicates.length + 1]; + for (int i = 0; i < branches.length; i++) { + branches[i] = new KStreamSource(partitioningInfo, context); + } + } + + public void receive(K key, V value, long timestamp) { + synchronized(this) { + for (int i = 0; i < predicates.length; i++) { + Predicate predicate = predicates[i]; + if (predicate.apply(key, value)) { + branches[i].receive(key, value, timestamp); + return; + } + } + branches[branches.length - 1].receive(key, value, timestamp); + return; + } + } + + public void punctuate(long timestamp) { + for (KStreamSource branch : branches) { + branch.punctuate(timestamp); + } + } + + public void flush() { + for (KStreamSource branch : branches) { + branch.flush(); + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java new file mode 100644 index 0000000000000..e3daec0eb56ba --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java @@ -0,0 +1,76 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.CopartitioningGroup; +import io.confluent.streaming.CopartitioningGroupFactory; +import io.confluent.streaming.KStreamConfig; +import io.confluent.streaming.StreamSynchronizer; + +import java.util.HashMap; + +/** + * Created by yasuhiro on 6/19/15. + */ +public class KStreamConfigImpl implements KStreamConfig { + + private StreamSynchronizer streamSynchronizer = null; + private CopartitioningGroupFactory copartitioningGroupFactory = null; + + private final HashMap copartitioningGroups = new HashMap(); + private final HashMap partitioningInfos = new HashMap(); + + KStreamConfigImpl(/* TODO: pass in a consumer */) { + + } + + public void setDefaultStreamSynchronizer(StreamSynchronizer streamSynchronizer) { + synchronized (this) { + if (streamSynchronizer == null) + throw new IllegalArgumentException("null StreamSynchronizer"); + if (this.streamSynchronizer != null) + throw new IllegalStateException("default StreamSynchronizer was already set"); + + this.streamSynchronizer = streamSynchronizer; + } + } + + public void setCopartitioningGroupFactory(CopartitioningGroupFactory copartitioningGroupFactory) { + synchronized (this) { + if (copartitioningGroupFactory == null) + throw new IllegalArgumentException("null CoPartitioningGroupFactory"); + if (this.copartitioningGroupFactory != null) + throw new IllegalStateException("CoPartitioningGroupFactory was already set"); + + this.copartitioningGroupFactory = copartitioningGroupFactory; + } + } + + public void addTopicToCopartitioningGroup(String topic, String groupName) { + synchronized (this) { + PartitioningInfo info = partitioningInfos.get(topic); + if (info != null) + throw new IllegalStateException("the topic was already added to the CopartitioningGroup: topic="+ topic); + + // TODO: use a consumer to get partition info from Kafka and set them in PartitioningInfo below + int numPartitions = 1; + + CopartitioningGroup group = copartitioningGroups.get(groupName); + + if (group == null) { + group = copartitioningGroupFactory.create(groupName, numPartitions); + copartitioningGroups.put(groupName, group); + } + else { + if (group.numPartitions != numPartitions) + throw new IllegalStateException("incompatible number of partitions"); + } + + info = new PartitioningInfo(group, numPartitions); + partitioningInfos.put(topic, info); + } + } + + HashMap getPartitioningInfos() { + return partitioningInfos; + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java new file mode 100644 index 0000000000000..dec4ca2b41a51 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -0,0 +1,78 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KStream; +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.RecordCollector; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import java.util.HashMap; + +/** + * Created by yasuhiro on 6/19/15. + */ +public class KStreamContextImpl implements KStreamContext { + + private final RecordCollector collector; + private final HashMap> sourceStreams = new HashMap>(); + private final HashMap partitioningInfos; + + KStreamContextImpl(/* TODO: pass in consumer */ RecordCollector collector) { + this(new HashMap(), collector); + } + + KStreamContextImpl(/* TODO: pass in consumer */ HashMap partitioningInfos, RecordCollector collector) { + this.collector = collector; + this.partitioningInfos = partitioningInfos; + } + + public KStream from(String topic) { + synchronized (this) { + KStreamSource stream = (KStreamSource)sourceStreams.get(topic); + + if (stream == null) { + PartitioningInfo partitioningInfo = partitioningInfos.get(topic); + + if (partitioningInfo == null) { + // TODO: use the consumer to get partitioning info + int numPartitions = 1; + + partitioningInfo = new PartitioningInfo(null, numPartitions); + partitioningInfos.put(topic, partitioningInfo); + } + + stream = new KStreamSource(partitioningInfo, this); + sourceStreams.put(topic, stream); + } + + return stream; + } + } + + public RecordCollector getRecordCollector() { + return collector; + } + + void process(ConsumerRecord record, long timestamp) { + String topic = record.topic(); + + KStreamSource stream = (KStreamSource)sourceStreams.get(record.topic()); + + if (stream == null) + throw new IllegalStateException("a stream for a topic not found: topic=" + topic); + + stream.receive(record.key(), record.value(), timestamp); + } + + void punctuate(long timestamp) { + for (KStreamSource stream : sourceStreams.values()) { + stream.punctuate(timestamp); + } + } + + void flush() { + for (KStreamSource stream : sourceStreams.values()) { + stream.flush(); + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java new file mode 100644 index 0000000000000..e8941e94c7369 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -0,0 +1,25 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.Predicate; + +/** + * Created by yasuhiro on 6/17/15. + */ +class KStreamFilter extends KStreamImpl { + + private final Predicate predicate; + + KStreamFilter(Predicate predicate, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + super(partitioningInfo, context); + this.predicate = predicate; + } + + public void receive(K key, V value, long timestamp) { + synchronized(this) { + if (predicate.apply(key, value)) { + forward(key, value, timestamp); + } + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java new file mode 100644 index 0000000000000..62a59f7f47bd3 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -0,0 +1,27 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KeyValueMapper; +import io.confluent.streaming.KeyValue; + +/** + * Created by yasuhiro on 6/17/15. + */ +class KStreamFlatMap extends KStreamImpl { + + private final KeyValueMapper, K1, V1> mapper; + + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamContextImpl context) { + super(PartitioningInfo.missing, context); + this.mapper = mapper; + } + + public void receive(K1 key, V1 value, long timestamp) { + synchronized(this) { + KeyValue> newPair = mapper.apply(key, value); + for (V v : newPair.value) { + forward(newPair.key, v, timestamp); + } + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java new file mode 100644 index 0000000000000..fca87d0e48843 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -0,0 +1,26 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.ValueMapper; + +/** + * Created by yasuhiro on 6/17/15. + */ +class KStreamFlatMapValues extends KStreamImpl { + + private final ValueMapper, V1> mapper; + + KStreamFlatMapValues(ValueMapper, V1> mapper, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + super(partitioningInfo, context); + this.mapper = mapper; + } + + public void receive(K key, V1 value, long timestamp) { + synchronized(this) { + Iterable newValues = mapper.apply(value); + for (V v : newValues) { + forward(key, v, timestamp); + } + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java new file mode 100644 index 0000000000000..70776f047b77a --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -0,0 +1,143 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import org.apache.kafka.clients.producer.ProducerRecord; + +import java.util.ArrayList; + +/** + * Created by yasuhiro on 6/17/15. + */ +abstract class KStreamImpl implements KStream, Receiver { + + private final ArrayList> nextReceivers = new ArrayList>(1); + private long punctuatedAt = 0L; + final PartitioningInfo partitioningInfo; + final KStreamContextImpl context; + + protected KStreamImpl(PartitioningInfo partitioningInfo, KStreamContextImpl context) { + this.partitioningInfo = partitioningInfo; + this.context = context; + } + + public KStream filter(Predicate predicate) { + return chain(new KStreamFilter(predicate, partitioningInfo, context)); + } + + public KStream filterOut(final Predicate predicate) { + return filter(new Predicate() { + public boolean apply(K key, V value) { + return !predicate.apply(key, value); + } + }); + } + + public KStream map(KeyValueMapper mapper) { + return chain(new KStreamMap(mapper, context)); + } + + public KStream mapValues(ValueMapper mapper) { + return chain(new KStreamMapValues(mapper, partitioningInfo, context)); + } + + public KStream flatMap(KeyValueMapper, K, V> mapper) { + return chain(new KStreamFlatMap(mapper, context)); + } + + public KStream flatMapValues(ValueMapper, V> mapper) { + return chain(new KStreamFlatMapValues(mapper, partitioningInfo, context)); + } + + public KStreamWindowed with(Window window) { + return (KStreamWindowed)chain(new KStreamWindowedImpl(window, partitioningInfo, context)); + } + + public KStream nestedLoop(KStreamWindowed other, ValueJoiner processor) + throws NotCopartitionedException { + + KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; + + if (!partitioningInfo.isJoinCompatibleWith(otherImpl.partitioningInfo)) throw new NotCopartitionedException(); + + return chain(new KStreamNestedLoop(otherImpl.window, processor, partitioningInfo, context)); + } + + public KStream[] branch(Predicate... predicates) { + KStreamBranch branch = new KStreamBranch(predicates, partitioningInfo, context); + registerReceiver(branch); + return branch.branches; + } + + public KStream through(String topic) { + process(this.getSendProcessor(topic)); + return context.from(topic); + } + + public void sendTo(String topic) { + process(this.getSendProcessor(topic)); + } + + private Processor getSendProcessor(final String topic) { + final RecordCollector collector = context.getRecordCollector(); + + return new Processor() { + public void apply(K key, V value) { + collector.send(new ProducerRecord(topic, key, value)); + } + public void punctuate(long timestamp) {} + public void flush() {} + }; + } + + public void process(final Processor processor) { + Receiver receiver = new Receiver() { + public void receive(K key, V value, long timestamp) { + processor.apply(key, value); + } + public void punctuate(long timestamp) { + processor.punctuate(timestamp); + } + public void flush() { + processor.flush(); + } + }; + registerReceiver(receiver); + } + + public void punctuate(long timestamp) { + if (timestamp != punctuatedAt) { + punctuatedAt = timestamp; + + int numReceivers = nextReceivers.size(); + for (int i = 0; i < numReceivers; i++) { + nextReceivers.get(i).punctuate(timestamp); + } + } + } + + public void flush() { + int numReceivers = nextReceivers.size(); + for (int i = 0; i < numReceivers; i++) { + nextReceivers.get(i).flush(); + } + } + + void registerReceiver(Receiver receiver) { + nextReceivers.add(receiver); + } + + protected void forward(K key, V value, long timestamp) { + int numReceivers = nextReceivers.size(); + for (int i = 0; i < numReceivers; i++) { + nextReceivers.get(i).receive(key, value, timestamp); + } + } + + protected KStream chain(KStreamImpl kstream) { + synchronized(this) { + nextReceivers.add(kstream); + return kstream; + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java new file mode 100644 index 0000000000000..ae671064c9693 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -0,0 +1,74 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.ValueJoiner; +import io.confluent.streaming.Window; + +import java.util.Iterator; + +/** + * Created by yasuhiro on 6/17/15. + */ +class KStreamJoin extends KStreamImpl { + + private final Window window1; + private final Window window2; + private final ValueJoiner joiner; + final Receiver receiverForOtherStream; + + private boolean flushed = true; + + KStreamJoin(final Window window1, Window window2, ValueJoiner joiner, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + super(partitioningInfo, context); + + this.window1 = window1; + this.window2 = window2; + this.joiner = joiner; + + this.receiverForOtherStream = getReceiverForOther(); + } + + public void receive(K key, V1 value, long timestamp) { + flushed = false; + Iterator iter = window2.find(key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + doJoin(key, value, iter.next(), timestamp); + } + } + } + + public void flush() { + if (!flushed) { + super.flush(); + flushed = true; + } + } + + private Receiver getReceiverForOther() { + return new Receiver() { + + public void receive(K key, V2 value2, long timestamp) { + flushed = false; + Iterator iter = window1.find(key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + doJoin(key, iter.next(), value2, timestamp); + } + } + } + + public void punctuate(long timestamp) { + KStreamJoin.this.punctuate(timestamp); + } + + public void flush() { + KStreamJoin.this.flush(); + } + }; + } + + private void doJoin(K key, V1 value1, V2 value2, long timestamp) { + forward(key, joiner.apply(value1, value2), timestamp); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java new file mode 100644 index 0000000000000..e9585f46e3117 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -0,0 +1,25 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KeyValueMapper; +import io.confluent.streaming.KeyValue; + +/** + * Created by yasuhiro on 6/17/15. + */ +class KStreamMap extends KStreamImpl { + + private final KeyValueMapper mapper; + + KStreamMap(KeyValueMapper mapper, KStreamContextImpl context) { + super(PartitioningInfo.missing, context); + this.mapper = mapper; + } + + public void receive(K1 key, V1 value, long timestamp) { + synchronized (this) { + KeyValue newPair = mapper.apply(key, value); + forward(newPair.key, newPair.value, timestamp); + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java new file mode 100644 index 0000000000000..98596552b5727 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -0,0 +1,24 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.ValueMapper; + +/** + * Created by yasuhiro on 6/17/15. + */ +class KStreamMapValues extends KStreamImpl { + + private final ValueMapper mapper; + + KStreamMapValues(ValueMapper mapper, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + super(partitioningInfo, context); + this.mapper = mapper; + } + + public void receive(K key, V1 value, long timestamp) { + synchronized (this) { + V newValue = mapper.apply(value); + forward(key, newValue, timestamp); + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java b/src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java new file mode 100644 index 0000000000000..f4eadb86668b3 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java @@ -0,0 +1,31 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.ValueJoiner; +import io.confluent.streaming.Window; + +import java.util.Iterator; + +/** + * Created by yasuhiro on 6/17/15. + */ +class KStreamNestedLoop extends KStreamImpl { + + private final Window window; + private final ValueJoiner joiner; + + KStreamNestedLoop(Window window, ValueJoiner joiner, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + super(partitioningInfo, context); + this.window = window; + this.joiner = joiner; + } + + public void receive(K key, V1 value, long timestamp) { + Iterator iter = window.find(key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + forward(key, joiner.apply(value, iter.next()), timestamp); + } + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java new file mode 100644 index 0000000000000..d73631a7bedde --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -0,0 +1,18 @@ +package io.confluent.streaming.internal; + +/** + * Created by yasuhiro on 6/17/15. + */ +class KStreamSource extends KStreamImpl { + + KStreamSource(PartitioningInfo partitioningInfo, KStreamContextImpl context) { + super(partitioningInfo, context); + } + + public void receive(K key, V value, long timestamp) { + synchronized(this) { + forward(key, value, timestamp); + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java new file mode 100644 index 0000000000000..cfb17f2ce916d --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -0,0 +1,47 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; + +/** + * Created by yasuhiro on 6/18/15. + */ +public class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { + + final Window window; + + KStreamWindowedImpl(Window window, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + super(partitioningInfo, context); + this.window = window; + } + + public void receive(K key, V value, long timestamp) { + synchronized(this) { + window.put(key, value, timestamp); + forward(key, value, timestamp); + } + } + + public void punctuate(long timestamp) { + window.punctuate(timestamp); + super.punctuate(timestamp); + } + + public void flush() { + window.flush(); + super.flush(); + } + + public KStream join(KStreamWindowed other, ValueJoiner processor) + throws NotCopartitionedException { + + KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; + + if (!partitioningInfo.isJoinCompatibleWith(otherImpl.partitioningInfo)) throw new NotCopartitionedException(); + + KStreamJoin stream = new KStreamJoin(this.window, otherImpl.window, processor, partitioningInfo, context); + otherImpl.registerReceiver(stream.receiverForOtherStream); + + return chain(stream); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java b/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java new file mode 100644 index 0000000000000..c37b4c53cdac8 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java @@ -0,0 +1,35 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.CopartitioningGroup; + +/** + * Created by yasuhiro on 6/19/15. + */ +class PartitioningInfo { + + public static PartitioningInfo missing = new PartitioningInfo(null, 0); + + private final CopartitioningGroup copartitioningGroup; + private final int numPartitions; + + PartitioningInfo(int numPartitions) { + this(null, numPartitions); + } + + PartitioningInfo(CopartitioningGroup copartitioningGroup, int numPartitions) { + this.copartitioningGroup = copartitioningGroup; + this.numPartitions = numPartitions; + } + + boolean isJoinCompatibleWith(PartitioningInfo other) { + if (copartitioningGroup != null) { + return copartitioningGroup == other.copartitioningGroup; + } + else { + return + (other.copartitioningGroup == null & + (numPartitions >= 0 || numPartitions == other.numPartitions)); + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java new file mode 100644 index 0000000000000..0fe8cf872bf28 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -0,0 +1,14 @@ +package io.confluent.streaming.internal; + +/** + * Created by yasuhiro on 6/17/15. + */ +interface Receiver { + + void receive(K key, V value, long timestamp); + + void punctuate(long timestamp); + + void flush(); + +} diff --git a/src/main/java/io/confluent/streaming/util/Stamped.java b/src/main/java/io/confluent/streaming/util/Stamped.java new file mode 100644 index 0000000000000..460ea76cca520 --- /dev/null +++ b/src/main/java/io/confluent/streaming/util/Stamped.java @@ -0,0 +1,33 @@ +package io.confluent.streaming.util; + +/** + * Created by yasuhiro on 6/23/15. + */ +public class Stamped implements Comparable { + + public final V value; + public final long timestamp; + private final long nanoTime = System.nanoTime(); // used for tie-breaking + + 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; + + // tie breaking + otherTimestamp = ((Stamped) other).nanoTime; + + if (nanoTime < otherTimestamp) return -1; + else if (nanoTime > otherTimestamp) return 1; + + return 0; + } + +} diff --git a/src/main/java/io/confluent/streaming/util/Util.java b/src/main/java/io/confluent/streaming/util/Util.java new file mode 100644 index 0000000000000..3fff745a5d1ee --- /dev/null +++ b/src/main/java/io/confluent/streaming/util/Util.java @@ -0,0 +1,16 @@ +package io.confluent.streaming.util; + +import java.util.HashSet; + +/** + * Created by yasuhiro on 6/22/15. + */ +public class Util { + + static HashSet mkSet(T... elems) { + HashSet set = new HashSet(); + for (T e : elems) set.add(e); + return set; + } + +} From a387642ec999454c8940d94414f073df6575213e Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 25 Jun 2015 10:56:58 -0700 Subject: [PATCH 002/275] stream synchronization --- .../java/io/confluent/streaming/Chooser.java | 16 ++ .../streaming/CopartitioningGroup.java | 5 +- .../streaming/CopartitioningGroupFactory.java | 10 -- .../DefaultCopartitioningGroupFactory.java | 18 --- .../io/confluent/streaming/KStreamConfig.java | 4 +- .../io/confluent/streaming/RecordQueue.java | 24 +++ .../streaming/StreamSynchronizer.java | 96 +++++++++++- .../streaming/StreamSynchronizerFactory.java | 23 +++ .../TimeBasedStreamSynchronizer.java | 48 ------ .../streaming/internal/ChooserImpl.java | 46 ++++++ .../streaming/internal/KStreamConfigImpl.java | 59 ++++---- .../internal/KStreamContextImpl.java | 2 +- .../streaming/internal/PartitioningInfo.java | 6 +- .../streaming/internal/RecordQueueImpl.java | 27 ++++ .../streaming/internal/RegulatedConsumer.java | 83 +++++++++++ .../io/confluent/streaming/internal/Test.java | 140 ++++++++++++++++++ .../util/QueueWithMinTimestampTracking.java | 63 ++++++++ .../io/confluent/streaming/util/Stamped.java | 16 +- 18 files changed, 563 insertions(+), 123 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/Chooser.java delete mode 100644 src/main/java/io/confluent/streaming/CopartitioningGroupFactory.java delete mode 100644 src/main/java/io/confluent/streaming/DefaultCopartitioningGroupFactory.java create mode 100644 src/main/java/io/confluent/streaming/RecordQueue.java create mode 100644 src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java delete mode 100644 src/main/java/io/confluent/streaming/TimeBasedStreamSynchronizer.java create mode 100644 src/main/java/io/confluent/streaming/internal/ChooserImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java create mode 100644 src/main/java/io/confluent/streaming/internal/Test.java create mode 100644 src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java diff --git a/src/main/java/io/confluent/streaming/Chooser.java b/src/main/java/io/confluent/streaming/Chooser.java new file mode 100644 index 0000000000000..995fcb2e40145 --- /dev/null +++ b/src/main/java/io/confluent/streaming/Chooser.java @@ -0,0 +1,16 @@ +package io.confluent.streaming; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +/** + * Created by yasuhiro on 6/25/15. + */ +public interface Chooser { + + void add(RecordQueue queue); + + RecordQueue next(); + + void close(); + +} diff --git a/src/main/java/io/confluent/streaming/CopartitioningGroup.java b/src/main/java/io/confluent/streaming/CopartitioningGroup.java index ea364c198f9a2..d36635a05bc6c 100644 --- a/src/main/java/io/confluent/streaming/CopartitioningGroup.java +++ b/src/main/java/io/confluent/streaming/CopartitioningGroup.java @@ -8,12 +8,9 @@ public final class CopartitioningGroup { public final String name; public final int numPartitions; - public final StreamSynchronizer streamSynchronizer; - - public CopartitioningGroup(String name, int numPartitions, StreamSynchronizer streamSynchronizer) { + public CopartitioningGroup(String name, int numPartitions) { this.name = name; this.numPartitions = numPartitions; - this.streamSynchronizer = streamSynchronizer; } } diff --git a/src/main/java/io/confluent/streaming/CopartitioningGroupFactory.java b/src/main/java/io/confluent/streaming/CopartitioningGroupFactory.java deleted file mode 100644 index c4c06660f2aed..0000000000000 --- a/src/main/java/io/confluent/streaming/CopartitioningGroupFactory.java +++ /dev/null @@ -1,10 +0,0 @@ -package io.confluent.streaming; - -/** - * Created by yasuhiro on 6/22/15. - */ -public interface CopartitioningGroupFactory { - - CopartitioningGroup create(String name, int numPartitions); - -} diff --git a/src/main/java/io/confluent/streaming/DefaultCopartitioningGroupFactory.java b/src/main/java/io/confluent/streaming/DefaultCopartitioningGroupFactory.java deleted file mode 100644 index 508a1684c96a3..0000000000000 --- a/src/main/java/io/confluent/streaming/DefaultCopartitioningGroupFactory.java +++ /dev/null @@ -1,18 +0,0 @@ -package io.confluent.streaming; - -/** - * Created by yasuhiro on 6/22/15. - */ -public class DefaultCopartitioningGroupFactory implements CopartitioningGroupFactory { - - private final TimestampExtractor timestampExtractor; - - public DefaultCopartitioningGroupFactory(TimestampExtractor timestampExtractor) { - this.timestampExtractor = timestampExtractor; - } - - public CopartitioningGroup create(String name, int numPartitions) { - return new CopartitioningGroup(name, numPartitions, new TimeBasedStreamSynchronizer(timestampExtractor)); - } - -} diff --git a/src/main/java/io/confluent/streaming/KStreamConfig.java b/src/main/java/io/confluent/streaming/KStreamConfig.java index d7fa390bb42c7..5ddb155e39d90 100644 --- a/src/main/java/io/confluent/streaming/KStreamConfig.java +++ b/src/main/java/io/confluent/streaming/KStreamConfig.java @@ -5,9 +5,7 @@ */ public interface KStreamConfig { - void setDefaultStreamSynchronizer(StreamSynchronizer streamSynchronizer); - - void setCopartitioningGroupFactory(CopartitioningGroupFactory copartitioningGroupFactory); + void setStreamSynchronizerFactory(StreamSynchronizerFactory streamSynchronizerFactory); void addTopicToCopartitioningGroup(String topic, String groupName); diff --git a/src/main/java/io/confluent/streaming/RecordQueue.java b/src/main/java/io/confluent/streaming/RecordQueue.java new file mode 100644 index 0000000000000..c187929a45372 --- /dev/null +++ b/src/main/java/io/confluent/streaming/RecordQueue.java @@ -0,0 +1,24 @@ +package io.confluent.streaming; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; + +/** + * Created by yasuhiro on 6/25/15. + */ +public interface RecordQueue { + + TopicPartition partition(); + + void add(ConsumerRecord value, long timestamp); + + ConsumerRecord next(); + + ConsumerRecord peekNext(); + + ConsumerRecord peekLast(); + + int size(); + + long currentStreamTime(); +} diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/StreamSynchronizer.java index 1d64b2fd5b550..7dd6fc17f5d39 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizer.java @@ -1,18 +1,104 @@ package io.confluent.streaming; +import io.confluent.streaming.internal.RecordQueueImpl; +import io.confluent.streaming.internal.RegulatedConsumer; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; /** * Created by yasuhiro on 6/23/15. */ -public interface StreamSynchronizer { +public class StreamSynchronizer { + + private final RegulatedConsumer consumer; + private final Chooser chooser; + private final TimestampExtractor timestampExtractor; + private final Map> stash = new HashMap>(); + private final int desiredUnprocessed; + + private long streamTime = -1; + private int buffered = 0; + + public StreamSynchronizer(RegulatedConsumer consumer, + Chooser chooser, + TimestampExtractor timestampExtractor, + int desiredNumberOfUnprocessedRecords) { + this.consumer = consumer; + this.chooser = chooser; + this.timestampExtractor = timestampExtractor; + this.desiredUnprocessed = desiredNumberOfUnprocessedRecords; + } + + public void add(TopicPartition partition, Iterator> iterator) { + RecordQueue queue = stash.get(partition); + if (queue == null) { + queue = createRecordQueue(partition); + this.stash.put(partition, queue); + } + + boolean wasEmpty = (queue.size() == 0); + + while (iterator.hasNext()) { + ConsumerRecord record = iterator.next(); + queue.add(record, timestampExtractor.extract(record.topic(), record.key(), record.value())); + buffered++; + } + + if (wasEmpty && queue.size() > 0) chooser.add(queue); + + // if we have buffered enough for this partition, pause + if (queue.size() > this.desiredUnprocessed) { + consumer.pause(partition); + } + } + + public ConsumerRecord next() { + RecordQueue queue = chooser.next(); + + if (queue == null) { + consumer.poll(); + return null; + } + + if (queue.size() == this.desiredUnprocessed) { + ConsumerRecord record = queue.peekLast(); + if (record != null) { + consumer.unpause(queue.partition(), record.offset()); + } + } + + if (queue.size() == 0) return null; + + long timestamp = queue.currentStreamTime(); + ConsumerRecord record = queue.next(); + + if (streamTime < timestamp) streamTime = timestamp; + + if (queue.size() > 0) chooser.add(queue); + buffered--; + + return record; + } - void add(ConsumerRecord record); + public long currentStreamTime() { + return streamTime; + } - ConsumerRecord next(); + public int buffered() { + return buffered; + } - long currentStreamTime(); + public void close() { + chooser.close(); + stash.clear(); + } - void close(); + protected RecordQueue createRecordQueue(TopicPartition partition) { + return new RecordQueueImpl(partition); + } } diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java b/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java new file mode 100644 index 0000000000000..b0a207913f6ff --- /dev/null +++ b/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java @@ -0,0 +1,23 @@ +package io.confluent.streaming; + +import io.confluent.streaming.internal.ChooserImpl; +import io.confluent.streaming.internal.RegulatedConsumer; + +/** + * Created by yasuhiro on 6/24/15. + */ +public class StreamSynchronizerFactory { + + private TimestampExtractor timestampExtractor; + private int desiredNumberOfUnprocessedRecords; + + public StreamSynchronizerFactory(TimestampExtractor timestampExtractor, int desiredNumberOfUnprocessedRecords) { + this.timestampExtractor = timestampExtractor; + this.desiredNumberOfUnprocessedRecords = desiredNumberOfUnprocessedRecords; + } + + public StreamSynchronizer create(String name, RegulatedConsumer consumer) { + return new StreamSynchronizer(consumer, new ChooserImpl(), timestampExtractor, desiredNumberOfUnprocessedRecords); + } + +} diff --git a/src/main/java/io/confluent/streaming/TimeBasedStreamSynchronizer.java b/src/main/java/io/confluent/streaming/TimeBasedStreamSynchronizer.java deleted file mode 100644 index f0702be93f12b..0000000000000 --- a/src/main/java/io/confluent/streaming/TimeBasedStreamSynchronizer.java +++ /dev/null @@ -1,48 +0,0 @@ -package io.confluent.streaming; - -import io.confluent.streaming.util.Stamped; -import org.apache.kafka.clients.consumer.ConsumerRecord; - -import java.util.PriorityQueue; - -/** - * Created by yasuhiro on 6/23/15. - */ -public class TimeBasedStreamSynchronizer implements StreamSynchronizer { - - private final TimestampExtractor timestampExtractor; - private final PriorityQueue>> pq = new PriorityQueue(); - private long streamTime = -1; - - public TimeBasedStreamSynchronizer(TimestampExtractor timestampExtractor) { - this.timestampExtractor = timestampExtractor; - } - - public void add(ConsumerRecord record) { - long timestamp = timestampExtractor.extract(record.topic(), record.key(), record.value()); - - if (timestamp < 0) timestamp = System.currentTimeMillis(); - - Stamped> stamped = new Stamped(record, timestamp); - pq.offer(stamped); - } - - public ConsumerRecord next() { - Stamped> stamped = pq.poll(); - - if (stamped == null) return null; - - if (streamTime < stamped.timestamp) streamTime = stamped.timestamp; - - return stamped.value; - } - - public long currentStreamTime() { - return streamTime; - } - - public void close() { - pq.clear(); - } - -} diff --git a/src/main/java/io/confluent/streaming/internal/ChooserImpl.java b/src/main/java/io/confluent/streaming/internal/ChooserImpl.java new file mode 100644 index 0000000000000..278bfdfa020c8 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/ChooserImpl.java @@ -0,0 +1,46 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.Chooser; +import io.confluent.streaming.RecordQueue; + +import java.util.Comparator; +import java.util.PriorityQueue; + +/** + * Created by yasuhiro on 6/25/15. + */ +public class ChooserImpl implements Chooser { + + private final PriorityQueue> pq; + + public ChooserImpl() { + this(new Comparator>() { + public int compare(RecordQueue queue1, RecordQueue queue2) { + long time1 = queue1.currentStreamTime(); + long time2 = queue2.currentStreamTime(); + + if (time1 < time2) return -1; + if (time1 > time2) return 1; + return 0; + } + }); + } + + public ChooserImpl(Comparator> comparator) { + pq = new PriorityQueue>(3, comparator); + } + + public void add(RecordQueue queue) { + pq.offer(queue); + } + + + public RecordQueue next() { + return pq.poll(); + } + + public void close() { + pq.clear(); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java index e3daec0eb56ba..48ca7a080257e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java @@ -1,53 +1,43 @@ package io.confluent.streaming.internal; import io.confluent.streaming.CopartitioningGroup; -import io.confluent.streaming.CopartitioningGroupFactory; import io.confluent.streaming.KStreamConfig; import io.confluent.streaming.StreamSynchronizer; +import io.confluent.streaming.StreamSynchronizerFactory; +import org.apache.kafka.common.TopicPartition; import java.util.HashMap; +import java.util.Map; +import java.util.Set; /** * Created by yasuhiro on 6/19/15. */ public class KStreamConfigImpl implements KStreamConfig { - private StreamSynchronizer streamSynchronizer = null; - private CopartitioningGroupFactory copartitioningGroupFactory = null; + private StreamSynchronizerFactory streamSynchronizerFactory = null; + private final HashMap topicCopartitioningGroupMap = new HashMap(); private final HashMap copartitioningGroups = new HashMap(); - private final HashMap partitioningInfos = new HashMap(); KStreamConfigImpl(/* TODO: pass in a consumer */) { } - public void setDefaultStreamSynchronizer(StreamSynchronizer streamSynchronizer) { + public void setStreamSynchronizerFactory(StreamSynchronizerFactory streamSynchronizerFactory) { synchronized (this) { - if (streamSynchronizer == null) - throw new IllegalArgumentException("null StreamSynchronizer"); - if (this.streamSynchronizer != null) - throw new IllegalStateException("default StreamSynchronizer was already set"); + if (streamSynchronizerFactory == null) + throw new IllegalArgumentException("null StreamSynchronizerFactory"); + if (this.streamSynchronizerFactory != null) + throw new IllegalStateException("default StreamSynchronizerFactory was already set"); - this.streamSynchronizer = streamSynchronizer; - } - } - - public void setCopartitioningGroupFactory(CopartitioningGroupFactory copartitioningGroupFactory) { - synchronized (this) { - if (copartitioningGroupFactory == null) - throw new IllegalArgumentException("null CoPartitioningGroupFactory"); - if (this.copartitioningGroupFactory != null) - throw new IllegalStateException("CoPartitioningGroupFactory was already set"); - - this.copartitioningGroupFactory = copartitioningGroupFactory; + this.streamSynchronizerFactory = streamSynchronizerFactory; } } public void addTopicToCopartitioningGroup(String topic, String groupName) { synchronized (this) { - PartitioningInfo info = partitioningInfos.get(topic); - if (info != null) + if (topicCopartitioningGroupMap.get(topic) != null) throw new IllegalStateException("the topic was already added to the CopartitioningGroup: topic="+ topic); // TODO: use a consumer to get partition info from Kafka and set them in PartitioningInfo below @@ -56,21 +46,36 @@ public void addTopicToCopartitioningGroup(String topic, String groupName) { CopartitioningGroup group = copartitioningGroups.get(groupName); if (group == null) { - group = copartitioningGroupFactory.create(groupName, numPartitions); + group = new CopartitioningGroup(groupName, numPartitions); copartitioningGroups.put(groupName, group); } else { if (group.numPartitions != numPartitions) throw new IllegalStateException("incompatible number of partitions"); } + } + } + + Map getPartitioningInfos() { + HashMap partitioningInfos = new HashMap(); - info = new PartitioningInfo(group, numPartitions); + for (Map.Entry entry : copartitioningGroups.entrySet()) { + String topic = entry.getKey(); + CopartitioningGroup group = entry.getValue(); + PartitioningInfo info = new PartitioningInfo(group); partitioningInfos.put(topic, info); } - } - HashMap getPartitioningInfos() { return partitioningInfos; } + Map> getStreamSynchronizers(RegulatedConsumer consumer, + Set partitions) { + HashMap> streamSynchronizers = new HashMap(); + for (TopicPartition partition : partitions) { + streamSynchronizers.put(partition, streamSynchronizerFactory.create(topicCopartitioningGroupMap.get(partition.topic()), consumer)); + } + return streamSynchronizers; + } + } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index dec4ca2b41a51..5a69bbe21d7ed 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -36,7 +36,7 @@ public KStream from(String topic) { // TODO: use the consumer to get partitioning info int numPartitions = 1; - partitioningInfo = new PartitioningInfo(null, numPartitions); + partitioningInfo = new PartitioningInfo(numPartitions); partitioningInfos.put(topic, partitioningInfo); } diff --git a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java b/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java index c37b4c53cdac8..d0eea0a127e70 100644 --- a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java +++ b/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java @@ -16,7 +16,11 @@ class PartitioningInfo { this(null, numPartitions); } - PartitioningInfo(CopartitioningGroup copartitioningGroup, int numPartitions) { + PartitioningInfo(CopartitioningGroup copartitioningGroup) { + this(copartitioningGroup, copartitioningGroup.numPartitions); + } + + private PartitioningInfo(CopartitioningGroup copartitioningGroup, int numPartitions) { this.copartitioningGroup = copartitioningGroup; this.numPartitions = numPartitions; } diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java b/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java new file mode 100644 index 0000000000000..a73720bfdc427 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java @@ -0,0 +1,27 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.RecordQueue; +import io.confluent.streaming.util.QueueWithMinTimestampTracking; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; + +/** + * Created by yasuhiro on 6/25/15. + */ +public class RecordQueueImpl extends QueueWithMinTimestampTracking> implements RecordQueue { + + private final TopicPartition partition; + + public RecordQueueImpl(TopicPartition partition) { + this.partition = partition; + } + + public TopicPartition partition() { + return partition; + } + + public long currentStreamTime() { + return super.timestamp(); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java b/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java new file mode 100644 index 0000000000000..0fe197ec03875 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java @@ -0,0 +1,83 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.StreamSynchronizer; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; + +import java.util.*; + +public class RegulatedConsumer { + + private final Consumer consumer; + private final Set unpaused = new HashSet(); + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + private final long pollTimeMs; + private final Map> streamSynchronizers = new HashMap(); + + public RegulatedConsumer(Consumer consumer, + Deserializer keyDeserializer, + Deserializer valueDeserializer, + long pollTimeMs) { + this.consumer = consumer; + this.keyDeserializer = keyDeserializer; + this.valueDeserializer = valueDeserializer; + this.pollTimeMs = pollTimeMs; + } + + public void init() { + unpaused.clear(); + unpaused.addAll(consumer.subscriptions()); + } + + public void addStreamSynchronizers(Map> streamSynchronizers) { + this.streamSynchronizers.putAll(streamSynchronizers); + } + + public void poll() { + poll(pollTimeMs); + } + + public void poll(long timeoutMs) { + ConsumerRecords records = consumer.poll(timeoutMs); + + for (TopicPartition partition : unpaused) { + + final Iterator> inner = records.records(partition).iterator(); + Iterator> iterator = new Iterator>() { + public boolean hasNext() { + return inner.hasNext(); + } + public ConsumerRecord next() { + ConsumerRecord record = inner.next(); + K key = keyDeserializer.deserialize(record.topic(), record.key()); + V value = valueDeserializer.deserialize(record.topic(), record.value()); + return new ConsumerRecord(record.topic(), record.partition(), record.offset(), key, value); + } + public void remove() { + throw new UnsupportedOperationException(); + } + }; + streamSynchronizers.get(partition).add(partition, iterator); + } + } + + public void pause(TopicPartition partition) { + consumer.seek(partition, Long.MAX_VALUE); // hack: stop consuming from this partition by setting a big offset + unpaused.remove(partition); + } + + public void unpause(TopicPartition partition, long lastOffset) { + consumer.seek(partition, lastOffset); + unpaused.add(partition); + } + + public void clear() { + unpaused.clear(); + streamSynchronizers.clear(); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/Test.java b/src/main/java/io/confluent/streaming/internal/Test.java new file mode 100644 index 0000000000000..b67c925175b8a --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/Test.java @@ -0,0 +1,140 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; + +import java.util.ArrayList; + +/** + * Created by yasuhiro on 6/17/15. + */ +public class Test { + + public static void main(String[] args) throws NotCopartitionedException { + + RecordCollector collector = new RecordCollector() { + public void send(ProducerRecord record) { + System.out.println(record.toString()); + } + }; + + KStreamContextImpl context = new KStreamContextImpl(collector); + + Predicate lengthCheck = new Predicate() { + public boolean apply(Integer i, String s) { + return s.length() > 1; + } + }; + + KeyValueMapper truncate = new KeyValueMapper() { + public KeyValue apply(Integer i, String s) { + if (i < s.length()) + return KeyValue.pair(i * 10, s.substring(0, i)); + else + return KeyValue.pair(i * 10, s); + } + public void flush() { + System.out.println("flushed truncate"); + } + }; + + KeyValueMapper, Integer, String> toCharacters = new KeyValueMapper, Integer, String>() { + public KeyValue> apply(Integer i, String s) { + ArrayList list = new ArrayList(s.length()); + for (char c : s.toCharArray()) { + list.add(c); + } + return KeyValue.pair(i, list); + } + + public void flush() { + System.out.println("flushed toCharacters"); + } + }; + + ValueMapper, String> toCharacters2 = new ValueMapper, String>() { + public Iterable apply(String s) { + ArrayList list = new ArrayList(s.length()); + for (char c : s.toCharArray()) { + list.add(c); + } + return list; + } + + public void flush() { + System.out.println("flushed toCharacters2"); + } + }; + + Processor printCharWithKey = new Processor() { + public void apply(Integer i, Character c) { + System.out.println(i + ":" + c); + } + public void punctuate(long time) {} + public void flush() { + System.out.println("flushed printCharWithKey"); + } + }; + + KStream stream = context.from("topicABC"); + + stream.filter(lengthCheck).map(truncate).flatMap(toCharacters).process(printCharWithKey); + + context.process(new ConsumerRecord("topicABC", 1, 0, 1, "s"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topicABC", 1, 1, 2, "abcd"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topicABC", 1, 2, 2, "t"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topicABC", 1, 3, 3, "abcd"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topicABC", 1, 4, 3, "u"), System.currentTimeMillis()); + + stream = context.from("topicXYZ"); + stream.filter(lengthCheck).map(truncate).flatMapValues(toCharacters2).process(printCharWithKey); + + context.process(new ConsumerRecord("topicXYZ", 1, 0, 1, "h"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topicXYZ", 1, 1, 2, "xyz"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topicXYZ", 1, 2, 3, "i"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topicXYZ", 1, 3, 4, "stu"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topicXYZ", 1, 4, 5, "j"), System.currentTimeMillis()); + + KStream stream1 = context.from("topic1"); + KStream stream2 = context.from("topic2"); + + stream1.with(new WindowByCount(3)).join(stream2.with(new WindowByCount(3)), + new ValueJoiner() { + public String apply(String v1, String v2) { + return v1 + v2; + } + } + ).process(new Processor() { + public void apply(Integer key, String value) { + System.out.println(key + ":" + value); + } + + public void punctuate(long timestamp) { + } + + public void flush() { + } + }); + + context.process(new ConsumerRecord("topic1", 1, 0, 1, "x"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topic2", 1, 0, 2, "A"), System.currentTimeMillis()); + + context.process(new ConsumerRecord("topic1", 1, 1, 2, "y"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topic2", 1, 1, 2, "B"), System.currentTimeMillis()); + + context.process(new ConsumerRecord("topic1", 1, 2, 3, "z"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topic2", 1, 2, 2, "C"), System.currentTimeMillis()); + + context.process(new ConsumerRecord("topic1", 1, 3, 4, "x"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topic2", 1, 3, 2, "D"), System.currentTimeMillis()); + + context.process(new ConsumerRecord("topic1", 1, 4, 5, "y"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topic2", 1, 4, 2, "E"), System.currentTimeMillis()); + + context.process(new ConsumerRecord("topic1", 1, 5, 6, "x"), System.currentTimeMillis()); + context.process(new ConsumerRecord("topic2", 1, 5, 2, "F"), System.currentTimeMillis()); + + } +} + diff --git a/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java b/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java new file mode 100644 index 0000000000000..0f35a91ddece0 --- /dev/null +++ b/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java @@ -0,0 +1,63 @@ +package io.confluent.streaming.util; + +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.LinkedList; + +/** + * Created by yasuhiro on 6/24/15. + */ +public class QueueWithMinTimestampTracking { + + private final Deque> queue = new ArrayDeque(); + private final LinkedList> descendingSubsequence = new LinkedList(); + + public void add(E value, long timestamp) { + + Stamped elem = new Stamped(value, timestamp); + queue.addLast(elem); + + Stamped minElem = descendingSubsequence.peekLast(); + while (minElem.compareTo(elem) >= 0) { + descendingSubsequence.removeLast(); + minElem = descendingSubsequence.peekLast(); + } + descendingSubsequence.offerLast(elem); + } + + public E next() { + Stamped stamped = queue.getFirst(); + + if (stamped == null) return null; + + if (descendingSubsequence.peekFirst() == stamped) + descendingSubsequence.removeFirst(); + + return stamped.value; + } + + public E peekNext() { + if (queue.size() > 0) + return queue.peekFirst().value; + else + return null; + } + + public E peekLast() { + if (queue.size() > 0) + return queue.peekLast().value; + else + return null; + } + + public int size() { + return queue.size(); + } + + public long timestamp() { + Stamped stamped = descendingSubsequence.peekFirst(); + if (stamped == null) return -1L; + + return stamped.timestamp; + } +} diff --git a/src/main/java/io/confluent/streaming/util/Stamped.java b/src/main/java/io/confluent/streaming/util/Stamped.java index 460ea76cca520..3e278fcc20371 100644 --- a/src/main/java/io/confluent/streaming/util/Stamped.java +++ b/src/main/java/io/confluent/streaming/util/Stamped.java @@ -7,25 +7,29 @@ public class Stamped implements Comparable { public final V value; public final long timestamp; - private final long nanoTime = System.nanoTime(); // used for tie-breaking + private final long tieBreaker; public Stamped(V value, long timestamp) { + this(value, timestamp, System.nanoTime()); + } + + public Stamped(V value, long timestamp, long tieBreaker) { this.value = value; this.timestamp = timestamp; + this.tieBreaker = tieBreaker; } - @Override public int compareTo(Object other) { - long otherTimestamp = ((Stamped) other).timestamp; + long otherTimestamp = ((Stamped) other).timestamp; if (timestamp < otherTimestamp) return -1; else if (timestamp > otherTimestamp) return 1; // tie breaking - otherTimestamp = ((Stamped) other).nanoTime; + otherTimestamp = ((Stamped) other).tieBreaker; - if (nanoTime < otherTimestamp) return -1; - else if (nanoTime > otherTimestamp) return 1; + if (tieBreaker < otherTimestamp) return -1; + else if (tieBreaker > otherTimestamp) return 1; return 0; } From 94da171ed6356b64ad8e4dbf7241bd6b58d959a5 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 29 Jun 2015 09:27:06 -0700 Subject: [PATCH 003/275] ported many stuff from Jay's streming prototype --- .../io/confluent/streaming/Coordinator.java | 81 ++++ .../streaming/CopartitioningGroup.java | 16 - .../io/confluent/streaming/KStreamConfig.java | 12 - .../confluent/streaming/KStreamContext.java | 24 +- .../io/confluent/streaming/KStreamJob.java | 2 - .../confluent/streaming/KafkaStreaming.java | 429 ++++++++++++++++++ .../confluent/streaming/RecordCollector.java | 4 +- .../io/confluent/streaming/StorageEngine.java | 64 +++ .../streaming/StreamSynchronizer.java | 151 ++++-- .../streaming/StreamSynchronizerFactory.java | 8 +- .../confluent/streaming/StreamingConfig.java | 139 ++++++ .../io/confluent/streaming/SyncGroup.java | 16 + .../streaming/internal/ChooserImpl.java | 4 +- .../streaming/internal/KStreamConfigImpl.java | 81 ---- .../internal/KStreamContextImpl.java | 166 ++++++- .../streaming/internal/KStreamImpl.java | 16 +- .../streaming/internal/PartitioningInfo.java | 29 +- .../streaming/internal/ProcessorConfig.java | 90 ++++ .../streaming/internal/ProcessorContext.java | 82 ++++ .../internal/ProcessorStateManager.java | 128 ++++++ .../streaming/internal/Receiver.java | 2 +- .../streaming/internal/RecordCollectors.java | 76 ++++ .../streaming/internal/RegulatedConsumer.java | 52 ++- .../io/confluent/streaming/internal/Test.java | 140 ------ .../java/io/confluent/streaming/kv/Entry.java | 28 ++ .../streaming/kv/InMemoryKeyValueStore.java | 162 +++++++ .../streaming/kv/KeyValueIterator.java | 30 ++ .../confluent/streaming/kv/KeyValueStore.java | 85 ++++ .../kv/internals/MeteredKeyValueStore.java | 194 ++++++++ .../streaming/util/OffsetCheckpoint.java | 162 +++++++ .../io/confluent/streaming/util/Util.java | 20 + 31 files changed, 2132 insertions(+), 361 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/Coordinator.java delete mode 100644 src/main/java/io/confluent/streaming/CopartitioningGroup.java delete mode 100644 src/main/java/io/confluent/streaming/KStreamConfig.java create mode 100644 src/main/java/io/confluent/streaming/KafkaStreaming.java create mode 100644 src/main/java/io/confluent/streaming/StorageEngine.java create mode 100644 src/main/java/io/confluent/streaming/StreamingConfig.java create mode 100644 src/main/java/io/confluent/streaming/SyncGroup.java delete mode 100644 src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/ProcessorConfig.java create mode 100644 src/main/java/io/confluent/streaming/internal/ProcessorContext.java create mode 100644 src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java create mode 100644 src/main/java/io/confluent/streaming/internal/RecordCollectors.java delete mode 100644 src/main/java/io/confluent/streaming/internal/Test.java create mode 100644 src/main/java/io/confluent/streaming/kv/Entry.java create mode 100644 src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java create mode 100644 src/main/java/io/confluent/streaming/kv/KeyValueIterator.java create mode 100644 src/main/java/io/confluent/streaming/kv/KeyValueStore.java create mode 100644 src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java create mode 100644 src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java diff --git a/src/main/java/io/confluent/streaming/Coordinator.java b/src/main/java/io/confluent/streaming/Coordinator.java new file mode 100644 index 0000000000000..536421a92e6ca --- /dev/null +++ b/src/main/java/io/confluent/streaming/Coordinator.java @@ -0,0 +1,81 @@ +/** + * 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 io.confluent.streaming; + +/** + * Coordinators are provided to the process methods of {@link StreamProcessor} implementations to allow + * the user code to request actions from the framework. + *

+ * This interface may evolve over time. + *

+ */ +public interface Coordinator { + + /** + * Requests that the stream processor flush all it's state as well as any buffered output and commit the offsets. + * + *

+ * If CURRENT_TASK is given, a checkpoint is only written for the current task. If + * ALL_TASKS_IN_CONTAINER is given, a checkpoint is written for all tasks in the current container. + * + *

+ * Note that if you also have also configured your job to commit in regular intervals (using the + * task.commit.ms property), those time-based commits are not affected by calling this method. Any + * commits you request explicitly are in addition to timer-based commits. You can set task.commit.ms=-1 + * if you don't want commits to happen automatically. + * + * @param scope Which tasks are being asked to commit. + */ + public void commit(RequestScope scope); + + /** + * Requests that the container should be shut down. + * + *

+ * If CURRENT_TASK is given, that indicates a willingness of the current task to shut down. All tasks + * in the container (including the one that requested shutdown) will continue processing messages. Only when every + * task in the container has called shutdown(CURRENT_TASK), the container is shut down. Once a task has + * called shutdown(CURRENT_TASK), it cannot change its mind (i.e. it cannot revoke its willingness to + * shut down). + * + *

+ * If ALL_TASKS_IN_CONTAINER is given, the container will shut down immediately after it has finished + * processing the current message. Any buffers of pending writes are flushed, but no further messages will be + * processed in this container. + * + * @param scope The approach we should use for shutting down the container. + */ + public void shutdown(RequestScope scope); + + /** + * A task can make requests to the Samza framework while processing messages, such as + * {@link Coordinator#commit(RequestScope)} and {@link Coordinator#shutdown(RequestScope)}. This enum is used to + * indicate whether those requests apply only to the current task, or to all tasks in the current container. + */ + public enum RequestScope { + /** + * Indicates that a request applies only to the task making the call. + */ + CURRENT_TASK, + + /** + * Indicates that a request applies to all tasks in the current container. + */ + ALL_TASKS_IN_CONTAINER; + } +} diff --git a/src/main/java/io/confluent/streaming/CopartitioningGroup.java b/src/main/java/io/confluent/streaming/CopartitioningGroup.java deleted file mode 100644 index d36635a05bc6c..0000000000000 --- a/src/main/java/io/confluent/streaming/CopartitioningGroup.java +++ /dev/null @@ -1,16 +0,0 @@ -package io.confluent.streaming; - -/** - * Created by yasuhiro on 6/22/15. - */ -public final class CopartitioningGroup { - - public final String name; - public final int numPartitions; - - public CopartitioningGroup(String name, int numPartitions) { - this.name = name; - this.numPartitions = numPartitions; - } - -} diff --git a/src/main/java/io/confluent/streaming/KStreamConfig.java b/src/main/java/io/confluent/streaming/KStreamConfig.java deleted file mode 100644 index 5ddb155e39d90..0000000000000 --- a/src/main/java/io/confluent/streaming/KStreamConfig.java +++ /dev/null @@ -1,12 +0,0 @@ -package io.confluent.streaming; - -/** - * Created by yasuhiro on 6/22/15. - */ -public interface KStreamConfig { - - void setStreamSynchronizerFactory(StreamSynchronizerFactory streamSynchronizerFactory); - - void addTopicToCopartitioningGroup(String topic, String groupName); - -} diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index d9da4593b6d42..fd1b514cf9ef7 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -1,12 +1,34 @@ package io.confluent.streaming; +import org.apache.kafka.common.metrics.Metrics; + +import java.io.File; +import java.util.Map; + /** * Created by yasuhiro on 6/19/15. */ public interface KStreamContext { + static final String DEFAULT_SYNCHRONIZATION_GROUP = "defaultSynchronizationGroup"; + KStream from(String topic); - RecordCollector getRecordCollector(); + KStream from(String topic, SyncGroup syncGroup); + + RecordCollector simpleRecordCollector(); + + RecordCollector recordCollector(); + + Coordinator coordinator(); + + Map getContext(); + + File stateDir(); + + Metrics metrics(); + + SyncGroup syncGroup(String name); + void restore(StorageEngine engine) throws Exception; } diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java index bb3781f5d950f..e54dfe9de4cb3 100644 --- a/src/main/java/io/confluent/streaming/KStreamJob.java +++ b/src/main/java/io/confluent/streaming/KStreamJob.java @@ -5,8 +5,6 @@ */ public interface KStreamJob { - void configure(KStreamConfig config); - void build(KStreamContext ksc); } diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java new file mode 100644 index 0000000000000..186ecba6fc826 --- /dev/null +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -0,0 +1,429 @@ +/** + * 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 io.confluent.streaming; + +import io.confluent.streaming.internal.KStreamContextImpl; +import io.confluent.streaming.internal.ProcessorConfig; +import io.confluent.streaming.internal.RecordCollectors; +import io.confluent.streaming.internal.RegulatedConsumer; +import io.confluent.streaming.util.Util; +import org.apache.kafka.clients.consumer.*; +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.errors.InterruptException; +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.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.*; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and + * sends output to zero or more output topics. + *

+ * This processing is done by implementing the {@link StreamProcessor} interface to specify the transformation. The + * {@link KafkaStreaming} instance will be responsible for the lifecycle of these processors. It will instantiate and + * start one or more of these processors to process the Kafka partitions assigned to this particular instance. + *

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

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

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

+ *    Properties props = new Properties();
+ *    props.put("bootstrap.servers", "localhost:4242");
+ *    StreamingConfig config = new StreamingConfig(props);
+ *    config.subscribe("test-topic-1", "test-topic-2");
+ *    config.processor(ExampleStreamProcessor.class);
+ *    config.serialization(new StringSerializer(), new StringDeserializer());
+ *    KafkaStreaming container = new KafkaStreaming(config);
+ *    container.run();
+ * 
+ * + */ +public class KafkaStreaming implements Runnable { + + private static final Logger log = LoggerFactory.getLogger(KafkaStreaming.class); + + private final KStreamJob job; + private final Map> syncGroups = new HashMap(); + private final Map kstreamContexts = new HashMap(); + protected final Producer producer; + protected final Consumer consumer; + private final RegulatedConsumer regulatedConsumer; + private final StreamingConfig streamingConfig; + private final ProcessorConfig config; + private final Metrics metrics; + private final KafkaStreamingMetrics streamingMetrics; + private final Time time; + private final List requestingCommit; + private final AtomicBoolean started = new AtomicBoolean(false); + private volatile boolean running; + private CountDownLatch shutdownComplete = new CountDownLatch(1); + private long lastCommit; + private long lastWindow; + private long nextStateCleaning; + private long recordsProcessed; + + protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { + @Override + public void onPartitionsAssigned(Consumer consumer, Collection assignment) { + addPartitions(assignment); + } + + @Override + public void onPartitionsRevoked(Consumer consumer, Collection assignment) { + removePartitions(assignment); + } + }; + + public KafkaStreaming(KStreamJob job, StreamingConfig config) { + this(job, config, null, null); + } + + protected KafkaStreaming(KStreamJob job, + StreamingConfig config, + Producer producer, + Consumer consumer) { + this.job = job; + this.producer = producer == null? new KafkaProducer(config.config(), new ByteArraySerializer(), new ByteArraySerializer()): producer; + this.consumer = consumer == null? new KafkaConsumer(config.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()): consumer; + this.streamingConfig = config; + this.metrics = new Metrics(); + this.streamingMetrics = new KafkaStreamingMetrics(); + this.requestingCommit = new ArrayList(); + this.config = new ProcessorConfig(config.config()); + this.regulatedConsumer = + new RegulatedConsumer(this.consumer, + (Deserializer) config.keyDeserializer(), + (Deserializer) config.valueDeserializer(), + this.config.pollTimeMs); + this.running = true; + this.lastCommit = 0; + this.lastWindow = 0; + this.nextStateCleaning = Long.MAX_VALUE; + this.recordsProcessed = 0; + this.time = new SystemTime(); + } + + /** + * Execute the stream processors + */ + public synchronized void run() { + init(); + try { + runLoop(); + } catch (RuntimeException e) { + log.error("Uncaught error during processing: ", e); + throw e; + } finally { + shutdown(); + } + } + + private void init() { + log.info("Starting container"); + if (this.started.compareAndSet(false, true)) { + if (!config.stateDir.exists() && !config.stateDir.mkdirs()) + throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); + + for (String topic : streamingConfig.topics()) + consumer.subscribe(topic); + log.info("Start-up complete"); + } else { + throw new IllegalStateException("This container was already started"); + } + } + + private void shutdown() { + log.info("Shutting down container"); + commitAll(time.milliseconds()); + + for (Map.Entry> entry : syncGroups.entrySet()) { + for (SyncGroup syncGroup : entry.getValue()) { + try { + syncGroup.streamSynchronizer.close(); + } + catch(Exception e) { + log.error("Error while closing stream synchronizers: ", e); + } + } + } + + producer.close(); + consumer.close(); + log.info("Shut down complete"); + } + + /** + * Shutdown this streaming instance. + */ + public synchronized void close() { + this.running = false; + try { + this.shutdownComplete.await(); + } catch (InterruptedException e) { + throw new InterruptException(e); + } + } + + private void runLoop() { + try { + while (stillRunning()) { + for (Map.Entry> entry : syncGroups.entrySet()) { + for (SyncGroup syncGroup : entry.getValue()) { + syncGroup.streamSynchronizer.process(); + } + } + maybeWindow(); + maybeCommit(); + maybeCleanState(); + } + } catch (Exception e) { + throw new KafkaException(e); + } + } + + private boolean stillRunning() { + if(!this.running) { + log.debug("Shutting down at user request."); + return false; + } + if(this.config.totalRecordsToProcess >= 0 && this.recordsProcessed >= this.config.totalRecordsToProcess) { + log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", this.config.totalRecordsToProcess); + return false; + } + return true; + } + + private void maybeWindow() throws Exception { + long now = time.milliseconds(); + if (this.config.windowTimeMs >= 0 && lastWindow + this.config.windowTimeMs < now) { + log.trace("Windowing all stream processors"); + this.lastWindow = now; + for (KStreamContextImpl context : kstreamContexts.values()) { + context.punctuate(now); + // check co-ordinator + } + this.streamingMetrics.windowTime.record(time.milliseconds() - now); + } + } + + private void maybeCommit() { + long now = time.milliseconds(); + if (this.config.commitTimeMs >= 0 && lastCommit + this.config.commitTimeMs < time.milliseconds()) { + log.trace("Committing processor instances because the commit interval has elapsed."); + commitAll(now); + } else { + if (!this.requestingCommit.isEmpty()) { + log.trace("Committing processor instances because of user request."); + commitRequesting(now); + } + } + } + + private void commitAll(long now) { + Map commit = new HashMap(); + for (KStreamContextImpl context : kstreamContexts.values()) { + context.flush(); + // check co-ordinator + } + for (Map.Entry> entry : syncGroups.entrySet()) { + for (SyncGroup syncGroup : entry.getValue()) { + try { + commit.putAll(syncGroup.streamSynchronizer.consumedOffsets()); + } + catch(Exception e) { + log.error("Error while closing processor: ", e); + } + } + } + + producer.flush(); + consumer.commit(commit, CommitType.SYNC); // TODO: can this be async? + this.streamingMetrics.commitTime.record(time.milliseconds() - lastCommit); + } + + private void commitRequesting(long now) { + Map commit = new HashMap(this.requestingCommit.size()); + for (Integer id : this.requestingCommit) { + KStreamContextImpl context = kstreamContexts.get(id); + context.flush(); + + for (SyncGroup synchGroup : this.syncGroups.get(id)) { + commit.putAll(synchGroup.streamSynchronizer.consumedOffsets()); // TODO: can this be async? + } + } + consumer.commit(commit, CommitType.SYNC); + this.requestingCommit.clear(); + this.streamingMetrics.commitTime.record(time.milliseconds() - now); + } + + /* delete any state dirs that aren't for active contexts */ + private void maybeCleanState() { + long now = time.milliseconds(); + if(now > this.nextStateCleaning) { + File[] stateDirs = this.config.stateDir.listFiles(); + if(stateDirs != null) { + for(File dir: stateDirs) { + try { + Integer id = Integer.parseInt(dir.getName()); + if(!this.kstreamContexts.keySet().contains(id)) { + log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); + Util.rm(dir); + } + } catch(NumberFormatException e) { + log.warn("Deleting unknown directory in state directory {}.", dir.getAbsolutePath()); + Util.rm(dir); + } + } + } + this.nextStateCleaning = Long.MAX_VALUE; + } + } + + private void addPartitions(Collection assignment) { + HashSet partitions = new HashSet(assignment); + + Consumer restoreConsumer = + new KafkaConsumer( + streamingConfig.config(), + null, + new ByteArrayDeserializer(), + new ByteArrayDeserializer()); + + for (TopicPartition partition : partitions) { + final Integer id = partition.partition(); + KStreamContextImpl kstreamContext = kstreamContexts.get(id); + if (kstreamContext == null) { + Coordinator coordinator = new Coordinator() { + @Override + public void commit(Coordinator.RequestScope scope) { + requestingCommit.add(id); + } + + @Override + public void shutdown(Coordinator.RequestScope scope) { + throw new IllegalStateException("Implement me"); + } + }; + + File stateDir = new File(config.stateDir, id.toString()); + + kstreamContext = + new KStreamContextImpl(id, regulatedConsumer, producer, coordinator, streamingConfig, config, stateDir, metrics); + + kstreamContext.init(restoreConsumer,job); + + kstreamContexts.put(id, kstreamContext); + syncGroups.put(id, kstreamContext.syncGroups()); + } + } + + restoreConsumer.close(); + this.regulatedConsumer.init(); + this.nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; + } + + private void removePartitions(Collection assignment) { + commitAll(time.milliseconds()); + // remove all partitions + for (TopicPartition partition : assignment) { + Collection syncGroups = this.syncGroups.remove(partition.partition()); + if (syncGroups != null) { + log.info("Removing synchronization groups {}", partition.partition()); + for (SyncGroup syncGroup : syncGroups) + syncGroup.streamSynchronizer.close(); + } + } + for (TopicPartition partition : assignment) { + KStreamContextImpl kstreamContext = kstreamContexts.remove(partition.partition()); + if (kstreamContext != null) { + log.info("Removing stream context {}", partition.partition()); + try { + kstreamContext.close(); + } + catch (Exception e) { + throw new KafkaException(e); + } + streamingMetrics.processorDestruction.record(); + } + } + // clear buffered records + this.regulatedConsumer.clear(); + } + + private class KafkaStreamingMetrics { + final Sensor commitTime; + final Sensor processTime; + final Sensor windowTime; + final Sensor processorCreation; + final Sensor processorDestruction; + + public KafkaStreamingMetrics() { + String group = "kafka-streaming"; + + this.commitTime = metrics.sensor("commit-time"); + this.commitTime.add(new MetricName(group, "commit-time-avg-ms"), new Avg()); + this.commitTime.add(new MetricName(group, "commits-time-max-ms"), new Max()); + this.commitTime.add(new MetricName(group, "commits-per-second"), new Rate(new Count())); + + this.processTime = metrics.sensor("process-time"); + this.commitTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); + this.commitTime.add(new MetricName(group, "process-time-max-ms"), new Max()); + this.commitTime.add(new MetricName(group, "process-calls-per-second"), new Rate(new Count())); + + this.windowTime = metrics.sensor("window-time"); + this.windowTime.add(new MetricName(group, "window-time-avg-ms"), new Avg()); + this.windowTime.add(new MetricName(group, "window-time-max-ms"), new Max()); + this.windowTime.add(new MetricName(group, "window-calls-per-second"), new Rate(new Count())); + + this.processorCreation = metrics.sensor("processor-creation"); + this.processorCreation.add(new MetricName(group, "processor-creation"), new Rate(new Count())); + + this.processorDestruction = metrics.sensor("processor-destruction"); + this.processorDestruction.add(new MetricName(group, "processor-destruction"), new Rate(new Count())); + + } + + } + +} diff --git a/src/main/java/io/confluent/streaming/RecordCollector.java b/src/main/java/io/confluent/streaming/RecordCollector.java index 7341395a4dd54..36104d705c5c2 100644 --- a/src/main/java/io/confluent/streaming/RecordCollector.java +++ b/src/main/java/io/confluent/streaming/RecordCollector.java @@ -5,8 +5,8 @@ /** * Created by yasuhiro on 6/19/15. */ -public interface RecordCollector { +public interface RecordCollector { - void send(ProducerRecord record); + void send(ProducerRecord record); } diff --git a/src/main/java/io/confluent/streaming/StorageEngine.java b/src/main/java/io/confluent/streaming/StorageEngine.java new file mode 100644 index 0000000000000..cbd825908cdb2 --- /dev/null +++ b/src/main/java/io/confluent/streaming/StorageEngine.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.confluent.streaming; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.common.TopicPartition; + +/** + * A storage engine for managing state maintained by a stream processor. + * + *

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

+ */ +public interface StorageEngine { + + /** + * The name of this store. + */ + public String name(); + + /** + * Register the given storage engine with the changelog and restore it's state using the given + * consumer instance. + * @param consumer The consumer to read with + * @param partition The partition to use as the change log + * @param checkpointedOffset The offset of the last save + * @param logEndOffset The last offset in the changelog + */ + public void registerAndRestore(RecordCollector collector, + Consumer consumer, + TopicPartition partition, + long checkpointedOffset, + long logEndOffset); + + /** + * Flush any cached data + */ + public void flush(); + + /** + * Close the storage engine + */ + public void close(); + +} diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/StreamSynchronizer.java index 7dd6fc17f5d39..d4424f8b78b2c 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizer.java @@ -2,6 +2,7 @@ import io.confluent.streaming.internal.RecordQueueImpl; import io.confluent.streaming.internal.RegulatedConsumer; +import io.confluent.streaming.internal.Receiver; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; @@ -14,80 +15,105 @@ */ public class StreamSynchronizer { + public final String name; private final RegulatedConsumer consumer; private final Chooser chooser; private final TimestampExtractor timestampExtractor; - private final Map> stash = new HashMap>(); + private final Map stash = new HashMap(); private final int desiredUnprocessed; + private final Map consumedOffsets; private long streamTime = -1; - private int buffered = 0; + private volatile int buffered = 0; - public StreamSynchronizer(RegulatedConsumer consumer, + public StreamSynchronizer(String name, + RegulatedConsumer consumer, Chooser chooser, TimestampExtractor timestampExtractor, int desiredNumberOfUnprocessedRecords) { + this.name = name; this.consumer = consumer; this.chooser = chooser; this.timestampExtractor = timestampExtractor; this.desiredUnprocessed = desiredNumberOfUnprocessedRecords; + this.consumedOffsets = new HashMap(); + } - public void add(TopicPartition partition, Iterator> iterator) { - RecordQueue queue = stash.get(partition); - if (queue == null) { - queue = createRecordQueue(partition); - this.stash.put(partition, queue); + public void addPartition(TopicPartition partition, final Receiver receiver) { + synchronized (this) { + RecordQueueWrapper queue = stash.get(partition); + + if (queue == null) { + queue = new RecordQueueWrapper(createRecordQueue(partition)) { + @Override + void doProcess(ConsumerRecord record, long streamTime) { + receiver.receive(record.key(), record.value(), streamTime); + } + }; + stash.put(partition, queue); + } else { + throw new IllegalStateException("duplicate partition"); + } } + } - boolean wasEmpty = (queue.size() == 0); + public void addRecords(TopicPartition partition, Iterator> iterator) { + synchronized (this) { + RecordQueueWrapper queue = stash.get(partition); + if (queue != null) { + boolean wasEmpty = (queue.size() == 0); - while (iterator.hasNext()) { - ConsumerRecord record = iterator.next(); - queue.add(record, timestampExtractor.extract(record.topic(), record.key(), record.value())); - buffered++; - } + while (iterator.hasNext()) { + ConsumerRecord record = iterator.next(); + queue.add(record, timestampExtractor.extract(record.topic(), record.key(), record.value())); + buffered++; + } - if (wasEmpty && queue.size() > 0) chooser.add(queue); + if (wasEmpty && queue.size() > 0) chooser.add(queue); - // if we have buffered enough for this partition, pause - if (queue.size() > this.desiredUnprocessed) { - consumer.pause(partition); + // if we have buffered enough for this partition, pause + if (queue.size() > this.desiredUnprocessed) { + consumer.pause(partition); + } + } } } - public ConsumerRecord next() { - RecordQueue queue = chooser.next(); + public void process() { + synchronized (this) { + RecordQueueWrapper queue = (RecordQueueWrapper)chooser.next(); - if (queue == null) { - consumer.poll(); - return null; - } - - if (queue.size() == this.desiredUnprocessed) { - ConsumerRecord record = queue.peekLast(); - if (record != null) { - consumer.unpause(queue.partition(), record.offset()); + if (queue == null) { + consumer.poll(); + return; } - } - if (queue.size() == 0) return null; + if (queue.size() == this.desiredUnprocessed) { + ConsumerRecord record = queue.peekLast(); + if (record != null) { + consumer.unpause(queue.partition(), record.offset()); + } + } - long timestamp = queue.currentStreamTime(); - ConsumerRecord record = queue.next(); + if (queue.size() == 0) return; - if (streamTime < timestamp) streamTime = timestamp; + queue.process(); - if (queue.size() > 0) chooser.add(queue); - buffered--; + if (queue.size() > 0) chooser.add(queue); - return record; + buffered--; + } } public long currentStreamTime() { return streamTime; } + public Map consumedOffsets() { + return this.consumedOffsets; + } + public int buffered() { return buffered; } @@ -101,4 +127,55 @@ protected RecordQueue createRecordQueue(TopicPartition partition) { return new RecordQueueImpl(partition); } + private abstract class RecordQueueWrapper implements RecordQueue { + + private final RecordQueue queue; + + RecordQueueWrapper(RecordQueue queue) { + this.queue = queue; + } + + void process() { + long timestamp = queue.currentStreamTime(); + ConsumerRecord record = queue.next(); + + if (streamTime < timestamp) streamTime = timestamp; + + doProcess(record, streamTime); + + consumedOffsets.put(queue.partition(), record.offset()); + } + + abstract void doProcess(ConsumerRecord record, long streamTime); + + public TopicPartition partition() { + return queue.partition(); + } + + public void add(ConsumerRecord value, long timestamp) { + queue.add(value, timestamp); + } + + public ConsumerRecord next() { + return queue.next(); + } + + public ConsumerRecord peekNext() { + return queue.peekNext(); + } + + public ConsumerRecord peekLast() { + return queue.peekLast(); + } + + public int size() { + return queue.size(); + } + + public long currentStreamTime() { + return queue.currentStreamTime(); + } + + } + } diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java b/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java index b0a207913f6ff..5de2137d874f5 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java @@ -9,15 +9,13 @@ public class StreamSynchronizerFactory { private TimestampExtractor timestampExtractor; - private int desiredNumberOfUnprocessedRecords; - public StreamSynchronizerFactory(TimestampExtractor timestampExtractor, int desiredNumberOfUnprocessedRecords) { + public StreamSynchronizerFactory(TimestampExtractor timestampExtractor) { this.timestampExtractor = timestampExtractor; - this.desiredNumberOfUnprocessedRecords = desiredNumberOfUnprocessedRecords; } - public StreamSynchronizer create(String name, RegulatedConsumer consumer) { - return new StreamSynchronizer(consumer, new ChooserImpl(), timestampExtractor, desiredNumberOfUnprocessedRecords); + public StreamSynchronizer create(String name, RegulatedConsumer consumer, int desiredNumberOfUnprocessedRecords) { + return new StreamSynchronizer(name, consumer, new ChooserImpl(), timestampExtractor, desiredNumberOfUnprocessedRecords); } } diff --git a/src/main/java/io/confluent/streaming/StreamingConfig.java b/src/main/java/io/confluent/streaming/StreamingConfig.java new file mode 100644 index 0000000000000..e4bf4ed191ae2 --- /dev/null +++ b/src/main/java/io/confluent/streaming/StreamingConfig.java @@ -0,0 +1,139 @@ +/** + * 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 io.confluent.streaming; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.*; + +/** + * Configuration information passed to the {@link KafkaStreaming} instance for configuring the associated + * {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer}, and + * {@link org.apache.kafka.clients.consumer.KafkaConsumer KafkaConsumer}, as + * well as the processor itself. + */ +public class StreamingConfig { + + /** window.time.ms */ + public static final String STATE_DIR_CONFIG = "state.dir"; + + /** poll.time.ms */ + public static final String POLL_TIME_MS_CONFIG = "poll.time.ms"; + + /** commit.time.ms */ + public static final String COMMIT_TIME_MS_CONFIG = "commit.time.ms"; + + /** window.time.ms */ + public static final String WINDOW_TIME_MS_CONFIG = "window.time.ms"; + + /** buffered.records.per.partition */ + public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; + + /** state.cleanup.delay */ + public static final String STATE_CLEANUP_DELAY_CONFIG = "state.cleanup.delay"; + + /** total.records.to.process */ + public static final String TOTAL_RECORDS_TO_PROCESS = "total.records.to.process"; + + private final Properties config; + private final Map context = new HashMap(); + private final Set topics = new HashSet(); + private Serializer keySerializer; + private Serializer valSerializer; + private Deserializer keyDeserializer; + private Deserializer valDeserializer; + private StreamSynchronizerFactory streamSynchronizerFactory; + + public StreamingConfig(Properties config) { + this.config = config; + config.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + config.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); + } + + public void addContextObject(String key, Object value) { + this.context.put(key, value); + } + + @SuppressWarnings("all") + + public void subscribe(String...topic) { + Collections.addAll(this.topics, topic); + } + + public void serialization(Serializer serializer, Deserializer deserializer) { + keySerializer(serializer); + valueSerializer(serializer); + keyDeserializer(deserializer); + valueDeserializer(deserializer); + } + + public void keySerializer(Serializer serializer) { + this.keySerializer = serializer; + } + + public void valueSerializer(Serializer serializer) { + this.valSerializer = serializer; + } + + public void keyDeserializer(Deserializer deserializer) { + this.keyDeserializer = deserializer; + } + + public void valueDeserializer(Deserializer deserializer) { + this.valDeserializer = deserializer; + } + + public Properties config() { + return this.config; + } + + public Map context() { + return this.context; + } + + public Set topics() { + return this.topics; + } + + public Serializer keySerializer() { + return this.keySerializer; + } + + public Serializer valueSerializer() { + return this.valSerializer; + } + + public Deserializer keyDeserializer() { + return this.keyDeserializer; + } + + public Deserializer valueDeserializer() { + return this.valDeserializer; + } + + public void streamSynchronizerFactory(StreamSynchronizerFactory streamSynchronizerFactory) { + this.streamSynchronizerFactory = streamSynchronizerFactory; + } + + public StreamSynchronizerFactory streamSynchronizerFactory() { + return this.streamSynchronizerFactory; + } +} diff --git a/src/main/java/io/confluent/streaming/SyncGroup.java b/src/main/java/io/confluent/streaming/SyncGroup.java new file mode 100644 index 0000000000000..a0755ea744325 --- /dev/null +++ b/src/main/java/io/confluent/streaming/SyncGroup.java @@ -0,0 +1,16 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/26/15. + */ +public class SyncGroup { + + public final String name; + public final StreamSynchronizer streamSynchronizer; + + public SyncGroup(String name, StreamSynchronizer streamSynchronizer) { + this.name = name; + this.streamSynchronizer = streamSynchronizer; + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/ChooserImpl.java b/src/main/java/io/confluent/streaming/internal/ChooserImpl.java index 278bfdfa020c8..36418db04bd63 100644 --- a/src/main/java/io/confluent/streaming/internal/ChooserImpl.java +++ b/src/main/java/io/confluent/streaming/internal/ChooserImpl.java @@ -30,15 +30,17 @@ public ChooserImpl(Comparator> comparator) { pq = new PriorityQueue>(3, comparator); } + @Override public void add(RecordQueue queue) { pq.offer(queue); } - + @Override public RecordQueue next() { return pq.poll(); } + @Override public void close() { pq.clear(); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java deleted file mode 100644 index 48ca7a080257e..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/KStreamConfigImpl.java +++ /dev/null @@ -1,81 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.CopartitioningGroup; -import io.confluent.streaming.KStreamConfig; -import io.confluent.streaming.StreamSynchronizer; -import io.confluent.streaming.StreamSynchronizerFactory; -import org.apache.kafka.common.TopicPartition; - -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -/** - * Created by yasuhiro on 6/19/15. - */ -public class KStreamConfigImpl implements KStreamConfig { - - private StreamSynchronizerFactory streamSynchronizerFactory = null; - - private final HashMap topicCopartitioningGroupMap = new HashMap(); - private final HashMap copartitioningGroups = new HashMap(); - - KStreamConfigImpl(/* TODO: pass in a consumer */) { - - } - - public void setStreamSynchronizerFactory(StreamSynchronizerFactory streamSynchronizerFactory) { - synchronized (this) { - if (streamSynchronizerFactory == null) - throw new IllegalArgumentException("null StreamSynchronizerFactory"); - if (this.streamSynchronizerFactory != null) - throw new IllegalStateException("default StreamSynchronizerFactory was already set"); - - this.streamSynchronizerFactory = streamSynchronizerFactory; - } - } - - public void addTopicToCopartitioningGroup(String topic, String groupName) { - synchronized (this) { - if (topicCopartitioningGroupMap.get(topic) != null) - throw new IllegalStateException("the topic was already added to the CopartitioningGroup: topic="+ topic); - - // TODO: use a consumer to get partition info from Kafka and set them in PartitioningInfo below - int numPartitions = 1; - - CopartitioningGroup group = copartitioningGroups.get(groupName); - - if (group == null) { - group = new CopartitioningGroup(groupName, numPartitions); - copartitioningGroups.put(groupName, group); - } - else { - if (group.numPartitions != numPartitions) - throw new IllegalStateException("incompatible number of partitions"); - } - } - } - - Map getPartitioningInfos() { - HashMap partitioningInfos = new HashMap(); - - for (Map.Entry entry : copartitioningGroups.entrySet()) { - String topic = entry.getKey(); - CopartitioningGroup group = entry.getValue(); - PartitioningInfo info = new PartitioningInfo(group); - partitioningInfos.put(topic, info); - } - - return partitioningInfos; - } - - Map> getStreamSynchronizers(RegulatedConsumer consumer, - Set partitions) { - HashMap> streamSynchronizers = new HashMap(); - for (TopicPartition partition : partitions) { - streamSynchronizers.put(partition, streamSynchronizerFactory.create(topicCopartitioningGroupMap.get(partition.topic()), consumer)); - } - return streamSynchronizers; - } - -} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 5a69bbe21d7ed..60ee738afff0d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -1,32 +1,91 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStream; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.RecordCollector; -import org.apache.kafka.clients.consumer.ConsumerRecord; - +import io.confluent.streaming.*; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Serializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.Collection; import java.util.HashMap; +import java.util.Map; +import java.util.Set; /** * Created by yasuhiro on 6/19/15. */ public class KStreamContextImpl implements KStreamContext { - private final RecordCollector collector; - private final HashMap> sourceStreams = new HashMap>(); - private final HashMap partitioningInfos; + private static final Logger log = LoggerFactory.getLogger(KStreamContextImpl.class); - KStreamContextImpl(/* TODO: pass in consumer */ RecordCollector collector) { - this(new HashMap(), collector); - } + public final int id; + private final Set topics; + + private final RegulatedConsumer regulatedConsumer; + private final RecordCollectors.SimpleRecordCollector simpleCollector; + private final RecordCollector collector; - KStreamContextImpl(/* TODO: pass in consumer */ HashMap partitioningInfos, RecordCollector collector) { - this.collector = collector; - this.partitioningInfos = partitioningInfos; + private final Coordinator coordinator; + private final HashMap> sourceStreams = new HashMap>(); + private final HashMap partitioningInfos = new HashMap(); + private final StreamSynchronizerFactory streamSynchronizerFactory; + private final HashMap syncGroups = new HashMap(); + private final StreamingConfig streamingConfig; + private final ProcessorConfig processorConfig; + private final Metrics metrics; + private final File stateDir; + private final ProcessorContext processorContext; + private final ProcessorStateManager state; + private final Map stores = new HashMap(); + private Consumer restoreConsumer; + + public KStreamContextImpl(int id, + RegulatedConsumer regulatedConsumer, + Producer producer, + Coordinator coordinator, + StreamingConfig streamingConfig, + ProcessorConfig processorConfig, + File stateDir, + Metrics metrics) { + this.id = id; + this.topics = streamingConfig.topics(); + this.regulatedConsumer = (RegulatedConsumer)regulatedConsumer; + + this.simpleCollector = new RecordCollectors.SimpleRecordCollector(producer); + this.collector = new RecordCollectors.SerializingRecordCollector( + simpleCollector, (Serializer) streamingConfig.keySerializer(), (Serializer) streamingConfig.valueSerializer()); + + this.coordinator = coordinator; + this.streamingConfig = streamingConfig; + this.processorConfig = processorConfig; + + this.streamSynchronizerFactory = (StreamSynchronizerFactory)this.streamingConfig.streamSynchronizerFactory(); + if (this.streamSynchronizerFactory == null) throw new NullPointerException(); + + this.stateDir = stateDir; + this.state = new ProcessorStateManager(id, stateDir); + this.processorContext = new ProcessorContext(id, streamingConfig, stateDir, metrics); + + this.metrics = metrics; } + @Override public KStream from(String topic) { + return from(topic, syncGroup(DEFAULT_SYNCHRONIZATION_GROUP)); + } + + @Override + public KStream from(String topic, SyncGroup syncGroup) { + if (syncGroup == null) throw new NullPointerException(); + synchronized (this) { + if (!topics.contains(topic)) + throw new IllegalArgumentException("topic not subscribed: " + topic); + KStreamSource stream = (KStreamSource)sourceStreams.get(topic); if (stream == null) { @@ -36,43 +95,102 @@ public KStream from(String topic) { // TODO: use the consumer to get partitioning info int numPartitions = 1; - partitioningInfo = new PartitioningInfo(numPartitions); + partitioningInfo = new PartitioningInfo(syncGroup, numPartitions); partitioningInfos.put(topic, partitioningInfo); } stream = new KStreamSource(partitioningInfo, this); sourceStreams.put(topic, stream); + + syncGroup.streamSynchronizer.addPartition(new TopicPartition(topic, id), (Receiver)stream); + } + else { + if (stream.partitioningInfo.syncGroup == syncGroup) + throw new IllegalStateException("topic is already assigned a different synchronization group"); } return stream; } } - public RecordCollector getRecordCollector() { + @Override + public RecordCollector simpleRecordCollector() { + return simpleCollector; + } + + @Override + public RecordCollector recordCollector() { return collector; } - void process(ConsumerRecord record, long timestamp) { - String topic = record.topic(); + @Override + public Coordinator coordinator() { + return coordinator; + } + + @Override + public Map getContext() { + return streamingConfig.context(); + } + + @Override + public File stateDir() { + return stateDir; + } + + @Override + public Metrics metrics() { + return metrics; + } + + @Override + public SyncGroup syncGroup(String name) { + synchronized (this) { + SyncGroup syncGroup = syncGroups.get(name); + if (syncGroup == null) { + StreamSynchronizer streamSynchronizer = + streamSynchronizerFactory.create(name, regulatedConsumer, processorConfig.bufferedRecordsPerPartition); + syncGroup = new SyncGroup(name, streamSynchronizer); + syncGroups.put(name, syncGroup); + } + return syncGroup; + } + } + + @Override + public void restore(StorageEngine engine) throws Exception { + if (restoreConsumer == null) throw new IllegalStateException(); + + state.registerAndRestore(simpleCollector, restoreConsumer, engine); + } + + public Collection syncGroups() { + return syncGroups.values(); + } - KStreamSource stream = (KStreamSource)sourceStreams.get(record.topic()); + public void init(Consumer restoreConsumer, KStreamJob job) { + this.restoreConsumer = restoreConsumer; - if (stream == null) - throw new IllegalStateException("a stream for a topic not found: topic=" + topic); + job.build(this); - stream.receive(record.key(), record.value(), timestamp); + this.restoreConsumer = null; } - void punctuate(long timestamp) { + public void punctuate(long timestamp) { for (KStreamSource stream : sourceStreams.values()) { stream.punctuate(timestamp); } } - void flush() { + public void flush() { for (KStreamSource stream : sourceStreams.values()) { stream.flush(); } + state.flush(); + } + + public void close() throws Exception { + state.close(simpleCollector.offsets()); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 70776f047b77a..ed6247946670a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -20,10 +20,12 @@ protected KStreamImpl(PartitioningInfo partitioningInfo, KStreamContextImpl cont this.context = context; } + @Override public KStream filter(Predicate predicate) { return chain(new KStreamFilter(predicate, partitioningInfo, context)); } + @Override public KStream filterOut(final Predicate predicate) { return filter(new Predicate() { public boolean apply(K key, V value) { @@ -32,26 +34,32 @@ public boolean apply(K key, V value) { }); } + @Override public KStream map(KeyValueMapper mapper) { return chain(new KStreamMap(mapper, context)); } + @Override public KStream mapValues(ValueMapper mapper) { return chain(new KStreamMapValues(mapper, partitioningInfo, context)); } + @Override public KStream flatMap(KeyValueMapper, K, V> mapper) { return chain(new KStreamFlatMap(mapper, context)); } + @Override public KStream flatMapValues(ValueMapper, V> mapper) { return chain(new KStreamFlatMapValues(mapper, partitioningInfo, context)); } + @Override public KStreamWindowed with(Window window) { return (KStreamWindowed)chain(new KStreamWindowedImpl(window, partitioningInfo, context)); } + @Override public KStream nestedLoop(KStreamWindowed other, ValueJoiner processor) throws NotCopartitionedException { @@ -62,23 +70,26 @@ public KStream nestedLoop(KStreamWindowed other, ValueJoi return chain(new KStreamNestedLoop(otherImpl.window, processor, partitioningInfo, context)); } + @Override public KStream[] branch(Predicate... predicates) { KStreamBranch branch = new KStreamBranch(predicates, partitioningInfo, context); registerReceiver(branch); return branch.branches; } + @Override public KStream through(String topic) { process(this.getSendProcessor(topic)); return context.from(topic); } + @Override public void sendTo(String topic) { process(this.getSendProcessor(topic)); } private Processor getSendProcessor(final String topic) { - final RecordCollector collector = context.getRecordCollector(); + final RecordCollector collector = context.recordCollector(); return new Processor() { public void apply(K key, V value) { @@ -89,6 +100,7 @@ public void flush() {} }; } + @Override public void process(final Processor processor) { Receiver receiver = new Receiver() { public void receive(K key, V value, long timestamp) { @@ -104,6 +116,7 @@ public void flush() { registerReceiver(receiver); } + @Override public void punctuate(long timestamp) { if (timestamp != punctuatedAt) { punctuatedAt = timestamp; @@ -115,6 +128,7 @@ public void punctuate(long timestamp) { } } + @Override public void flush() { int numReceivers = nextReceivers.size(); for (int i = 0; i < numReceivers; i++) { diff --git a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java b/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java index d0eea0a127e70..c4d6817352092 100644 --- a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java +++ b/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java @@ -1,39 +1,26 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.CopartitioningGroup; +import io.confluent.streaming.SyncGroup; /** * Created by yasuhiro on 6/19/15. */ class PartitioningInfo { - public static PartitioningInfo missing = new PartitioningInfo(null, 0); + public static PartitioningInfo missing = new PartitioningInfo(null, -1); - private final CopartitioningGroup copartitioningGroup; - private final int numPartitions; + public final SyncGroup syncGroup; + public final int numPartitions; - PartitioningInfo(int numPartitions) { - this(null, numPartitions); - } - - PartitioningInfo(CopartitioningGroup copartitioningGroup) { - this(copartitioningGroup, copartitioningGroup.numPartitions); - } + PartitioningInfo(SyncGroup syncGroup, int numPartitions) { + if (syncGroup == null) throw new NullPointerException(); - private PartitioningInfo(CopartitioningGroup copartitioningGroup, int numPartitions) { - this.copartitioningGroup = copartitioningGroup; + this.syncGroup = syncGroup; this.numPartitions = numPartitions; } boolean isJoinCompatibleWith(PartitioningInfo other) { - if (copartitioningGroup != null) { - return copartitioningGroup == other.copartitioningGroup; - } - else { - return - (other.copartitioningGroup == null & - (numPartitions >= 0 || numPartitions == other.numPartitions)); - } + return syncGroup == other.syncGroup && numPartitions >= 0 && numPartitions == other.numPartitions; } } diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java b/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java new file mode 100644 index 0000000000000..4729fccb2ef04 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.confluent.streaming.internal; + +import io.confluent.streaming.StreamingConfig; +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.io.File; +import java.util.Properties; + +public class ProcessorConfig extends AbstractConfig { + + private static final ConfigDef CONFIG; + + static { + CONFIG = new ConfigDef().define(StreamingConfig.STATE_DIR_CONFIG, + Type.STRING, + System.getProperty("java.io.tmpdir"), + Importance.MEDIUM, + "") + .define(StreamingConfig.POLL_TIME_MS_CONFIG, + Type.LONG, + 100, + Importance.LOW, + "The amount of time to block waiting for input.") + .define(StreamingConfig.COMMIT_TIME_MS_CONFIG, + Type.LONG, + 30000, + Importance.HIGH, + "The frequency with which to save the position of the processor.") + .define(StreamingConfig.WINDOW_TIME_MS_CONFIG, + Type.LONG, + -1L, + Importance.MEDIUM, + "Setting this to a non-negative value will cause the processor to get called with this frequency even if there is no message.") + .define(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + 1000, + Importance.LOW, + "The maximum number of records to buffer per partition") + .define(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG, + Type.LONG, + 60000, + Importance.LOW, + "The amount of time to wait before deleting state when a partition has migrated.") + .define(StreamingConfig.TOTAL_RECORDS_TO_PROCESS, + Type.LONG, + -1L, + Importance.LOW, + "Exit after processing this many records."); + } + + public final File stateDir; + public final long pollTimeMs; + public final long commitTimeMs; + public final long windowTimeMs; + public final int bufferedRecordsPerPartition; + public final long stateCleanupDelay; + public final long totalRecordsToProcess; + + public ProcessorConfig(Properties processor) { + super(CONFIG, processor); + this.stateDir = new File(this.getString(StreamingConfig.STATE_DIR_CONFIG)); + this.pollTimeMs = this.getLong(StreamingConfig.POLL_TIME_MS_CONFIG); + this.commitTimeMs = this.getLong(StreamingConfig.COMMIT_TIME_MS_CONFIG); + this.windowTimeMs = this.getLong(StreamingConfig.WINDOW_TIME_MS_CONFIG); + this.bufferedRecordsPerPartition = this.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + this.stateCleanupDelay = this.getLong(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG); + this.totalRecordsToProcess = this.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContext.java b/src/main/java/io/confluent/streaming/internal/ProcessorContext.java new file mode 100644 index 0000000000000..342315d739082 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/ProcessorContext.java @@ -0,0 +1,82 @@ +/** + * 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 io.confluent.streaming.internal; + +import io.confluent.streaming.StorageEngine; +import io.confluent.streaming.StreamingConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; + +import java.io.File; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * A collection of values and helper methods available to the {@link StreamProcessor} during initialization. + */ +public class ProcessorContext { + + private final int id; + private final File stateDir; + private final StreamingConfig config; + private final Map stores; + private final Metrics metrics; + + public ProcessorContext(int id, + StreamingConfig config, + File stateDir, + Metrics metrics) { + this.id = id; + this.config = config; + this.stateDir = stateDir; + this.stores = new HashMap(); + this.metrics = metrics; + } + + /** + * The id of this {@link StreamProcessor} + */ + public int id() { + return this.id; + } + + /** + * The streaming config provided for this job + */ + public StreamingConfig config() { + return this.config; + } + + /** + * The directory allocated to this processor for maintaining any local state + */ + public File stateDir() { + return this.stateDir; + } + + /** + * Get the metrics associated with this processor instance + */ + public Metrics metrics() { + return metrics; + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java new file mode 100644 index 0000000000000..a2a0d114d4294 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -0,0 +1,128 @@ +/** + * 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 io.confluent.streaming.internal; + +import io.confluent.streaming.RecordCollector; +import io.confluent.streaming.StorageEngine; +import io.confluent.streaming.util.OffsetCheckpoint; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class ProcessorStateManager { + + private static final Logger log = LoggerFactory.getLogger(ProcessorStateManager.class); + private static final String CHECKPOINT_FILE_NAME = ".checkpoint"; + + private final int id; + private final File baseDir; + private final Map stores; + private final Map checkpointedOffsets; + private final Map restoredOffsets; + + public ProcessorStateManager(int id, File baseDir) { + this.id = id; + this.baseDir = baseDir; + this.stores = new HashMap(); + this.checkpointedOffsets = new HashMap(); + this.restoredOffsets = new HashMap(); + } + + public void init() throws IOException { + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + this.checkpointedOffsets.putAll(checkpoint.read()); + checkpoint.delete(); + } + + public void registerAndRestore(RecordCollector collector, Consumer consumer, StorageEngine engine) { + if (engine.name().equals(CHECKPOINT_FILE_NAME)) + throw new IllegalArgumentException("Illegal store name: " + CHECKPOINT_FILE_NAME); + + if(this.stores.containsKey(engine.name())) + throw new IllegalArgumentException("Store " + engine.name() + " has already been registered."); + + // register store + this.stores.put(engine.name(), engine); + + TopicPartition storePartition = new TopicPartition(engine.name(), id); + consumer.subscribe(storePartition); + + // calculate the end offset of the partition + consumer.seekToEnd(storePartition); + long partitionEndOffset = consumer.position(storePartition); + + // what was the last-written offset when we shutdown last? + long checkpointedOffset = 0; + if(checkpointedOffsets.containsKey(storePartition)) + checkpointedOffset = checkpointedOffsets.get(storePartition); + + // restore + consumer.subscribe(storePartition); + consumer.seekToBeginning(storePartition); + engine.registerAndRestore(collector, consumer, storePartition, checkpointedOffset, partitionEndOffset); + consumer.unsubscribe(storePartition); + restoredOffsets.put(storePartition, partitionEndOffset); + } + + 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 (StorageEngine engine : this.stores.values()) + engine.flush(); + } + } + + public void close(Map ackedOffsets) throws IOException { + if(!stores.isEmpty()) { + log.debug("Closing stores."); + for (Map.Entry entry : stores.entrySet()) { + log.debug("Closing storage engine {}", entry.getKey()); + entry.getValue().flush(); + entry.getValue().close(); + } + + Map checkpointOffsets = new HashMap(restoredOffsets); + for(String storeName: stores.keySet()) { + TopicPartition part = new TopicPartition(storeName, id); + if(ackedOffsets.containsKey(part)) + // store the last ack'd offset + 1 (the log position after restoration) + checkpointOffsets.put(part, ackedOffsets.get(part) + 1); + } + + // record that shutdown was clean + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + checkpoint.write(checkpointOffsets); + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 0fe8cf872bf28..f94d9ceb79afd 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -3,7 +3,7 @@ /** * Created by yasuhiro on 6/17/15. */ -interface Receiver { +public interface Receiver { void receive(K key, V value, long timestamp); diff --git a/src/main/java/io/confluent/streaming/internal/RecordCollectors.java b/src/main/java/io/confluent/streaming/internal/RecordCollectors.java new file mode 100644 index 0000000000000..b618d78247c0e --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/RecordCollectors.java @@ -0,0 +1,76 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.RecordCollector; +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; + +/** + * Created by yasuhiro on 6/26/15. + */ +public class RecordCollectors { + + private static final Logger log = LoggerFactory.getLogger(RecordCollectors.class); + + public static class SimpleRecordCollector implements RecordCollector { + + private final Producer producer; + private final Map offsets; + private final Callback callback = new Callback(){ + public void onCompletion(RecordMetadata metadata, Exception exception) { + if(exception == null) { + TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); + offsets.put(tp, metadata.offset()); + } else { + log.error("Error sending record: ", exception); + } + } + }; + + public SimpleRecordCollector(Producer producer) { + this.producer = producer; + this.offsets = new HashMap(); + } + + public void send(ProducerRecord record) { + // TODO: need to compute partition + this.producer.send(record, callback); + } + + /** + * The last ack'd offset from the producer + */ + public Map offsets() { + return this.offsets; + } + } + + public static class SerializingRecordCollector implements RecordCollector { + + private final RecordCollector collector; + private final Serializer keySerializer; + private final Serializer valueSerializer; + + public SerializingRecordCollector(RecordCollector collector, Serializer keySerializer, Serializer valueSerializer) { + super(); + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.collector = collector; + } + + public void send(ProducerRecord record) { + byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); + byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); + collector.send(new ProducerRecord(record.topic(), keyBytes, valBytes)); + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java b/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java index 0fe197ec03875..ccfda28cc4200 100644 --- a/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java +++ b/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java @@ -13,6 +13,7 @@ public class RegulatedConsumer { private final Consumer consumer; private final Set unpaused = new HashSet(); + private final Set toBePaused = new HashSet(); private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; private final long pollTimeMs; @@ -42,30 +43,23 @@ public void poll() { } public void poll(long timeoutMs) { + for (TopicPartition partition : toBePaused) { + doPause(partition); + } + toBePaused.clear(); + ConsumerRecords records = consumer.poll(timeoutMs); for (TopicPartition partition : unpaused) { - - final Iterator> inner = records.records(partition).iterator(); - Iterator> iterator = new Iterator>() { - public boolean hasNext() { - return inner.hasNext(); - } - public ConsumerRecord next() { - ConsumerRecord record = inner.next(); - K key = keyDeserializer.deserialize(record.topic(), record.key()); - V value = valueDeserializer.deserialize(record.topic(), record.value()); - return new ConsumerRecord(record.topic(), record.partition(), record.offset(), key, value); - } - public void remove() { - throw new UnsupportedOperationException(); - } - }; - streamSynchronizers.get(partition).add(partition, iterator); + streamSynchronizers.get(partition).addRecords(partition, new DeserializingIterator(records.records(partition).iterator())); } } public void pause(TopicPartition partition) { + toBePaused.add(partition); + } + + private void doPause(TopicPartition partition) { consumer.seek(partition, Long.MAX_VALUE); // hack: stop consuming from this partition by setting a big offset unpaused.remove(partition); } @@ -77,7 +71,31 @@ public void unpause(TopicPartition partition, long lastOffset) { public void clear() { unpaused.clear(); + toBePaused.clear(); streamSynchronizers.clear(); } + private class DeserializingIterator implements Iterator> { + + private final Iterator> inner; + + DeserializingIterator(Iterator> inner) { + this.inner = inner; + } + + public boolean hasNext() { + return inner.hasNext(); + } + + public ConsumerRecord next() { + ConsumerRecord record = inner.next(); + K key = keyDeserializer.deserialize(record.topic(), record.key()); + V value = valueDeserializer.deserialize(record.topic(), record.value()); + return new ConsumerRecord(record.topic(), record.partition(), record.offset(), key, value); + } + public void remove() { + throw new UnsupportedOperationException(); + } + } + } diff --git a/src/main/java/io/confluent/streaming/internal/Test.java b/src/main/java/io/confluent/streaming/internal/Test.java deleted file mode 100644 index b67c925175b8a..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/Test.java +++ /dev/null @@ -1,140 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.*; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.producer.ProducerRecord; - -import java.util.ArrayList; - -/** - * Created by yasuhiro on 6/17/15. - */ -public class Test { - - public static void main(String[] args) throws NotCopartitionedException { - - RecordCollector collector = new RecordCollector() { - public void send(ProducerRecord record) { - System.out.println(record.toString()); - } - }; - - KStreamContextImpl context = new KStreamContextImpl(collector); - - Predicate lengthCheck = new Predicate() { - public boolean apply(Integer i, String s) { - return s.length() > 1; - } - }; - - KeyValueMapper truncate = new KeyValueMapper() { - public KeyValue apply(Integer i, String s) { - if (i < s.length()) - return KeyValue.pair(i * 10, s.substring(0, i)); - else - return KeyValue.pair(i * 10, s); - } - public void flush() { - System.out.println("flushed truncate"); - } - }; - - KeyValueMapper, Integer, String> toCharacters = new KeyValueMapper, Integer, String>() { - public KeyValue> apply(Integer i, String s) { - ArrayList list = new ArrayList(s.length()); - for (char c : s.toCharArray()) { - list.add(c); - } - return KeyValue.pair(i, list); - } - - public void flush() { - System.out.println("flushed toCharacters"); - } - }; - - ValueMapper, String> toCharacters2 = new ValueMapper, String>() { - public Iterable apply(String s) { - ArrayList list = new ArrayList(s.length()); - for (char c : s.toCharArray()) { - list.add(c); - } - return list; - } - - public void flush() { - System.out.println("flushed toCharacters2"); - } - }; - - Processor printCharWithKey = new Processor() { - public void apply(Integer i, Character c) { - System.out.println(i + ":" + c); - } - public void punctuate(long time) {} - public void flush() { - System.out.println("flushed printCharWithKey"); - } - }; - - KStream stream = context.from("topicABC"); - - stream.filter(lengthCheck).map(truncate).flatMap(toCharacters).process(printCharWithKey); - - context.process(new ConsumerRecord("topicABC", 1, 0, 1, "s"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topicABC", 1, 1, 2, "abcd"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topicABC", 1, 2, 2, "t"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topicABC", 1, 3, 3, "abcd"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topicABC", 1, 4, 3, "u"), System.currentTimeMillis()); - - stream = context.from("topicXYZ"); - stream.filter(lengthCheck).map(truncate).flatMapValues(toCharacters2).process(printCharWithKey); - - context.process(new ConsumerRecord("topicXYZ", 1, 0, 1, "h"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topicXYZ", 1, 1, 2, "xyz"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topicXYZ", 1, 2, 3, "i"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topicXYZ", 1, 3, 4, "stu"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topicXYZ", 1, 4, 5, "j"), System.currentTimeMillis()); - - KStream stream1 = context.from("topic1"); - KStream stream2 = context.from("topic2"); - - stream1.with(new WindowByCount(3)).join(stream2.with(new WindowByCount(3)), - new ValueJoiner() { - public String apply(String v1, String v2) { - return v1 + v2; - } - } - ).process(new Processor() { - public void apply(Integer key, String value) { - System.out.println(key + ":" + value); - } - - public void punctuate(long timestamp) { - } - - public void flush() { - } - }); - - context.process(new ConsumerRecord("topic1", 1, 0, 1, "x"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topic2", 1, 0, 2, "A"), System.currentTimeMillis()); - - context.process(new ConsumerRecord("topic1", 1, 1, 2, "y"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topic2", 1, 1, 2, "B"), System.currentTimeMillis()); - - context.process(new ConsumerRecord("topic1", 1, 2, 3, "z"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topic2", 1, 2, 2, "C"), System.currentTimeMillis()); - - context.process(new ConsumerRecord("topic1", 1, 3, 4, "x"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topic2", 1, 3, 2, "D"), System.currentTimeMillis()); - - context.process(new ConsumerRecord("topic1", 1, 4, 5, "y"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topic2", 1, 4, 2, "E"), System.currentTimeMillis()); - - context.process(new ConsumerRecord("topic1", 1, 5, 6, "x"), System.currentTimeMillis()); - context.process(new ConsumerRecord("topic2", 1, 5, 2, "F"), System.currentTimeMillis()); - - } -} - diff --git a/src/main/java/io/confluent/streaming/kv/Entry.java b/src/main/java/io/confluent/streaming/kv/Entry.java new file mode 100644 index 0000000000000..53cd874d86ca3 --- /dev/null +++ b/src/main/java/io/confluent/streaming/kv/Entry.java @@ -0,0 +1,28 @@ +package io.confluent.streaming.kv; + +/** + * Created by yasuhiro on 6/26/15. + */ +public class Entry { + + private final K key; + private final V value; + + public Entry(K key, V value) { + this.key = key; + this.value = value; + } + + public K key() { + return key; + } + + public V value() { + return value; + } + + public String toString() { + return "Entry(" + key() + ", " + value() + ")"; + } + +} diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java new file mode 100644 index 0000000000000..f3a666ee8b877 --- /dev/null +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -0,0 +1,162 @@ +package io.confluent.streaming.kv; + +import io.confluent.streaming.RecordCollector; +import io.confluent.streaming.StorageEngine; +import io.confluent.streaming.internal.ProcessorContext; +import io.confluent.streaming.kv.internals.MeteredKeyValueStore; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.SystemTime; + +import java.util.*; + +/** + * An in-memory key-value store based on a TreeMap + * + * @param The key type + * @param The value type + */ +public class InMemoryKeyValueStore extends MeteredKeyValueStore implements KeyValueStore, StorageEngine { + + public InMemoryKeyValueStore(String name, ProcessorContext context) { + super(name, "kafka-streams", new MemoryStore(name, context), context.metrics(), new SystemTime()); + } + + private static class MemoryStore implements KeyValueStore, StorageEngine { + + private final String topic; + private final int partition; + private final NavigableMap store; + private final Set dirty; + private final int maxDirty; + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + private RecordCollector collector; + + public MemoryStore(String name, ProcessorContext context) { + this.topic = name; + this.partition = context.id(); + this.store = new TreeMap(); + this.dirty = new HashSet(); + this.collector = null; + this.maxDirty = 100; + this.keySerializer = (Serializer) context.config().keySerializer(); + this.valueSerializer = (Serializer) context.config().valueSerializer(); + this.keyDeserializer = (Deserializer) context.config().keyDeserializer(); + this.valueDeserializer = (Deserializer) context.config().valueDeserializer(); + } + + @Override + public String name() { + return this.topic; + } + + @Override + public V get(K key) { + return this.store.get(key); + } + + @Override + public void put(K key, V value) { + this.store.put(key, value); + if(this.collector != null) { + this.dirty.add(key); + if (this.dirty.size() > this.maxDirty) + flush(); + } + } + + @Override + public void putAll(List> entries) { + for (Entry entry : entries) + put(entry.key(), entry.value()); + } + + @Override + public void delete(K key) { + put(key, null); + } + + @Override + public KeyValueIterator range(K from, K to) { + return new MemoryStoreIterator(this.store.subMap(from, true, to, false).entrySet().iterator()); + } + + @Override + public KeyValueIterator all() { + return new MemoryStoreIterator(this.store.entrySet().iterator()); + } + + @Override + public void registerAndRestore(RecordCollector collector, + Consumer consumer, + TopicPartition partition, + long checkpointedOffset, + long endOffset) { + this.collector = collector; + while (true) { + for(ConsumerRecord record: consumer.poll(100)) + this.store.put(keyDeserializer.deserialize(partition.topic(), record.key()), + valueDeserializer.deserialize(partition.topic(), record.value())); + long position = consumer.position(partition); + if (position == endOffset) + break; + else if(position > endOffset) + throw new IllegalStateException("This should not happen."); + } + } + + @Override + public void flush() { + if(this.collector != null) { + for (K k : this.dirty) { + V v = this.store.get(k); + byte[] key = this.keySerializer.serialize(this.topic, k); + byte[] value = this.valueSerializer.serialize(this.topic, v); + this.collector.send(new ProducerRecord(this.topic, this.partition, key, value)); + } + this.dirty.clear(); + } + } + + @Override + public void close() { + flush(); + } + + private static class MemoryStoreIterator implements KeyValueIterator { + private final Iterator> iter; + + public MemoryStoreIterator(Iterator> iter) { + this.iter = iter; + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public Entry next() { + Map.Entry entry = iter.next(); + return new Entry(entry.getKey(), entry.getValue()); + } + + @Override + public void remove() { + iter.remove(); + } + + @Override + public void close() {} + + } + } + +} diff --git a/src/main/java/io/confluent/streaming/kv/KeyValueIterator.java b/src/main/java/io/confluent/streaming/kv/KeyValueIterator.java new file mode 100644 index 0000000000000..9b2e1795e2f75 --- /dev/null +++ b/src/main/java/io/confluent/streaming/kv/KeyValueIterator.java @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.confluent.streaming.kv; + + +import java.io.Closeable; +import java.util.Iterator; + +public interface KeyValueIterator extends Iterator>, Closeable { + + @Override + public void close(); +} diff --git a/src/main/java/io/confluent/streaming/kv/KeyValueStore.java b/src/main/java/io/confluent/streaming/kv/KeyValueStore.java new file mode 100644 index 0000000000000..1d03aea1f1304 --- /dev/null +++ b/src/main/java/io/confluent/streaming/kv/KeyValueStore.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.confluent.streaming.kv; + +import io.confluent.streaming.StorageEngine; + +import java.util.List; + +/** + * A key-value store that supports put/get/delete and range queries. + * + * @param The key type + * @param The value type + */ +public interface KeyValueStore extends StorageEngine { + + /** + * 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. + */ + 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. + */ + 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. + */ + public void putAll(List> entries); + + /** + * Delete the value from the store (if there is one) + * + * @param key The key + * @throws NullPointerException If null is used for key. + */ + public void 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. + */ + public KeyValueIterator range(K from, K to); + + /** + * Return an iterator over all keys in the database. This iterator MUST be closed after use. + * + * @return An iterator of all key/value pairs in the store. + */ + public KeyValueIterator all(); + +} diff --git a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java new file mode 100644 index 0000000000000..e1543b0803d80 --- /dev/null +++ b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java @@ -0,0 +1,194 @@ +package io.confluent.streaming.kv.internals; + +import io.confluent.streaming.RecordCollector; +import io.confluent.streaming.kv.Entry; +import io.confluent.streaming.kv.KeyValueIterator; +import io.confluent.streaming.kv.KeyValueStore; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.TopicPartition; +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.utils.Time; + +import java.util.List; + +public class MeteredKeyValueStore implements KeyValueStore { + + protected final KeyValueStore inner; + private final Time time; + private final Metrics metrics; + 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; + + public MeteredKeyValueStore(String name, String group, KeyValueStore inner, Metrics metrics, Time time) { + this.inner = inner; + this.time = time; + this.metrics = metrics; + this.group = group; + 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"); + } + + 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 void registerAndRestore(RecordCollector collector, + Consumer consumer, + TopicPartition partition, + long checkpointedOffset, + long logEndOffset) { + long startNs = time.nanoseconds(); + try { + inner.registerAndRestore(collector, consumer, partition, checkpointedOffset, logEndOffset); + } finally { + recordLatency(this.restoreTime, startNs, time.nanoseconds()); + } + } + + @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); + } finally { + recordLatency(this.putTime, startNs, time.nanoseconds()); + } + } + + @Override + public void putAll(List> entries) { + long startNs = time.nanoseconds(); + try { + this.inner.putAll(entries); + } finally { + recordLatency(this.putAllTime, startNs, time.nanoseconds()); + } + } + + @Override + public void delete(K key) { + long startNs = time.nanoseconds(); + try { + this.inner.delete(key); + } finally { + recordLatency(this.deleteTime, startNs, time.nanoseconds()); + } + } + + @Override + public KeyValueIterator range(K from, K to) { + return new MeteredKeyValueIterator(this.inner.range(from, to), this.rangeTime); + } + + @Override + public KeyValueIterator all() { + return new MeteredKeyValueIterator(this.inner.all(), this.allTime); + } + + @Override + public void close() {} + + @Override + public void flush() { + long startNs = time.nanoseconds(); + try { + this.inner.flush(); + } finally { + recordLatency(this.flushTime, startNs, time.nanoseconds()); + } + } + + private void recordLatency(Sensor sensor, long startNs, long endNs) { + sensor.record((endNs - startNs) / 1000000, endNs); + } + + private class MeteredKeyValueIterator implements KeyValueIterator { + + private final KeyValueIterator iter; + private final Sensor sensor; + private final long startNs; + + public MeteredKeyValueIterator(KeyValueIterator iter, Sensor sensor) { + this.iter = iter; + this.sensor = sensor; + this.startNs = time.nanoseconds(); + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public Entry next() { + return iter.next(); + } + + @Override + public void remove() { + iter.remove(); + } + + @Override + public void close() { + try { + iter.close(); + } finally { + recordLatency(this.sensor, this.startNs, time.nanoseconds()); + } + } + + } + +} diff --git a/src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java b/src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java new file mode 100644 index 0000000000000..a2d29b89c7553 --- /dev/null +++ b/src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java @@ -0,0 +1,162 @@ +/** + * 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 io.confluent.streaming.util; + +import org.apache.kafka.common.TopicPartition; + +import java.io.*; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** + * This class saves out a map of topic/partition=>offsets to a file. The format of the file is UTF-8 text containing the following: + * + * + * + * . + * . + * . + * + * + * The first line contains a number designating the format version (currently 0), the next line contains + * a number giving the total number of offsets. Each successive line gives a topic/partition/offset triple + * separated by spaces. + */ +public class OffsetCheckpoint { + + private static final int VERSION = 0; + + private final File file; + private final Object lock; + + public OffsetCheckpoint(File file) throws IOException { + new File(file + ".tmp").delete(); // try to delete any existing temp files for cleanliness + this.file = file; + this.lock = new Object(); + } + + public void write(Map offsets) throws IOException { + synchronized (lock) { + // write to temp file and then swap with the existing file + File temp = new File(file.getAbsolutePath() + ".tmp"); + + FileOutputStream fileOutputStream = new FileOutputStream(temp); + BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream)); + try { + writeIntLine(writer, VERSION); + writeIntLine(writer, offsets.size()); + + // write the entries + for (Map.Entry entry : offsets.entrySet()) + writeEntry(writer, entry.getKey(), entry.getValue()); + + // flush the buffer and then fsync the underlying file + writer.flush(); + fileOutputStream.getFD().sync(); + } finally { + writer.close(); + } + + // swap new offset checkpoint file with previous one + if (!temp.renameTo(file)) { + // renameTo() fails on Windows if the destination file exists. + file.delete(); + if (!temp.renameTo(file)) + throw new IOException(String.format("File rename from %s to %s failed.", + temp.getAbsolutePath(), + file.getAbsolutePath())); + } + } + } + + private void writeIntLine(BufferedWriter writer, int number) throws IOException { + writer.write(Integer.toString(number)); + writer.newLine(); + } + + private void writeEntry(BufferedWriter writer, TopicPartition part, long offset) throws IOException { + writer.write(part.topic()); + writer.write(' '); + writer.write(Integer.toString(part.partition())); + writer.write(' '); + writer.write(Long.toString(offset)); + writer.newLine(); + } + + public Map read() throws IOException { + synchronized (lock) { + BufferedReader reader = null; + try { + reader = new BufferedReader(new FileReader(file)); + } catch(FileNotFoundException e) { + return Collections.emptyMap(); + } + + try { + int version = readInt(reader); + switch (version) { + case 0: + int expectedSize = readInt(reader); + Map offsets = new HashMap(); + String line = reader.readLine(); + while (line != null) { + String[] pieces = line.split("\\s+"); + if (pieces.length != 3) + throw new IOException(String.format("Malformed line in offset checkpoint file: '%s'.", + line)); + + String topic = pieces[0]; + int partition = Integer.parseInt(pieces[1]); + long offset = Long.parseLong(pieces[2]); + offsets.put(new TopicPartition(topic, partition), offset); + line = reader.readLine(); + } + if (offsets.size() != expectedSize) + throw new IOException(String.format("Expected %d entries but found only %d", + expectedSize, + offsets.size())); + return offsets; + + default: + throw new IllegalArgumentException("Unknown offset checkpoint version: " + version); + } + } finally { + if(reader != null) + reader.close(); + } + } + } + + private int readInt(BufferedReader reader) throws IOException { + String line = reader.readLine(); + if (line == null) + throw new EOFException("File ended prematurely."); + int val = Integer.parseInt(line); + return val; + } + + public void delete() throws IOException { + file.delete(); + } + + @Override + public String toString() { + return this.file.getAbsolutePath(); + } + +} diff --git a/src/main/java/io/confluent/streaming/util/Util.java b/src/main/java/io/confluent/streaming/util/Util.java index 3fff745a5d1ee..734785a5ca2e0 100644 --- a/src/main/java/io/confluent/streaming/util/Util.java +++ b/src/main/java/io/confluent/streaming/util/Util.java @@ -1,5 +1,6 @@ package io.confluent.streaming.util; +import java.io.File; import java.util.HashSet; /** @@ -13,4 +14,23 @@ static HashSet mkSet(T... elems) { return set; } + /** + * 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 rm(File file) { + if (file == null) { + return; + } else if (file.isDirectory()) { + File[] files = file.listFiles(); + if (files != null) { + for (File f : files) + rm(f); + } + file.delete(); + } else { + file.delete(); + } + } } From c336a4cde15c7d57d00ea83399669b56d6b3d515 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 29 Jun 2015 13:52:49 -0700 Subject: [PATCH 004/275] removed punctuate method, added punctuationqueue per streamsynchronizer --- .../java/io/confluent/streaming/KStream.java | 2 +- .../confluent/streaming/KStreamContext.java | 8 +- .../confluent/streaming/KafkaStreaming.java | 14 ---- .../io/confluent/streaming/Processor.java | 4 +- .../streaming/PunctuationScheduler.java | 14 ++++ .../streaming/StreamSynchronizer.java | 27 +++--- .../streaming/internal/KStreamBranch.java | 6 -- .../internal/KStreamContextImpl.java | 26 ++++-- .../streaming/internal/KStreamImpl.java | 34 ++++---- .../streaming/internal/KStreamJoin.java | 4 - .../internal/KStreamWindowedImpl.java | 5 -- .../streaming/internal/ProcessorContext.java | 82 ------------------- .../streaming/internal/PunctuationQueue.java | 49 +++++++++++ .../internal/PunctuationSchedulerImpl.java | 39 +++++++++ .../streaming/internal/Receiver.java | 2 - .../streaming/kv/InMemoryKeyValueStore.java | 14 ++-- 16 files changed, 168 insertions(+), 162 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/PunctuationScheduler.java delete mode 100644 src/main/java/io/confluent/streaming/internal/ProcessorContext.java create mode 100644 src/main/java/io/confluent/streaming/internal/PunctuationQueue.java create mode 100644 src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index bf9f7c2fe83b8..0bf0ea316fe8c 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -22,7 +22,7 @@ public interface KStream { KStream nestedLoop(KStreamWindowed other, ValueJoiner processor) throws NotCopartitionedException; - KStream[] branch(Predicate... predicates); + KStream[] branch(Predicate... predicates); KStream through(String topic); diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index fd1b514cf9ef7..a656bb726de82 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -10,7 +10,11 @@ */ public interface KStreamContext { - static final String DEFAULT_SYNCHRONIZATION_GROUP = "defaultSynchronizationGroup"; + String DEFAULT_SYNCHRONIZATION_GROUP = "defaultSynchronizationGroup"; + + int id(); + + StreamingConfig streamingConfig(); KStream from(String topic); @@ -31,4 +35,6 @@ public interface KStreamContext { SyncGroup syncGroup(String name); void restore(StorageEngine engine) throws Exception; + + void schedule(Processor processor, long time); } diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 186ecba6fc826..d624049506465 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -213,7 +213,6 @@ private void runLoop() { syncGroup.streamSynchronizer.process(); } } - maybeWindow(); maybeCommit(); maybeCleanState(); } @@ -234,19 +233,6 @@ private boolean stillRunning() { return true; } - private void maybeWindow() throws Exception { - long now = time.milliseconds(); - if (this.config.windowTimeMs >= 0 && lastWindow + this.config.windowTimeMs < now) { - log.trace("Windowing all stream processors"); - this.lastWindow = now; - for (KStreamContextImpl context : kstreamContexts.values()) { - context.punctuate(now); - // check co-ordinator - } - this.streamingMetrics.windowTime.record(time.milliseconds() - now); - } - } - private void maybeCommit() { long now = time.milliseconds(); if (this.config.commitTimeMs >= 0 && lastCommit + this.config.commitTimeMs < time.milliseconds()) { diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 373d86ca2413f..3d51b1a5baf21 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -7,7 +7,9 @@ public interface Processor { void apply(K key, V value); - void punctuate(long timestamp); + void init(PunctuationScheduler punctuationScheduler); + + void punctuate(); void flush(); diff --git a/src/main/java/io/confluent/streaming/PunctuationScheduler.java b/src/main/java/io/confluent/streaming/PunctuationScheduler.java new file mode 100644 index 0000000000000..cec3fd312f775 --- /dev/null +++ b/src/main/java/io/confluent/streaming/PunctuationScheduler.java @@ -0,0 +1,14 @@ +package io.confluent.streaming; + +import io.confluent.streaming.internal.PunctuationQueue; + +/** + * Created by yasuhiro on 6/29/15. + */ +public interface PunctuationScheduler { + + void schedule(long time); + + void cancel(); + +} diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/StreamSynchronizer.java index d4424f8b78b2c..0fca1064487d2 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizer.java @@ -1,8 +1,6 @@ package io.confluent.streaming; -import io.confluent.streaming.internal.RecordQueueImpl; -import io.confluent.streaming.internal.RegulatedConsumer; -import io.confluent.streaming.internal.Receiver; +import io.confluent.streaming.internal.*; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; @@ -22,6 +20,7 @@ public class StreamSynchronizer { private final Map stash = new HashMap(); private final int desiredUnprocessed; private final Map consumedOffsets; + private final PunctuationQueue punctuationQueue = new PunctuationQueue(); private long streamTime = -1; private volatile int buffered = 0; @@ -80,29 +79,35 @@ public void addRecords(TopicPartition partition, Iterator> } } + public PunctuationScheduler getPunctuationScheduler(Processor processor) { + return new PunctuationSchedulerImpl(punctuationQueue, processor); + } + public void process() { synchronized (this) { - RecordQueueWrapper queue = (RecordQueueWrapper)chooser.next(); + RecordQueueWrapper recordQueue = (RecordQueueWrapper)chooser.next(); - if (queue == null) { + if (recordQueue == null) { consumer.poll(); return; } - if (queue.size() == this.desiredUnprocessed) { - ConsumerRecord record = queue.peekLast(); + if (recordQueue.size() == this.desiredUnprocessed) { + ConsumerRecord record = recordQueue.peekLast(); if (record != null) { - consumer.unpause(queue.partition(), record.offset()); + consumer.unpause(recordQueue.partition(), record.offset()); } } - if (queue.size() == 0) return; + if (recordQueue.size() == 0) return; - queue.process(); + recordQueue.process(); - if (queue.size() > 0) chooser.add(queue); + if (recordQueue.size() > 0) chooser.add(recordQueue); buffered--; + + punctuationQueue.mayPunctuate(streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 1755eec768441..d26c144e767f2 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -34,12 +34,6 @@ public void receive(K key, V value, long timestamp) { } } - public void punctuate(long timestamp) { - for (KStreamSource branch : branches) { - branch.punctuate(timestamp); - } - } - public void flush() { for (KStreamSource branch : branches) { branch.flush(); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 60ee738afff0d..31055cb1dc682 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -38,11 +38,11 @@ public class KStreamContextImpl implements KStreamContext { private final ProcessorConfig processorConfig; private final Metrics metrics; private final File stateDir; - private final ProcessorContext processorContext; private final ProcessorStateManager state; private final Map stores = new HashMap(); private Consumer restoreConsumer; + @SuppressWarnings("unchecked") public KStreamContextImpl(int id, RegulatedConsumer regulatedConsumer, Producer producer, @@ -68,17 +68,27 @@ public KStreamContextImpl(int id, this.stateDir = stateDir; this.state = new ProcessorStateManager(id, stateDir); - this.processorContext = new ProcessorContext(id, streamingConfig, stateDir, metrics); this.metrics = metrics; } + @Override + public int id() { + return id; + } + + @Override + public StreamingConfig streamingConfig() { + return streamingConfig; + } + @Override public KStream from(String topic) { return from(topic, syncGroup(DEFAULT_SYNCHRONIZATION_GROUP)); } @Override + @SuppressWarnings("unchecked") public KStream from(String topic, SyncGroup syncGroup) { if (syncGroup == null) throw new NullPointerException(); @@ -164,6 +174,12 @@ public void restore(StorageEngine engine) throws Exception { state.registerAndRestore(simpleCollector, restoreConsumer, engine); } + + @Override + public void schedule(Processor processor, long time) { + + } + public Collection syncGroups() { return syncGroups.values(); } @@ -176,12 +192,6 @@ public void init(Consumer restoreConsumer, KStreamJob job) { this.restoreConsumer = null; } - public void punctuate(long timestamp) { - for (KStreamSource stream : sourceStreams.values()) { - stream.punctuate(timestamp); - } - } - public void flush() { for (KStreamSource stream : sourceStreams.values()) { stream.flush(); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index ed6247946670a..be3f25fdf9964 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -11,7 +11,6 @@ abstract class KStreamImpl implements KStream, Receiver { private final ArrayList> nextReceivers = new ArrayList>(1); - private long punctuatedAt = 0L; final PartitioningInfo partitioningInfo; final KStreamContextImpl context; @@ -34,6 +33,7 @@ public boolean apply(K key, V value) { }); } + @Override public KStream map(KeyValueMapper mapper) { return chain(new KStreamMap(mapper, context)); @@ -56,7 +56,7 @@ public KStream flatMapValues(ValueMapper, V> @Override public KStreamWindowed with(Window window) { - return (KStreamWindowed)chain(new KStreamWindowedImpl(window, partitioningInfo, context)); + return (KStreamWindowed)chain(new KStreamWindowedImpl(window, partitioningInfo, context)); } @Override @@ -71,8 +71,8 @@ public KStream nestedLoop(KStreamWindowed other, ValueJoi } @Override - public KStream[] branch(Predicate... predicates) { - KStreamBranch branch = new KStreamBranch(predicates, partitioningInfo, context); + public KStream[] branch(Predicate... predicates) { + KStreamBranch branch = new KStreamBranch(predicates, partitioningInfo, context); registerReceiver(branch); return branch.branches; } @@ -88,14 +88,20 @@ public void sendTo(String topic) { process(this.getSendProcessor(topic)); } + @SuppressWarnings("unchecked") private Processor getSendProcessor(final String topic) { - final RecordCollector collector = context.recordCollector(); + final RecordCollector collector = (RecordCollector) context.recordCollector(); return new Processor() { + @Override public void apply(K key, V value) { collector.send(new ProducerRecord(topic, key, value)); } - public void punctuate(long timestamp) {} + @Override + public void init(PunctuationScheduler scheduler) {} + @Override + public void punctuate() {} + @Override public void flush() {} }; } @@ -106,26 +112,14 @@ public void process(final Processor processor) { public void receive(K key, V value, long timestamp) { processor.apply(key, value); } - public void punctuate(long timestamp) { - processor.punctuate(timestamp); - } public void flush() { processor.flush(); } }; registerReceiver(receiver); - } - @Override - public void punctuate(long timestamp) { - if (timestamp != punctuatedAt) { - punctuatedAt = timestamp; - - int numReceivers = nextReceivers.size(); - for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).punctuate(timestamp); - } - } + PunctuationScheduler scheduler = partitioningInfo.syncGroup.streamSynchronizer.getPunctuationScheduler(processor); + processor.init(scheduler); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index ae671064c9693..ba1c4ea69daab 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -57,10 +57,6 @@ public void receive(K key, V2 value2, long timestamp) { } } - public void punctuate(long timestamp) { - KStreamJoin.this.punctuate(timestamp); - } - public void flush() { KStreamJoin.this.flush(); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index cfb17f2ce916d..cfc350265f6c5 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -21,11 +21,6 @@ public void receive(K key, V value, long timestamp) { } } - public void punctuate(long timestamp) { - window.punctuate(timestamp); - super.punctuate(timestamp); - } - public void flush() { window.flush(); super.flush(); diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContext.java b/src/main/java/io/confluent/streaming/internal/ProcessorContext.java deleted file mode 100644 index 342315d739082..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/ProcessorContext.java +++ /dev/null @@ -1,82 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.confluent.streaming.internal; - -import io.confluent.streaming.StorageEngine; -import io.confluent.streaming.StreamingConfig; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.metrics.Metrics; - -import java.io.File; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -/** - * A collection of values and helper methods available to the {@link StreamProcessor} during initialization. - */ -public class ProcessorContext { - - private final int id; - private final File stateDir; - private final StreamingConfig config; - private final Map stores; - private final Metrics metrics; - - public ProcessorContext(int id, - StreamingConfig config, - File stateDir, - Metrics metrics) { - this.id = id; - this.config = config; - this.stateDir = stateDir; - this.stores = new HashMap(); - this.metrics = metrics; - } - - /** - * The id of this {@link StreamProcessor} - */ - public int id() { - return this.id; - } - - /** - * The streaming config provided for this job - */ - public StreamingConfig config() { - return this.config; - } - - /** - * The directory allocated to this processor for maintaining any local state - */ - public File stateDir() { - return this.stateDir; - } - - /** - * Get the metrics associated with this processor instance - */ - public Metrics metrics() { - return metrics; - } - -} diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java b/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java new file mode 100644 index 0000000000000..a11ad3ba06ae8 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java @@ -0,0 +1,49 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.util.Stamped; + +import java.util.PriorityQueue; + +/** + * Created by yasuhiro on 6/29/15. + */ +public class PunctuationQueue { + + private PriorityQueue> pq = new PriorityQueue>(); + + public Stamped schedule(PunctuationSchedulerImpl scheduler, long time) { + synchronized (pq) { + Stamped stamped = new Stamped(scheduler, time); + pq.add(stamped); + return stamped; + } + } + + public void cancel(Stamped stamped) { + synchronized (pq) { + pq.remove(stamped); + } + + } + + public void close() { + synchronized (pq) { + pq.clear(); + } + } + + public void mayPunctuate(long streamTime) { + synchronized (pq) { + Stamped top = pq.peek(); + while (top.timestamp <= streamTime) { + PunctuationSchedulerImpl scheduler = top.value; + pq.poll(); + scheduler.processor.punctuate(); + scheduler.processed(); + + top = pq.peek(); + } + } + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java b/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java new file mode 100644 index 0000000000000..f47f7f32f9fff --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java @@ -0,0 +1,39 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.Processor; +import io.confluent.streaming.PunctuationScheduler; +import io.confluent.streaming.util.Stamped; + +/** + * Created by yasuhiro on 6/29/15. + */ +public class PunctuationSchedulerImpl implements PunctuationScheduler { + + private Stamped scheduled; + private final PunctuationQueue queue; + final Processor processor; + + public PunctuationSchedulerImpl(PunctuationQueue queue, Processor processor) { + this.queue = queue; + this.processor = processor; + } + + @Override + public void schedule(long time) { + if (scheduled != null) + throw new IllegalStateException("punctuation is already scheduled"); + + scheduled = queue.schedule(this, time); + } + + @Override + public void cancel() { + queue.cancel(scheduled); + scheduled = null; + } + + public void processed() { + scheduled = null; + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index f94d9ceb79afd..6ad7ff2f3efb1 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -7,8 +7,6 @@ public interface Receiver { void receive(K key, V value, long timestamp); - void punctuate(long timestamp); - void flush(); } diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index f3a666ee8b877..90529d28eaa05 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -1,8 +1,8 @@ package io.confluent.streaming.kv; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StorageEngine; -import io.confluent.streaming.internal.ProcessorContext; import io.confluent.streaming.kv.internals.MeteredKeyValueStore; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -22,7 +22,7 @@ */ public class InMemoryKeyValueStore extends MeteredKeyValueStore implements KeyValueStore, StorageEngine { - public InMemoryKeyValueStore(String name, ProcessorContext context) { + public InMemoryKeyValueStore(String name, KStreamContext context) { super(name, "kafka-streams", new MemoryStore(name, context), context.metrics(), new SystemTime()); } @@ -39,17 +39,17 @@ private static class MemoryStore implements KeyValueStore, StorageEn private final Deserializer valueDeserializer; private RecordCollector collector; - public MemoryStore(String name, ProcessorContext context) { + public MemoryStore(String name, KStreamContext context) { this.topic = name; this.partition = context.id(); this.store = new TreeMap(); this.dirty = new HashSet(); this.collector = null; this.maxDirty = 100; - this.keySerializer = (Serializer) context.config().keySerializer(); - this.valueSerializer = (Serializer) context.config().valueSerializer(); - this.keyDeserializer = (Deserializer) context.config().keyDeserializer(); - this.valueDeserializer = (Deserializer) context.config().valueDeserializer(); + this.keySerializer = (Serializer) context.streamingConfig().keySerializer(); + this.valueSerializer = (Serializer) context.streamingConfig().valueSerializer(); + this.keyDeserializer = (Deserializer) context.streamingConfig().keyDeserializer(); + this.valueDeserializer = (Deserializer) context.streamingConfig().valueDeserializer(); } @Override From fb3454abe095ae68c2329ec11d4031db59824fe1 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 29 Jun 2015 14:39:13 -0700 Subject: [PATCH 005/275] fixed compile warnings --- .../confluent/streaming/KafkaStreaming.java | 7 +-- .../streaming/StreamSynchronizer.java | 4 +- .../java/io/confluent/streaming/Window.java | 2 - .../io/confluent/streaming/WindowByCount.java | 57 ------------------- .../io/confluent/streaming/WindowByTime.java | 25 ++++---- .../streaming/internal/KStreamBranch.java | 4 +- .../streaming/internal/RecordCollectors.java | 4 +- .../streaming/internal/RegulatedConsumer.java | 3 +- .../streaming/kv/InMemoryKeyValueStore.java | 1 + .../streaming/util/FilteredIterator.java | 48 ++++++++++++++++ .../util/QueueWithMinTimestampTracking.java | 4 +- 11 files changed, 73 insertions(+), 86 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/WindowByCount.java create mode 100644 src/main/java/io/confluent/streaming/util/FilteredIterator.java diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index d624049506465..c24178b7d2dbe 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -84,8 +84,8 @@ public class KafkaStreaming implements Runnable { private static final Logger log = LoggerFactory.getLogger(KafkaStreaming.class); private final KStreamJob job; - private final Map> syncGroups = new HashMap(); - private final Map kstreamContexts = new HashMap(); + private final Map> syncGroups = new HashMap>(); + private final Map kstreamContexts = new HashMap(); protected final Producer producer; protected final Consumer consumer; private final RegulatedConsumer regulatedConsumer; @@ -99,7 +99,6 @@ public class KafkaStreaming implements Runnable { private volatile boolean running; private CountDownLatch shutdownComplete = new CountDownLatch(1); private long lastCommit; - private long lastWindow; private long nextStateCleaning; private long recordsProcessed; @@ -119,6 +118,7 @@ public KafkaStreaming(KStreamJob job, StreamingConfig config) { this(job, config, null, null); } + @SuppressWarnings("unchecked") protected KafkaStreaming(KStreamJob job, StreamingConfig config, Producer producer, @@ -138,7 +138,6 @@ protected KafkaStreaming(KStreamJob job, this.config.pollTimeMs); this.running = true; this.lastCommit = 0; - this.lastWindow = 0; this.nextStateCleaning = Long.MAX_VALUE; this.recordsProcessed = 0; this.time = new SystemTime(); diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/StreamSynchronizer.java index 0fca1064487d2..f7221a6806ea5 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizer.java @@ -16,7 +16,7 @@ public class StreamSynchronizer { public final String name; private final RegulatedConsumer consumer; private final Chooser chooser; - private final TimestampExtractor timestampExtractor; + private final TimestampExtractor timestampExtractor; private final Map stash = new HashMap(); private final int desiredUnprocessed; private final Map consumedOffsets; @@ -28,7 +28,7 @@ public class StreamSynchronizer { public StreamSynchronizer(String name, RegulatedConsumer consumer, Chooser chooser, - TimestampExtractor timestampExtractor, + TimestampExtractor timestampExtractor, int desiredNumberOfUnprocessedRecords) { this.name = name; this.consumer = consumer; diff --git a/src/main/java/io/confluent/streaming/Window.java b/src/main/java/io/confluent/streaming/Window.java index f213df00a6878..9b886e566f985 100644 --- a/src/main/java/io/confluent/streaming/Window.java +++ b/src/main/java/io/confluent/streaming/Window.java @@ -11,8 +11,6 @@ public interface Window { void put(K key, V value, long timestamp); - void punctuate(long timestamp); - void flush(); } diff --git a/src/main/java/io/confluent/streaming/WindowByCount.java b/src/main/java/io/confluent/streaming/WindowByCount.java deleted file mode 100644 index 18e2ae74807fd..0000000000000 --- a/src/main/java/io/confluent/streaming/WindowByCount.java +++ /dev/null @@ -1,57 +0,0 @@ -package io.confluent.streaming; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; - -/** - * Created by yasuhiro on 6/18/15. - */ -public class WindowByCount implements Window { - - private final int maxCount; - private LinkedList list = new LinkedList(); - private HashMap> map = new HashMap>(); - private int count = 0; - - public WindowByCount(int maxCount) { - this.maxCount = maxCount; - } - - public Iterator find(K key, long timestamp) { - LinkedList values = map.get(key); - if (values == null) - return null; - else - return values.iterator(); - } - - public void put(K key, V value, long timestamp) { - list.offerLast(key); - - LinkedList values = map.get(key); - if (values == null) { - values = new LinkedList(); - map.put(key, values); - } - values.offerLast(value); - count++; - - mayEvict(); - } - - private void mayEvict() { - while (count > maxCount) { - K oldestKey = list.poll(); - LinkedList values = map.get(oldestKey); - values.removeFirst(); - if (values.isEmpty()) map.remove(oldestKey); - count--; - } - } - - public void punctuate(long timestamp) {} - - public void flush() {} - -} diff --git a/src/main/java/io/confluent/streaming/WindowByTime.java b/src/main/java/io/confluent/streaming/WindowByTime.java index 8215d235193d1..c2273feb0e69b 100644 --- a/src/main/java/io/confluent/streaming/WindowByTime.java +++ b/src/main/java/io/confluent/streaming/WindowByTime.java @@ -1,5 +1,6 @@ package io.confluent.streaming; +import io.confluent.streaming.util.FilteredIterator; import io.confluent.streaming.util.Stamped; import java.util.HashMap; @@ -21,24 +22,20 @@ public WindowByTime(long duration, int maxCount) { this.maxCount = maxCount; } - public Iterator find(K key, long timestamp) { + public Iterator find(K key, final long timestamp) { final LinkedList> values = map.get(key); if (values == null) { return null; } else { - final Iterator> inner = values.iterator(); - - return new Iterator() { - public boolean hasNext() { - return inner.hasNext(); - } - public V next() { - return inner.next().value; - } - public void remove() { - throw new UnsupportedOperationException(); + return new FilteredIterator>(values.iterator()) { + @Override + protected V filter(Stamped item) { + if (item.timestamp <= timestamp) + return item.value; + else + return null; } }; } @@ -53,7 +50,7 @@ public void put(K key, V value, long timestamp) { map.put(key, values); } - values.offerLast(new Stamped(value, timestamp)); + values.offerLast(new Stamped(value, timestamp)); evictExcess(); evictExpired(timestamp - duration); @@ -89,8 +86,6 @@ private void evictExpired(long cutoffTime) { } } - public void punctuate(long timestamp) {} - public void flush() {} } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index d26c144e767f2..148a1c1c7cd6b 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -2,6 +2,7 @@ import io.confluent.streaming.Predicate; +import java.lang.reflect.Array; import java.util.Arrays; /** @@ -12,9 +13,10 @@ class KStreamBranch implements Receiver { private final Predicate[] predicates; final KStreamSource[] branches; + @SuppressWarnings("unchecked") KStreamBranch(Predicate[] predicates, PartitioningInfo partitioningInfo, KStreamContextImpl context) { this.predicates = Arrays.copyOf(predicates, predicates.length); - branches = (KStreamSource[]) new Object[predicates.length + 1]; + this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length + 1); for (int i = 0; i < branches.length; i++) { branches[i] = new KStreamSource(partitioningInfo, context); } diff --git a/src/main/java/io/confluent/streaming/internal/RecordCollectors.java b/src/main/java/io/confluent/streaming/internal/RecordCollectors.java index b618d78247c0e..6bc330681d962 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordCollectors.java +++ b/src/main/java/io/confluent/streaming/internal/RecordCollectors.java @@ -55,11 +55,11 @@ public Map offsets() { public static class SerializingRecordCollector implements RecordCollector { - private final RecordCollector collector; + private final RecordCollector collector; private final Serializer keySerializer; private final Serializer valueSerializer; - public SerializingRecordCollector(RecordCollector collector, Serializer keySerializer, Serializer valueSerializer) { + public SerializingRecordCollector(RecordCollector collector, Serializer keySerializer, Serializer valueSerializer) { super(); this.keySerializer = keySerializer; this.valueSerializer = valueSerializer; diff --git a/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java b/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java index ccfda28cc4200..db9e69c7342c9 100644 --- a/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java +++ b/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java @@ -17,7 +17,8 @@ public class RegulatedConsumer { private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; private final long pollTimeMs; - private final Map> streamSynchronizers = new HashMap(); + private final Map> streamSynchronizers = + new HashMap>(); public RegulatedConsumer(Consumer consumer, Deserializer keyDeserializer, diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index 90529d28eaa05..003bc4c795f38 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -39,6 +39,7 @@ private static class MemoryStore implements KeyValueStore, StorageEn private final Deserializer valueDeserializer; private RecordCollector collector; + @SuppressWarnings("unchecked") public MemoryStore(String name, KStreamContext context) { this.topic = name; this.partition = context.id(); diff --git a/src/main/java/io/confluent/streaming/util/FilteredIterator.java b/src/main/java/io/confluent/streaming/util/FilteredIterator.java new file mode 100644 index 0000000000000..e79def2e68c13 --- /dev/null +++ b/src/main/java/io/confluent/streaming/util/FilteredIterator.java @@ -0,0 +1,48 @@ +package io.confluent.streaming.util; + +import java.util.Iterator; + +/** + * Created by yasuhiro on 6/29/15. + */ +public abstract class FilteredIterator implements Iterator { + + private Iterator inner; + private T nextValue = null; + + public FilteredIterator(Iterator inner) { + this.inner = inner; + + findNext(); + } + + @Override + public boolean hasNext() { + return nextValue != null; + } + + @Override + public T next() { + T value = nextValue; + findNext(); + + return value; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + private void findNext() { + while (inner.hasNext()) { + S item = inner.next(); + nextValue = filter(item); + if (nextValue != null) { + return; + } + } + } + + protected abstract T filter(S item); +} diff --git a/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java b/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java index 0f35a91ddece0..0ec1a397e81b8 100644 --- a/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java +++ b/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java @@ -9,8 +9,8 @@ */ public class QueueWithMinTimestampTracking { - private final Deque> queue = new ArrayDeque(); - private final LinkedList> descendingSubsequence = new LinkedList(); + private final Deque> queue = new ArrayDeque>(); + private final LinkedList> descendingSubsequence = new LinkedList>(); public void add(E value, long timestamp) { From 2cf744c590e74108e62a25ee4ccca0c010ee8b67 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 29 Jun 2015 16:19:49 -0700 Subject: [PATCH 006/275] added sync to PunctuationSchedulerImpl --- .../internal/PunctuationSchedulerImpl.java | 20 ++++++++++++----- .../streaming/internal/RegulatedConsumer.java | 22 +++++-------------- 2 files changed, 19 insertions(+), 23 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java b/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java index f47f7f32f9fff..d617f0caf2a10 100644 --- a/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java +++ b/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java @@ -20,20 +20,28 @@ public PunctuationSchedulerImpl(PunctuationQueue queue, Processor processo @Override public void schedule(long time) { - if (scheduled != null) - throw new IllegalStateException("punctuation is already scheduled"); + synchronized (this) { + if (scheduled != null) + throw new IllegalStateException("punctuation is already scheduled"); - scheduled = queue.schedule(this, time); + scheduled = queue.schedule(this, time); + } } @Override public void cancel() { - queue.cancel(scheduled); - scheduled = null; + synchronized (this) { + if (scheduled != null) { + queue.cancel(scheduled); + scheduled = null; + } + } } public void processed() { - scheduled = null; + synchronized (this) { + scheduled = null; + } } } diff --git a/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java b/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java index db9e69c7342c9..25945382de375 100644 --- a/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java +++ b/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java @@ -1,6 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.StreamSynchronizer; +import io.confluent.streaming.util.FilteredIterator; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -35,10 +36,6 @@ public void init() { unpaused.addAll(consumer.subscriptions()); } - public void addStreamSynchronizers(Map> streamSynchronizers) { - this.streamSynchronizers.putAll(streamSynchronizers); - } - public void poll() { poll(pollTimeMs); } @@ -76,27 +73,18 @@ public void clear() { streamSynchronizers.clear(); } - private class DeserializingIterator implements Iterator> { - - private final Iterator> inner; + private class DeserializingIterator extends FilteredIterator, ConsumerRecord> { DeserializingIterator(Iterator> inner) { - this.inner = inner; + super(inner); } - public boolean hasNext() { - return inner.hasNext(); - } - - public ConsumerRecord next() { - ConsumerRecord record = inner.next(); + protected ConsumerRecord filter(ConsumerRecord record) { K key = keyDeserializer.deserialize(record.topic(), record.key()); V value = valueDeserializer.deserialize(record.topic(), record.value()); return new ConsumerRecord(record.topic(), record.partition(), record.offset(), key, value); } - public void remove() { - throw new UnsupportedOperationException(); - } + } } From edc9f3c5c2dabff1ece8c34e13cb83c782115ddf Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 29 Jun 2015 17:09:23 -0700 Subject: [PATCH 007/275] pass stream time to puctuate() --- src/main/java/io/confluent/streaming/Processor.java | 2 +- src/main/java/io/confluent/streaming/internal/KStreamImpl.java | 2 +- .../java/io/confluent/streaming/internal/PunctuationQueue.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 3d51b1a5baf21..7c13e067a91cb 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -9,7 +9,7 @@ public interface Processor { void init(PunctuationScheduler punctuationScheduler); - void punctuate(); + void punctuate(long streamTime); void flush(); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index be3f25fdf9964..f596ab5e840dd 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -100,7 +100,7 @@ public void apply(K key, V value) { @Override public void init(PunctuationScheduler scheduler) {} @Override - public void punctuate() {} + public void punctuate(long streamTime) {} @Override public void flush() {} }; diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java b/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java index a11ad3ba06ae8..262271a0eb471 100644 --- a/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java +++ b/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java @@ -38,7 +38,7 @@ public void mayPunctuate(long streamTime) { while (top.timestamp <= streamTime) { PunctuationSchedulerImpl scheduler = top.value; pq.poll(); - scheduler.processor.punctuate(); + scheduler.processor.punctuate(streamTime); scheduler.processed(); top = pq.peek(); From 10fd4641497eecd641961941302fa5863d5feb9e Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 30 Jun 2015 09:46:16 -0700 Subject: [PATCH 008/275] removed flush method from KStream and Processor --- .../java/io/confluent/streaming/Processor.java | 2 -- .../streaming/internal/KStreamBranch.java | 7 +------ .../streaming/internal/KStreamContextImpl.java | 3 --- .../streaming/internal/KStreamFilter.java | 1 + .../streaming/internal/KStreamFlatMap.java | 1 + .../internal/KStreamFlatMapValues.java | 1 + .../streaming/internal/KStreamImpl.java | 13 ------------- .../streaming/internal/KStreamJoin.java | 17 ++--------------- .../streaming/internal/KStreamMap.java | 1 + .../streaming/internal/KStreamMapValues.java | 1 + .../streaming/internal/KStreamNestedLoop.java | 1 + .../streaming/internal/KStreamSource.java | 1 + .../streaming/internal/KStreamWindowedImpl.java | 7 ++----- .../confluent/streaming/internal/Receiver.java | 2 -- 14 files changed, 12 insertions(+), 46 deletions(-) diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 7c13e067a91cb..b77514020bc3c 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -11,6 +11,4 @@ public interface Processor { void punctuate(long streamTime); - void flush(); - } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 148a1c1c7cd6b..d0b1f7d7edcaa 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -22,6 +22,7 @@ class KStreamBranch implements Receiver { } } + @Override public void receive(K key, V value, long timestamp) { synchronized(this) { for (int i = 0; i < predicates.length; i++) { @@ -36,10 +37,4 @@ public void receive(K key, V value, long timestamp) { } } - public void flush() { - for (KStreamSource branch : branches) { - branch.flush(); - } - } - } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 31055cb1dc682..94c4d1d55d900 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -193,9 +193,6 @@ public void init(Consumer restoreConsumer, KStreamJob job) { } public void flush() { - for (KStreamSource stream : sourceStreams.values()) { - stream.flush(); - } state.flush(); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index e8941e94c7369..11add37faf623 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -14,6 +14,7 @@ class KStreamFilter extends KStreamImpl { this.predicate = predicate; } + @Override public void receive(K key, V value, long timestamp) { synchronized(this) { if (predicate.apply(key, value)) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 62a59f7f47bd3..f0f340c1e81b8 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -15,6 +15,7 @@ class KStreamFlatMap extends KStreamImpl { this.mapper = mapper; } + @Override public void receive(K1 key, V1 value, long timestamp) { synchronized(this) { KeyValue> newPair = mapper.apply(key, value); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index fca87d0e48843..b52455c4632cf 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -14,6 +14,7 @@ class KStreamFlatMapValues extends KStreamImpl { this.mapper = mapper; } + @Override public void receive(K key, V1 value, long timestamp) { synchronized(this) { Iterable newValues = mapper.apply(value); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index f596ab5e840dd..f9f86b5bafb35 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -101,8 +101,6 @@ public void apply(K key, V value) { public void init(PunctuationScheduler scheduler) {} @Override public void punctuate(long streamTime) {} - @Override - public void flush() {} }; } @@ -112,9 +110,6 @@ public void process(final Processor processor) { public void receive(K key, V value, long timestamp) { processor.apply(key, value); } - public void flush() { - processor.flush(); - } }; registerReceiver(receiver); @@ -122,14 +117,6 @@ public void flush() { processor.init(scheduler); } - @Override - public void flush() { - int numReceivers = nextReceivers.size(); - for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).flush(); - } - } - void registerReceiver(Receiver receiver) { nextReceivers.add(receiver); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index ba1c4ea69daab..ba1215ffbbf83 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -15,8 +15,6 @@ class KStreamJoin extends KStreamImpl { private final ValueJoiner joiner; final Receiver receiverForOtherStream; - private boolean flushed = true; - KStreamJoin(final Window window1, Window window2, ValueJoiner joiner, PartitioningInfo partitioningInfo, KStreamContextImpl context) { super(partitioningInfo, context); @@ -27,8 +25,8 @@ class KStreamJoin extends KStreamImpl { this.receiverForOtherStream = getReceiverForOther(); } + @Override public void receive(K key, V1 value, long timestamp) { - flushed = false; Iterator iter = window2.find(key, timestamp); if (iter != null) { while (iter.hasNext()) { @@ -37,18 +35,11 @@ public void receive(K key, V1 value, long timestamp) { } } - public void flush() { - if (!flushed) { - super.flush(); - flushed = true; - } - } - private Receiver getReceiverForOther() { return new Receiver() { + @Override public void receive(K key, V2 value2, long timestamp) { - flushed = false; Iterator iter = window1.find(key, timestamp); if (iter != null) { while (iter.hasNext()) { @@ -56,10 +47,6 @@ public void receive(K key, V2 value2, long timestamp) { } } } - - public void flush() { - KStreamJoin.this.flush(); - } }; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index e9585f46e3117..26198ad89ea2d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -15,6 +15,7 @@ class KStreamMap extends KStreamImpl { this.mapper = mapper; } + @Override public void receive(K1 key, V1 value, long timestamp) { synchronized (this) { KeyValue newPair = mapper.apply(key, value); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index 98596552b5727..a2c481c042d77 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -14,6 +14,7 @@ class KStreamMapValues extends KStreamImpl { this.mapper = mapper; } + @Override public void receive(K key, V1 value, long timestamp) { synchronized (this) { V newValue = mapper.apply(value); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java b/src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java index f4eadb86668b3..9be776bfa27c9 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java @@ -19,6 +19,7 @@ class KStreamNestedLoop extends KStreamImpl { this.joiner = joiner; } + @Override public void receive(K key, V1 value, long timestamp) { Iterator iter = window.find(key, timestamp); if (iter != null) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index d73631a7bedde..e1d565d8d3037 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -9,6 +9,7 @@ class KStreamSource extends KStreamImpl { super(partitioningInfo, context); } + @Override public void receive(K key, V value, long timestamp) { synchronized(this) { forward(key, value, timestamp); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index cfc350265f6c5..101cd018041c3 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -14,6 +14,7 @@ public class KStreamWindowedImpl extends KStreamImpl implement this.window = window; } + @Override public void receive(K key, V value, long timestamp) { synchronized(this) { window.put(key, value, timestamp); @@ -21,11 +22,7 @@ public void receive(K key, V value, long timestamp) { } } - public void flush() { - window.flush(); - super.flush(); - } - + @Override public KStream join(KStreamWindowed other, ValueJoiner processor) throws NotCopartitionedException { diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 6ad7ff2f3efb1..ae73c4f59fecb 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -7,6 +7,4 @@ public interface Receiver { void receive(K key, V value, long timestamp); - void flush(); - } From 47676e48429ceed44c86fcb7aeba8ac64fff6297 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 30 Jun 2015 10:09:35 -0700 Subject: [PATCH 009/275] simplified recordqueue interface --- .../java/io/confluent/streaming/RecordQueue.java | 4 +--- .../io/confluent/streaming/StreamSynchronizer.java | 13 +++---------- .../streaming/internal/RecordQueueImpl.java | 9 +++++++++ .../util/QueueWithMinTimestampTracking.java | 14 -------------- 4 files changed, 13 insertions(+), 27 deletions(-) diff --git a/src/main/java/io/confluent/streaming/RecordQueue.java b/src/main/java/io/confluent/streaming/RecordQueue.java index c187929a45372..59a962ab90347 100644 --- a/src/main/java/io/confluent/streaming/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/RecordQueue.java @@ -14,9 +14,7 @@ public interface RecordQueue { ConsumerRecord next(); - ConsumerRecord peekNext(); - - ConsumerRecord peekLast(); + long offset(); int size(); diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/StreamSynchronizer.java index f7221a6806ea5..e32c99911f5d2 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizer.java @@ -93,10 +93,7 @@ public void process() { } if (recordQueue.size() == this.desiredUnprocessed) { - ConsumerRecord record = recordQueue.peekLast(); - if (record != null) { - consumer.unpause(recordQueue.partition(), record.offset()); - } + consumer.unpause(recordQueue.partition(), recordQueue.offset()); } if (recordQueue.size() == 0) return; @@ -165,12 +162,8 @@ public ConsumerRecord next() { return queue.next(); } - public ConsumerRecord peekNext() { - return queue.peekNext(); - } - - public ConsumerRecord peekLast() { - return queue.peekLast(); + public long offset() { + return queue.offset(); } public int size() { diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java b/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java index a73720bfdc427..04b02b135e05b 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java @@ -11,6 +11,7 @@ public class RecordQueueImpl extends QueueWithMinTimestampTracking> implements RecordQueue { private final TopicPartition partition; + private long offset; public RecordQueueImpl(TopicPartition partition) { this.partition = partition; @@ -20,6 +21,14 @@ public TopicPartition partition() { return partition; } + public void add(ConsumerRecord record, long timestamp) { + offset = record.offset(); + } + + public long offset() { + return offset; + } + public long currentStreamTime() { return super.timestamp(); } diff --git a/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java b/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java index 0ec1a397e81b8..6c1a10f6623a1 100644 --- a/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java +++ b/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java @@ -36,20 +36,6 @@ public E next() { return stamped.value; } - public E peekNext() { - if (queue.size() > 0) - return queue.peekFirst().value; - else - return null; - } - - public E peekLast() { - if (queue.size() > 0) - return queue.peekLast().value; - else - return null; - } - public int size() { return queue.size(); } From 968b0d8c0fc196b4cf3649050f43634f039e8a42 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 30 Jun 2015 11:03:40 -0700 Subject: [PATCH 010/275] separated timestamp stacking from queue impl --- .../streaming/StreamSynchronizer.java | 33 ++++++++----------- .../streaming/internal/RecordQueueImpl.java | 32 +++++++++++++++--- ...Tracking.java => MinTimestampTracker.java} | 28 +++++----------- .../streaming/util/TimestampTracker.java | 32 ++++++++++++++++++ 4 files changed, 82 insertions(+), 43 deletions(-) rename src/main/java/io/confluent/streaming/util/{QueueWithMinTimestampTracking.java => MinTimestampTracker.java} (52%) create mode 100644 src/main/java/io/confluent/streaming/util/TimestampTracker.java diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/StreamSynchronizer.java index e32c99911f5d2..d8020d7a01da6 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizer.java @@ -1,6 +1,7 @@ package io.confluent.streaming; import io.confluent.streaming.internal.*; +import io.confluent.streaming.util.MinTimestampTracker; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; @@ -44,13 +45,7 @@ public void addPartition(TopicPartition partition, final Receiver record, long streamTime) { - receiver.receive(record.key(), record.value(), streamTime); - } - }; - stash.put(partition, queue); + stash.put(partition, new RecordQueueWrapper(createRecordQueue(partition), receiver)); } else { throw new IllegalStateException("duplicate partition"); } @@ -108,10 +103,6 @@ public void process() { } } - public long currentStreamTime() { - return streamTime; - } - public Map consumedOffsets() { return this.consumedOffsets; } @@ -126,15 +117,17 @@ public void close() { } protected RecordQueue createRecordQueue(TopicPartition partition) { - return new RecordQueueImpl(partition); + return new RecordQueueImpl(partition, new MinTimestampTracker>()); } - private abstract class RecordQueueWrapper implements RecordQueue { + private class RecordQueueWrapper implements RecordQueue { private final RecordQueue queue; + private final Receiver receiver; - RecordQueueWrapper(RecordQueue queue) { - this.queue = queue; + RecordQueueWrapper(RecordQueue queue, Receiver receiver) { + this.queue = queue; + this.receiver = receiver; } void process() { @@ -143,17 +136,16 @@ void process() { if (streamTime < timestamp) streamTime = timestamp; - doProcess(record, streamTime); - + receiver.receive(record.key(), record.value(), streamTime); consumedOffsets.put(queue.partition(), record.offset()); } - abstract void doProcess(ConsumerRecord record, long streamTime); - + @Override public TopicPartition partition() { return queue.partition(); } + @Override public void add(ConsumerRecord value, long timestamp) { queue.add(value, timestamp); } @@ -162,14 +154,17 @@ public ConsumerRecord next() { return queue.next(); } + @Override public long offset() { return queue.offset(); } + @Override public int size() { return queue.size(); } + @Override public long currentStreamTime() { return queue.currentStreamTime(); } diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java b/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java index 04b02b135e05b..7e3da315bb0a3 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java @@ -1,20 +1,27 @@ package io.confluent.streaming.internal; import io.confluent.streaming.RecordQueue; -import io.confluent.streaming.util.QueueWithMinTimestampTracking; +import io.confluent.streaming.util.Stamped; +import io.confluent.streaming.util.TimestampTracker; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import java.util.ArrayDeque; +import java.util.Deque; + /** * Created by yasuhiro on 6/25/15. */ -public class RecordQueueImpl extends QueueWithMinTimestampTracking> implements RecordQueue { +public class RecordQueueImpl implements RecordQueue { + private final Deque>> queue = new ArrayDeque>>(); private final TopicPartition partition; + private TimestampTracker> timestampTracker; private long offset; - public RecordQueueImpl(TopicPartition partition) { + public RecordQueueImpl(TopicPartition partition, TimestampTracker> timestampTracker) { this.partition = partition; + this.timestampTracker = timestampTracker; } public TopicPartition partition() { @@ -22,15 +29,32 @@ public TopicPartition partition() { } public void add(ConsumerRecord record, long timestamp) { + Stamped> elem = new Stamped>(record, timestamp); + queue.addLast(elem); + timestampTracker.addStampedElement(elem); offset = record.offset(); } + public ConsumerRecord next() { + Stamped> elem = queue.getFirst(); + + if (elem == null) return null; + + timestampTracker.removeStampedElement(elem); + + return elem.value; + } + public long offset() { return offset; } + public int size() { + return queue.size(); + } + public long currentStreamTime() { - return super.timestamp(); + return timestampTracker.get(); } } diff --git a/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java b/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java similarity index 52% rename from src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java rename to src/main/java/io/confluent/streaming/util/MinTimestampTracker.java index 6c1a10f6623a1..5f5e50e2330ca 100644 --- a/src/main/java/io/confluent/streaming/util/QueueWithMinTimestampTracking.java +++ b/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java @@ -1,22 +1,15 @@ package io.confluent.streaming.util; -import java.util.ArrayDeque; -import java.util.Deque; import java.util.LinkedList; /** - * Created by yasuhiro on 6/24/15. + * MinTimestampTracker maintains the minimum timestamp of stamped elements that were added but not yet removed. */ -public class QueueWithMinTimestampTracking { +public class MinTimestampTracker implements TimestampTracker { - private final Deque> queue = new ArrayDeque>(); private final LinkedList> descendingSubsequence = new LinkedList>(); - public void add(E value, long timestamp) { - - Stamped elem = new Stamped(value, timestamp); - queue.addLast(elem); - + public void addStampedElement(Stamped elem) { Stamped minElem = descendingSubsequence.peekLast(); while (minElem.compareTo(elem) >= 0) { descendingSubsequence.removeLast(); @@ -25,25 +18,20 @@ public void add(E value, long timestamp) { descendingSubsequence.offerLast(elem); } - public E next() { - Stamped stamped = queue.getFirst(); - - if (stamped == null) return null; - - if (descendingSubsequence.peekFirst() == stamped) + public void removeStampedElement(Stamped elem) { + if (elem != null && descendingSubsequence.peekFirst() == elem) descendingSubsequence.removeFirst(); - - return stamped.value; } public int size() { - return queue.size(); + return descendingSubsequence.size(); } - public long timestamp() { + public long get() { Stamped stamped = descendingSubsequence.peekFirst(); if (stamped == null) return -1L; return stamped.timestamp; } + } diff --git a/src/main/java/io/confluent/streaming/util/TimestampTracker.java b/src/main/java/io/confluent/streaming/util/TimestampTracker.java new file mode 100644 index 0000000000000..612da6b82d9f8 --- /dev/null +++ b/src/main/java/io/confluent/streaming/util/TimestampTracker.java @@ -0,0 +1,32 @@ +package io.confluent.streaming.util; + +/** + * TimestampTracker maintains the timestamp, like the min timestamp, the max timestamp, etc. of stamped elements + * that were added but not yet removed. + */ +public interface TimestampTracker { + + /** + * Adds a stamped elements to this tracker. + * @param elem + */ + void addStampedElement(Stamped elem); + + /** + * Removed a stamped elements to this tracker. + * @param elem + */ + void removeStampedElement(Stamped elem); + + /** + * Returns the timestamp + * @return + */ + long get(); + + /** + * Returns the size of internal structure. The meaning of "size" depends on the implementation. + */ + int size(); + +} From abb5efee9e6e8c9a130953602d83b2b1ab9930b8 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 30 Jun 2015 12:03:18 -0700 Subject: [PATCH 011/275] comments --- .../java/io/confluent/streaming/KStream.java | 66 +++++++++++++++++++ .../streaming/internal/KStreamBranch.java | 3 +- 2 files changed, 67 insertions(+), 2 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index 0bf0ea316fe8c..952470e79d6f7 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -5,29 +5,95 @@ */ public interface KStream { + /** + * Creates a new stream consists of all elements of this stream which satisfy a predicate + * @param predicate + * @return KStream + */ KStream filter(Predicate predicate); + /** + * Creates a new stream consists all elements of this stream which do not satisfy a predicate + * @param predicate + * @return + */ KStream filterOut(Predicate predicate); + /** + * Creates a new stream by transforming key-value pairs by a mapper to all elements of this stream + * @param mapper + * @return KStream + */ KStream map(KeyValueMapper mapper); + /** + * Creates a new stream by transforming valuesa by a mapper to all values of this stream + * @param mapper + * @return + */ KStream mapValues(ValueMapper mapper); + /** + * Creates a new stream by applying a mapper to all elements of this stream and using the values in the resulting Iterable + * @param mapper + * @return + */ KStream flatMap(KeyValueMapper, K, V> 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 + * @return + */ KStream flatMapValues(ValueMapper, V> processor); + /** + * Creates a new windowed stream using a specified window object. + * @param window + * @return + */ KStreamWindowed with(Window window); + /** + * Creates a new stream by joining this stream with the other stream. + * Each element in this stream are joined with elements in other streams window by applying a function + * + * @param other + * @param processor + * @return KStream + * @throws NotCopartitionedException + */ KStream nestedLoop(KStreamWindowed other, ValueJoiner processor) throws NotCopartitionedException; + /** + * 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 + * @return + */ KStream[] branch(Predicate... predicates); + /** + * Sends key-value to a topic, also creates a new stream from the topic. + * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). + * @param topic + * @return + */ KStream through(String topic); + /** + * Sends key-value to a topic. + * @param topic + */ void sendTo(String topic); + /** + * Processes all elements in this stream by applying a processor. + * @param processor + */ void process(Processor processor); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index d0b1f7d7edcaa..2b0c14dce42b0 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -16,7 +16,7 @@ class KStreamBranch implements Receiver { @SuppressWarnings("unchecked") KStreamBranch(Predicate[] predicates, PartitioningInfo partitioningInfo, KStreamContextImpl context) { this.predicates = Arrays.copyOf(predicates, predicates.length); - this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length + 1); + this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { branches[i] = new KStreamSource(partitioningInfo, context); } @@ -32,7 +32,6 @@ public void receive(K key, V value, long timestamp) { return; } } - branches[branches.length - 1].receive(key, value, timestamp); return; } } From a5a2ab450507fe0d7ea33dc8adf07439d52f0a03 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 30 Jun 2015 12:33:59 -0700 Subject: [PATCH 012/275] comments --- src/main/java/io/confluent/streaming/KStream.java | 11 ++++++----- .../io/confluent/streaming/KStreamWindowed.java | 14 ++++++++++++-- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index 952470e79d6f7..f27a10bca8d07 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -1,7 +1,7 @@ package io.confluent.streaming; /** - * Created by yasuhiro on 6/17/15. + * KStream is an abstraction of a stream of key-value pairs. */ public interface KStream { @@ -55,15 +55,16 @@ public interface KStream { KStreamWindowed with(Window window); /** - * Creates a new stream by joining this stream with the other stream. - * Each element in this stream are joined with elements in other streams window by applying a function + * Creates a new stream by joining this stream with the other windowed stream. + * Each element in this stream is joined with elements in the other stream's window. + * The resulting values are computed by applying a joiner. * * @param other - * @param processor + * @param joiner * @return KStream * @throws NotCopartitionedException */ - KStream nestedLoop(KStreamWindowed other, ValueJoiner processor) + KStream nestedLoop(KStreamWindowed other, ValueJoiner joiner) throws NotCopartitionedException; /** diff --git a/src/main/java/io/confluent/streaming/KStreamWindowed.java b/src/main/java/io/confluent/streaming/KStreamWindowed.java index 83b0886df7709..102580d91fc94 100644 --- a/src/main/java/io/confluent/streaming/KStreamWindowed.java +++ b/src/main/java/io/confluent/streaming/KStreamWindowed.java @@ -1,11 +1,21 @@ package io.confluent.streaming; /** - * Created by yasuhiro on 6/18/15. + * KStreamWindowed is an abstraction of a stream of key-value pairs with a window. */ public interface KStreamWindowed extends KStream { - KStream join(KStreamWindowed other, ValueJoiner processor) + /** + * 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 + * @param joiner + * @return KStream + * @throws NotCopartitionedException + */ + KStream join(KStreamWindowed other, ValueJoiner joiner) throws NotCopartitionedException; } From 05e4d1b2e10aaaad7f6696a17c7c2b9e64bf1a1e Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 30 Jun 2015 14:24:40 -0700 Subject: [PATCH 013/275] added Ingestor interface, renamed RegulatedConsumer to IngestorImpl --- pom.xml | 16 +++- .../confluent/streaming/KafkaStreaming.java | 22 +++--- .../streaming/StreamSynchronizer.java | 12 +-- .../streaming/StreamSynchronizerFactory.java | 4 +- .../io/confluent/streaming/SyncGroup.java | 2 +- .../streaming/internal/Ingestor.java | 18 +++++ ...gulatedConsumer.java => IngestorImpl.java} | 10 +-- .../internal/KStreamContextImpl.java | 8 +- .../streaming/internal/KStreamFlatMap.java | 5 +- .../streaming/internal/KStreamImpl.java | 4 +- .../streaming/internal/KStreamMap.java | 5 +- .../streaming/internal/PartitioningInfo.java | 6 +- .../streaming/internal/KStreamSourceTest.java | 79 +++++++++++++++++++ 13 files changed, 151 insertions(+), 40 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/internal/Ingestor.java rename src/main/java/io/confluent/streaming/internal/{RegulatedConsumer.java => IngestorImpl.java} (91%) create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java diff --git a/pom.xml b/pom.xml index 108039a64bba6..2997743d9c15d 100644 --- a/pom.xml +++ b/pom.xml @@ -28,8 +28,22 @@ junit junit - 3.8.1 + 4.8.1 test + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.3 + + 1.6 + 1.6 + + + + diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index c24178b7d2dbe..317e24a22defc 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -19,8 +19,7 @@ import io.confluent.streaming.internal.KStreamContextImpl; import io.confluent.streaming.internal.ProcessorConfig; -import io.confluent.streaming.internal.RecordCollectors; -import io.confluent.streaming.internal.RegulatedConsumer; +import io.confluent.streaming.internal.IngestorImpl; import io.confluent.streaming.util.Util; import org.apache.kafka.clients.consumer.*; import org.apache.kafka.clients.producer.KafkaProducer; @@ -38,10 +37,8 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,7 +85,7 @@ public class KafkaStreaming implements Runnable { private final Map kstreamContexts = new HashMap(); protected final Producer producer; protected final Consumer consumer; - private final RegulatedConsumer regulatedConsumer; + private final IngestorImpl ingestor; private final StreamingConfig streamingConfig; private final ProcessorConfig config; private final Metrics metrics; @@ -131,8 +128,8 @@ protected KafkaStreaming(KStreamJob job, this.streamingMetrics = new KafkaStreamingMetrics(); this.requestingCommit = new ArrayList(); this.config = new ProcessorConfig(config.config()); - this.regulatedConsumer = - new RegulatedConsumer(this.consumer, + this.ingestor = + new IngestorImpl(this.consumer, (Deserializer) config.keyDeserializer(), (Deserializer) config.valueDeserializer(), this.config.pollTimeMs); @@ -273,8 +270,8 @@ private void commitRequesting(long now) { KStreamContextImpl context = kstreamContexts.get(id); context.flush(); - for (SyncGroup synchGroup : this.syncGroups.get(id)) { - commit.putAll(synchGroup.streamSynchronizer.consumedOffsets()); // TODO: can this be async? + for (SyncGroup syncGroup : this.syncGroups.get(id)) { + commit.putAll(syncGroup.streamSynchronizer.consumedOffsets()); // TODO: can this be async? } } consumer.commit(commit, CommitType.SYNC); @@ -308,6 +305,8 @@ private void maybeCleanState() { private void addPartitions(Collection assignment) { HashSet partitions = new HashSet(assignment); + this.ingestor.init(); + Consumer restoreConsumer = new KafkaConsumer( streamingConfig.config(), @@ -334,7 +333,7 @@ public void shutdown(Coordinator.RequestScope scope) { File stateDir = new File(config.stateDir, id.toString()); kstreamContext = - new KStreamContextImpl(id, regulatedConsumer, producer, coordinator, streamingConfig, config, stateDir, metrics); + new KStreamContextImpl(id, ingestor, producer, coordinator, streamingConfig, config, stateDir, metrics); kstreamContext.init(restoreConsumer,job); @@ -344,7 +343,6 @@ public void shutdown(Coordinator.RequestScope scope) { } restoreConsumer.close(); - this.regulatedConsumer.init(); this.nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; } @@ -373,7 +371,7 @@ private void removePartitions(Collection assignment) { } } // clear buffered records - this.regulatedConsumer.clear(); + this.ingestor.clear(); } private class KafkaStreamingMetrics { diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/StreamSynchronizer.java index d8020d7a01da6..ecb5d642fbe5e 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizer.java @@ -15,7 +15,7 @@ public class StreamSynchronizer { public final String name; - private final RegulatedConsumer consumer; + private final Ingestor ingestor; private final Chooser chooser; private final TimestampExtractor timestampExtractor; private final Map stash = new HashMap(); @@ -27,12 +27,12 @@ public class StreamSynchronizer { private volatile int buffered = 0; public StreamSynchronizer(String name, - RegulatedConsumer consumer, + Ingestor ingestor, Chooser chooser, TimestampExtractor timestampExtractor, int desiredNumberOfUnprocessedRecords) { this.name = name; - this.consumer = consumer; + this.ingestor = ingestor; this.chooser = chooser; this.timestampExtractor = timestampExtractor; this.desiredUnprocessed = desiredNumberOfUnprocessedRecords; @@ -68,7 +68,7 @@ public void addRecords(TopicPartition partition, Iterator> // if we have buffered enough for this partition, pause if (queue.size() > this.desiredUnprocessed) { - consumer.pause(partition); + ingestor.pause(partition); } } } @@ -83,12 +83,12 @@ public void process() { RecordQueueWrapper recordQueue = (RecordQueueWrapper)chooser.next(); if (recordQueue == null) { - consumer.poll(); + ingestor.poll(); return; } if (recordQueue.size() == this.desiredUnprocessed) { - consumer.unpause(recordQueue.partition(), recordQueue.offset()); + ingestor.unpause(recordQueue.partition(), recordQueue.offset()); } if (recordQueue.size() == 0) return; diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java b/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java index 5de2137d874f5..c8c006589091f 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java @@ -1,7 +1,7 @@ package io.confluent.streaming; import io.confluent.streaming.internal.ChooserImpl; -import io.confluent.streaming.internal.RegulatedConsumer; +import io.confluent.streaming.internal.IngestorImpl; /** * Created by yasuhiro on 6/24/15. @@ -14,7 +14,7 @@ public StreamSynchronizerFactory(TimestampExtractor timestampExtractor) { this.timestampExtractor = timestampExtractor; } - public StreamSynchronizer create(String name, RegulatedConsumer consumer, int desiredNumberOfUnprocessedRecords) { + public StreamSynchronizer create(String name, IngestorImpl consumer, int desiredNumberOfUnprocessedRecords) { return new StreamSynchronizer(name, consumer, new ChooserImpl(), timestampExtractor, desiredNumberOfUnprocessedRecords); } diff --git a/src/main/java/io/confluent/streaming/SyncGroup.java b/src/main/java/io/confluent/streaming/SyncGroup.java index a0755ea744325..d29954c6e88e3 100644 --- a/src/main/java/io/confluent/streaming/SyncGroup.java +++ b/src/main/java/io/confluent/streaming/SyncGroup.java @@ -8,7 +8,7 @@ public class SyncGroup { public final String name; public final StreamSynchronizer streamSynchronizer; - public SyncGroup(String name, StreamSynchronizer streamSynchronizer) { + public SyncGroup(String name, StreamSynchronizer streamSynchronizer) { this.name = name; this.streamSynchronizer = streamSynchronizer; } diff --git a/src/main/java/io/confluent/streaming/internal/Ingestor.java b/src/main/java/io/confluent/streaming/internal/Ingestor.java new file mode 100644 index 0000000000000..c76ac23a98fe4 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/Ingestor.java @@ -0,0 +1,18 @@ +package io.confluent.streaming.internal; + +import org.apache.kafka.common.TopicPartition; + +/** + * Created by yasuhiro on 6/30/15. + */ +public interface Ingestor { + + void poll(); + + void poll(long timeoutMs); + + void pause(TopicPartition partition); + + void unpause(TopicPartition partition, long offset); + +} diff --git a/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java similarity index 91% rename from src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java rename to src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 25945382de375..34a146d89b96b 100644 --- a/src/main/java/io/confluent/streaming/internal/RegulatedConsumer.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -10,7 +10,7 @@ import java.util.*; -public class RegulatedConsumer { +public class IngestorImpl implements Ingestor { private final Consumer consumer; private final Set unpaused = new HashSet(); @@ -21,10 +21,10 @@ public class RegulatedConsumer { private final Map> streamSynchronizers = new HashMap>(); - public RegulatedConsumer(Consumer consumer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, - long pollTimeMs) { + public IngestorImpl(Consumer consumer, + Deserializer keyDeserializer, + Deserializer valueDeserializer, + long pollTimeMs) { this.consumer = consumer; this.keyDeserializer = keyDeserializer; this.valueDeserializer = valueDeserializer; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 94c4d1d55d900..8b4955228c862 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -25,7 +25,7 @@ public class KStreamContextImpl implements KStreamContext { public final int id; private final Set topics; - private final RegulatedConsumer regulatedConsumer; + private final IngestorImpl ingestor; private final RecordCollectors.SimpleRecordCollector simpleCollector; private final RecordCollector collector; @@ -44,7 +44,7 @@ public class KStreamContextImpl implements KStreamContext { @SuppressWarnings("unchecked") public KStreamContextImpl(int id, - RegulatedConsumer regulatedConsumer, + IngestorImpl ingestor, Producer producer, Coordinator coordinator, StreamingConfig streamingConfig, @@ -53,7 +53,7 @@ public KStreamContextImpl(int id, Metrics metrics) { this.id = id; this.topics = streamingConfig.topics(); - this.regulatedConsumer = (RegulatedConsumer)regulatedConsumer; + this.ingestor = (IngestorImpl) ingestor; this.simpleCollector = new RecordCollectors.SimpleRecordCollector(producer); this.collector = new RecordCollectors.SerializingRecordCollector( @@ -159,7 +159,7 @@ public SyncGroup syncGroup(String name) { SyncGroup syncGroup = syncGroups.get(name); if (syncGroup == null) { StreamSynchronizer streamSynchronizer = - streamSynchronizerFactory.create(name, regulatedConsumer, processorConfig.bufferedRecordsPerPartition); + streamSynchronizerFactory.create(name, ingestor, processorConfig.bufferedRecordsPerPartition); syncGroup = new SyncGroup(name, streamSynchronizer); syncGroups.put(name, syncGroup); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index f0f340c1e81b8..cdad3a0badf79 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -2,6 +2,7 @@ import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; +import io.confluent.streaming.SyncGroup; /** * Created by yasuhiro on 6/17/15. @@ -10,8 +11,8 @@ class KStreamFlatMap extends KStreamImpl { private final KeyValueMapper, K1, V1> mapper; - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamContextImpl context) { - super(PartitioningInfo.missing, context); + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, SyncGroup syncGroup, KStreamContextImpl context) { + super(PartitioningInfo.unjoinable(syncGroup), context); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index f9f86b5bafb35..b123385378a16 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -36,7 +36,7 @@ public boolean apply(K key, V value) { @Override public KStream map(KeyValueMapper mapper) { - return chain(new KStreamMap(mapper, context)); + return chain(new KStreamMap(mapper, partitioningInfo.syncGroup, context)); } @Override @@ -46,7 +46,7 @@ public KStream mapValues(ValueMapper mapper) { @Override public KStream flatMap(KeyValueMapper, K, V> mapper) { - return chain(new KStreamFlatMap(mapper, context)); + return chain(new KStreamFlatMap(mapper, partitioningInfo.syncGroup, context)); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index 26198ad89ea2d..f4fec60afd92a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -2,6 +2,7 @@ import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; +import io.confluent.streaming.SyncGroup; /** * Created by yasuhiro on 6/17/15. @@ -10,8 +11,8 @@ class KStreamMap extends KStreamImpl { private final KeyValueMapper mapper; - KStreamMap(KeyValueMapper mapper, KStreamContextImpl context) { - super(PartitioningInfo.missing, context); + KStreamMap(KeyValueMapper mapper, SyncGroup syncGroup, KStreamContextImpl context) { + super(PartitioningInfo.unjoinable(syncGroup), context); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java b/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java index c4d6817352092..30de31ffa819c 100644 --- a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java +++ b/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java @@ -7,14 +7,14 @@ */ class PartitioningInfo { - public static PartitioningInfo missing = new PartitioningInfo(null, -1); + public static PartitioningInfo unjoinable(SyncGroup syncGroup) { + return new PartitioningInfo(syncGroup, -1); + } public final SyncGroup syncGroup; public final int numPartitions; PartitioningInfo(SyncGroup syncGroup, int numPartitions) { - if (syncGroup == null) throw new NullPointerException(); - this.syncGroup = syncGroup; this.numPartitions = numPartitions; } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java new file mode 100644 index 0000000000000..86418bc6cfbe3 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -0,0 +1,79 @@ +package io.confluent.streaming.internal; + +import static org.junit.Assert.assertEquals; + +import io.confluent.streaming.*; + +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import java.util.ArrayList; + +public class KStreamSourceTest { + + @Test + public void testKStreamSource() { + + Ingestor ingestor = new Ingestor() { + + @Override + public void poll() {} + + @Override + public void poll(long timeoutMs) {} + + @Override + public void pause(TopicPartition partition) {} + + @Override + public void unpause(TopicPartition partition, long offset) {} + }; + + StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; + } + }, + 10 + ); + + PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + final ArrayList processed = new ArrayList(); + + Processor processor = new Processor() { + + @Override + public void apply(String key, String value) { + processed.add(key + ":" + value); + } + + @Override + public void init(PunctuationScheduler punctuationScheduler) {} + + @Override + public void punctuate(long streamTime) {} + }; + + KStreamSource stream = new KStreamSource(partitioningInfo, null); + stream.process(processor); + + final String[] expectedKeys = new String[] { "k1", "k2", "k3" }; + final String[] expectedValues = new String[] { "v1", "v2", "v3" }; + + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], expectedValues[i], 0L); + } + + assertEquals(3, processed.size()); + + for (int i = 0; i < expectedKeys.length; i++) { + assertEquals(expectedKeys[i] + ":" + expectedValues[i], processed.get(i)); + } + } + +} From 1b9fb5e3df8f50aef565b8943e8fbf874d107e82 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 30 Jun 2015 14:54:18 -0700 Subject: [PATCH 014/275] added KStreamBranchTest --- .../streaming/internal/KStreamBranchTest.java | 109 ++++++++++++++++++ .../streaming/internal/KStreamSourceTest.java | 71 +++++------- .../streaming/internal/TestProcessor.java | 26 +++++ 3 files changed, 162 insertions(+), 44 deletions(-) create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java create mode 100644 src/test/java/io/confluent/streaming/internal/TestProcessor.java diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java new file mode 100644 index 0000000000000..03fe74b9ae0e5 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -0,0 +1,109 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import java.lang.reflect.Array; + +import static org.junit.Assert.assertEquals; + +public class KStreamBranchTest { + + private Ingestor ingestor = new Ingestor() { + @Override + public void poll() {} + + @Override + public void poll(long timeoutMs) {} + + @Override + public void pause(TopicPartition partition) {} + + @Override + public void unpause(TopicPartition partition, long offset) {} + }; + + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; + } + }, + 10 + ); + + private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + @SuppressWarnings("unchecked") + @Test + public void testKStreamBranch() { + + Predicate isEven = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 2) == 0; + } + }; + Predicate isMultipleOfThree = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 3) == 0; + } + }; + Predicate isOdd = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 2) != 0; + } + }; + + final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; + + KStreamSource stream; + KStream[] branches; + TestProcessor[] processors; + + stream = new KStreamSource(partitioningInfo, null); + branches = stream.branch(isEven, isMultipleOfThree, isOdd); + + assertEquals(3, branches.length); + + processors = (TestProcessor[]) Array.newInstance(TestProcessor.class, branches.length); + for (int i = 0; i < branches.length; i++) { + processors[i] = new TestProcessor(); + branches[i].process(processors[i]); + } + + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(3, processors[0].processed.size()); + assertEquals(1, processors[1].processed.size()); + assertEquals(3, processors[2].processed.size()); + + stream = new KStreamSource(partitioningInfo, null); + branches = stream.branch(isEven, isOdd, isMultipleOfThree); + + assertEquals(3, branches.length); + + processors = (TestProcessor[]) Array.newInstance(TestProcessor.class, branches.length); + for (int i = 0; i < branches.length; i++) { + processors[i] = new TestProcessor(); + branches[i].process(processors[i]); + } + + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(3, processors[0].processed.size()); + assertEquals(4, processors[1].processed.size()); + assertEquals(0, processors[2].processed.size()); + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 86418bc6cfbe3..4400a78502eaa 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -7,57 +7,40 @@ import org.apache.kafka.common.TopicPartition; import org.junit.Test; -import java.util.ArrayList; - public class KStreamSourceTest { - @Test - public void testKStreamSource() { - - Ingestor ingestor = new Ingestor() { - - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} + private Ingestor ingestor = new Ingestor() { + @Override + public void poll() {} - @Override - public void pause(TopicPartition partition) {} + @Override + public void poll(long timeoutMs) {} - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + @Override + public void pause(TopicPartition partition) {} - StreamSynchronizer streamSynchronizer = new StreamSynchronizer( - "group", - ingestor, - new ChooserImpl(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { - return 0L; - } - }, - 10 - ); + @Override + public void unpause(TopicPartition partition, long offset) {} + }; - PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); - - final ArrayList processed = new ArrayList(); - - Processor processor = new Processor() { - - @Override - public void apply(String key, String value) { - processed.add(key + ":" + value); + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; } + }, + 10 + ); - @Override - public void init(PunctuationScheduler punctuationScheduler) {} + private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + @Test + public void testKStreamSource() { - @Override - public void punctuate(long streamTime) {} - }; + TestProcessor processor = new TestProcessor(); KStreamSource stream = new KStreamSource(partitioningInfo, null); stream.process(processor); @@ -69,10 +52,10 @@ public void punctuate(long streamTime) {} stream.receive(expectedKeys[i], expectedValues[i], 0L); } - assertEquals(3, processed.size()); + assertEquals(3, processor.processed.size()); for (int i = 0; i < expectedKeys.length; i++) { - assertEquals(expectedKeys[i] + ":" + expectedValues[i], processed.get(i)); + assertEquals(expectedKeys[i] + ":" + expectedValues[i], processor.processed.get(i)); } } diff --git a/src/test/java/io/confluent/streaming/internal/TestProcessor.java b/src/test/java/io/confluent/streaming/internal/TestProcessor.java new file mode 100644 index 0000000000000..a5951de6a4074 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/TestProcessor.java @@ -0,0 +1,26 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.Processor; +import io.confluent.streaming.PunctuationScheduler; + +import java.util.ArrayList; + +/** + * Created by yasuhiro on 6/30/15. + */ +class TestProcessor implements Processor { + public final ArrayList processed = new ArrayList(); + + @Override + public void apply(K key, V value) { + processed.add(key + ":" + value); + } + + @Override + public void init(PunctuationScheduler punctuationScheduler) { + } + + @Override + public void punctuate(long streamTime) { + } +} From 464e29ec1b765ac73aae1f4933fc7da39deb96c0 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 30 Jun 2015 16:10:06 -0700 Subject: [PATCH 015/275] more tests --- .../java/io/confluent/streaming/Window.java | 2 - .../io/confluent/streaming/WindowByTime.java | 4 +- .../streaming/util/FilteredIterator.java | 1 + .../streaming/internal/KStreamFilterTest.java | 84 ++++++++++++ .../internal/KStreamFlatMapTest.java | 78 +++++++++++ .../internal/KStreamFlatMapValuesTest.java | 77 +++++++++++ .../streaming/internal/KStreamMapTest.java | 74 ++++++++++ .../internal/KStreamMapValuesTest.java | 77 +++++++++++ .../internal/KStreamWindowedTest.java | 129 ++++++++++++++++++ .../streaming/internal/UnlimitedWindow.java | 34 +++++ 10 files changed, 556 insertions(+), 4 deletions(-) create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamMapTest.java create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java create mode 100644 src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java diff --git a/src/main/java/io/confluent/streaming/Window.java b/src/main/java/io/confluent/streaming/Window.java index 9b886e566f985..c6232d3ec9a7f 100644 --- a/src/main/java/io/confluent/streaming/Window.java +++ b/src/main/java/io/confluent/streaming/Window.java @@ -11,6 +11,4 @@ public interface Window { void put(K key, V value, long timestamp); - void flush(); - } diff --git a/src/main/java/io/confluent/streaming/WindowByTime.java b/src/main/java/io/confluent/streaming/WindowByTime.java index c2273feb0e69b..f02aeffc6012a 100644 --- a/src/main/java/io/confluent/streaming/WindowByTime.java +++ b/src/main/java/io/confluent/streaming/WindowByTime.java @@ -22,6 +22,7 @@ public WindowByTime(long duration, int maxCount) { this.maxCount = maxCount; } + @Override public Iterator find(K key, final long timestamp) { final LinkedList> values = map.get(key); @@ -41,6 +42,7 @@ protected V filter(Stamped item) { } } + @Override public void put(K key, V value, long timestamp) { list.offerLast(key); @@ -86,6 +88,4 @@ private void evictExpired(long cutoffTime) { } } - public void flush() {} - } diff --git a/src/main/java/io/confluent/streaming/util/FilteredIterator.java b/src/main/java/io/confluent/streaming/util/FilteredIterator.java index e79def2e68c13..76fe098780415 100644 --- a/src/main/java/io/confluent/streaming/util/FilteredIterator.java +++ b/src/main/java/io/confluent/streaming/util/FilteredIterator.java @@ -42,6 +42,7 @@ private void findNext() { return; } } + nextValue = null; } protected abstract T filter(S item); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java new file mode 100644 index 0000000000000..7919774eeddcf --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -0,0 +1,84 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import java.lang.reflect.Array; + +import static org.junit.Assert.assertEquals; + +public class KStreamFilterTest { + + private Ingestor ingestor = new Ingestor() { + @Override + public void poll() {} + + @Override + public void poll(long timeoutMs) {} + + @Override + public void pause(TopicPartition partition) {} + + @Override + public void unpause(TopicPartition partition, long offset) {} + }; + + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; + } + }, + 10 + ); + + private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + private Predicate isMultipleOfThree = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 3) == 0; + } + }; + + @Test + public void testFilter() { + final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; + + KStreamSource stream; + TestProcessor processor; + + processor = new TestProcessor(); + stream = new KStreamSource(partitioningInfo, null); + stream.filter(isMultipleOfThree).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(2, processor.processed.size()); + } + + @Test + public void testFilterOut() { + final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; + + KStreamSource stream; + TestProcessor processor; + + processor = new TestProcessor(); + stream = new KStreamSource(partitioningInfo, null); + stream.filterOut(isMultipleOfThree).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(5, processor.processed.size()); + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java new file mode 100644 index 0000000000000..a02e3104e4c78 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -0,0 +1,78 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +public class KStreamFlatMapTest { + + private Ingestor ingestor = new Ingestor() { + @Override + public void poll() {} + + @Override + public void poll(long timeoutMs) {} + + @Override + public void pause(TopicPartition partition) {} + + @Override + public void unpause(TopicPartition partition, long offset) {} + }; + + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; + } + }, + 10 + ); + + private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + @Test + public void testFlatMap() { + + KeyValueMapper, Integer, String> mapper = + new KeyValueMapper, Integer, String>() { + @Override + public KeyValue> apply(Integer key, String value) { + ArrayList result = new ArrayList(); + for (int i = 0; i < key; i++) { + result.add(value); + } + return KeyValue.pair(Integer.toString(key * 10), (Iterable)result); + } + }; + + final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + + KStreamSource stream; + TestProcessor processor; + + processor = new TestProcessor(); + stream = new KStreamSource(partitioningInfo, null); + stream.flatMap(mapper).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(6, processor.processed.size()); + + String[] expected = new String[] { "10:V1", "20:V2", "20:V2", "30:V3", "30:V3", "30:V3" }; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java new file mode 100644 index 0000000000000..6f50e3a482934 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -0,0 +1,77 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +public class KStreamFlatMapValuesTest { + + private Ingestor ingestor = new Ingestor() { + @Override + public void poll() {} + + @Override + public void poll(long timeoutMs) {} + + @Override + public void pause(TopicPartition partition) {} + + @Override + public void unpause(TopicPartition partition, long offset) {} + }; + + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; + } + }, + 10 + ); + + private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + @Test + public void testFlatMapValues() { + + ValueMapper, String> mapper = + new ValueMapper, String>() { + @Override + public Iterable apply(String value) { + ArrayList result = new ArrayList(); + result.add(value.toLowerCase()); + result.add(value); + return result; + } + }; + + final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + + KStreamSource stream; + TestProcessor processor; + + processor = new TestProcessor(); + stream = new KStreamSource(partitioningInfo, null); + stream.flatMapValues(mapper).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(8, processor.processed.size()); + + String[] expected = new String[] { "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)); + } + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java new file mode 100644 index 0000000000000..315a35a97928e --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -0,0 +1,74 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +public class KStreamMapTest { + + private Ingestor ingestor = new Ingestor() { + @Override + public void poll() {} + + @Override + public void poll(long timeoutMs) {} + + @Override + public void pause(TopicPartition partition) {} + + @Override + public void unpause(TopicPartition partition, long offset) {} + }; + + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; + } + }, + 10 + ); + + private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + @Test + public void testMap() { + + KeyValueMapper mapper = + new KeyValueMapper() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(value, key); + } + }; + + final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + + KStreamSource stream; + TestProcessor processor; + + processor = new TestProcessor(); + stream = new KStreamSource(partitioningInfo, null); + stream.map(mapper).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(4, processor.processed.size()); + + String[] expected = new String[] { "V0:0", "V1:1", "V2:2", "V3:3" }; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java new file mode 100644 index 0000000000000..c5195daf2ff6f --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -0,0 +1,77 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.StreamSynchronizer; +import io.confluent.streaming.SyncGroup; +import io.confluent.streaming.TimestampExtractor; +import io.confluent.streaming.ValueMapper; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +public class KStreamMapValuesTest { + + private Ingestor ingestor = new Ingestor() { + @Override + public void poll() {} + + @Override + public void poll(long timeoutMs) {} + + @Override + public void pause(TopicPartition partition) {} + + @Override + public void unpause(TopicPartition partition, long offset) {} + }; + + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; + } + }, + 10 + ); + + private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + @Test + public void testFlatMapValues() { + + ValueMapper mapper = + new ValueMapper() { + @Override + public Integer apply(String value) { + return value.length(); + } + }; + + final int[] expectedKeys = new int[] { 1, 10, 100, 1000 }; + + KStreamSource stream; + TestProcessor processor; + + processor = new TestProcessor(); + stream = new KStreamSource(partitioningInfo, null); + stream.mapValues(mapper).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L); + } + + assertEquals(4, processor.processed.size()); + + String[] expected = new String[] { "1:1", "10:2", "100:3", "1000:4" }; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java new file mode 100644 index 0000000000000..2d11fa275f2ae --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -0,0 +1,129 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class KStreamWindowedTest { + + private Ingestor ingestor = new Ingestor() { + @Override + public void poll() {} + + @Override + public void poll(long timeoutMs) {} + + @Override + public void pause(TopicPartition partition) {} + + @Override + public void unpause(TopicPartition partition, long offset) {} + }; + + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; + } + }, + 10 + ); + + private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + @Test + public void testNestedLoop() { + + KeyValueMapper mapper = + new KeyValueMapper() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(value, key); + } + }; + + ValueJoiner joiner = new ValueJoiner() { + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } + }; + + final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + + KStreamSource stream1; + KStreamSource stream2; + KStreamWindowed windowed; + TestProcessor processor; + String[] expected; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + windowed = stream2.with(new UnlimitedWindow()); + + boolean exceptionRaised = false; + + try { + stream1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + // empty window + + for (int i = 0; i < expectedKeys.length; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + } + + assertEquals(0, processor.processed.size()); + + // two items in the window + + for (int i = 0; i < 2; i++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + } + + for (int i = 0; i < expectedKeys.length; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + } + + 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(); + + // previous two items + all items, thus two are duplicates, in the window + + for (int i = 0; i < expectedKeys.length; i++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + } + + for (int i = 0; i < expectedKeys.length; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + } + + 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)); + } + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java b/src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java new file mode 100644 index 0000000000000..fdbd3c648f03a --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java @@ -0,0 +1,34 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KeyValue; +import io.confluent.streaming.Window; +import io.confluent.streaming.util.FilteredIterator; + +import java.util.Iterator; +import java.util.LinkedList; + +/** + * Created by yasuhiro on 6/30/15. + */ +class UnlimitedWindow implements Window { + + private LinkedList> list = new LinkedList>(); + + @Override + public Iterator find(final K key, long timestamp) { + return new FilteredIterator>(list.iterator()) { + protected V filter(KeyValue item) { + if (item.key.equals(key)) + return item.value; + else + return null; + } + }; + } + + @Override + public void put(K key, V value, long timestamp) { + list.add(KeyValue.pair(key, value)); + } + +} From f628706761bb176bedd2641e0f5ce43ef551734e Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 30 Jun 2015 16:29:01 -0700 Subject: [PATCH 016/275] more tests --- .../streaming/internal/KStreamJoinTest.java | 132 ++++++++++++++++++ .../internal/KStreamNestedLoopTest.java | 121 ++++++++++++++++ .../internal/KStreamWindowedTest.java | 90 ++++-------- 3 files changed, 278 insertions(+), 65 deletions(-) create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java new file mode 100644 index 0000000000000..5ab1cf69260c5 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -0,0 +1,132 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class KStreamJoinTest { + + private Ingestor ingestor = new Ingestor() { + @Override + public void poll() {} + + @Override + public void poll(long timeoutMs) {} + + @Override + public void pause(TopicPartition partition) {} + + @Override + public void unpause(TopicPartition partition, long offset) {} + }; + + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; + } + }, + 10 + ); + + private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + @Test + public void testJoin() { + + ValueJoiner joiner = new ValueJoiner() { + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } + }; + + final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + + KStreamSource stream1; + KStreamSource stream2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + TestProcessor processor; + String[] expected; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); + + boolean exceptionRaised = false; + + try { + windowed1.join(windowed2, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + // push two items to the main stream. the other stream's window is empty + + for (int i = 0; i < 2; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + } + + 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++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + } + + 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++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + } + + 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++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + } + + 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)); + } + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java new file mode 100644 index 0000000000000..f6107e8ac3d06 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java @@ -0,0 +1,121 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class KStreamNestedLoopTest { + + private Ingestor ingestor = new Ingestor() { + @Override + public void poll() {} + + @Override + public void poll(long timeoutMs) {} + + @Override + public void pause(TopicPartition partition) {} + + @Override + public void unpause(TopicPartition partition, long offset) {} + }; + + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + ingestor, + new ChooserImpl(), + new TimestampExtractor() { + public long extract(String topic, String key, String value) { + return 0L; + } + }, + 10 + ); + + private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + + @Test + public void testNestedLoop() { + + ValueJoiner joiner = new ValueJoiner() { + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } + }; + + final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + + KStreamSource stream1; + KStreamSource stream2; + KStreamWindowed windowed; + TestProcessor processor; + String[] expected; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + windowed = stream2.with(new UnlimitedWindow()); + + boolean exceptionRaised = false; + + try { + stream1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + // empty window + + for (int i = 0; i < expectedKeys.length; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + } + + assertEquals(0, processor.processed.size()); + + // two items in the window + + for (int i = 0; i < 2; i++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + } + + for (int i = 0; i < expectedKeys.length; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + } + + 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(); + + // previous two items + all items, thus two are duplicates, in the window + + for (int i = 0; i < expectedKeys.length; i++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + } + + for (int i = 0; i < expectedKeys.length; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + } + + 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)); + } + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 2d11fa275f2ae..08e0edf0695d8 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -4,6 +4,8 @@ import org.apache.kafka.common.TopicPartition; import org.junit.Test; +import java.util.Iterator; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -38,92 +40,50 @@ public long extract(String topic, String key, String value) { private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); @Test - public void testNestedLoop() { - - KeyValueMapper mapper = - new KeyValueMapper() { - @Override - public KeyValue apply(Integer key, String value) { - return KeyValue.pair(value, key); - } - }; - - ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; - } - }; + public void testWindowedStream() { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - KStreamSource stream1; - KStreamSource stream2; - KStreamWindowed windowed; - TestProcessor processor; + KStreamSource stream; + Window window; String[] expected; - processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); - windowed = stream2.with(new UnlimitedWindow()); + window = new UnlimitedWindow(); + stream = new KStreamSource(partitioningInfo, null); + stream.with(window); boolean exceptionRaised = false; - try { - stream1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - - // empty window - - for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); - } - - assertEquals(0, processor.processed.size()); - // two items in the window for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); - } - - for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); } - 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)); - } + 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))); - processor.processed.clear(); - // previous two items + all items, thus two are duplicates, in the window for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); - } - - for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); } - assertEquals(6, processor.processed.size()); + 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))); + } - 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)); + private int countItem(Iterator iter) { + int i = 0; + while (iter.hasNext()) { + i++; + iter.next(); } + return i; } - } From 14c13ea8f07b89225a6e66344fc3f18f0b07e3c0 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 09:04:03 -0700 Subject: [PATCH 017/275] use Ingestor interface in StreamSycnhronizer and KStreamContextImpl --- .../io/confluent/streaming/StreamSynchronizerFactory.java | 6 +++--- .../io/confluent/streaming/internal/KStreamContextImpl.java | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java b/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java index c8c006589091f..ba6e12e780093 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java @@ -1,7 +1,7 @@ package io.confluent.streaming; import io.confluent.streaming.internal.ChooserImpl; -import io.confluent.streaming.internal.IngestorImpl; +import io.confluent.streaming.internal.Ingestor; /** * Created by yasuhiro on 6/24/15. @@ -14,8 +14,8 @@ public StreamSynchronizerFactory(TimestampExtractor timestampExtractor) { this.timestampExtractor = timestampExtractor; } - public StreamSynchronizer create(String name, IngestorImpl consumer, int desiredNumberOfUnprocessedRecords) { - return new StreamSynchronizer(name, consumer, new ChooserImpl(), timestampExtractor, desiredNumberOfUnprocessedRecords); + public StreamSynchronizer create(String name, Ingestor ingestor, int desiredNumberOfUnprocessedRecords) { + return new StreamSynchronizer(name, ingestor, new ChooserImpl(), timestampExtractor, desiredNumberOfUnprocessedRecords); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 8b4955228c862..db738c64a3b59 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -25,7 +25,7 @@ public class KStreamContextImpl implements KStreamContext { public final int id; private final Set topics; - private final IngestorImpl ingestor; + private final Ingestor ingestor; private final RecordCollectors.SimpleRecordCollector simpleCollector; private final RecordCollector collector; @@ -44,7 +44,7 @@ public class KStreamContextImpl implements KStreamContext { @SuppressWarnings("unchecked") public KStreamContextImpl(int id, - IngestorImpl ingestor, + Ingestor ingestor, Producer producer, Coordinator coordinator, StreamingConfig streamingConfig, @@ -53,7 +53,7 @@ public KStreamContextImpl(int id, Metrics metrics) { this.id = id; this.topics = streamingConfig.topics(); - this.ingestor = (IngestorImpl) ingestor; + this.ingestor = ingestor; this.simpleCollector = new RecordCollectors.SimpleRecordCollector(producer); this.collector = new RecordCollectors.SerializingRecordCollector( From 653b35b9ca60003b08937765ed53763234dcee91 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 09:51:42 -0700 Subject: [PATCH 018/275] more join tests --- .../io/confluent/streaming/util/Util.java | 2 +- .../streaming/internal/KStreamJoinTest.java | 273 +++++++++++++++++- .../internal/KStreamNestedLoopTest.java | 257 ++++++++++++++++- 3 files changed, 515 insertions(+), 17 deletions(-) diff --git a/src/main/java/io/confluent/streaming/util/Util.java b/src/main/java/io/confluent/streaming/util/Util.java index 734785a5ca2e0..d477888b1e6ad 100644 --- a/src/main/java/io/confluent/streaming/util/Util.java +++ b/src/main/java/io/confluent/streaming/util/Util.java @@ -8,7 +8,7 @@ */ public class Util { - static HashSet mkSet(T... elems) { + public static HashSet mkSet(T... elems) { HashSet set = new HashSet(); for (T e : elems) set.add(e); return set; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 5ab1cf69260c5..72993a4b0d006 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -1,11 +1,13 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.util.Util; import org.apache.kafka.common.TopicPartition; import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; public class KStreamJoinTest { @@ -37,17 +39,47 @@ public long extract(String topic, String key, String value) { private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); - @Test - public void testJoin() { + private ValueJoiner joiner = new ValueJoiner() { + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } + }; - ValueJoiner joiner = new ValueJoiner() { + private ValueMapper valueMapper = new ValueMapper() { + @Override + public String apply(String value) { + return "#" + value; + } + }; + + private ValueMapper, String> valueMapper2 = new ValueMapper, String>() { + @Override + public Iterable apply(String value) { + return (Iterable) Util.mkSet(value); + } + }; + + private KeyValueMapper keyValueMapper = + new KeyValueMapper() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(key, value); + } + }; + + KeyValueMapper, Integer, String> keyValueMapper2 = + new KeyValueMapper, Integer, String>() { @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; + public KeyValue> apply(Integer key, String value) { + return KeyValue.pair(key, (Iterable) Util.mkSet(value)); } }; - final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + @Test + public void testJoin() { + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; KStreamSource stream1; KStreamSource stream2; @@ -66,8 +98,7 @@ public String apply(String value1, String value2) { try { windowed1.join(windowed2, joiner).process(processor); - } - catch (NotCopartitionedException e) { + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -89,7 +120,7 @@ public String apply(String value1, String value2) { assertEquals(2, processor.processed.size()); - expected = new String[] { "0:X0+Y0", "1:X1+Y1" }; + expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; for (int i = 0; i < expected.length; i++) { assertEquals(expected[i], processor.processed.get(i)); @@ -105,7 +136,7 @@ public String apply(String value1, String value2) { assertEquals(2, processor.processed.size()); - expected = new String[] { "0:X0+Y0", "1:X1+Y1" }; + expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; for (int i = 0; i < expected.length; i++) { assertEquals(expected[i], processor.processed.get(i)); @@ -122,11 +153,231 @@ public String apply(String value1, String value2) { 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" }; + 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)); } } + @Test + public void testMap() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + TestProcessor processor; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + mapped1 = stream1.map(keyValueMapper); + mapped2 = stream2.map(keyValueMapper); + + boolean exceptionRaised; + + try { + exceptionRaised = false; + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + + try { + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + + try { + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + } + + @Test + public void testFlatMap() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + TestProcessor processor; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + mapped1 = stream1.flatMap(keyValueMapper2); + mapped2 = stream2.flatMap(keyValueMapper2); + + boolean exceptionRaised; + + try { + exceptionRaised = false; + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + + try { + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + + try { + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + } + + @Test + public void testMapValues() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + TestProcessor processor; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + mapped1 = stream1.mapValues(valueMapper); + mapped2 = stream2.mapValues(valueMapper); + + boolean exceptionRaised; + + try { + exceptionRaised = false; + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + try { + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + try { + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + } + + @Test + public void testFlatMapValues() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + TestProcessor processor; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + mapped1 = stream1.flatMapValues(valueMapper2); + mapped2 = stream2.flatMapValues(valueMapper2); + + boolean exceptionRaised; + + try { + exceptionRaised = false; + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + try { + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + try { + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + } + } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java index f6107e8ac3d06..42dcf7012352e 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java @@ -1,11 +1,13 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.util.Util; import org.apache.kafka.common.TopicPartition; import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; public class KStreamNestedLoopTest { @@ -37,16 +39,46 @@ public long extract(String topic, String key, String value) { private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); - @Test - public void testNestedLoop() { - ValueJoiner joiner = new ValueJoiner() { + private ValueJoiner joiner = new ValueJoiner() { + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } + }; + + private ValueMapper valueMapper = new ValueMapper() { + @Override + public String apply(String value) { + return "#" + value; + } + }; + + private ValueMapper, String> valueMapper2 = new ValueMapper, String>() { + @Override + public Iterable apply(String value) { + return (Iterable) Util.mkSet(value); + } + }; + + private KeyValueMapper keyValueMapper = + new KeyValueMapper() { @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(key, value); } }; + KeyValueMapper, Integer, String> keyValueMapper2 = + new KeyValueMapper, Integer, String>() { + @Override + public KeyValue> apply(Integer key, String value) { + return KeyValue.pair(key, (Iterable) Util.mkSet(value)); + } + }; + + @Test + public void testNestedLoop() { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; KStreamSource stream1; @@ -118,4 +150,219 @@ public String apply(String value1, String value2) { } } + @Test + public void testMap() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed; + TestProcessor processor; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + mapped1 = stream1.map(keyValueMapper); + mapped2 = stream2.map(keyValueMapper); + + boolean exceptionRaised; + + try { + exceptionRaised = false; + windowed = stream2.with(new UnlimitedWindow()); + + mapped1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + + try { + exceptionRaised = false; + windowed = mapped2.with(new UnlimitedWindow()); + + stream1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + + try { + exceptionRaised = false; + windowed = mapped2.with(new UnlimitedWindow()); + + mapped1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + } + + @Test + public void testFlatMap() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed; + TestProcessor processor; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + mapped1 = stream1.flatMap(keyValueMapper2); + mapped2 = stream2.flatMap(keyValueMapper2); + + boolean exceptionRaised; + + try { + exceptionRaised = false; + windowed = stream2.with(new UnlimitedWindow()); + + mapped1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + + try { + exceptionRaised = false; + windowed = mapped2.with(new UnlimitedWindow()); + + stream1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + + try { + exceptionRaised = false; + windowed = mapped2.with(new UnlimitedWindow()); + + mapped1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + } + + @Test + public void testMapValues() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed; + TestProcessor processor; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + mapped1 = stream1.mapValues(valueMapper); + mapped2 = stream2.mapValues(valueMapper); + + boolean exceptionRaised; + + try { + exceptionRaised = false; + windowed = stream2.with(new UnlimitedWindow()); + + mapped1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + try { + exceptionRaised = false; + windowed = mapped2.with(new UnlimitedWindow()); + + stream1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + try { + exceptionRaised = false; + windowed = mapped2.with(new UnlimitedWindow()); + + mapped1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + } + + @Test + public void testFlatMapValues() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed; + TestProcessor processor; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + mapped1 = stream1.flatMapValues(valueMapper2); + mapped2 = stream2.flatMapValues(valueMapper2); + + boolean exceptionRaised; + + try { + exceptionRaised = false; + windowed = stream2.with(new UnlimitedWindow()); + + mapped1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + try { + exceptionRaised = false; + windowed = mapped2.with(new UnlimitedWindow()); + + stream1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + try { + exceptionRaised = false; + windowed = mapped2.with(new UnlimitedWindow()); + + mapped1.nestedLoop(windowed, joiner).process(processor); + } + catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + } } From 607404b87cd63db9bac7fc6aca3b30b8dad266bc Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 10:16:28 -0700 Subject: [PATCH 019/275] test FilteredIterator --- .../streaming/util/FilteredIteratorTest.java | 77 +++++++++++++++++++ 1 file changed, 77 insertions(+) create mode 100644 src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java diff --git a/src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java b/src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java new file mode 100644 index 0000000000000..097bf749189b7 --- /dev/null +++ b/src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java @@ -0,0 +1,77 @@ +package io.confluent.streaming.util; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class FilteredIteratorTest { + + @Test + public void testFiltering() { + List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 3 == 0) return i.toString(); + return null; + } + }; + + List expected = Arrays.asList("3", "9", "6", "3"); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + + @Test + public void testEmptySource() { + List list = Arrays.asList(); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 3 == 0) return i.toString(); + return null; + } + }; + + List expected = Arrays.asList(); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + + @Test + public void testNoMatch() { + List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 7 == 0) return i.toString(); + return null; + } + }; + + List expected = Arrays.asList(); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + +} From 785876bd87c43854bb7a29eb0268ae377be210d3 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 10:48:34 -0700 Subject: [PATCH 020/275] test MinTimestampTracker --- .../streaming/util/MinTimestampTracker.java | 8 +- .../streaming/util/TimestampTracker.java | 6 +- .../util/MinTimestampTrackerTest.java | 78 +++++++++++++++++++ 3 files changed, 88 insertions(+), 4 deletions(-) create mode 100644 src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java diff --git a/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java b/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java index 5f5e50e2330ca..42e62531ee40e 100644 --- a/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java +++ b/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java @@ -3,15 +3,19 @@ import java.util.LinkedList; /** - * MinTimestampTracker maintains the minimum timestamp of stamped elements that were added but not yet removed. + * MinTimestampTracker is a helper class for a sliding window implementation. + * It is assumed that elements are added or removed in a FIFO manner. + * It maintains the minimum timestamp of stamped elements that were added but not yet removed. */ public class MinTimestampTracker implements TimestampTracker { private final LinkedList> descendingSubsequence = new LinkedList>(); public void addStampedElement(Stamped elem) { + if (elem == null) throw new NullPointerException(); + Stamped minElem = descendingSubsequence.peekLast(); - while (minElem.compareTo(elem) >= 0) { + while (minElem != null && minElem.compareTo(elem) >= 0) { descendingSubsequence.removeLast(); minElem = descendingSubsequence.peekLast(); } diff --git a/src/main/java/io/confluent/streaming/util/TimestampTracker.java b/src/main/java/io/confluent/streaming/util/TimestampTracker.java index 612da6b82d9f8..87169bf2c7989 100644 --- a/src/main/java/io/confluent/streaming/util/TimestampTracker.java +++ b/src/main/java/io/confluent/streaming/util/TimestampTracker.java @@ -1,7 +1,9 @@ package io.confluent.streaming.util; /** - * TimestampTracker maintains the timestamp, like the min timestamp, the max timestamp, etc. of stamped elements + * TimestampTracker is a helper class for a sliding window implementation. + * It is assumed that elements are added or removed in a FIFO manner. + * It maintains the timestamp, like the min timestamp, the max timestamp, etc. of stamped elements * that were added but not yet removed. */ public interface TimestampTracker { @@ -20,7 +22,7 @@ public interface TimestampTracker { /** * Returns the timestamp - * @return + * @return timestamp, or -1L when empty */ long get(); diff --git a/src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java b/src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java new file mode 100644 index 0000000000000..d0f84fe24bc4f --- /dev/null +++ b/src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java @@ -0,0 +1,78 @@ +package io.confluent.streaming.util; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +import java.lang.reflect.Array; + +public class MinTimestampTrackerTest { + + private Stamped elem(long timestamp) { + return new Stamped("", timestamp); + } + + @SuppressWarnings("unchecked") + @Test + public void testTracking() { + MinTimestampTracker 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.addStampedElement((Stamped)elems[insertionIndex++]); + assertEquals(100L, tracker.get()); + + // add 101 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); + + // remove 100 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(101L, tracker.get()); + + // add 102 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(101L, tracker.get()); + + // add 98 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // add 99 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // add 100 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // remove 101 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); + + // remove 102 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); + + // remove 98 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(99L, tracker.get()); + + // remove 99 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(100L, tracker.get()); + + // remove 100 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(-1L, tracker.get()); + + assertEquals(insertionIndex, removalIndex); + } + +} From d2ff445f1533d8a599b24db07f87c7e1d260fb2b Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 13:47:38 -0700 Subject: [PATCH 021/275] more test, and removed StreamSynchronizerFactory --- .../confluent/streaming/KStreamContext.java | 2 ++ .../streaming/StreamSynchronizer.java | 4 +-- .../streaming/StreamSynchronizerFactory.java | 21 ------------ .../confluent/streaming/StreamingConfig.java | 10 +++--- .../internal/KStreamContextImpl.java | 20 ++++++++--- .../streaming/internal/RoundRobinChooser.java | 34 +++++++++++++++++++ ...ChooserImpl.java => TimeBasedChooser.java} | 6 ++-- .../streaming/internal/KStreamBranchTest.java | 2 +- .../streaming/internal/KStreamFilterTest.java | 4 +-- .../internal/KStreamFlatMapTest.java | 2 +- .../internal/KStreamFlatMapValuesTest.java | 2 +- .../streaming/internal/KStreamJoinTest.java | 2 +- .../streaming/internal/KStreamMapTest.java | 4 +-- .../internal/KStreamMapValuesTest.java | 4 +-- .../internal/KStreamNestedLoopTest.java | 2 +- .../streaming/internal/KStreamSourceTest.java | 2 +- .../internal/KStreamWindowedTest.java | 2 +- .../streaming/util/FilteredIteratorTest.java | 6 ++-- .../util/MinTimestampTrackerTest.java | 4 +-- 19 files changed, 76 insertions(+), 57 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java create mode 100644 src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java rename src/main/java/io/confluent/streaming/internal/{ChooserImpl.java => TimeBasedChooser.java} (84%) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index a656bb726de82..8908560d1f7a3 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -34,6 +34,8 @@ public interface KStreamContext { SyncGroup syncGroup(String name); + SyncGroup roundRobinSyncGroup(String name); + void restore(StorageEngine engine) throws Exception; void schedule(Processor processor, long time); diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/StreamSynchronizer.java index ecb5d642fbe5e..8937e9265a6c1 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/StreamSynchronizer.java @@ -30,12 +30,12 @@ public StreamSynchronizer(String name, Ingestor ingestor, Chooser chooser, TimestampExtractor timestampExtractor, - int desiredNumberOfUnprocessedRecords) { + int desiredUnprocessedPerPartition) { this.name = name; this.ingestor = ingestor; this.chooser = chooser; this.timestampExtractor = timestampExtractor; - this.desiredUnprocessed = desiredNumberOfUnprocessedRecords; + this.desiredUnprocessed = desiredUnprocessedPerPartition; this.consumedOffsets = new HashMap(); } diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java b/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java deleted file mode 100644 index ba6e12e780093..0000000000000 --- a/src/main/java/io/confluent/streaming/StreamSynchronizerFactory.java +++ /dev/null @@ -1,21 +0,0 @@ -package io.confluent.streaming; - -import io.confluent.streaming.internal.ChooserImpl; -import io.confluent.streaming.internal.Ingestor; - -/** - * Created by yasuhiro on 6/24/15. - */ -public class StreamSynchronizerFactory { - - private TimestampExtractor timestampExtractor; - - public StreamSynchronizerFactory(TimestampExtractor timestampExtractor) { - this.timestampExtractor = timestampExtractor; - } - - public StreamSynchronizer create(String name, Ingestor ingestor, int desiredNumberOfUnprocessedRecords) { - return new StreamSynchronizer(name, ingestor, new ChooserImpl(), timestampExtractor, desiredNumberOfUnprocessedRecords); - } - -} diff --git a/src/main/java/io/confluent/streaming/StreamingConfig.java b/src/main/java/io/confluent/streaming/StreamingConfig.java index e4bf4ed191ae2..bb98dd4c6fb4c 100644 --- a/src/main/java/io/confluent/streaming/StreamingConfig.java +++ b/src/main/java/io/confluent/streaming/StreamingConfig.java @@ -60,7 +60,7 @@ public class StreamingConfig { private Serializer valSerializer; private Deserializer keyDeserializer; private Deserializer valDeserializer; - private StreamSynchronizerFactory streamSynchronizerFactory; + private TimestampExtractor timestampExtractor; public StreamingConfig(Properties config) { this.config = config; @@ -129,11 +129,11 @@ public Deserializer valueDeserializer() { return this.valDeserializer; } - public void streamSynchronizerFactory(StreamSynchronizerFactory streamSynchronizerFactory) { - this.streamSynchronizerFactory = streamSynchronizerFactory; + public void timestampExtractor(TimestampExtractor streamSynchronizerFactory) { + this.timestampExtractor = streamSynchronizerFactory; } - public StreamSynchronizerFactory streamSynchronizerFactory() { - return this.streamSynchronizerFactory; + public TimestampExtractor timestampExtractor() { + return this.timestampExtractor; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index db738c64a3b59..65e43b28fc959 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -32,7 +32,7 @@ public class KStreamContextImpl implements KStreamContext { private final Coordinator coordinator; private final HashMap> sourceStreams = new HashMap>(); private final HashMap partitioningInfos = new HashMap(); - private final StreamSynchronizerFactory streamSynchronizerFactory; + private final TimestampExtractor timestampExtractor; private final HashMap syncGroups = new HashMap(); private final StreamingConfig streamingConfig; private final ProcessorConfig processorConfig; @@ -63,8 +63,8 @@ public KStreamContextImpl(int id, this.streamingConfig = streamingConfig; this.processorConfig = processorConfig; - this.streamSynchronizerFactory = (StreamSynchronizerFactory)this.streamingConfig.streamSynchronizerFactory(); - if (this.streamSynchronizerFactory == null) throw new NullPointerException(); + this.timestampExtractor = (TimestampExtractor)this.streamingConfig.timestampExtractor(); + if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractoris missing"); this.stateDir = stateDir; this.state = new ProcessorStateManager(id, stateDir); @@ -155,11 +155,22 @@ public Metrics metrics() { @Override public SyncGroup syncGroup(String name) { + return syncGroup(name, new TimeBasedChooser()); + } + + @Override + public SyncGroup roundRobinSyncGroup(String name) { + return syncGroup(name, new RoundRobinChooser()); + } + + private SyncGroup syncGroup(String name, Chooser chooser) { + int desiredUnprocessedPerPartition = processorConfig.bufferedRecordsPerPartition; + synchronized (this) { SyncGroup syncGroup = syncGroups.get(name); if (syncGroup == null) { StreamSynchronizer streamSynchronizer = - streamSynchronizerFactory.create(name, ingestor, processorConfig.bufferedRecordsPerPartition); + new StreamSynchronizer(name, ingestor, chooser, timestampExtractor, desiredUnprocessedPerPartition); syncGroup = new SyncGroup(name, streamSynchronizer); syncGroups.put(name, syncGroup); } @@ -167,6 +178,7 @@ public SyncGroup syncGroup(String name) { } } + @Override public void restore(StorageEngine engine) throws Exception { if (restoreConsumer == null) throw new IllegalStateException(); diff --git a/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java b/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java new file mode 100644 index 0000000000000..34a38583d0b92 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java @@ -0,0 +1,34 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.Chooser; +import io.confluent.streaming.RecordQueue; + +import java.util.ArrayDeque; + +/** + * Created by yasuhiro on 6/25/15. + */ +public class RoundRobinChooser implements Chooser { + + private final ArrayDeque> deque; + + public RoundRobinChooser() { + deque = new ArrayDeque>(); + } + + @Override + public void add(RecordQueue queue) { + deque.offer(queue); + } + + @Override + public RecordQueue next() { + return deque.poll(); + } + + @Override + public void close() { + deque.clear(); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/ChooserImpl.java b/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java similarity index 84% rename from src/main/java/io/confluent/streaming/internal/ChooserImpl.java rename to src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java index 36418db04bd63..4f8c97a0d74d0 100644 --- a/src/main/java/io/confluent/streaming/internal/ChooserImpl.java +++ b/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java @@ -9,11 +9,11 @@ /** * Created by yasuhiro on 6/25/15. */ -public class ChooserImpl implements Chooser { +public class TimeBasedChooser implements Chooser { private final PriorityQueue> pq; - public ChooserImpl() { + public TimeBasedChooser() { this(new Comparator>() { public int compare(RecordQueue queue1, RecordQueue queue2) { long time1 = queue1.currentStreamTime(); @@ -26,7 +26,7 @@ public int compare(RecordQueue queue1, RecordQueue queue2) { }); } - public ChooserImpl(Comparator> comparator) { + private TimeBasedChooser(Comparator> comparator) { pq = new PriorityQueue>(3, comparator); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index 03fe74b9ae0e5..e05070920c67c 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -27,7 +27,7 @@ public void unpause(TopicPartition partition, long offset) {} private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new ChooserImpl(), + new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, String key, String value) { return 0L; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 7919774eeddcf..f90a6f80e4259 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -4,8 +4,6 @@ import org.apache.kafka.common.TopicPartition; import org.junit.Test; -import java.lang.reflect.Array; - import static org.junit.Assert.assertEquals; public class KStreamFilterTest { @@ -27,7 +25,7 @@ public void unpause(TopicPartition partition, long offset) {} private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new ChooserImpl(), + new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, String key, String value) { return 0L; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index a02e3104e4c78..a746baf4965dc 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -27,7 +27,7 @@ public void unpause(TopicPartition partition, long offset) {} private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new ChooserImpl(), + new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, String key, String value) { return 0L; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 6f50e3a482934..63fc60d4f14af 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -27,7 +27,7 @@ public void unpause(TopicPartition partition, long offset) {} private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new ChooserImpl(), + new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, String key, String value) { return 0L; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 72993a4b0d006..7037dc22501c3 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -28,7 +28,7 @@ public void unpause(TopicPartition partition, long offset) {} private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new ChooserImpl(), + new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, String key, String value) { return 0L; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 315a35a97928e..7112016ca026b 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -4,8 +4,6 @@ import org.apache.kafka.common.TopicPartition; import org.junit.Test; -import java.util.ArrayList; - import static org.junit.Assert.assertEquals; public class KStreamMapTest { @@ -27,7 +25,7 @@ public void unpause(TopicPartition partition, long offset) {} private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new ChooserImpl(), + new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, String key, String value) { return 0L; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index c5195daf2ff6f..1bcf2cb86c41a 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -7,8 +7,6 @@ import org.apache.kafka.common.TopicPartition; import org.junit.Test; -import java.util.ArrayList; - import static org.junit.Assert.assertEquals; public class KStreamMapValuesTest { @@ -30,7 +28,7 @@ public void unpause(TopicPartition partition, long offset) {} private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new ChooserImpl(), + new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, String key, String value) { return 0L; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java index 42dcf7012352e..04969b9cee78c 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java @@ -28,7 +28,7 @@ public void unpause(TopicPartition partition, long offset) {} private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new ChooserImpl(), + new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, String key, String value) { return 0L; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 4400a78502eaa..e93a69ef6f123 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -26,7 +26,7 @@ public void unpause(TopicPartition partition, long offset) {} private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new ChooserImpl(), + new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, String key, String value) { return 0L; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 08e0edf0695d8..4bcd9163ad10d 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -28,7 +28,7 @@ public void unpause(TopicPartition partition, long offset) {} private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new ChooserImpl(), + new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, String key, String value) { return 0L; diff --git a/src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java b/src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java index 097bf749189b7..ab54b0728e7fc 100644 --- a/src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java +++ b/src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java @@ -34,7 +34,7 @@ protected String filter(Integer i) { @Test public void testEmptySource() { - List list = Arrays.asList(); + List list = new ArrayList(); Iterator filtered = new FilteredIterator(list.iterator()) { protected String filter(Integer i) { @@ -43,7 +43,7 @@ protected String filter(Integer i) { } }; - List expected = Arrays.asList(); + List expected = new ArrayList(); List result = new ArrayList(); while (filtered.hasNext()) { @@ -64,7 +64,7 @@ protected String filter(Integer i) { } }; - List expected = Arrays.asList(); + List expected = new ArrayList(); List result = new ArrayList(); while (filtered.hasNext()) { diff --git a/src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java b/src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java index d0f84fe24bc4f..8476092a5a49c 100644 --- a/src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java +++ b/src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java @@ -4,12 +4,10 @@ import org.junit.Test; -import java.lang.reflect.Array; - public class MinTimestampTrackerTest { private Stamped elem(long timestamp) { - return new Stamped("", timestamp); + return new Stamped("", timestamp); } @SuppressWarnings("unchecked") From 5374030b1074be70ed7426c4a3f6af08089ce95a Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 14:07:49 -0700 Subject: [PATCH 022/275] removed KStreamContext.steamingConfig() method --- .../confluent/streaming/KStreamContext.java | 15 ++++++++-- .../internal/KStreamContextImpl.java | 28 +++++++++++++++---- .../streaming/internal/KStreamImpl.java | 3 +- .../streaming/kv/InMemoryKeyValueStore.java | 8 +++--- 4 files changed, 40 insertions(+), 14 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 8908560d1f7a3..d000f9f8aee68 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -1,6 +1,8 @@ package io.confluent.streaming; 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.Map; @@ -14,11 +16,17 @@ public interface KStreamContext { int id(); - StreamingConfig streamingConfig(); + Serializer keySerializer(); - KStream from(String topic); + Serializer valueSerializer(); - KStream from(String topic, SyncGroup syncGroup); + Deserializer keyDeserializer(); + + Deserializer valueDeserializer(); + + KStream from(String topic); + + KStream from(String topic, SyncGroup syncGroup); RecordCollector simpleRecordCollector(); @@ -39,4 +47,5 @@ public interface KStreamContext { void restore(StorageEngine engine) throws Exception; void schedule(Processor processor, long time); + } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 65e43b28fc959..8bba4075d4aef 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -5,6 +5,7 @@ import org.apache.kafka.clients.producer.Producer; 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.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,25 +79,40 @@ public int id() { } @Override - public StreamingConfig streamingConfig() { - return streamingConfig; + public Serializer keySerializer() { + return streamingConfig.keySerializer(); } @Override - public KStream from(String topic) { + public Serializer valueSerializer() { + return streamingConfig.valueSerializer(); + } + + @Override + public Deserializer keyDeserializer() { + return streamingConfig.keyDeserializer(); + } + + @Override + public Deserializer valueDeserializer() { + return streamingConfig.valueDeserializer(); + } + + @Override + public KStream from(String topic) { return from(topic, syncGroup(DEFAULT_SYNCHRONIZATION_GROUP)); } @Override @SuppressWarnings("unchecked") - public KStream from(String topic, SyncGroup syncGroup) { + public KStream from(String topic, SyncGroup syncGroup) { if (syncGroup == null) throw new NullPointerException(); synchronized (this) { if (!topics.contains(topic)) throw new IllegalArgumentException("topic not subscribed: " + topic); - KStreamSource stream = (KStreamSource)sourceStreams.get(topic); + KStreamSource stream = sourceStreams.get(topic); if (stream == null) { PartitioningInfo partitioningInfo = partitioningInfos.get(topic); @@ -109,7 +125,7 @@ public KStream from(String topic, SyncGroup syncGroup) { partitioningInfos.put(topic, partitioningInfo); } - stream = new KStreamSource(partitioningInfo, this); + stream = new KStreamSource(partitioningInfo, this); sourceStreams.put(topic, stream); syncGroup.streamSynchronizer.addPartition(new TopicPartition(topic, id), (Receiver)stream); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index b123385378a16..5dc488c45f623 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -77,10 +77,11 @@ public KStream[] branch(Predicate... predicates) { return branch.branches; } + @SuppressWarnings("unchecked") @Override public KStream through(String topic) { process(this.getSendProcessor(topic)); - return context.from(topic); + return (KStream)context.from(topic); } @Override diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index 003bc4c795f38..8a51f61ebfc3a 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -47,10 +47,10 @@ public MemoryStore(String name, KStreamContext context) { this.dirty = new HashSet(); this.collector = null; this.maxDirty = 100; - this.keySerializer = (Serializer) context.streamingConfig().keySerializer(); - this.valueSerializer = (Serializer) context.streamingConfig().valueSerializer(); - this.keyDeserializer = (Deserializer) context.streamingConfig().keyDeserializer(); - this.valueDeserializer = (Deserializer) context.streamingConfig().valueDeserializer(); + this.keySerializer = (Serializer) context.keySerializer(); + this.valueSerializer = (Serializer) context.valueSerializer(); + this.keyDeserializer = (Deserializer) context.keyDeserializer(); + this.valueDeserializer = (Deserializer) context.valueDeserializer(); } @Override From b57800714fd889240452241f76054988d45768f8 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 14:17:21 -0700 Subject: [PATCH 023/275] parameter name --- src/main/java/io/confluent/streaming/StreamingConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/io/confluent/streaming/StreamingConfig.java b/src/main/java/io/confluent/streaming/StreamingConfig.java index bb98dd4c6fb4c..1f650cb9f97fb 100644 --- a/src/main/java/io/confluent/streaming/StreamingConfig.java +++ b/src/main/java/io/confluent/streaming/StreamingConfig.java @@ -129,8 +129,8 @@ public Deserializer valueDeserializer() { return this.valDeserializer; } - public void timestampExtractor(TimestampExtractor streamSynchronizerFactory) { - this.timestampExtractor = streamSynchronizerFactory; + public void timestampExtractor(TimestampExtractor timestampExtractor) { + this.timestampExtractor = timestampExtractor; } public TimestampExtractor timestampExtractor() { From e53fe8472f7cff2ba27b0194366f95bab062a38a Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 14:33:38 -0700 Subject: [PATCH 024/275] added Ingestor.numPartition(topic) method --- .../streaming/internal/Ingestor.java | 2 ++ .../streaming/internal/IngestorImpl.java | 9 +++++++ .../internal/KStreamContextImpl.java | 5 +--- .../streaming/internal/KStreamBranchTest.java | 15 +---------- .../streaming/internal/KStreamFilterTest.java | 15 +---------- .../internal/KStreamFlatMapTest.java | 15 +---------- .../internal/KStreamFlatMapValuesTest.java | 15 +---------- .../streaming/internal/KStreamJoinTest.java | 15 +---------- .../streaming/internal/KStreamMapTest.java | 15 +---------- .../internal/KStreamMapValuesTest.java | 15 +---------- .../internal/KStreamNestedLoopTest.java | 15 +---------- .../streaming/internal/KStreamSourceTest.java | 15 +---------- .../internal/KStreamWindowedTest.java | 16 +----------- .../streaming/internal/NoopIngestor.java | 26 +++++++++++++++++++ .../streaming/internal/TestProcessor.java | 3 --- .../streaming/internal/UnlimitedWindow.java | 3 --- 16 files changed, 48 insertions(+), 151 deletions(-) create mode 100644 src/test/java/io/confluent/streaming/internal/NoopIngestor.java diff --git a/src/main/java/io/confluent/streaming/internal/Ingestor.java b/src/main/java/io/confluent/streaming/internal/Ingestor.java index c76ac23a98fe4..571e3b49022ae 100644 --- a/src/main/java/io/confluent/streaming/internal/Ingestor.java +++ b/src/main/java/io/confluent/streaming/internal/Ingestor.java @@ -15,4 +15,6 @@ public interface Ingestor { void unpause(TopicPartition partition, long offset); + int numPartitions(String topic); + } diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 34a146d89b96b..20bf54b32072e 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -36,10 +36,12 @@ public void init() { unpaused.addAll(consumer.subscriptions()); } + @Override public void poll() { poll(pollTimeMs); } + @Override public void poll(long timeoutMs) { for (TopicPartition partition : toBePaused) { doPause(partition); @@ -53,6 +55,7 @@ public void poll(long timeoutMs) { } } + @Override public void pause(TopicPartition partition) { toBePaused.add(partition); } @@ -62,11 +65,17 @@ private void doPause(TopicPartition partition) { unpaused.remove(partition); } + @Override public void unpause(TopicPartition partition, long lastOffset) { consumer.seek(partition, lastOffset); unpaused.add(partition); } + @Override + public int numPartitions(String topic) { + return consumer.partitionsFor(topic).size(); + } + public void clear() { unpaused.clear(); toBePaused.clear(); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 8bba4075d4aef..7f2a3412ae4c8 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -118,10 +118,7 @@ public Deserializer valueDeserializer() { PartitioningInfo partitioningInfo = partitioningInfos.get(topic); if (partitioningInfo == null) { - // TODO: use the consumer to get partitioning info - int numPartitions = 1; - - partitioningInfo = new PartitioningInfo(syncGroup, numPartitions); + partitioningInfo = new PartitioningInfo(syncGroup, ingestor.numPartitions(topic)); partitioningInfos.put(topic, partitioningInfo); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index e05070920c67c..aa0b5c3ac3865 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import org.apache.kafka.common.TopicPartition; import org.junit.Test; import java.lang.reflect.Array; @@ -10,19 +9,7 @@ public class KStreamBranchTest { - private Ingestor ingestor = new Ingestor() { - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} - - @Override - public void pause(TopicPartition partition) {} - - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + private Ingestor ingestor = new NoopIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index f90a6f80e4259..1bc36cc43c0c7 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -1,26 +1,13 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import org.apache.kafka.common.TopicPartition; import org.junit.Test; import static org.junit.Assert.assertEquals; public class KStreamFilterTest { - private Ingestor ingestor = new Ingestor() { - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} - - @Override - public void pause(TopicPartition partition) {} - - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + private Ingestor ingestor = new NoopIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index a746baf4965dc..ade9605513ca4 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import org.apache.kafka.common.TopicPartition; import org.junit.Test; import java.util.ArrayList; @@ -10,19 +9,7 @@ public class KStreamFlatMapTest { - private Ingestor ingestor = new Ingestor() { - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} - - @Override - public void pause(TopicPartition partition) {} - - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + private Ingestor ingestor = new NoopIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 63fc60d4f14af..df2330164515d 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import org.apache.kafka.common.TopicPartition; import org.junit.Test; import java.util.ArrayList; @@ -10,19 +9,7 @@ public class KStreamFlatMapValuesTest { - private Ingestor ingestor = new Ingestor() { - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} - - @Override - public void pause(TopicPartition partition) {} - - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + private Ingestor ingestor = new NoopIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 7037dc22501c3..595ed457475ba 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -2,7 +2,6 @@ import io.confluent.streaming.*; import io.confluent.streaming.util.Util; -import org.apache.kafka.common.TopicPartition; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -11,19 +10,7 @@ public class KStreamJoinTest { - private Ingestor ingestor = new Ingestor() { - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} - - @Override - public void pause(TopicPartition partition) {} - - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + private Ingestor ingestor = new NoopIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 7112016ca026b..f872a1e95515f 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -1,26 +1,13 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import org.apache.kafka.common.TopicPartition; import org.junit.Test; import static org.junit.Assert.assertEquals; public class KStreamMapTest { - private Ingestor ingestor = new Ingestor() { - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} - - @Override - public void pause(TopicPartition partition) {} - - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + private Ingestor ingestor = new NoopIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 1bcf2cb86c41a..1410e539f4364 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -4,26 +4,13 @@ import io.confluent.streaming.SyncGroup; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.ValueMapper; -import org.apache.kafka.common.TopicPartition; import org.junit.Test; import static org.junit.Assert.assertEquals; public class KStreamMapValuesTest { - private Ingestor ingestor = new Ingestor() { - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} - - @Override - public void pause(TopicPartition partition) {} - - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + private Ingestor ingestor = new NoopIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java index 04969b9cee78c..53ae212ab9821 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java @@ -2,7 +2,6 @@ import io.confluent.streaming.*; import io.confluent.streaming.util.Util; -import org.apache.kafka.common.TopicPartition; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -11,19 +10,7 @@ public class KStreamNestedLoopTest { - private Ingestor ingestor = new Ingestor() { - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} - - @Override - public void pause(TopicPartition partition) {} - - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + private Ingestor ingestor = new NoopIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index e93a69ef6f123..cf557fd46f5c3 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -4,24 +4,11 @@ import io.confluent.streaming.*; -import org.apache.kafka.common.TopicPartition; import org.junit.Test; public class KStreamSourceTest { - private Ingestor ingestor = new Ingestor() { - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} - - @Override - public void pause(TopicPartition partition) {} - - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + private Ingestor ingestor = new NoopIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 4bcd9163ad10d..79e8244a500eb 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -1,29 +1,15 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import org.apache.kafka.common.TopicPartition; import org.junit.Test; import java.util.Iterator; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; public class KStreamWindowedTest { - private Ingestor ingestor = new Ingestor() { - @Override - public void poll() {} - - @Override - public void poll(long timeoutMs) {} - - @Override - public void pause(TopicPartition partition) {} - - @Override - public void unpause(TopicPartition partition, long offset) {} - }; + private Ingestor ingestor = new NoopIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/NoopIngestor.java b/src/test/java/io/confluent/streaming/internal/NoopIngestor.java new file mode 100644 index 0000000000000..fc58a196a9a25 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/NoopIngestor.java @@ -0,0 +1,26 @@ +package io.confluent.streaming.internal; + +import org.apache.kafka.common.TopicPartition; + +class NoopIngestor implements Ingestor { + @Override + public void poll() { + } + + @Override + public void poll(long timeoutMs) { + } + + @Override + public void pause(TopicPartition partition) { + } + + @Override + public void unpause(TopicPartition partition, long offset) { + } + + @Override + public int numPartitions(String topic) { + return 1; + } +} diff --git a/src/test/java/io/confluent/streaming/internal/TestProcessor.java b/src/test/java/io/confluent/streaming/internal/TestProcessor.java index a5951de6a4074..889c829d816ba 100644 --- a/src/test/java/io/confluent/streaming/internal/TestProcessor.java +++ b/src/test/java/io/confluent/streaming/internal/TestProcessor.java @@ -5,9 +5,6 @@ import java.util.ArrayList; -/** - * Created by yasuhiro on 6/30/15. - */ class TestProcessor implements Processor { public final ArrayList processed = new ArrayList(); diff --git a/src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java b/src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java index fdbd3c648f03a..090e318b77708 100644 --- a/src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java +++ b/src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java @@ -7,9 +7,6 @@ import java.util.Iterator; import java.util.LinkedList; -/** - * Created by yasuhiro on 6/30/15. - */ class UnlimitedWindow implements Window { private LinkedList> list = new LinkedList>(); From 7501f0c7274db73b70de7d62b13d20b9e5864c3a Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 15:09:08 -0700 Subject: [PATCH 025/275] removed unused member variables --- .../confluent/streaming/KafkaStreaming.java | 14 ++++++--- .../internal/KStreamContextImpl.java | 31 ++++++++++--------- 2 files changed, 25 insertions(+), 20 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 317e24a22defc..d95e6115d7c0e 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -330,14 +330,18 @@ public void shutdown(Coordinator.RequestScope scope) { } }; - File stateDir = new File(config.stateDir, id.toString()); - kstreamContext = - new KStreamContextImpl(id, ingestor, producer, coordinator, streamingConfig, config, stateDir, metrics); - - kstreamContext.init(restoreConsumer,job); + new KStreamContextImpl(id, ingestor, producer, coordinator, streamingConfig, config, metrics); kstreamContexts.put(id, kstreamContext); + + try { + kstreamContext.init(restoreConsumer, job); + } + catch (Exception e) { + throw new KafkaException(e); + } + syncGroups.put(id, kstreamContext.syncGroups()); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 7f2a3412ae4c8..5341e30c4b54d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -11,6 +11,7 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.io.IOException; import java.util.Collection; import java.util.HashMap; import java.util.Map; @@ -39,8 +40,7 @@ public class KStreamContextImpl implements KStreamContext { private final ProcessorConfig processorConfig; private final Metrics metrics; private final File stateDir; - private final ProcessorStateManager state; - private final Map stores = new HashMap(); + private final ProcessorStateManager stateMgr; private Consumer restoreConsumer; @SuppressWarnings("unchecked") @@ -50,7 +50,6 @@ public KStreamContextImpl(int id, Coordinator coordinator, StreamingConfig streamingConfig, ProcessorConfig processorConfig, - File stateDir, Metrics metrics) { this.id = id; this.topics = streamingConfig.topics(); @@ -67,9 +66,8 @@ public KStreamContextImpl(int id, this.timestampExtractor = (TimestampExtractor)this.streamingConfig.timestampExtractor(); if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractoris missing"); - this.stateDir = stateDir; - this.state = new ProcessorStateManager(id, stateDir); - + this.stateDir = new File(processorConfig.stateDir, Integer.toString(id)); + this.stateMgr = new ProcessorStateManager(id, stateDir); this.metrics = metrics; } @@ -196,7 +194,7 @@ private SyncGroup syncGroup(String name, Chooser chooser) { public void restore(StorageEngine engine) throws Exception { if (restoreConsumer == null) throw new IllegalStateException(); - state.registerAndRestore(simpleCollector, restoreConsumer, engine); + stateMgr.registerAndRestore(simpleCollector, restoreConsumer, engine); } @@ -209,20 +207,23 @@ public Collection syncGroups() { return syncGroups.values(); } - public void init(Consumer restoreConsumer, KStreamJob job) { - this.restoreConsumer = restoreConsumer; - - job.build(this); - - this.restoreConsumer = null; + public void init(Consumer restoreConsumer, KStreamJob job) throws IOException { + stateMgr.init(); + try { + this.restoreConsumer = restoreConsumer; + job.build(this); + } + finally { + this.restoreConsumer = null; + } } public void flush() { - state.flush(); + stateMgr.flush(); } public void close() throws Exception { - state.close(simpleCollector.offsets()); + stateMgr.close(simpleCollector.offsets()); } } From 8f918eda3a9c097e80601350e7e86ca921c28574 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 16:02:48 -0700 Subject: [PATCH 026/275] more test --- .../streaming/PunctuationScheduler.java | 6 +- .../streaming/internal/PunctuationQueue.java | 2 +- .../internal/PunctuationSchedulerImpl.java | 7 +- .../streaming/internal/KStreamBranchTest.java | 2 + .../streaming/internal/KStreamFilterTest.java | 2 + .../internal/KStreamFlatMapTest.java | 2 + .../internal/KStreamFlatMapValuesTest.java | 2 + .../streaming/internal/KStreamJoinTest.java | 3 + .../streaming/internal/KStreamMapTest.java | 2 + .../internal/KStreamMapValuesTest.java | 2 + .../internal/KStreamNestedLoopTest.java | 3 + .../streaming/internal/KStreamSourceTest.java | 2 + .../internal/KStreamWindowedTest.java | 2 + .../PunctuationSchedulerImplTest.java | 125 ++++++++++++++++++ .../{internal => testutil}/NoopIngestor.java | 5 +- .../{internal => testutil}/TestProcessor.java | 6 +- .../UnlimitedWindow.java | 4 +- 17 files changed, 161 insertions(+), 16 deletions(-) create mode 100644 src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java rename src/test/java/io/confluent/streaming/{internal => testutil}/NoopIngestor.java (72%) rename src/test/java/io/confluent/streaming/{internal => testutil}/TestProcessor.java (68%) rename src/test/java/io/confluent/streaming/{internal => testutil}/UnlimitedWindow.java (87%) diff --git a/src/main/java/io/confluent/streaming/PunctuationScheduler.java b/src/main/java/io/confluent/streaming/PunctuationScheduler.java index cec3fd312f775..a79a4b746581c 100644 --- a/src/main/java/io/confluent/streaming/PunctuationScheduler.java +++ b/src/main/java/io/confluent/streaming/PunctuationScheduler.java @@ -1,13 +1,11 @@ package io.confluent.streaming; -import io.confluent.streaming.internal.PunctuationQueue; - /** - * Created by yasuhiro on 6/29/15. + * An interface that allows {@link Processor} to schedule a notification at a specified stream time. */ public interface PunctuationScheduler { - void schedule(long time); + void schedule(long timestamp); void cancel(); diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java b/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java index 262271a0eb471..4f32d8002526d 100644 --- a/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java +++ b/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java @@ -35,7 +35,7 @@ public void close() { public void mayPunctuate(long streamTime) { synchronized (pq) { Stamped top = pq.peek(); - while (top.timestamp <= streamTime) { + while (top != null && top.timestamp <= streamTime) { PunctuationSchedulerImpl scheduler = top.value; pq.poll(); scheduler.processor.punctuate(streamTime); diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java b/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java index d617f0caf2a10..5df8ec4b7a845 100644 --- a/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java +++ b/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java @@ -4,9 +4,6 @@ import io.confluent.streaming.PunctuationScheduler; import io.confluent.streaming.util.Stamped; -/** - * Created by yasuhiro on 6/29/15. - */ public class PunctuationSchedulerImpl implements PunctuationScheduler { private Stamped scheduled; @@ -19,12 +16,12 @@ public PunctuationSchedulerImpl(PunctuationQueue queue, Processor processo } @Override - public void schedule(long time) { + public void schedule(long timestamp) { synchronized (this) { if (scheduled != null) throw new IllegalStateException("punctuation is already scheduled"); - scheduled = queue.schedule(this, time); + scheduled = queue.schedule(this, timestamp); } } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index aa0b5c3ac3865..b486c3db6e241 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -1,6 +1,8 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; import java.lang.reflect.Array; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 1bc36cc43c0c7..911e618ae55ff 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -1,6 +1,8 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; import static org.junit.Assert.assertEquals; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index ade9605513ca4..114453a407f3c 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -1,6 +1,8 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; import java.util.ArrayList; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index df2330164515d..fced0ba5eda1a 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -1,6 +1,8 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; import java.util.ArrayList; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 595ed457475ba..1a420eceb705b 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -1,6 +1,9 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.TestProcessor; +import io.confluent.streaming.testutil.UnlimitedWindow; import io.confluent.streaming.util.Util; import org.junit.Test; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index f872a1e95515f..6b34d696904e3 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -1,6 +1,8 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; import static org.junit.Assert.assertEquals; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 1410e539f4364..7685f670e8bfb 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -4,6 +4,8 @@ import io.confluent.streaming.SyncGroup; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.ValueMapper; +import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; import static org.junit.Assert.assertEquals; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java index 53ae212ab9821..32cae0367494f 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java @@ -1,6 +1,9 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.TestProcessor; +import io.confluent.streaming.testutil.UnlimitedWindow; import io.confluent.streaming.util.Util; import org.junit.Test; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index cf557fd46f5c3..5aac8e9abe58c 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -4,6 +4,8 @@ import io.confluent.streaming.*; +import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; public class KStreamSourceTest { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 79e8244a500eb..d1cc012d11166 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -1,6 +1,8 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.UnlimitedWindow; import org.junit.Test; import java.util.Iterator; diff --git a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java new file mode 100644 index 0000000000000..94358aed7887d --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java @@ -0,0 +1,125 @@ +package io.confluent.streaming.internal; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import io.confluent.streaming.PunctuationScheduler; +import io.confluent.streaming.testutil.TestProcessor; +import org.junit.Test; + +public class PunctuationSchedulerImplTest { + + @Test + public void testScheduling() { + PunctuationQueue queue = new PunctuationQueue(); + + TestProcessor proc1 = new TestProcessor(); + PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); + + TestProcessor proc2 = new TestProcessor(); + PunctuationScheduler sched2 = new PunctuationSchedulerImpl(queue, proc2); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + + proc1.init(sched1); + proc2.init(sched2); + + sched1.schedule(500); + sched2.schedule(1000); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + + queue.mayPunctuate(999); + + assertEquals(1, proc1.punctuated.size()); + assertEquals(999, (long) proc1.punctuated.get(0)); + assertEquals(0, proc2.punctuated.size()); + + proc1.punctuated.clear(); + queue.mayPunctuate(1000); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(1, proc2.punctuated.size()); + assertEquals(1000, (long) proc2.punctuated.get(0)); + + proc2.punctuated.clear(); + queue.mayPunctuate(2000); + assertEquals(0, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + + sched1.schedule(3000); + queue.mayPunctuate(4000); + + assertEquals(1, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + assertEquals(4000, (long) proc1.punctuated.get(0)); + } + + @Test + public void testCanceling() { + PunctuationQueue queue = new PunctuationQueue(); + + TestProcessor proc1 = new TestProcessor(); + PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); + + TestProcessor proc2 = new TestProcessor(); + PunctuationScheduler sched2 = new PunctuationSchedulerImpl(queue, proc2); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + + proc1.init(sched1); + proc2.init(sched2); + + sched1.schedule(500); + sched2.schedule(1000); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + + sched1.cancel(); + + queue.mayPunctuate(1000); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(1, proc2.punctuated.size()); + assertEquals(1000, (long) proc2.punctuated.get(0)); + + sched1.schedule(2000); + sched1.cancel(); + sched1.schedule(3000); + + queue.mayPunctuate(2000); + assertEquals(0, proc1.punctuated.size()); + + queue.mayPunctuate(3000); + assertEquals(1, proc1.punctuated.size()); + } + + @Test + public void testDuplicateScheduling() { + PunctuationQueue queue = new PunctuationQueue(); + + TestProcessor proc1 = new TestProcessor(); + PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); + + assertEquals(0, proc1.punctuated.size()); + + proc1.init(sched1); + + sched1.schedule(500); + + boolean exceptionRaised = false; + try { + sched1.schedule(1000); + } + catch (IllegalStateException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/NoopIngestor.java b/src/test/java/io/confluent/streaming/testutil/NoopIngestor.java similarity index 72% rename from src/test/java/io/confluent/streaming/internal/NoopIngestor.java rename to src/test/java/io/confluent/streaming/testutil/NoopIngestor.java index fc58a196a9a25..3df6fd4628389 100644 --- a/src/test/java/io/confluent/streaming/internal/NoopIngestor.java +++ b/src/test/java/io/confluent/streaming/testutil/NoopIngestor.java @@ -1,8 +1,9 @@ -package io.confluent.streaming.internal; +package io.confluent.streaming.testutil; +import io.confluent.streaming.internal.Ingestor; import org.apache.kafka.common.TopicPartition; -class NoopIngestor implements Ingestor { +public class NoopIngestor implements Ingestor { @Override public void poll() { } diff --git a/src/test/java/io/confluent/streaming/internal/TestProcessor.java b/src/test/java/io/confluent/streaming/testutil/TestProcessor.java similarity index 68% rename from src/test/java/io/confluent/streaming/internal/TestProcessor.java rename to src/test/java/io/confluent/streaming/testutil/TestProcessor.java index 889c829d816ba..aa8fcbbcaf242 100644 --- a/src/test/java/io/confluent/streaming/internal/TestProcessor.java +++ b/src/test/java/io/confluent/streaming/testutil/TestProcessor.java @@ -1,12 +1,13 @@ -package io.confluent.streaming.internal; +package io.confluent.streaming.testutil; import io.confluent.streaming.Processor; import io.confluent.streaming.PunctuationScheduler; import java.util.ArrayList; -class TestProcessor implements Processor { +public class TestProcessor implements Processor { public final ArrayList processed = new ArrayList(); + public final ArrayList punctuated = new ArrayList(); @Override public void apply(K key, V value) { @@ -19,5 +20,6 @@ public void init(PunctuationScheduler punctuationScheduler) { @Override public void punctuate(long streamTime) { + punctuated.add(streamTime); } } diff --git a/src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java b/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java similarity index 87% rename from src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java rename to src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java index 090e318b77708..fb4a1d5c2306c 100644 --- a/src/test/java/io/confluent/streaming/internal/UnlimitedWindow.java +++ b/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package io.confluent.streaming.testutil; import io.confluent.streaming.KeyValue; import io.confluent.streaming.Window; @@ -7,7 +7,7 @@ import java.util.Iterator; import java.util.LinkedList; -class UnlimitedWindow implements Window { +public class UnlimitedWindow implements Window { private LinkedList> list = new LinkedList>(); From a222567a8ce220b13986f7366c9819a39c9d8ebb Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 16:15:50 -0700 Subject: [PATCH 027/275] moved StreamSynchronizer to internal --- .../java/io/confluent/streaming/SyncGroup.java | 4 +++- .../confluent/streaming/internal/IngestorImpl.java | 1 - .../{ => internal}/StreamSynchronizer.java | 14 +++++++------- .../streaming/internal/KStreamMapValuesTest.java | 1 - 4 files changed, 10 insertions(+), 10 deletions(-) rename src/main/java/io/confluent/streaming/{ => internal}/StreamSynchronizer.java (93%) diff --git a/src/main/java/io/confluent/streaming/SyncGroup.java b/src/main/java/io/confluent/streaming/SyncGroup.java index d29954c6e88e3..c7bc6fb846b0d 100644 --- a/src/main/java/io/confluent/streaming/SyncGroup.java +++ b/src/main/java/io/confluent/streaming/SyncGroup.java @@ -1,7 +1,9 @@ package io.confluent.streaming; +import io.confluent.streaming.internal.StreamSynchronizer; + /** - * Created by yasuhiro on 6/26/15. + * SyncGroup represents a group of streams synchronized together. */ public class SyncGroup { diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 20bf54b32072e..0a67ec25da7b6 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -1,6 +1,5 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.StreamSynchronizer; import io.confluent.streaming.util.FilteredIterator; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/src/main/java/io/confluent/streaming/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java similarity index 93% rename from src/main/java/io/confluent/streaming/StreamSynchronizer.java rename to src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index 8937e9265a6c1..01d264c8b72d7 100644 --- a/src/main/java/io/confluent/streaming/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -1,5 +1,6 @@ -package io.confluent.streaming; +package io.confluent.streaming.internal; +import io.confluent.streaming.*; import io.confluent.streaming.internal.*; import io.confluent.streaming.util.MinTimestampTracker; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -26,18 +27,17 @@ public class StreamSynchronizer { private long streamTime = -1; private volatile int buffered = 0; - public StreamSynchronizer(String name, - Ingestor ingestor, - Chooser chooser, - TimestampExtractor timestampExtractor, - int desiredUnprocessedPerPartition) { + StreamSynchronizer(String name, + Ingestor ingestor, + Chooser chooser, + TimestampExtractor timestampExtractor, + int desiredUnprocessedPerPartition) { this.name = name; this.ingestor = ingestor; this.chooser = chooser; this.timestampExtractor = timestampExtractor; this.desiredUnprocessed = desiredUnprocessedPerPartition; this.consumedOffsets = new HashMap(); - } public void addPartition(TopicPartition partition, final Receiver receiver) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 7685f670e8bfb..fd17bf1abc96a 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -1,6 +1,5 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.StreamSynchronizer; import io.confluent.streaming.SyncGroup; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.ValueMapper; From 37fd7bd34d684d7e3b945e0fccbdd9b19790732d Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 1 Jul 2015 16:32:31 -0700 Subject: [PATCH 028/275] moved Chooser and RecordQueue to internal --- .../io/confluent/streaming/RecordQueue.java | 22 ----- .../streaming/{ => internal}/Chooser.java | 4 +- ...{RecordQueueImpl.java => RecordQueue.java} | 7 +- .../streaming/internal/RoundRobinChooser.java | 3 - .../internal/StreamSynchronizer.java | 91 +++++-------------- .../streaming/internal/TimeBasedChooser.java | 3 - 6 files changed, 29 insertions(+), 101 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/RecordQueue.java rename src/main/java/io/confluent/streaming/{ => internal}/Chooser.java (64%) rename src/main/java/io/confluent/streaming/internal/{RecordQueueImpl.java => RecordQueue.java} (85%) diff --git a/src/main/java/io/confluent/streaming/RecordQueue.java b/src/main/java/io/confluent/streaming/RecordQueue.java deleted file mode 100644 index 59a962ab90347..0000000000000 --- a/src/main/java/io/confluent/streaming/RecordQueue.java +++ /dev/null @@ -1,22 +0,0 @@ -package io.confluent.streaming; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.TopicPartition; - -/** - * Created by yasuhiro on 6/25/15. - */ -public interface RecordQueue { - - TopicPartition partition(); - - void add(ConsumerRecord value, long timestamp); - - ConsumerRecord next(); - - long offset(); - - int size(); - - long currentStreamTime(); -} diff --git a/src/main/java/io/confluent/streaming/Chooser.java b/src/main/java/io/confluent/streaming/internal/Chooser.java similarity index 64% rename from src/main/java/io/confluent/streaming/Chooser.java rename to src/main/java/io/confluent/streaming/internal/Chooser.java index 995fcb2e40145..37a4a88b14c2d 100644 --- a/src/main/java/io/confluent/streaming/Chooser.java +++ b/src/main/java/io/confluent/streaming/internal/Chooser.java @@ -1,6 +1,4 @@ -package io.confluent.streaming; - -import org.apache.kafka.clients.consumer.ConsumerRecord; +package io.confluent.streaming.internal; /** * Created by yasuhiro on 6/25/15. diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java similarity index 85% rename from src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java rename to src/main/java/io/confluent/streaming/internal/RecordQueue.java index 7e3da315bb0a3..43bdb4f0ca1a1 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueueImpl.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -1,6 +1,5 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.RecordQueue; import io.confluent.streaming.util.Stamped; import io.confluent.streaming.util.TimestampTracker; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -12,15 +11,17 @@ /** * Created by yasuhiro on 6/25/15. */ -public class RecordQueueImpl implements RecordQueue { +public class RecordQueue { private final Deque>> queue = new ArrayDeque>>(); + public final Receiver receiver; private final TopicPartition partition; private TimestampTracker> timestampTracker; private long offset; - public RecordQueueImpl(TopicPartition partition, TimestampTracker> timestampTracker) { + public RecordQueue(TopicPartition partition, Receiver receiver, TimestampTracker> timestampTracker) { this.partition = partition; + this.receiver = receiver; this.timestampTracker = timestampTracker; } diff --git a/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java b/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java index 34a38583d0b92..3409a0ac9513a 100644 --- a/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java +++ b/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java @@ -1,8 +1,5 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.Chooser; -import io.confluent.streaming.RecordQueue; - import java.util.ArrayDeque; /** diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index 01d264c8b72d7..88927b6e1d022 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.internal.*; import io.confluent.streaming.util.MinTimestampTracker; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; @@ -19,7 +18,7 @@ public class StreamSynchronizer { private final Ingestor ingestor; private final Chooser chooser; private final TimestampExtractor timestampExtractor; - private final Map stash = new HashMap(); + private final Map> stash = new HashMap>(); private final int desiredUnprocessed; private final Map consumedOffsets; private final PunctuationQueue punctuationQueue = new PunctuationQueue(); @@ -40,34 +39,36 @@ public class StreamSynchronizer { this.consumedOffsets = new HashMap(); } - public void addPartition(TopicPartition partition, final Receiver receiver) { + @SuppressWarnings("unchecked") + public void addPartition(TopicPartition partition, Receiver receiver) { synchronized (this) { - RecordQueueWrapper queue = stash.get(partition); + RecordQueue recordQueue = stash.get(partition); - if (queue == null) { - stash.put(partition, new RecordQueueWrapper(createRecordQueue(partition), receiver)); + if (recordQueue == null) { + stash.put(partition, createRecordQueue(partition, (Receiver) receiver)); } else { throw new IllegalStateException("duplicate partition"); } } } + @SuppressWarnings("unchecked") public void addRecords(TopicPartition partition, Iterator> iterator) { synchronized (this) { - RecordQueueWrapper queue = stash.get(partition); - if (queue != null) { - boolean wasEmpty = (queue.size() == 0); + RecordQueue recordQueue = stash.get(partition); + if (recordQueue != null) { + boolean wasEmpty = (recordQueue.size() == 0); while (iterator.hasNext()) { ConsumerRecord record = iterator.next(); - queue.add(record, timestampExtractor.extract(record.topic(), record.key(), record.value())); + recordQueue.add(record, timestampExtractor.extract(record.topic(), record.key(), record.value())); buffered++; } - if (wasEmpty && queue.size() > 0) chooser.add(queue); + if (wasEmpty && recordQueue.size() > 0) chooser.add(recordQueue); // if we have buffered enough for this partition, pause - if (queue.size() > this.desiredUnprocessed) { + if (recordQueue.size() > this.desiredUnprocessed) { ingestor.pause(partition); } } @@ -78,9 +79,10 @@ public PunctuationScheduler getPunctuationScheduler(Processor processor) { return new PunctuationSchedulerImpl(punctuationQueue, processor); } + @SuppressWarnings("unchecked") public void process() { synchronized (this) { - RecordQueueWrapper recordQueue = (RecordQueueWrapper)chooser.next(); + RecordQueue recordQueue = chooser.next(); if (recordQueue == null) { ingestor.poll(); @@ -93,7 +95,13 @@ public void process() { if (recordQueue.size() == 0) return; - recordQueue.process(); + long timestamp = recordQueue.currentStreamTime(); + ConsumerRecord record = recordQueue.next(); + + if (streamTime < timestamp) streamTime = timestamp; + + recordQueue.receiver.receive(record.key(), record.value(), streamTime); + consumedOffsets.put(recordQueue.partition(), record.offset()); if (recordQueue.size() > 0) chooser.add(recordQueue); @@ -116,59 +124,8 @@ public void close() { stash.clear(); } - protected RecordQueue createRecordQueue(TopicPartition partition) { - return new RecordQueueImpl(partition, new MinTimestampTracker>()); - } - - private class RecordQueueWrapper implements RecordQueue { - - private final RecordQueue queue; - private final Receiver receiver; - - RecordQueueWrapper(RecordQueue queue, Receiver receiver) { - this.queue = queue; - this.receiver = receiver; - } - - void process() { - long timestamp = queue.currentStreamTime(); - ConsumerRecord record = queue.next(); - - if (streamTime < timestamp) streamTime = timestamp; - - receiver.receive(record.key(), record.value(), streamTime); - consumedOffsets.put(queue.partition(), record.offset()); - } - - @Override - public TopicPartition partition() { - return queue.partition(); - } - - @Override - public void add(ConsumerRecord value, long timestamp) { - queue.add(value, timestamp); - } - - public ConsumerRecord next() { - return queue.next(); - } - - @Override - public long offset() { - return queue.offset(); - } - - @Override - public int size() { - return queue.size(); - } - - @Override - public long currentStreamTime() { - return queue.currentStreamTime(); - } - + protected RecordQueue createRecordQueue(TopicPartition partition, Receiver receiver) { + return new RecordQueue(partition, receiver, new MinTimestampTracker>()); } } diff --git a/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java b/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java index 4f8c97a0d74d0..0823862127355 100644 --- a/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java +++ b/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java @@ -1,8 +1,5 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.Chooser; -import io.confluent.streaming.RecordQueue; - import java.util.Comparator; import java.util.PriorityQueue; From beb7d13a409638005242b9101686a6dff01c754e Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 2 Jul 2015 11:05:12 -0700 Subject: [PATCH 029/275] removed nestedLoop, added joinPrior --- .../java/io/confluent/streaming/KStream.java | 13 - .../confluent/streaming/KStreamWindowed.java | 14 + .../java/io/confluent/streaming/Window.java | 4 + .../io/confluent/streaming/WindowByTime.java | 19 +- .../streaming/internal/KStreamBranch.java | 4 +- .../streaming/internal/KStreamFilter.java | 4 +- .../streaming/internal/KStreamFlatMap.java | 4 +- .../internal/KStreamFlatMapValues.java | 4 +- .../streaming/internal/KStreamImpl.java | 17 +- .../streaming/internal/KStreamJoin.java | 53 ++- .../streaming/internal/KStreamMap.java | 4 +- .../streaming/internal/KStreamMapValues.java | 4 +- .../streaming/internal/KStreamNestedLoop.java | 32 -- .../streaming/internal/KStreamSource.java | 4 +- .../internal/KStreamWindowedImpl.java | 20 +- .../streaming/internal/Receiver.java | 2 +- .../streaming/internal/RecordQueue.java | 19 +- .../streaming/internal/StampedRecord.java | 23 ++ .../internal/StreamSynchronizer.java | 14 +- .../streaming/internal/KStreamBranchTest.java | 4 +- .../streaming/internal/KStreamFilterTest.java | 4 +- .../internal/KStreamFlatMapTest.java | 2 +- .../internal/KStreamFlatMapValuesTest.java | 2 +- .../streaming/internal/KStreamJoinTest.java | 8 +- .../streaming/internal/KStreamMapTest.java | 2 +- .../internal/KStreamMapValuesTest.java | 2 +- .../internal/KStreamNestedLoopTest.java | 358 ------------------ .../streaming/internal/KStreamSourceTest.java | 2 +- .../internal/KStreamWindowedTest.java | 4 +- .../streaming/testutil/UnlimitedWindow.java | 28 +- 30 files changed, 189 insertions(+), 485 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java create mode 100644 src/main/java/io/confluent/streaming/internal/StampedRecord.java delete mode 100644 src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index f27a10bca8d07..d6277029db12f 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -54,19 +54,6 @@ public interface KStream { */ KStreamWindowed with(Window window); - /** - * Creates a new stream by joining this stream with the other windowed stream. - * Each element in this stream is joined with elements in the other stream's window. - * The resulting values are computed by applying a joiner. - * - * @param other - * @param joiner - * @return KStream - * @throws NotCopartitionedException - */ - KStream nestedLoop(KStreamWindowed other, ValueJoiner joiner) - throws NotCopartitionedException; - /** * 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 diff --git a/src/main/java/io/confluent/streaming/KStreamWindowed.java b/src/main/java/io/confluent/streaming/KStreamWindowed.java index 102580d91fc94..8baf56b9c34d1 100644 --- a/src/main/java/io/confluent/streaming/KStreamWindowed.java +++ b/src/main/java/io/confluent/streaming/KStreamWindowed.java @@ -18,4 +18,18 @@ public interface KStreamWindowed extends KStream { KStream join(KStreamWindowed other, ValueJoiner joiner) throws NotCopartitionedException; + /** + * 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 if + * the element from the other stream has an older timestamp. + * The resulting values are computed by applying a joiner. + * + * @param other + * @param joiner + * @return KStream + * @throws NotCopartitionedException + */ + KStream joinPrior(KStreamWindowed other, ValueJoiner joiner) + throws NotCopartitionedException; + } diff --git a/src/main/java/io/confluent/streaming/Window.java b/src/main/java/io/confluent/streaming/Window.java index c6232d3ec9a7f..dc949568d9f0f 100644 --- a/src/main/java/io/confluent/streaming/Window.java +++ b/src/main/java/io/confluent/streaming/Window.java @@ -9,6 +9,10 @@ public interface Window { Iterator find(K key, long timestamp); + Iterator findAfter(K key, long timestamp); + + Iterator findBefore(K key, long timestamp); + void put(K key, V value, long timestamp); } diff --git a/src/main/java/io/confluent/streaming/WindowByTime.java b/src/main/java/io/confluent/streaming/WindowByTime.java index f02aeffc6012a..ba42f3771d845 100644 --- a/src/main/java/io/confluent/streaming/WindowByTime.java +++ b/src/main/java/io/confluent/streaming/WindowByTime.java @@ -22,8 +22,25 @@ public WindowByTime(long duration, int maxCount) { this.maxCount = maxCount; } + @Override + public Iterator findAfter(K key, final long timestamp) { + return find(key, timestamp, timestamp + duration); + } + + @Override + public Iterator findBefore(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp); + } + @Override public Iterator find(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp + duration); + } + + /* + * finds items in the window between startTime and endTime (both inclusive) + */ + private Iterator find(K key, final long startTime, final long endTime) { final LinkedList> values = map.get(key); if (values == null) { @@ -33,7 +50,7 @@ public Iterator find(K key, final long timestamp) { return new FilteredIterator>(values.iterator()) { @Override protected V filter(Stamped item) { - if (item.timestamp <= timestamp) + if (startTime <= item.timestamp && item.timestamp <= endTime) return item.value; else return null; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 2b0c14dce42b0..7446944955d38 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -23,12 +23,12 @@ class KStreamBranch implements Receiver { } @Override - public void receive(K key, V value, long timestamp) { + public void receive(K key, V value, long timestamp, long streamTime) { synchronized(this) { for (int i = 0; i < predicates.length; i++) { Predicate predicate = predicates[i]; if (predicate.apply(key, value)) { - branches[i].receive(key, value, timestamp); + branches[i].receive(key, value, timestamp, streamTime); return; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 11add37faf623..dbf43e7531c9e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -15,10 +15,10 @@ class KStreamFilter extends KStreamImpl { } @Override - public void receive(K key, V value, long timestamp) { + public void receive(K key, V value, long timestamp,long streamTime) { synchronized(this) { if (predicate.apply(key, value)) { - forward(key, value, timestamp); + forward(key, value, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index cdad3a0badf79..6e20483b44055 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -17,11 +17,11 @@ class KStreamFlatMap extends KStreamImpl { } @Override - public void receive(K1 key, V1 value, long timestamp) { + public void receive(K1 key, V1 value, long timestamp, long streamTime) { synchronized(this) { KeyValue> newPair = mapper.apply(key, value); for (V v : newPair.value) { - forward(newPair.key, v, timestamp); + forward(newPair.key, v, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index b52455c4632cf..7cc413ccbe609 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -15,11 +15,11 @@ class KStreamFlatMapValues extends KStreamImpl { } @Override - public void receive(K key, V1 value, long timestamp) { + public void receive(K key, V1 value, long timestamp, long streamTime) { synchronized(this) { Iterable newValues = mapper.apply(value); for (V v : newValues) { - forward(key, v, timestamp); + forward(key, v, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 5dc488c45f623..3504fbf7fb35a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -59,17 +59,6 @@ public KStreamWindowed with(Window window) { return (KStreamWindowed)chain(new KStreamWindowedImpl(window, partitioningInfo, context)); } - @Override - public KStream nestedLoop(KStreamWindowed other, ValueJoiner processor) - throws NotCopartitionedException { - - KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; - - if (!partitioningInfo.isJoinCompatibleWith(otherImpl.partitioningInfo)) throw new NotCopartitionedException(); - - return chain(new KStreamNestedLoop(otherImpl.window, processor, partitioningInfo, context)); - } - @Override public KStream[] branch(Predicate... predicates) { KStreamBranch branch = new KStreamBranch(predicates, partitioningInfo, context); @@ -108,7 +97,7 @@ public void punctuate(long streamTime) {} @Override public void process(final Processor processor) { Receiver receiver = new Receiver() { - public void receive(K key, V value, long timestamp) { + public void receive(K key, V value, long timestamp, long streamTime) { processor.apply(key, value); } }; @@ -122,10 +111,10 @@ void registerReceiver(Receiver receiver) { nextReceivers.add(receiver); } - protected void forward(K key, V value, long timestamp) { + protected void forward(K key, V value, long timestamp, long streamTime) { int numReceivers = nextReceivers.size(); for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).receive(key, value, timestamp); + nextReceivers.get(i).receive(key, value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index ba1215ffbbf83..a00db2cea1f4d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -10,27 +10,54 @@ */ class KStreamJoin extends KStreamImpl { - private final Window window1; - private final Window window2; + private static abstract class Finder { + abstract Iterator find(K key, long timestamp); + } + + private final Finder finder1; + private final Finder finder2; private final ValueJoiner joiner; final Receiver receiverForOtherStream; - KStreamJoin(final Window window1, Window window2, ValueJoiner joiner, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + KStreamJoin(final Window window1, final Window window2, boolean prior, ValueJoiner joiner, PartitioningInfo partitioningInfo, KStreamContextImpl context) { super(partitioningInfo, context); - this.window1 = window1; - this.window2 = window2; + if (prior) { + this.finder1 = new Finder() { + Iterator find(K key, long timestamp) { + return window1.findAfter(key, timestamp); + } + }; + this.finder2 = new Finder() { + Iterator find(K key, long timestamp) { + return window2.findBefore(key, timestamp); + } + }; + } + else { + this.finder1 = new Finder() { + Iterator find(K key, long timestamp) { + return window1.find(key, timestamp); + } + }; + this.finder2 = new Finder() { + Iterator find(K key, long timestamp) { + return window2.find(key, timestamp); + } + }; + } + this.joiner = joiner; this.receiverForOtherStream = getReceiverForOther(); } @Override - public void receive(K key, V1 value, long timestamp) { - Iterator iter = window2.find(key, timestamp); + public void receive(K key, V1 value, long timestamp, long streamTime) { + Iterator iter = finder2.find(key, timestamp); if (iter != null) { while (iter.hasNext()) { - doJoin(key, value, iter.next(), timestamp); + doJoin(key, value, iter.next(), timestamp, streamTime); } } } @@ -39,19 +66,19 @@ private Receiver getReceiverForOther() { return new Receiver() { @Override - public void receive(K key, V2 value2, long timestamp) { - Iterator iter = window1.find(key, timestamp); + public void receive(K key, V2 value2, long timestamp, long streamTime) { + Iterator iter = finder1.find(key, timestamp); if (iter != null) { while (iter.hasNext()) { - doJoin(key, iter.next(), value2, timestamp); + doJoin(key, iter.next(), value2, timestamp, streamTime); } } } }; } - private void doJoin(K key, V1 value1, V2 value2, long timestamp) { - forward(key, joiner.apply(value1, value2), timestamp); + private void doJoin(K key, V1 value1, V2 value2, long timestamp, long streamTime) { + forward(key, joiner.apply(value1, value2), timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index f4fec60afd92a..20e83ca69538c 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -17,10 +17,10 @@ class KStreamMap extends KStreamImpl { } @Override - public void receive(K1 key, V1 value, long timestamp) { + public void receive(K1 key, V1 value, long timestamp, long streamTime) { synchronized (this) { KeyValue newPair = mapper.apply(key, value); - forward(newPair.key, newPair.value, timestamp); + forward(newPair.key, newPair.value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index a2c481c042d77..2c99205eddc91 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -15,10 +15,10 @@ class KStreamMapValues extends KStreamImpl { } @Override - public void receive(K key, V1 value, long timestamp) { + public void receive(K key, V1 value, long timestamp, long streamTime) { synchronized (this) { V newValue = mapper.apply(value); - forward(key, newValue, timestamp); + forward(key, newValue, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java b/src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java deleted file mode 100644 index 9be776bfa27c9..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/KStreamNestedLoop.java +++ /dev/null @@ -1,32 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.ValueJoiner; -import io.confluent.streaming.Window; - -import java.util.Iterator; - -/** - * Created by yasuhiro on 6/17/15. - */ -class KStreamNestedLoop extends KStreamImpl { - - private final Window window; - private final ValueJoiner joiner; - - KStreamNestedLoop(Window window, ValueJoiner joiner, PartitioningInfo partitioningInfo, KStreamContextImpl context) { - super(partitioningInfo, context); - this.window = window; - this.joiner = joiner; - } - - @Override - public void receive(K key, V1 value, long timestamp) { - Iterator iter = window.find(key, timestamp); - if (iter != null) { - while (iter.hasNext()) { - forward(key, joiner.apply(value, iter.next()), timestamp); - } - } - } - -} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index e1d565d8d3037..7652ac15dcfcb 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -10,9 +10,9 @@ class KStreamSource extends KStreamImpl { } @Override - public void receive(K key, V value, long timestamp) { + public void receive(K key, V value, long timestamp, long streamTime) { synchronized(this) { - forward(key, value, timestamp); + forward(key, value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 101cd018041c3..50a362285fb18 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -15,22 +15,36 @@ public class KStreamWindowedImpl extends KStreamImpl implement } @Override - public void receive(K key, V value, long timestamp) { + public void receive(K key, V value, long timestamp, long streamTime) { synchronized(this) { window.put(key, value, timestamp); - forward(key, value, timestamp); + forward(key, value, timestamp, streamTime); } } @Override public KStream join(KStreamWindowed other, ValueJoiner processor) + throws NotCopartitionedException { + + return join(other, false, processor); + } + + @Override + public KStream joinPrior(KStreamWindowed other, ValueJoiner processor) + throws NotCopartitionedException { + + return join(other, true, processor); + } + + private KStream join(KStreamWindowed other, boolean prior, ValueJoiner processor) throws NotCopartitionedException { KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; if (!partitioningInfo.isJoinCompatibleWith(otherImpl.partitioningInfo)) throw new NotCopartitionedException(); - KStreamJoin stream = new KStreamJoin(this.window, otherImpl.window, processor, partitioningInfo, context); + KStreamJoin stream = + new KStreamJoin(this.window, otherImpl.window, prior, processor, partitioningInfo, context); otherImpl.registerReceiver(stream.receiverForOtherStream); return chain(stream); diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index ae73c4f59fecb..90997088cb42d 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -5,6 +5,6 @@ */ public interface Receiver { - void receive(K key, V value, long timestamp); + void receive(K key, V value, long timestamp, long streamTime); } diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java index 43bdb4f0ca1a1..6511093d0d8b3 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -13,7 +13,7 @@ */ public class RecordQueue { - private final Deque>> queue = new ArrayDeque>>(); + private final Deque> queue = new ArrayDeque>(); public final Receiver receiver; private final TopicPartition partition; private TimestampTracker> timestampTracker; @@ -29,21 +29,20 @@ public TopicPartition partition() { return partition; } - public void add(ConsumerRecord record, long timestamp) { - Stamped> elem = new Stamped>(record, timestamp); - queue.addLast(elem); - timestampTracker.addStampedElement(elem); + public void add(StampedRecord record) { + queue.addLast(record); + timestampTracker.addStampedElement(record); offset = record.offset(); } - public ConsumerRecord next() { - Stamped> elem = queue.getFirst(); + public StampedRecord next() { + StampedRecord elem = queue.getFirst(); if (elem == null) return null; timestampTracker.removeStampedElement(elem); - return elem.value; + return elem; } public long offset() { @@ -54,6 +53,10 @@ public int size() { return queue.size(); } + public boolean isEmpty() { + return queue.isEmpty(); + } + public long currentStreamTime() { return timestampTracker.get(); } diff --git a/src/main/java/io/confluent/streaming/internal/StampedRecord.java b/src/main/java/io/confluent/streaming/internal/StampedRecord.java new file mode 100644 index 0000000000000..1ed961c455061 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/StampedRecord.java @@ -0,0 +1,23 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.util.Stamped; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +public class StampedRecord extends Stamped> { + + StampedRecord(ConsumerRecord record, long timestamp) { + super(record, timestamp); + } + + public K key() { + return value.key(); + } + + public V value() { + return value.value(); + } + + public long offset() { + return value.offset(); + } +} diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index 88927b6e1d022..ee821ed89bebc 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -57,18 +57,20 @@ public void addRecords(TopicPartition partition, Iterator> synchronized (this) { RecordQueue recordQueue = stash.get(partition); if (recordQueue != null) { - boolean wasEmpty = (recordQueue.size() == 0); + boolean wasEmpty = recordQueue.isEmpty(); while (iterator.hasNext()) { ConsumerRecord record = iterator.next(); - recordQueue.add(record, timestampExtractor.extract(record.topic(), record.key(), record.value())); + long timestamp = timestampExtractor.extract(record.topic(), record.key(), record.value()); + recordQueue.add(new StampedRecord(record, timestamp)); buffered++; } - if (wasEmpty && recordQueue.size() > 0) chooser.add(recordQueue); + int queueSize = recordQueue.size(); + if (wasEmpty && queueSize > 0) chooser.add(recordQueue); // if we have buffered enough for this partition, pause - if (recordQueue.size() > this.desiredUnprocessed) { + if (queueSize >= this.desiredUnprocessed) { ingestor.pause(partition); } } @@ -96,11 +98,11 @@ public void process() { if (recordQueue.size() == 0) return; long timestamp = recordQueue.currentStreamTime(); - ConsumerRecord record = recordQueue.next(); + StampedRecord record = recordQueue.next(); if (streamTime < timestamp) streamTime = timestamp; - recordQueue.receiver.receive(record.key(), record.value(), streamTime); + recordQueue.receiver.receive(record.key(), record.value(), record.timestamp, streamTime); consumedOffsets.put(recordQueue.partition(), record.offset()); if (recordQueue.size() > 0) chooser.add(recordQueue); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index b486c3db6e241..e19b06d660c20 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -68,7 +68,7 @@ public boolean apply(Integer key, String value) { } for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(3, processors[0].processed.size()); @@ -87,7 +87,7 @@ public boolean apply(Integer key, String value) { } for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(3, processors[0].processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 911e618ae55ff..13c17cfbe0e0c 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -44,7 +44,7 @@ public void testFilter() { stream.filter(isMultipleOfThree).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(2, processor.processed.size()); @@ -62,7 +62,7 @@ public void testFilterOut() { stream.filterOut(isMultipleOfThree).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(5, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 114453a407f3c..263ac3602c9fd 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -52,7 +52,7 @@ public KeyValue> apply(Integer key, String value) { stream.flatMap(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(6, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index fced0ba5eda1a..0eb39cee59887 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -51,7 +51,7 @@ public Iterable apply(String value) { stream.flatMapValues(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(8, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 1a420eceb705b..9cae60a30967a 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -97,7 +97,7 @@ public void testJoin() { // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); } assertEquals(0, processor.processed.size()); @@ -105,7 +105,7 @@ public void testJoin() { // push two items to the other stream. the main stream's window has two items for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); } assertEquals(2, processor.processed.size()); @@ -121,7 +121,7 @@ public void testJoin() { // push all items to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); } assertEquals(2, processor.processed.size()); @@ -138,7 +138,7 @@ public void testJoin() { // push all items to the other stream. this should produce 6 items for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); } assertEquals(6, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 6b34d696904e3..ba7de4ec319a6 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -46,7 +46,7 @@ public KeyValue apply(Integer key, String value) { stream.map(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(4, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index fd17bf1abc96a..4954d8b6d0209 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -48,7 +48,7 @@ public Integer apply(String value) { stream.mapValues(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L); + stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L, 0L); } assertEquals(4, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java b/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java deleted file mode 100644 index 32cae0367494f..0000000000000 --- a/src/test/java/io/confluent/streaming/internal/KStreamNestedLoopTest.java +++ /dev/null @@ -1,358 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.*; -import io.confluent.streaming.testutil.NoopIngestor; -import io.confluent.streaming.testutil.TestProcessor; -import io.confluent.streaming.testutil.UnlimitedWindow; -import io.confluent.streaming.util.Util; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -public class KStreamNestedLoopTest { - - private Ingestor ingestor = new NoopIngestor(); - - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( - "group", - ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { - return 0L; - } - }, - 10 - ); - - private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); - - - private ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; - } - }; - - private ValueMapper valueMapper = new ValueMapper() { - @Override - public String apply(String value) { - return "#" + value; - } - }; - - private ValueMapper, String> valueMapper2 = new ValueMapper, String>() { - @Override - public Iterable apply(String value) { - return (Iterable) Util.mkSet(value); - } - }; - - private KeyValueMapper keyValueMapper = - new KeyValueMapper() { - @Override - public KeyValue apply(Integer key, String value) { - return KeyValue.pair(key, value); - } - }; - - KeyValueMapper, Integer, String> keyValueMapper2 = - new KeyValueMapper, Integer, String>() { - @Override - public KeyValue> apply(Integer key, String value) { - return KeyValue.pair(key, (Iterable) Util.mkSet(value)); - } - }; - - @Test - public void testNestedLoop() { - final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - - KStreamSource stream1; - KStreamSource stream2; - KStreamWindowed windowed; - TestProcessor processor; - String[] expected; - - processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); - windowed = stream2.with(new UnlimitedWindow()); - - boolean exceptionRaised = false; - - try { - stream1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - - // empty window - - for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); - } - - assertEquals(0, processor.processed.size()); - - // two items in the window - - for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); - } - - for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); - } - - 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(); - - // previous two items + all items, thus two are duplicates, in the window - - for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); - } - - for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); - } - - 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)); - } - } - - @Test - public void testMap() { - KStreamSource stream1; - KStreamSource stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed; - TestProcessor processor; - - processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); - mapped1 = stream1.map(keyValueMapper); - mapped2 = stream2.map(keyValueMapper); - - boolean exceptionRaised; - - try { - exceptionRaised = false; - windowed = stream2.with(new UnlimitedWindow()); - - mapped1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - - try { - exceptionRaised = false; - windowed = mapped2.with(new UnlimitedWindow()); - - stream1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - - try { - exceptionRaised = false; - windowed = mapped2.with(new UnlimitedWindow()); - - mapped1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - } - - @Test - public void testFlatMap() { - KStreamSource stream1; - KStreamSource stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed; - TestProcessor processor; - - processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); - mapped1 = stream1.flatMap(keyValueMapper2); - mapped2 = stream2.flatMap(keyValueMapper2); - - boolean exceptionRaised; - - try { - exceptionRaised = false; - windowed = stream2.with(new UnlimitedWindow()); - - mapped1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - - try { - exceptionRaised = false; - windowed = mapped2.with(new UnlimitedWindow()); - - stream1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - - try { - exceptionRaised = false; - windowed = mapped2.with(new UnlimitedWindow()); - - mapped1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - } - - @Test - public void testMapValues() { - KStreamSource stream1; - KStreamSource stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed; - TestProcessor processor; - - processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); - mapped1 = stream1.mapValues(valueMapper); - mapped2 = stream2.mapValues(valueMapper); - - boolean exceptionRaised; - - try { - exceptionRaised = false; - windowed = stream2.with(new UnlimitedWindow()); - - mapped1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - - try { - exceptionRaised = false; - windowed = mapped2.with(new UnlimitedWindow()); - - stream1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - - try { - exceptionRaised = false; - windowed = mapped2.with(new UnlimitedWindow()); - - mapped1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - } - - @Test - public void testFlatMapValues() { - KStreamSource stream1; - KStreamSource stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed; - TestProcessor processor; - - processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); - mapped1 = stream1.flatMapValues(valueMapper2); - mapped2 = stream2.flatMapValues(valueMapper2); - - boolean exceptionRaised; - - try { - exceptionRaised = false; - windowed = stream2.with(new UnlimitedWindow()); - - mapped1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - - try { - exceptionRaised = false; - windowed = mapped2.with(new UnlimitedWindow()); - - stream1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - - try { - exceptionRaised = false; - windowed = mapped2.with(new UnlimitedWindow()); - - mapped1.nestedLoop(windowed, joiner).process(processor); - } - catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - } -} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 5aac8e9abe58c..cb1753068f9b4 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -38,7 +38,7 @@ public void testKStreamSource() { final String[] expectedValues = new String[] { "v1", "v2", "v3" }; for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], expectedValues[i], 0L); + stream.receive(expectedKeys[i], expectedValues[i], 0L, 0L); } assertEquals(3, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index d1cc012d11166..86347c574ebd4 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -45,7 +45,7 @@ public void testWindowedStream() { // two items in the window for (int i = 0; i < 2; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(1, countItem(window.find(0, 0L))); @@ -56,7 +56,7 @@ public void testWindowedStream() { // previous two items + all items, thus two are duplicates, in the window for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + stream.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); } assertEquals(2, countItem(window.find(0, 0L))); diff --git a/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java b/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java index fb4a1d5c2306c..98a897dc8a77d 100644 --- a/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java +++ b/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java @@ -3,29 +3,43 @@ import io.confluent.streaming.KeyValue; import io.confluent.streaming.Window; import io.confluent.streaming.util.FilteredIterator; +import io.confluent.streaming.util.Stamped; import java.util.Iterator; import java.util.LinkedList; public class UnlimitedWindow implements Window { - private LinkedList> list = new LinkedList>(); + private LinkedList>> list = new LinkedList>>(); @Override public Iterator find(final K key, long timestamp) { - return new FilteredIterator>(list.iterator()) { - protected V filter(KeyValue item) { - if (item.key.equals(key)) - return item.value; + return find(key, Long.MIN_VALUE, timestamp); + } + + @Override + public Iterator findAfter(final K key, long timestamp) { + return find(key, timestamp, Long.MAX_VALUE); + } + + @Override + public Iterator findBefore(final K key, long timestamp) { + return find(key, Long.MIN_VALUE, Long.MAX_VALUE); + } + + private Iterator find(final K key, final long startTime, final long endTime) { + return new FilteredIterator>>(list.iterator()) { + protected V filter(Stamped> item) { + if (item.value.key.equals(key) && startTime <= item.timestamp && item.timestamp <= endTime) + return item.value.value; else return null; } }; } - @Override public void put(K key, V value, long timestamp) { - list.add(KeyValue.pair(key, value)); + list.add(new Stamped>(KeyValue.pair(key, value), timestamp)); } } From cfa7f487298dc2a3a338c685a67a27bec2187117 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 2 Jul 2015 11:29:44 -0700 Subject: [PATCH 030/275] test joinPior --- .../streaming/internal/KStreamJoinTest.java | 80 +++++++++++++++++++ 1 file changed, 80 insertions(+) diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 9cae60a30967a..5bcb3d73b7517 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -150,6 +150,86 @@ public void testJoin() { } } + @Test + public void testJoinPrior() { + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStreamSource stream1; + KStreamSource stream2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + TestProcessor processor; + String[] expected; + + processor = new TestProcessor(); + stream1 = new KStreamSource(partitioningInfo, null); + stream2 = new KStreamSource(partitioningInfo, null); + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); + + boolean exceptionRaised = false; + + try { + windowed1.joinPrior(windowed2, joiner).process(processor); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } + + assertFalse(exceptionRaised); + + // push two items to the main stream. the other stream's window is empty + + for (int i = 0; i < 2; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i, 0L); + } + + assertEquals(0, processor.processed.size()); + + // push two items to the other stream. the main stream's window has two items + // no corresponding item in the main window has a newer timestamp + + for (int i = 0; i < 2; i++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i + 1, 0L); + } + + assertEquals(0, processor.processed.size()); + + processor.processed.clear(); + + // push all items with newer timestamps to the main stream. this should produce two items. + + for (int i = 0; i < expectedKeys.length; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i + 2, 0L); + } + + 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 with older timestamps to the other stream. this should produce six items + for (int i = 0; i < expectedKeys.length; i++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i, 0L); + } + + 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)); + } + + } + @Test public void testMap() { KStreamSource stream1; From 4159d42c5f4e910e850d73117e23c58d4b192583 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 2 Jul 2015 11:42:04 -0700 Subject: [PATCH 031/275] removed unused method --- src/main/java/io/confluent/streaming/KStreamContext.java | 2 -- .../io/confluent/streaming/internal/KStreamContextImpl.java | 5 ----- 2 files changed, 7 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index d000f9f8aee68..cc73592df0449 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -46,6 +46,4 @@ public interface KStreamContext { void restore(StorageEngine engine) throws Exception; - void schedule(Processor processor, long time); - } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 5341e30c4b54d..37726719a6b04 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -198,11 +198,6 @@ public void restore(StorageEngine engine) throws Exception { } - @Override - public void schedule(Processor processor, long time) { - - } - public Collection syncGroups() { return syncGroups.values(); } From 66da0b1d0caf1b38f1c28e89c637c00ba779b358 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 2 Jul 2015 12:09:12 -0700 Subject: [PATCH 032/275] register a StreamSynchronizer to Ingestor --- .../java/io/confluent/streaming/KafkaStreaming.java | 6 ++++-- .../io/confluent/streaming/internal/Ingestor.java | 4 ++++ .../confluent/streaming/internal/IngestorImpl.java | 11 +++++++++++ .../streaming/internal/KStreamContextImpl.java | 4 +++- .../streaming/internal/KStreamBranchTest.java | 4 ++-- .../streaming/internal/KStreamFilterTest.java | 4 ++-- .../streaming/internal/KStreamFlatMapTest.java | 4 ++-- .../streaming/internal/KStreamFlatMapValuesTest.java | 4 ++-- .../streaming/internal/KStreamJoinTest.java | 4 ++-- .../confluent/streaming/internal/KStreamMapTest.java | 4 ++-- .../streaming/internal/KStreamMapValuesTest.java | 4 ++-- .../streaming/internal/KStreamSourceTest.java | 4 ++-- .../streaming/internal/KStreamWindowedTest.java | 4 ++-- .../{NoopIngestor.java => MockIngestor.java} | 12 +++++++++++- 14 files changed, 51 insertions(+), 22 deletions(-) rename src/test/java/io/confluent/streaming/testutil/{NoopIngestor.java => MockIngestor.java} (56%) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index d95e6115d7c0e..dc6d49c7fce45 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -270,7 +270,7 @@ private void commitRequesting(long now) { KStreamContextImpl context = kstreamContexts.get(id); context.flush(); - for (SyncGroup syncGroup : this.syncGroups.get(id)) { + for (SyncGroup syncGroup : syncGroups.get(id)) { commit.putAll(syncGroup.streamSynchronizer.consumedOffsets()); // TODO: can this be async? } } @@ -305,7 +305,7 @@ private void maybeCleanState() { private void addPartitions(Collection assignment) { HashSet partitions = new HashSet(assignment); - this.ingestor.init(); + ingestor.init(); Consumer restoreConsumer = new KafkaConsumer( @@ -363,6 +363,8 @@ private void removePartitions(Collection assignment) { } for (TopicPartition partition : assignment) { KStreamContextImpl kstreamContext = kstreamContexts.remove(partition.partition()); + ingestor.removeStreamSynchronizerForPartition(partition); + if (kstreamContext != null) { log.info("Removing stream context {}", partition.partition()); try { diff --git a/src/main/java/io/confluent/streaming/internal/Ingestor.java b/src/main/java/io/confluent/streaming/internal/Ingestor.java index 571e3b49022ae..5080c4f91f2e6 100644 --- a/src/main/java/io/confluent/streaming/internal/Ingestor.java +++ b/src/main/java/io/confluent/streaming/internal/Ingestor.java @@ -17,4 +17,8 @@ public interface Ingestor { int numPartitions(String topic); + void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition); + + void removeStreamSynchronizerForPartition(TopicPartition partition); + } diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 0a67ec25da7b6..b676ab8a2af1a 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -75,6 +75,17 @@ public int numPartitions(String topic) { return consumer.partitionsFor(topic).size(); } + @SuppressWarnings("unchecked") + @Override + public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition) { + streamSynchronizers.put(partition, (StreamSynchronizer) streamSynchronizer); + } + + @Override + public void removeStreamSynchronizerForPartition(TopicPartition partition) { + streamSynchronizers.remove(partition); + } + public void clear() { unpaused.clear(); toBePaused.clear(); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 37726719a6b04..7fdecb0ecf7f0 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -123,7 +123,9 @@ public Deserializer valueDeserializer() { stream = new KStreamSource(partitioningInfo, this); sourceStreams.put(topic, stream); - syncGroup.streamSynchronizer.addPartition(new TopicPartition(topic, id), (Receiver)stream); + TopicPartition partition = new TopicPartition(topic, id); + syncGroup.streamSynchronizer.addPartition(partition, (Receiver)stream); + ingestor.addStreamSynchronizerForPartition(syncGroup.streamSynchronizer, partition); } else { if (stream.partitioningInfo.syncGroup == syncGroup) diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index e19b06d660c20..767f4c74bec78 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -11,7 +11,7 @@ public class KStreamBranchTest { - private Ingestor ingestor = new NoopIngestor(); + private Ingestor ingestor = new MockIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 13c17cfbe0e0c..cb630a611d263 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -9,7 +9,7 @@ public class KStreamFilterTest { - private Ingestor ingestor = new NoopIngestor(); + private Ingestor ingestor = new MockIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 263ac3602c9fd..685662fb2f9f6 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -11,7 +11,7 @@ public class KStreamFlatMapTest { - private Ingestor ingestor = new NoopIngestor(); + private Ingestor ingestor = new MockIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 0eb39cee59887..bed3fde82ba11 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -11,7 +11,7 @@ public class KStreamFlatMapValuesTest { - private Ingestor ingestor = new NoopIngestor(); + private Ingestor ingestor = new MockIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 5bcb3d73b7517..1b196908eaa44 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.TestProcessor; import io.confluent.streaming.testutil.UnlimitedWindow; import io.confluent.streaming.util.Util; @@ -13,7 +13,7 @@ public class KStreamJoinTest { - private Ingestor ingestor = new NoopIngestor(); + private Ingestor ingestor = new MockIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index ba7de4ec319a6..27f677f001814 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -9,7 +9,7 @@ public class KStreamMapTest { - private Ingestor ingestor = new NoopIngestor(); + private Ingestor ingestor = new MockIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 4954d8b6d0209..19832849f83d8 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -3,7 +3,7 @@ import io.confluent.streaming.SyncGroup; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.ValueMapper; -import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -11,7 +11,7 @@ public class KStreamMapValuesTest { - private Ingestor ingestor = new NoopIngestor(); + private Ingestor ingestor = new MockIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index cb1753068f9b4..4c1e9d5ce72ee 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -4,13 +4,13 @@ import io.confluent.streaming.*; -import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; public class KStreamSourceTest { - private Ingestor ingestor = new NoopIngestor(); + private Ingestor ingestor = new MockIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 86347c574ebd4..87849177c7971 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.NoopIngestor; +import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.UnlimitedWindow; import org.junit.Test; @@ -11,7 +11,7 @@ public class KStreamWindowedTest { - private Ingestor ingestor = new NoopIngestor(); + private Ingestor ingestor = new MockIngestor(); private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", diff --git a/src/test/java/io/confluent/streaming/testutil/NoopIngestor.java b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java similarity index 56% rename from src/test/java/io/confluent/streaming/testutil/NoopIngestor.java rename to src/test/java/io/confluent/streaming/testutil/MockIngestor.java index 3df6fd4628389..dcfb6435540ea 100644 --- a/src/test/java/io/confluent/streaming/testutil/NoopIngestor.java +++ b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java @@ -1,9 +1,10 @@ package io.confluent.streaming.testutil; import io.confluent.streaming.internal.Ingestor; +import io.confluent.streaming.internal.StreamSynchronizer; import org.apache.kafka.common.TopicPartition; -public class NoopIngestor implements Ingestor { +public class MockIngestor implements Ingestor { @Override public void poll() { } @@ -24,4 +25,13 @@ public void unpause(TopicPartition partition, long offset) { public int numPartitions(String topic) { return 1; } + + @Override + public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition) { + } + + @Override + public void removeStreamSynchronizerForPartition(TopicPartition partition) { + } + } From 84b301483db603a89c7c4b8fbd93f5bc69227e3b Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 2 Jul 2015 12:14:03 -0700 Subject: [PATCH 033/275] log unused topic --- .../io/confluent/streaming/internal/IngestorImpl.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index b676ab8a2af1a..bd3a8da9b900d 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -6,11 +6,15 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.*; public class IngestorImpl implements Ingestor { + private static final Logger log = LoggerFactory.getLogger(IngestorImpl.class); + private final Consumer consumer; private final Set unpaused = new HashSet(); private final Set toBePaused = new HashSet(); @@ -50,7 +54,12 @@ public void poll(long timeoutMs) { ConsumerRecords records = consumer.poll(timeoutMs); for (TopicPartition partition : unpaused) { - streamSynchronizers.get(partition).addRecords(partition, new DeserializingIterator(records.records(partition).iterator())); + StreamSynchronizer streamSynchronizer = streamSynchronizers.get(partition); + + if (streamSynchronizer != null) + streamSynchronizer.addRecords(partition, new DeserializingIterator(records.records(partition).iterator())); + else + log.warn("unused topic: " + partition.topic()); } } From 619604b2582fa51d8d70143950df3ef5404c77ba Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 2 Jul 2015 14:58:49 -0700 Subject: [PATCH 034/275] removed the subscribe method from StreamingConfig --- .../confluent/streaming/KStreamException.java | 19 ++++++++ .../io/confluent/streaming/KStreamJob.java | 47 +++++++++++++++++-- .../confluent/streaming/KafkaStreaming.java | 36 ++++++++++---- .../confluent/streaming/StreamingConfig.java | 9 ---- .../streaming/internal/IngestorImpl.java | 2 +- .../internal/KStreamContextImpl.java | 5 +- .../io/confluent/streaming/util/Util.java | 19 ++++++++ 7 files changed, 112 insertions(+), 25 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/KStreamException.java diff --git a/src/main/java/io/confluent/streaming/KStreamException.java b/src/main/java/io/confluent/streaming/KStreamException.java new file mode 100644 index 0000000000000..7442df714b3c1 --- /dev/null +++ b/src/main/java/io/confluent/streaming/KStreamException.java @@ -0,0 +1,19 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 7/2/15. + */ +public class KStreamException extends RuntimeException { + + public KStreamException(String msg) { + super(msg); + } + + public KStreamException(Throwable t) { + super(t); + } + + public KStreamException(String msg, Throwable t) { + super(msg, t); + } +} diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java index e54dfe9de4cb3..6cb009a90ca51 100644 --- a/src/main/java/io/confluent/streaming/KStreamJob.java +++ b/src/main/java/io/confluent/streaming/KStreamJob.java @@ -1,10 +1,51 @@ package io.confluent.streaming; +import io.confluent.streaming.util.Util; + +import java.util.Collections; +import java.util.Set; + /** - * Created by yasuhiro on 6/22/15. + * An interface to implement an application logic of a stream processing. + * + * Each subclass must declare a public static final member variable topics like this: + *
+ *   class MyKStreamJob {
+ *     public static final Topics topics = new Topics("pageView", "adClick");
+ *     ...
+ *   }
+ * 
+ * + * Topics represents a set of topics used in the application. */ -public interface KStreamJob { +public abstract class KStreamJob { + + /** + * A class represents a set of topic names. + */ + public static class Topics { + + public final Set set; + + Topics(String... topics) { + set = Collections.unmodifiableSet(Util.mkSet(topics)); + } + + } - void build(KStreamContext ksc); + /** + * Initializes a streaming job by constructing a processing logic using KStream API. + *

+ * For exmaple, + *

+ *
+   *   public init(KStreamContext context) {
+   *     KStream<Integer, PageView> pageViewStream = context.from("pageView").mapValues(...)
+   *     KStream<Integer, AdClick> adClickStream = context.from("adClick").join(pageViewStream, ...).process(...);
+   *   }
+   * 
+ * @param context + */ + public abstract void init(KStreamContext context); } diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index dc6d49c7fce45..ba5b1e15f2e8d 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -39,6 +39,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,7 +81,8 @@ public class KafkaStreaming implements Runnable { private static final Logger log = LoggerFactory.getLogger(KafkaStreaming.class); - private final KStreamJob job; + private final Class jobClass; + private final Set topics; private final Map> syncGroups = new HashMap>(); private final Map kstreamContexts = new HashMap(); protected final Producer producer; @@ -111,16 +113,17 @@ public void onPartitionsRevoked(Consumer consumer, Collection jobClass, StreamingConfig config) { + this(jobClass, config, null, null); } @SuppressWarnings("unchecked") - protected KafkaStreaming(KStreamJob job, + protected KafkaStreaming(Class jobClass, StreamingConfig config, Producer producer, Consumer consumer) { - this.job = job; + this.jobClass = jobClass; + this.topics = extractTopics(jobClass); this.producer = producer == null? new KafkaProducer(config.config(), new ByteArraySerializer(), new ByteArraySerializer()): producer; this.consumer = consumer == null? new KafkaConsumer(config.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()): consumer; this.streamingConfig = config; @@ -130,9 +133,9 @@ protected KafkaStreaming(KStreamJob job, this.config = new ProcessorConfig(config.config()); this.ingestor = new IngestorImpl(this.consumer, - (Deserializer) config.keyDeserializer(), - (Deserializer) config.valueDeserializer(), - this.config.pollTimeMs); + (Deserializer) config.keyDeserializer(), + (Deserializer) config.valueDeserializer(), + this.config.pollTimeMs); this.running = true; this.lastCommit = 0; this.nextStateCleaning = Long.MAX_VALUE; @@ -161,8 +164,9 @@ private void init() { if (!config.stateDir.exists() && !config.stateDir.mkdirs()) throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); - for (String topic : streamingConfig.topics()) + for (String topic : topics) consumer.subscribe(topic); + log.info("Start-up complete"); } else { throw new IllegalStateException("This container was already started"); @@ -318,6 +322,8 @@ private void addPartitions(Collection assignment) { final Integer id = partition.partition(); KStreamContextImpl kstreamContext = kstreamContexts.get(id); if (kstreamContext == null) { + KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); + Coordinator coordinator = new Coordinator() { @Override public void commit(Coordinator.RequestScope scope) { @@ -331,7 +337,7 @@ public void shutdown(Coordinator.RequestScope scope) { }; kstreamContext = - new KStreamContextImpl(id, ingestor, producer, coordinator, streamingConfig, config, metrics); + new KStreamContextImpl(id, topics, ingestor, producer, coordinator, streamingConfig, config, metrics); kstreamContexts.put(id, kstreamContext); @@ -380,6 +386,16 @@ private void removePartitions(Collection assignment) { this.ingestor.clear(); } + private static Set extractTopics(Class jobClass) { + // extract topics from a jobClass's static member field, topics + try { + return ((KStreamJob.Topics)Util.getFieldValue(jobClass, null, "topics")).set; + } + catch (Exception e) { + throw new KStreamException("failed to get a topic list from the job", e); + } + } + private class KafkaStreamingMetrics { final Sensor commitTime; final Sensor processTime; diff --git a/src/main/java/io/confluent/streaming/StreamingConfig.java b/src/main/java/io/confluent/streaming/StreamingConfig.java index 1f650cb9f97fb..966204a83c35f 100644 --- a/src/main/java/io/confluent/streaming/StreamingConfig.java +++ b/src/main/java/io/confluent/streaming/StreamingConfig.java @@ -55,7 +55,6 @@ public class StreamingConfig { private final Properties config; private final Map context = new HashMap(); - private final Set topics = new HashSet(); private Serializer keySerializer; private Serializer valSerializer; private Deserializer keyDeserializer; @@ -74,10 +73,6 @@ public void addContextObject(String key, Object value) { @SuppressWarnings("all") - public void subscribe(String...topic) { - Collections.addAll(this.topics, topic); - } - public void serialization(Serializer serializer, Deserializer deserializer) { keySerializer(serializer); valueSerializer(serializer); @@ -109,10 +104,6 @@ public Map context() { return this.context; } - public Set topics() { - return this.topics; - } - public Serializer keySerializer() { return this.keySerializer; } diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index bd3a8da9b900d..206009cd4b37b 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -55,7 +55,7 @@ public void poll(long timeoutMs) { for (TopicPartition partition : unpaused) { StreamSynchronizer streamSynchronizer = streamSynchronizers.get(partition); - + if (streamSynchronizer != null) streamSynchronizer.addRecords(partition, new DeserializingIterator(records.records(partition).iterator())); else diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 7fdecb0ecf7f0..128285c24e6ea 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -45,6 +45,7 @@ public class KStreamContextImpl implements KStreamContext { @SuppressWarnings("unchecked") public KStreamContextImpl(int id, + Set topics, Ingestor ingestor, Producer producer, Coordinator coordinator, @@ -52,7 +53,7 @@ public KStreamContextImpl(int id, ProcessorConfig processorConfig, Metrics metrics) { this.id = id; - this.topics = streamingConfig.topics(); + this.topics = topics; this.ingestor = ingestor; this.simpleCollector = new RecordCollectors.SimpleRecordCollector(producer); @@ -208,7 +209,7 @@ public void init(Consumer restoreConsumer, KStreamJob job) throw stateMgr.init(); try { this.restoreConsumer = restoreConsumer; - job.build(this); + job.init(this); } finally { this.restoreConsumer = null; diff --git a/src/main/java/io/confluent/streaming/util/Util.java b/src/main/java/io/confluent/streaming/util/Util.java index d477888b1e6ad..1d548ffa3a84d 100644 --- a/src/main/java/io/confluent/streaming/util/Util.java +++ b/src/main/java/io/confluent/streaming/util/Util.java @@ -1,6 +1,7 @@ package io.confluent.streaming.util; import java.io.File; +import java.lang.reflect.Field; import java.util.HashSet; /** @@ -8,12 +9,30 @@ */ public class Util { + /** + * Creates a set + * @param elems + * @return Set + */ public static HashSet mkSet(T... elems) { HashSet set = new HashSet(); for (T e : elems) set.add(e); return set; } + /** + * Gets a value from a field of a class + * @param clazz the class object + * @param obj the instance object, or maybe null for a static field + * @Param fieldName the name of the field + * @return Object + * @throws Exception + */ + public static Object getFieldValue(Class clazz, Object obj, String fieldName) throws Exception { + Field myField = clazz.getDeclaredField(fieldName); + return myField.get(obj); + } + /** * Recursively delete the given file/directory and any subfiles (if any exist) * From 5638eb9b5fc04045a22d43eea5ff20432d733a52 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 2 Jul 2015 16:17:25 -0700 Subject: [PATCH 035/275] raise exception if there is an unused topic --- .../streaming/internal/KStreamContextImpl.java | 15 +++++++++++---- .../java/io/confluent/streaming/util/Util.java | 17 +++++++++++++++++ 2 files changed, 28 insertions(+), 4 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 128285c24e6ea..2b50ef8698e26 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -1,6 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; +import io.confluent.streaming.util.Util; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.TopicPartition; @@ -12,10 +13,7 @@ import java.io.File; import java.io.IOException; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; +import java.util.*; /** * Created by yasuhiro on 6/19/15. @@ -214,6 +212,15 @@ public void init(Consumer restoreConsumer, KStreamJob job) throw finally { this.restoreConsumer = null; } + + if (!topics.equals(sourceStreams.keySet())) { + LinkedList unusedTopics = new LinkedList(); + for (String topic : topics) { + if (!sourceStreams.containsKey(topic)) + unusedTopics.add(topic); + } + throw new KStreamException("unused topics: " + Util.mkString(unusedTopics)); + } } public void flush() { diff --git a/src/main/java/io/confluent/streaming/util/Util.java b/src/main/java/io/confluent/streaming/util/Util.java index 1d548ffa3a84d..4a6a71eea11a7 100644 --- a/src/main/java/io/confluent/streaming/util/Util.java +++ b/src/main/java/io/confluent/streaming/util/Util.java @@ -2,6 +2,7 @@ import java.io.File; import java.lang.reflect.Field; +import java.util.Collection; import java.util.HashSet; /** @@ -33,6 +34,22 @@ public static Object getFieldValue(Class clazz, Object obj, String fieldName) return myField.get(obj); } + /** + * Makes a srtring of a comma separated list of collection elements + * @param collection + * @return String + */ + public static String mkString(Collection collection) { + StringBuilder sb = new StringBuilder(); + int count = collection.size(); + for (E elem : collection) { + sb.append(elem.toString()); + count--; + if (count > 0) sb.append(", "); + } + return sb.toString(); + } + /** * Recursively delete the given file/directory and any subfiles (if any exist) * From a91486c7c716884696c75bae0c849f5521a6bce9 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 6 Jul 2015 08:02:24 -0700 Subject: [PATCH 036/275] made NotCopartitionedException a subclass of KStreamException --- .../java/io/confluent/streaming/KStreamException.java | 4 ++++ .../java/io/confluent/streaming/KStreamWindowed.java | 8 ++------ .../streaming/NotCopartitionedException.java | 6 +++++- .../streaming/internal/KStreamWindowedImpl.java | 11 +++-------- 4 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStreamException.java b/src/main/java/io/confluent/streaming/KStreamException.java index 7442df714b3c1..2273a510edbe8 100644 --- a/src/main/java/io/confluent/streaming/KStreamException.java +++ b/src/main/java/io/confluent/streaming/KStreamException.java @@ -5,6 +5,10 @@ */ public class KStreamException extends RuntimeException { + public KStreamException() { + super(); + } + public KStreamException(String msg) { super(msg); } diff --git a/src/main/java/io/confluent/streaming/KStreamWindowed.java b/src/main/java/io/confluent/streaming/KStreamWindowed.java index 8baf56b9c34d1..2556065f3bba6 100644 --- a/src/main/java/io/confluent/streaming/KStreamWindowed.java +++ b/src/main/java/io/confluent/streaming/KStreamWindowed.java @@ -13,10 +13,8 @@ public interface KStreamWindowed extends KStream { * @param other * @param joiner * @return KStream - * @throws NotCopartitionedException */ - KStream join(KStreamWindowed other, ValueJoiner joiner) - throws NotCopartitionedException; + KStream join(KStreamWindowed other, ValueJoiner joiner); /** * Creates a new stream by joining this windowed stream with the other windowed stream. @@ -27,9 +25,7 @@ KStream join(KStreamWindowed other, ValueJoiner KStream joinPrior(KStreamWindowed other, ValueJoiner joiner) - throws NotCopartitionedException; + KStream joinPrior(KStreamWindowed other, ValueJoiner joiner); } diff --git a/src/main/java/io/confluent/streaming/NotCopartitionedException.java b/src/main/java/io/confluent/streaming/NotCopartitionedException.java index 849f614506888..0a8a2fabdfe9f 100644 --- a/src/main/java/io/confluent/streaming/NotCopartitionedException.java +++ b/src/main/java/io/confluent/streaming/NotCopartitionedException.java @@ -3,6 +3,10 @@ /** * Created by yasuhiro on 6/19/15. */ -public class NotCopartitionedException extends Exception { +public class NotCopartitionedException extends KStreamException { + + public NotCopartitionedException() { + super(); + } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 50a362285fb18..0b9f92e4aec78 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -23,21 +23,16 @@ public void receive(K key, V value, long timestamp, long streamTime) { } @Override - public KStream join(KStreamWindowed other, ValueJoiner processor) - throws NotCopartitionedException { - + public KStream join(KStreamWindowed other, ValueJoiner processor) { return join(other, false, processor); } @Override - public KStream joinPrior(KStreamWindowed other, ValueJoiner processor) - throws NotCopartitionedException { - + public KStream joinPrior(KStreamWindowed other, ValueJoiner processor) { return join(other, true, processor); } - private KStream join(KStreamWindowed other, boolean prior, ValueJoiner processor) - throws NotCopartitionedException { + private KStream join(KStreamWindowed other, boolean prior, ValueJoiner processor) { KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; From d8816b4e9a00fe73149c33b37d97e5e46e9eb994 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 6 Jul 2015 09:03:49 -0700 Subject: [PATCH 037/275] less polling --- .../confluent/streaming/KafkaStreaming.java | 5 +++ .../streaming/internal/IngestorImpl.java | 31 +++++++++++++------ .../streaming/internal/RecordQueue.java | 6 +++- .../internal/StreamSynchronizer.java | 21 +++++++++---- .../streaming/internal/TimeBasedChooser.java | 4 +-- 5 files changed, 49 insertions(+), 18 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index ba5b1e15f2e8d..c4e9627ee0822 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -208,13 +208,18 @@ public synchronized void close() { private void runLoop() { try { while (stillRunning()) { + boolean pollRequired = false; + for (Map.Entry> entry : syncGroups.entrySet()) { for (SyncGroup syncGroup : entry.getValue()) { syncGroup.streamSynchronizer.process(); + pollRequired = pollRequired || syncGroup.streamSynchronizer.requiresPoll(); } } maybeCommit(); maybeCleanState(); + + if (pollRequired) ingestor.poll(); } } catch (Exception e) { throw new KafkaException(e); diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 206009cd4b37b..9e28a6e68e302 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -46,12 +46,16 @@ public void poll() { @Override public void poll(long timeoutMs) { - for (TopicPartition partition : toBePaused) { - doPause(partition); - } - toBePaused.clear(); + ConsumerRecords records; - ConsumerRecords records = consumer.poll(timeoutMs); + synchronized (this) { + for (TopicPartition partition : toBePaused) { + doPause(partition); + } + toBePaused.clear(); + + records = consumer.poll(timeoutMs); + } for (TopicPartition partition : unpaused) { StreamSynchronizer streamSynchronizer = streamSynchronizers.get(partition); @@ -75,8 +79,10 @@ private void doPause(TopicPartition partition) { @Override public void unpause(TopicPartition partition, long lastOffset) { - consumer.seek(partition, lastOffset); - unpaused.add(partition); + synchronized (this) { + consumer.seek(partition, lastOffset); + unpaused.add(partition); + } } @Override @@ -87,12 +93,19 @@ public int numPartitions(String topic) { @SuppressWarnings("unchecked") @Override public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition) { - streamSynchronizers.put(partition, (StreamSynchronizer) streamSynchronizer); + synchronized (this) { + streamSynchronizers.put(partition, (StreamSynchronizer) streamSynchronizer); + unpaused.add(partition); + } } @Override public void removeStreamSynchronizerForPartition(TopicPartition partition) { - streamSynchronizers.remove(partition); + synchronized (this) { + streamSynchronizers.remove(partition); + unpaused.remove(partition); + toBePaused.remove(partition); + } } public void clear() { diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java index 6511093d0d8b3..86b2abc913929 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -57,7 +57,11 @@ public boolean isEmpty() { return queue.isEmpty(); } - public long currentStreamTime() { + /** + * Returns a timestamp tracked by the TimestampTracker + * @return timestamp + */ + public long trackedTimestamp() { return timestampTracker.get(); } diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index ee821ed89bebc..269bad6db2779 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -24,6 +24,7 @@ public class StreamSynchronizer { private final PunctuationQueue punctuationQueue = new PunctuationQueue(); private long streamTime = -1; + private boolean pollRequired = false; private volatile int buffered = 0; StreamSynchronizer(String name, @@ -77,6 +78,10 @@ public void addRecords(TopicPartition partition, Iterator> } } + public boolean requiresPoll() { + return pollRequired; + } + public PunctuationScheduler getPunctuationScheduler(Processor processor) { return new PunctuationSchedulerImpl(punctuationQueue, processor); } @@ -84,23 +89,25 @@ public PunctuationScheduler getPunctuationScheduler(Processor processor) { @SuppressWarnings("unchecked") public void process() { synchronized (this) { - RecordQueue recordQueue = chooser.next(); + pollRequired = false; + RecordQueue recordQueue = chooser.next(); if (recordQueue == null) { - ingestor.poll(); + pollRequired = true; return; } + if (recordQueue.size() == 0) throw new IllegalStateException("empty record queue"); + if (recordQueue.size() == this.desiredUnprocessed) { ingestor.unpause(recordQueue.partition(), recordQueue.offset()); + pollRequired = true; } - if (recordQueue.size() == 0) return; - - long timestamp = recordQueue.currentStreamTime(); + long trackedTimestamp = recordQueue.trackedTimestamp(); StampedRecord record = recordQueue.next(); - if (streamTime < timestamp) streamTime = timestamp; + if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; recordQueue.receiver.receive(record.key(), record.value(), record.timestamp, streamTime); consumedOffsets.put(recordQueue.partition(), record.offset()); @@ -110,6 +117,8 @@ public void process() { buffered--; punctuationQueue.mayPunctuate(streamTime); + + return; } } diff --git a/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java b/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java index 0823862127355..ccb96c9709035 100644 --- a/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java +++ b/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java @@ -13,8 +13,8 @@ public class TimeBasedChooser implements Chooser { public TimeBasedChooser() { this(new Comparator>() { public int compare(RecordQueue queue1, RecordQueue queue2) { - long time1 = queue1.currentStreamTime(); - long time2 = queue2.currentStreamTime(); + long time1 = queue1.trackedTimestamp(); + long time2 = queue2.trackedTimestamp(); if (time1 < time2) return -1; if (time1 > time2) return 1; From 75e048c8c3bde75683e47ee6f107b68d1cdb0f50 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 7 Jul 2015 11:24:14 -0700 Subject: [PATCH 038/275] shutdown latch --- .../confluent/streaming/KafkaStreaming.java | 52 ++++++++++--------- 1 file changed, 28 insertions(+), 24 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index c4e9627ee0822..3827bdbe19797 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -160,7 +160,7 @@ public synchronized void run() { private void init() { log.info("Starting container"); - if (this.started.compareAndSet(false, true)) { + if (started.compareAndSet(false, true)) { if (!config.stateDir.exists() && !config.stateDir.mkdirs()) throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); @@ -190,6 +190,7 @@ private void shutdown() { producer.close(); consumer.close(); + shutdownComplete.countDown(); log.info("Shut down complete"); } @@ -197,9 +198,9 @@ private void shutdown() { * Shutdown this streaming instance. */ public synchronized void close() { - this.running = false; + running = false; try { - this.shutdownComplete.await(); + shutdownComplete.await(); } catch (InterruptedException e) { throw new InterruptException(e); } @@ -207,9 +208,14 @@ public synchronized void close() { private void runLoop() { try { + boolean pollRequired = true; + while (stillRunning()) { - boolean pollRequired = false; - + if (pollRequired) { + ingestor.poll(); + pollRequired = false; + } + for (Map.Entry> entry : syncGroups.entrySet()) { for (SyncGroup syncGroup : entry.getValue()) { syncGroup.streamSynchronizer.process(); @@ -218,8 +224,6 @@ private void runLoop() { } maybeCommit(); maybeCleanState(); - - if (pollRequired) ingestor.poll(); } } catch (Exception e) { throw new KafkaException(e); @@ -227,12 +231,12 @@ private void runLoop() { } private boolean stillRunning() { - if(!this.running) { + if(!running) { log.debug("Shutting down at user request."); return false; } - if(this.config.totalRecordsToProcess >= 0 && this.recordsProcessed >= this.config.totalRecordsToProcess) { - log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", this.config.totalRecordsToProcess); + if(config.totalRecordsToProcess >= 0 && recordsProcessed >= config.totalRecordsToProcess) { + log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", config.totalRecordsToProcess); return false; } return true; @@ -240,11 +244,11 @@ private boolean stillRunning() { private void maybeCommit() { long now = time.milliseconds(); - if (this.config.commitTimeMs >= 0 && lastCommit + this.config.commitTimeMs < time.milliseconds()) { + if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < time.milliseconds()) { log.trace("Committing processor instances because the commit interval has elapsed."); commitAll(now); } else { - if (!this.requestingCommit.isEmpty()) { + if (!requestingCommit.isEmpty()) { log.trace("Committing processor instances because of user request."); commitRequesting(now); } @@ -270,12 +274,12 @@ private void commitAll(long now) { producer.flush(); consumer.commit(commit, CommitType.SYNC); // TODO: can this be async? - this.streamingMetrics.commitTime.record(time.milliseconds() - lastCommit); + streamingMetrics.commitTime.record(time.milliseconds() - lastCommit); } private void commitRequesting(long now) { - Map commit = new HashMap(this.requestingCommit.size()); - for (Integer id : this.requestingCommit) { + Map commit = new HashMap(requestingCommit.size()); + for (Integer id : requestingCommit) { KStreamContextImpl context = kstreamContexts.get(id); context.flush(); @@ -284,20 +288,20 @@ private void commitRequesting(long now) { } } consumer.commit(commit, CommitType.SYNC); - this.requestingCommit.clear(); - this.streamingMetrics.commitTime.record(time.milliseconds() - now); + requestingCommit.clear(); + streamingMetrics.commitTime.record(time.milliseconds() - now); } /* delete any state dirs that aren't for active contexts */ private void maybeCleanState() { long now = time.milliseconds(); - if(now > this.nextStateCleaning) { - File[] stateDirs = this.config.stateDir.listFiles(); + if(now > nextStateCleaning) { + File[] stateDirs = config.stateDir.listFiles(); if(stateDirs != null) { for(File dir: stateDirs) { try { Integer id = Integer.parseInt(dir.getName()); - if(!this.kstreamContexts.keySet().contains(id)) { + if(!kstreamContexts.keySet().contains(id)) { log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); Util.rm(dir); } @@ -307,7 +311,7 @@ private void maybeCleanState() { } } } - this.nextStateCleaning = Long.MAX_VALUE; + nextStateCleaning = Long.MAX_VALUE; } } @@ -337,7 +341,7 @@ public void commit(Coordinator.RequestScope scope) { @Override public void shutdown(Coordinator.RequestScope scope) { - throw new IllegalStateException("Implement me"); + running = true; } }; @@ -358,7 +362,7 @@ public void shutdown(Coordinator.RequestScope scope) { } restoreConsumer.close(); - this.nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; + nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; } private void removePartitions(Collection assignment) { @@ -388,7 +392,7 @@ private void removePartitions(Collection assignment) { } } // clear buffered records - this.ingestor.clear(); + ingestor.clear(); } private static Set extractTopics(Class jobClass) { From 8165c280a32f565bc9950bad9abe0f74e78080e7 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 7 Jul 2015 12:50:03 -0700 Subject: [PATCH 039/275] removed tie-breaker from Stamped. fixed MinTimestampTracker --- .../streaming/internal/KStreamContextImpl.java | 2 +- .../streaming/util/MinTimestampTracker.java | 2 +- .../java/io/confluent/streaming/util/Stamped.java | 13 ------------- 3 files changed, 2 insertions(+), 15 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 2b50ef8698e26..ee8f489b99d88 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -63,7 +63,7 @@ public KStreamContextImpl(int id, this.processorConfig = processorConfig; this.timestampExtractor = (TimestampExtractor)this.streamingConfig.timestampExtractor(); - if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractoris missing"); + if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractor is missing"); this.stateDir = new File(processorConfig.stateDir, Integer.toString(id)); this.stateMgr = new ProcessorStateManager(id, stateDir); diff --git a/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java b/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java index 42e62531ee40e..2ec701995b8ed 100644 --- a/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java +++ b/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java @@ -15,7 +15,7 @@ public void addStampedElement(Stamped elem) { if (elem == null) throw new NullPointerException(); Stamped minElem = descendingSubsequence.peekLast(); - while (minElem != null && minElem.compareTo(elem) >= 0) { + while (minElem != null && minElem.timestamp >= elem.timestamp) { descendingSubsequence.removeLast(); minElem = descendingSubsequence.peekLast(); } diff --git a/src/main/java/io/confluent/streaming/util/Stamped.java b/src/main/java/io/confluent/streaming/util/Stamped.java index 3e278fcc20371..2926f271a211f 100644 --- a/src/main/java/io/confluent/streaming/util/Stamped.java +++ b/src/main/java/io/confluent/streaming/util/Stamped.java @@ -7,16 +7,10 @@ public class Stamped implements Comparable { public final V value; public final long timestamp; - private final long tieBreaker; public Stamped(V value, long timestamp) { - this(value, timestamp, System.nanoTime()); - } - - public Stamped(V value, long timestamp, long tieBreaker) { this.value = value; this.timestamp = timestamp; - this.tieBreaker = tieBreaker; } public int compareTo(Object other) { @@ -24,13 +18,6 @@ public int compareTo(Object other) { if (timestamp < otherTimestamp) return -1; else if (timestamp > otherTimestamp) return 1; - - // tie breaking - otherTimestamp = ((Stamped) other).tieBreaker; - - if (tieBreaker < otherTimestamp) return -1; - else if (tieBreaker > otherTimestamp) return 1; - return 0; } From 9c39bbb328ad087ecc300dc59eb0c5f93019139f Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 7 Jul 2015 13:07:32 -0700 Subject: [PATCH 040/275] force range partition assignor --- src/main/java/io/confluent/streaming/StreamingConfig.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/io/confluent/streaming/StreamingConfig.java b/src/main/java/io/confluent/streaming/StreamingConfig.java index 966204a83c35f..af29c0280fecf 100644 --- a/src/main/java/io/confluent/streaming/StreamingConfig.java +++ b/src/main/java/io/confluent/streaming/StreamingConfig.java @@ -64,6 +64,7 @@ public class StreamingConfig { public StreamingConfig(Properties config) { this.config = config; config.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + config.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); config.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); } From 7b3f0496e1d7bc40e59f03c6e82d47f019764704 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 7 Jul 2015 15:07:06 -0700 Subject: [PATCH 041/275] javadoc --- pom.xml | 5 +++ .../io/confluent/streaming/Coordinator.java | 2 +- .../java/io/confluent/streaming/KStream.java | 44 +++++++++++-------- .../io/confluent/streaming/KStreamJob.java | 6 +-- .../confluent/streaming/KStreamWindowed.java | 12 +++-- .../confluent/streaming/KafkaStreaming.java | 7 ++- .../io/confluent/streaming/StorageEngine.java | 18 ++++---- .../streaming/internal/RecordCollectors.java | 1 + .../streaming/util/OffsetCheckpoint.java | 13 +++--- .../streaming/util/TimestampTracker.java | 5 ++- .../io/confluent/streaming/util/Util.java | 12 ++--- 11 files changed, 73 insertions(+), 52 deletions(-) diff --git a/pom.xml b/pom.xml index 2997743d9c15d..3f1c0661f6779 100644 --- a/pom.xml +++ b/pom.xml @@ -44,6 +44,11 @@ 1.6 + + org.apache.maven.plugins + maven-javadoc-plugin + 2.10.3 + diff --git a/src/main/java/io/confluent/streaming/Coordinator.java b/src/main/java/io/confluent/streaming/Coordinator.java index 536421a92e6ca..0676199e35b67 100644 --- a/src/main/java/io/confluent/streaming/Coordinator.java +++ b/src/main/java/io/confluent/streaming/Coordinator.java @@ -18,7 +18,7 @@ package io.confluent.streaming; /** - * Coordinators are provided to the process methods of {@link StreamProcessor} implementations to allow + * Coordinators are provided by the coordiantor methods of {@link KStreamContext} to allow * the user code to request actions from the framework. *

* This interface may evolve over time. diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index d6277029db12f..34f361c75ec87 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -7,50 +7,56 @@ public interface KStream { /** * Creates a new stream consists of all elements of this stream which satisfy a predicate - * @param predicate + * @param predicate the instance of Predicate * @return KStream */ KStream filter(Predicate predicate); /** * Creates a new stream consists all elements of this stream which do not satisfy a predicate - * @param predicate - * @return + * @param predicate the instance of Predicate + * @return KStream */ KStream filterOut(Predicate predicate); /** * Creates a new stream by transforming key-value pairs by a mapper to all elements of this stream - * @param mapper + * @param mapper the instance of KeyValueMapper + * @param the key type of the new stream + * @param the value type of the new stream * @return KStream */ KStream map(KeyValueMapper mapper); /** * Creates a new stream by transforming valuesa by a mapper to all values of this stream - * @param mapper - * @return + * @param mapper the instance of ValueMapper + * @param the value type of the new stream + * @return KStream */ KStream mapValues(ValueMapper mapper); /** * Creates a new stream by applying a mapper to all elements of this stream and using the values in the resulting Iterable - * @param mapper - * @return + * @param mapper the instance of KeyValueMapper + * @param the key type of the new stream + * @param the value type of the new stream + * @return KStream */ KStream flatMap(KeyValueMapper, K, V> 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 - * @return + * @param processor the instance of Processor + * @param the value type of the new stream + * @return KStream */ KStream flatMapValues(ValueMapper, V> processor); /** - * Creates a new windowed stream using a specified window object. - * @param window - * @return + * Creates a new windowed stream using a specified window instance. + * @param window the instance of Window + * @return KStream */ KStreamWindowed with(Window window); @@ -59,28 +65,28 @@ public interface KStream { * 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 - * @return + * @param predicates Instances of Predicate + * @return KStream */ KStream[] branch(Predicate... predicates); /** * Sends key-value to a topic, also creates a new stream from the topic. * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). - * @param topic - * @return + * @param topic the topic name + * @return KStream */ KStream through(String topic); /** * Sends key-value to a topic. - * @param topic + * @param topic the topic name */ void sendTo(String topic); /** * Processes all elements in this stream by applying a processor. - * @param processor + * @param processor the instance of Processor */ void process(Processor processor); diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java index 6cb009a90ca51..2e2bab10eed25 100644 --- a/src/main/java/io/confluent/streaming/KStreamJob.java +++ b/src/main/java/io/confluent/streaming/KStreamJob.java @@ -40,11 +40,11 @@ public static class Topics { *

*
    *   public init(KStreamContext context) {
-   *     KStream<Integer, PageView> pageViewStream = context.from("pageView").mapValues(...)
-   *     KStream<Integer, AdClick> adClickStream = context.from("adClick").join(pageViewStream, ...).process(...);
+   *     KStream<Integer, PageView> pageViewStream = context.from("pageView").mapValues(...);
+   *     KStream<Integer, AdClick> adClickStream = context.from("adClick").join(pageViewStream, ...).process(...);
    *   }
    * 
- * @param context + * @param context KStreamContext */ public abstract void init(KStreamContext context); diff --git a/src/main/java/io/confluent/streaming/KStreamWindowed.java b/src/main/java/io/confluent/streaming/KStreamWindowed.java index 2556065f3bba6..c058d082c947b 100644 --- a/src/main/java/io/confluent/streaming/KStreamWindowed.java +++ b/src/main/java/io/confluent/streaming/KStreamWindowed.java @@ -10,8 +10,10 @@ public interface KStreamWindowed extends KStream { * 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 - * @param joiner + * @param other the other windowed stream + * @param joiner ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream * @return KStream */ KStream join(KStreamWindowed other, ValueJoiner joiner); @@ -22,8 +24,10 @@ public interface KStreamWindowed extends KStream { * the element from the other stream has an older timestamp. * The resulting values are computed by applying a joiner. * - * @param other - * @param joiner + * @param other the other windowed stream + * @param joiner the instance ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream * @return KStream */ KStream joinPrior(KStreamWindowed other, ValueJoiner joiner); diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 3827bdbe19797..ce3fc024f3cb8 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -52,13 +52,13 @@ * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and * sends output to zero or more output topics. *

- * This processing is done by implementing the {@link StreamProcessor} interface to specify the transformation. The + * This processing is done by implementing the {@link KStreamJob} interface to specify the transformation. The * {@link KafkaStreaming} instance will be responsible for the lifecycle of these processors. It will instantiate and * start one or more of these processors to process the Kafka partitions assigned to this particular instance. *

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

* Internally the {@link KafkaStreaming} instance contains a normal {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer} @@ -69,10 +69,9 @@ * Properties props = new Properties(); * props.put("bootstrap.servers", "localhost:4242"); * StreamingConfig config = new StreamingConfig(props); - * config.subscribe("test-topic-1", "test-topic-2"); * config.processor(ExampleStreamProcessor.class); * config.serialization(new StringSerializer(), new StringDeserializer()); - * KafkaStreaming container = new KafkaStreaming(config); + * KafkaStreaming container = new KafkaStreaming(MyKStreamJob.class, config); * container.run(); * * diff --git a/src/main/java/io/confluent/streaming/StorageEngine.java b/src/main/java/io/confluent/streaming/StorageEngine.java index cbd825908cdb2..0f6ab6e3e2f21 100644 --- a/src/main/java/io/confluent/streaming/StorageEngine.java +++ b/src/main/java/io/confluent/streaming/StorageEngine.java @@ -34,31 +34,33 @@ public interface StorageEngine { /** * The name of this store. + * @return the storage name */ - public String name(); + String name(); /** * Register the given storage engine with the changelog and restore it's state using the given * consumer instance. + * @param collector The record collector to write records to * @param consumer The consumer to read with * @param partition The partition to use as the change log * @param checkpointedOffset The offset of the last save * @param logEndOffset The last offset in the changelog */ - public void registerAndRestore(RecordCollector collector, - Consumer consumer, - TopicPartition partition, - long checkpointedOffset, - long logEndOffset); + void registerAndRestore(RecordCollector collector, + Consumer consumer, + TopicPartition partition, + long checkpointedOffset, + long logEndOffset); /** * Flush any cached data */ - public void flush(); + void flush(); /** * Close the storage engine */ - public void close(); + void close(); } diff --git a/src/main/java/io/confluent/streaming/internal/RecordCollectors.java b/src/main/java/io/confluent/streaming/internal/RecordCollectors.java index 6bc330681d962..ab0162ea84c5a 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordCollectors.java +++ b/src/main/java/io/confluent/streaming/internal/RecordCollectors.java @@ -47,6 +47,7 @@ public void send(ProducerRecord record) { /** * The last ack'd offset from the producer + * @return the map from TopicPartition to offset */ public Map offsets() { return this.offsets; diff --git a/src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java b/src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java index a2d29b89c7553..946c259eaf4ea 100644 --- a/src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java +++ b/src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java @@ -24,15 +24,16 @@ import java.util.Map; /** - * This class saves out a map of topic/partition=>offsets to a file. The format of the file is UTF-8 text containing the following: - * - * - * + * This class saves out a map of topic/partition=>offsets to a file. The format of the file is UTF-8 text containing the following: + *

+ *   <version>
+ *   <n>
+ *   <topic_name_1> <partition_1> <offset_1>
  *   .
  *   .
  *   .
- *     
- *
+ *   <topic_name_n> <partition_n> <offset_n>
+ * 
* The first line contains a number designating the format version (currently 0), the next line contains * a number giving the total number of offsets. Each successive line gives a topic/partition/offset triple * separated by spaces. diff --git a/src/main/java/io/confluent/streaming/util/TimestampTracker.java b/src/main/java/io/confluent/streaming/util/TimestampTracker.java index 87169bf2c7989..97a34221b8431 100644 --- a/src/main/java/io/confluent/streaming/util/TimestampTracker.java +++ b/src/main/java/io/confluent/streaming/util/TimestampTracker.java @@ -10,13 +10,13 @@ public interface TimestampTracker { /** * Adds a stamped elements to this tracker. - * @param elem + * @param elem the added element */ void addStampedElement(Stamped elem); /** * Removed a stamped elements to this tracker. - * @param elem + * @param elem the removed element */ void removeStampedElement(Stamped elem); @@ -28,6 +28,7 @@ public interface TimestampTracker { /** * Returns the size of internal structure. The meaning of "size" depends on the implementation. + * @return size */ int size(); diff --git a/src/main/java/io/confluent/streaming/util/Util.java b/src/main/java/io/confluent/streaming/util/Util.java index 4a6a71eea11a7..280fa87e24b71 100644 --- a/src/main/java/io/confluent/streaming/util/Util.java +++ b/src/main/java/io/confluent/streaming/util/Util.java @@ -12,7 +12,8 @@ public class Util { /** * Creates a set - * @param elems + * @param elems the elements + * @param the type of element * @return Set */ public static HashSet mkSet(T... elems) { @@ -25,9 +26,9 @@ public static HashSet mkSet(T... elems) { * Gets a value from a field of a class * @param clazz the class object * @param obj the instance object, or maybe null for a static field - * @Param fieldName the name of the field + * @param fieldName the name of the field * @return Object - * @throws Exception + * @throws Exception the exception */ public static Object getFieldValue(Class clazz, Object obj, String fieldName) throws Exception { Field myField = clazz.getDeclaredField(fieldName); @@ -35,8 +36,9 @@ public static Object getFieldValue(Class clazz, Object obj, String fieldName) } /** - * Makes a srtring of a comma separated list of collection elements - * @param collection + * Makes a string of a comma separated list of collection elements + * @param collection the collection + * @param the type of collection elements * @return String */ public static String mkString(Collection collection) { From cef40bda786e14cd214ea53ae71f0325255c1d84 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 7 Jul 2015 18:06:22 -0700 Subject: [PATCH 042/275] add kafka-clients and log4j to pom.xml --- pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/pom.xml b/pom.xml index 3f1c0661f6779..00fd0c924dbad 100644 --- a/pom.xml +++ b/pom.xml @@ -25,12 +25,22 @@ kafka_${kafka.scala.version} ${kafka.version} + + org.apache.kafka + kafka-clients + ${kafka.version} + junit junit 4.8.1 test + + org.slf4j + slf4j-log4j12 + 1.7.6 + From 13fe3dfa1e8d1b74a62426a7186b1d08e1f7aa3c Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 9 Jul 2015 11:07:54 +0100 Subject: [PATCH 043/275] Require Java 7 --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 00fd0c924dbad..7b4ae64a314a8 100644 --- a/pom.xml +++ b/pom.xml @@ -50,8 +50,8 @@ maven-compiler-plugin 3.3 - 1.6 - 1.6 + 1.7 + 1.7 From 7f031478e2f452a1e884385c738b08120d4d2aad Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 9 Jul 2015 10:53:28 -0700 Subject: [PATCH 044/275] added close method to kstreamjob --- .../confluent/streaming/KStreamContext.java | 73 ++++++++++++++++++- .../io/confluent/streaming/KStreamJob.java | 46 ++++-------- .../confluent/streaming/KafkaStreaming.java | 7 +- .../java/io/confluent/streaming/Topics.java | 19 +++++ .../internal/KStreamContextImpl.java | 6 +- .../io/confluent/streaming/util/Util.java | 13 ---- 6 files changed, 116 insertions(+), 48 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/Topics.java diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index cc73592df0449..289c50ea0f442 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -8,42 +8,113 @@ import java.util.Map; /** - * Created by yasuhiro on 6/19/15. + * KStreamContext is the interface that allows an implementation of {@link KStreamJob#init(KStreamContext)} to create KStream instances. + * It also provides access to the system resources for a stream processing job. + * An instance of KStreamContext is created for each partition. */ public interface KStreamContext { String DEFAULT_SYNCHRONIZATION_GROUP = "defaultSynchronizationGroup"; + /** + * Returns the partition id + * @return partition 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(); + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * @param topic + * @return KStream + */ KStream from(String topic); + /** + * Creates a KStream instance for the specified topic. The stream is added to the specified synchronization group. + * @param topic + * @return KStream + */ KStream from(String topic, SyncGroup syncGroup); + /** + * Returns a RecordCollector which takes binary (byte array) key and value. + * @return + */ RecordCollector simpleRecordCollector(); + /** + * Returns a RecordCollector which applies the serializer to key and value. + * @return + */ RecordCollector recordCollector(); + /** + * Returns {@link Coordinator}. + * @return Coordinator + */ Coordinator coordinator(); + /** + * Returns an application context registered to {@link StreamingConfig}. + * @return an application context + */ Map getContext(); + /** + * Returns the state directory for the partition. + * @return the state directory + */ File stateDir(); + /** + * Returns Metrics instance + * @return Metrics + */ Metrics metrics(); + /** + * Creates a synchronization group with the given name. + * @param name + * @return a synchronization group + */ SyncGroup syncGroup(String name); + /** + * Creates a round robin synchronization group with the given name. + * @param name + * @return a round robin synchronization group + */ SyncGroup roundRobinSyncGroup(String name); + /** + * Restores the specified storage engine. + * @param engine + * @throws Exception + */ void restore(StorageEngine engine) throws Exception; } diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java index 2e2bab10eed25..528785a05354d 100644 --- a/src/main/java/io/confluent/streaming/KStreamJob.java +++ b/src/main/java/io/confluent/streaming/KStreamJob.java @@ -1,42 +1,22 @@ package io.confluent.streaming; -import io.confluent.streaming.util.Util; - -import java.util.Collections; -import java.util.Set; - /** * An interface to implement an application logic of a stream processing. - * - * Each subclass must declare a public static final member variable topics like this: - *
- *   class MyKStreamJob {
- *     public static final Topics topics = new Topics("pageView", "adClick");
- *     ...
- *   }
- * 
- * - * Topics represents a set of topics used in the application. + * An instance is created and initialized by the framework for each partition. */ -public abstract class KStreamJob { +public interface KStreamJob { /** - * A class represents a set of topic names. + * Returns a set of topics used in this job + * @return */ - public static class Topics { - - public final Set set; - - Topics(String... topics) { - set = Collections.unmodifiableSet(Util.mkSet(topics)); - } - - } + Topics topics(); /** - * Initializes a streaming job by constructing a processing logic using KStream API. + * Initializes a stream processing job for a partition. This method is called for each partition. + * An application constructs a processing logic using KStream API. *

- * For exmaple, + * For example, *

*
    *   public init(KStreamContext context) {
@@ -44,8 +24,14 @@ public static class Topics {
    *     KStream<Integer, AdClick> adClickStream = context.from("adClick").join(pageViewStream, ...).process(...);
    *   }
    * 
- * @param context KStreamContext + * @param context KStreamContext for this partition + */ + void init(KStreamContext context); + + /** + * Closes this partition of the stream processing job. + * An application can perform its special clean up here. */ - public abstract void init(KStreamContext context); + void close(); } diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index ce3fc024f3cb8..ccb1bc9f751e8 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -345,12 +345,12 @@ public void shutdown(Coordinator.RequestScope scope) { }; kstreamContext = - new KStreamContextImpl(id, topics, ingestor, producer, coordinator, streamingConfig, config, metrics); + new KStreamContextImpl(id, job, topics, ingestor, producer, coordinator, streamingConfig, config, metrics); kstreamContexts.put(id, kstreamContext); try { - kstreamContext.init(restoreConsumer, job); + kstreamContext.init(restoreConsumer); } catch (Exception e) { throw new KafkaException(e); @@ -397,7 +397,8 @@ private void removePartitions(Collection assignment) { private static Set extractTopics(Class jobClass) { // extract topics from a jobClass's static member field, topics try { - return ((KStreamJob.Topics)Util.getFieldValue(jobClass, null, "topics")).set; + Object instance = Utils.newInstance(jobClass); + return ((Topics)instance).topics; } catch (Exception e) { throw new KStreamException("failed to get a topic list from the job", e); diff --git a/src/main/java/io/confluent/streaming/Topics.java b/src/main/java/io/confluent/streaming/Topics.java new file mode 100644 index 0000000000000..55df44f484c6b --- /dev/null +++ b/src/main/java/io/confluent/streaming/Topics.java @@ -0,0 +1,19 @@ +package io.confluent.streaming; + +import io.confluent.streaming.util.Util; + +import java.util.Collections; +import java.util.Set; + +/** + * A class represents a set of topic names. + */ +public class Topics { + + public final Set topics; + + Topics(String... topics) { + this.topics = Collections.unmodifiableSet(Util.mkSet(topics)); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index ee8f489b99d88..5b31efc0ac4a4 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -23,6 +23,7 @@ public class KStreamContextImpl implements KStreamContext { private static final Logger log = LoggerFactory.getLogger(KStreamContextImpl.class); public final int id; + private final KStreamJob job; private final Set topics; private final Ingestor ingestor; @@ -43,6 +44,7 @@ public class KStreamContextImpl implements KStreamContext { @SuppressWarnings("unchecked") public KStreamContextImpl(int id, + KStreamJob job, Set topics, Ingestor ingestor, Producer producer, @@ -51,6 +53,7 @@ public KStreamContextImpl(int id, ProcessorConfig processorConfig, Metrics metrics) { this.id = id; + this.job = job; this.topics = topics; this.ingestor = ingestor; @@ -203,7 +206,7 @@ public Collection syncGroups() { return syncGroups.values(); } - public void init(Consumer restoreConsumer, KStreamJob job) throws IOException { + public void init(Consumer restoreConsumer) throws IOException { stateMgr.init(); try { this.restoreConsumer = restoreConsumer; @@ -229,6 +232,7 @@ public void flush() { public void close() throws Exception { stateMgr.close(simpleCollector.offsets()); + job.close(); } } diff --git a/src/main/java/io/confluent/streaming/util/Util.java b/src/main/java/io/confluent/streaming/util/Util.java index 280fa87e24b71..071edc8de2d9a 100644 --- a/src/main/java/io/confluent/streaming/util/Util.java +++ b/src/main/java/io/confluent/streaming/util/Util.java @@ -22,19 +22,6 @@ public static HashSet mkSet(T... elems) { return set; } - /** - * Gets a value from a field of a class - * @param clazz the class object - * @param obj the instance object, or maybe null for a static field - * @param fieldName the name of the field - * @return Object - * @throws Exception the exception - */ - public static Object getFieldValue(Class clazz, Object obj, String fieldName) throws Exception { - Field myField = clazz.getDeclaredField(fieldName); - return myField.get(obj); - } - /** * Makes a string of a comma separated list of collection elements * @param collection the collection From d6edb1840e389ec037bcacea6b15cb72645c2907 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 9 Jul 2015 11:09:28 -0700 Subject: [PATCH 045/275] fix javadoc --- .../io/confluent/streaming/KStreamContext.java | 17 +++++++++-------- .../java/io/confluent/streaming/KStreamJob.java | 2 +- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 289c50ea0f442..287b7ad8543d6 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -48,27 +48,28 @@ public interface KStreamContext { /** * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. - * @param topic + * @param topic the topic name * @return KStream */ KStream from(String topic); /** * Creates a KStream instance for the specified topic. The stream is added to the specified synchronization group. - * @param topic + * @param topic the topic name + * @param syncGroup the synchronization group * @return KStream */ KStream from(String topic, SyncGroup syncGroup); /** * Returns a RecordCollector which takes binary (byte array) key and value. - * @return + * @return RecordCollector */ RecordCollector simpleRecordCollector(); /** * Returns a RecordCollector which applies the serializer to key and value. - * @return + * @return RecordCollector */ RecordCollector recordCollector(); @@ -98,22 +99,22 @@ public interface KStreamContext { /** * Creates a synchronization group with the given name. - * @param name + * @param name the synchronization group name * @return a synchronization group */ SyncGroup syncGroup(String name); /** * Creates a round robin synchronization group with the given name. - * @param name + * @param name the synchronization group name * @return a round robin synchronization group */ SyncGroup roundRobinSyncGroup(String name); /** * Restores the specified storage engine. - * @param engine - * @throws Exception + * @param engine the storage engine + * @throws Exception an exception thrown by the engine */ void restore(StorageEngine engine) throws Exception; diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java index 528785a05354d..e61b991c418d8 100644 --- a/src/main/java/io/confluent/streaming/KStreamJob.java +++ b/src/main/java/io/confluent/streaming/KStreamJob.java @@ -8,7 +8,7 @@ public interface KStreamJob { /** * Returns a set of topics used in this job - * @return + * @return Topics */ Topics topics(); From 5227b630099533cfe45ffbe69cf2e7aa73693b42 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 9 Jul 2015 16:58:15 -0700 Subject: [PATCH 046/275] parallel execution --- .../confluent/streaming/KafkaStreaming.java | 89 ++++++------ .../confluent/streaming/StreamingConfig.java | 3 + .../streaming/internal/ProcessorConfig.java | 9 +- .../internal/StreamSynchronizer.java | 61 +++++--- .../streaming/util/ParallelExecutor.java | 135 ++++++++++++++++++ .../streaming/util/ParallelExecutorTest.java | 128 +++++++++++++++++ 6 files changed, 364 insertions(+), 61 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/util/ParallelExecutor.java create mode 100644 src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index ccb1bc9f751e8..8f807dd8e4d34 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -20,6 +20,8 @@ import io.confluent.streaming.internal.KStreamContextImpl; import io.confluent.streaming.internal.ProcessorConfig; import io.confluent.streaming.internal.IngestorImpl; +import io.confluent.streaming.internal.StreamSynchronizer; +import io.confluent.streaming.util.ParallelExecutor; import io.confluent.streaming.util.Util; import org.apache.kafka.clients.consumer.*; import org.apache.kafka.clients.producer.KafkaProducer; @@ -82,8 +84,10 @@ public class KafkaStreaming implements Runnable { private final Class jobClass; private final Set topics; - private final Map> syncGroups = new HashMap>(); - private final Map kstreamContexts = new HashMap(); + private final Map> syncGroups = new HashMap<>(); + private final ArrayList> streamSynchronizers = new ArrayList<>(); + private final ParallelExecutor parallelExecutor; + private final Map kstreamContexts = new HashMap<>(); protected final Producer producer; protected final Consumer consumer; private final IngestorImpl ingestor; @@ -123,23 +127,24 @@ protected KafkaStreaming(Class jobClass, Consumer consumer) { this.jobClass = jobClass; this.topics = extractTopics(jobClass); - this.producer = producer == null? new KafkaProducer(config.config(), new ByteArraySerializer(), new ByteArraySerializer()): producer; - this.consumer = consumer == null? new KafkaConsumer(config.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()): consumer; + this.producer = producer == null? new KafkaProducer<>(config.config(), new ByteArraySerializer(), new ByteArraySerializer()): producer; + this.consumer = consumer == null? new KafkaConsumer<>(config.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()): consumer; this.streamingConfig = config; this.metrics = new Metrics(); this.streamingMetrics = new KafkaStreamingMetrics(); - this.requestingCommit = new ArrayList(); + this.requestingCommit = new ArrayList<>(); this.config = new ProcessorConfig(config.config()); this.ingestor = - new IngestorImpl(this.consumer, - (Deserializer) config.keyDeserializer(), - (Deserializer) config.valueDeserializer(), - this.config.pollTimeMs); + new IngestorImpl<>(this.consumer, + (Deserializer) config.keyDeserializer(), + (Deserializer) config.valueDeserializer(), + this.config.pollTimeMs); this.running = true; this.lastCommit = 0; this.nextStateCleaning = Long.MAX_VALUE; this.recordsProcessed = 0; this.time = new SystemTime(); + this.parallelExecutor = new ParallelExecutor(this.config.numStreamThreads); } /** @@ -176,19 +181,20 @@ private void shutdown() { log.info("Shutting down container"); commitAll(time.milliseconds()); - for (Map.Entry> entry : syncGroups.entrySet()) { - for (SyncGroup syncGroup : entry.getValue()) { - try { - syncGroup.streamSynchronizer.close(); - } - catch(Exception e) { - log.error("Error while closing stream synchronizers: ", e); - } + for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { + try { + streamSynchronizer.close(); + } + catch(Exception e) { + log.error("Error while closing stream synchronizers: ", e); } } producer.close(); consumer.close(); + parallelExecutor.shutdown(); + syncGroups.clear(); + streamSynchronizers.clear(); shutdownComplete.countDown(); log.info("Shut down complete"); } @@ -207,20 +213,17 @@ public synchronized void close() { private void runLoop() { try { - boolean pollRequired = true; + StreamSynchronizer.Status status = new StreamSynchronizer.Status(); + status.pollRequired(true); while (stillRunning()) { - if (pollRequired) { + if (status.pollRequired()) { ingestor.poll(); - pollRequired = false; + status.pollRequired(false); } - for (Map.Entry> entry : syncGroups.entrySet()) { - for (SyncGroup syncGroup : entry.getValue()) { - syncGroup.streamSynchronizer.process(); - pollRequired = pollRequired || syncGroup.streamSynchronizer.requiresPoll(); - } - } + parallelExecutor.execute(streamSynchronizers, status); + maybeCommit(); maybeCleanState(); } @@ -255,19 +258,17 @@ private void maybeCommit() { } private void commitAll(long now) { - Map commit = new HashMap(); + Map commit = new HashMap<>(); for (KStreamContextImpl context : kstreamContexts.values()) { context.flush(); // check co-ordinator } - for (Map.Entry> entry : syncGroups.entrySet()) { - for (SyncGroup syncGroup : entry.getValue()) { - try { - commit.putAll(syncGroup.streamSynchronizer.consumedOffsets()); - } - catch(Exception e) { - log.error("Error while closing processor: ", e); - } + for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { + try { + commit.putAll(streamSynchronizer.consumedOffsets()); + } + catch(Exception e) { + log.error("Error while closing processor: ", e); } } @@ -277,7 +278,7 @@ private void commitAll(long now) { } private void commitRequesting(long now) { - Map commit = new HashMap(requestingCommit.size()); + Map commit = new HashMap<>(requestingCommit.size()); for (Integer id : requestingCommit) { KStreamContextImpl context = kstreamContexts.get(id); context.flush(); @@ -315,16 +316,12 @@ private void maybeCleanState() { } private void addPartitions(Collection assignment) { - HashSet partitions = new HashSet(assignment); + HashSet partitions = new HashSet<>(assignment); ingestor.init(); Consumer restoreConsumer = - new KafkaConsumer( - streamingConfig.config(), - null, - new ByteArrayDeserializer(), - new ByteArrayDeserializer()); + new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); for (TopicPartition partition : partitions) { final Integer id = partition.partition(); @@ -356,7 +353,11 @@ public void shutdown(Coordinator.RequestScope scope) { throw new KafkaException(e); } - syncGroups.put(id, kstreamContext.syncGroups()); + Collection syncGroups = kstreamContext.syncGroups(); + this.syncGroups.put(id, syncGroups); + for (SyncGroup syncGroup : syncGroups) { + streamSynchronizers.add(syncGroup.streamSynchronizer); + } } } @@ -390,7 +391,7 @@ private void removePartitions(Collection assignment) { streamingMetrics.processorDestruction.record(); } } - // clear buffered records + streamSynchronizers.clear(); ingestor.clear(); } diff --git a/src/main/java/io/confluent/streaming/StreamingConfig.java b/src/main/java/io/confluent/streaming/StreamingConfig.java index af29c0280fecf..1ae31691ab600 100644 --- a/src/main/java/io/confluent/streaming/StreamingConfig.java +++ b/src/main/java/io/confluent/streaming/StreamingConfig.java @@ -53,6 +53,9 @@ public class StreamingConfig { /** total.records.to.process */ public static final String TOTAL_RECORDS_TO_PROCESS = "total.records.to.process"; + /** num.of.stream.threads */ + public static final String NUM_STREAM_THREADS = "num.of.stream.threads"; + private final Properties config; private final Map context = new HashMap(); private Serializer keySerializer; diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java b/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java index 4729fccb2ef04..84679e6d65c87 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java @@ -65,7 +65,12 @@ public class ProcessorConfig extends AbstractConfig { Type.LONG, -1L, Importance.LOW, - "Exit after processing this many records."); + "Exit after processing this many records.") + .define(StreamingConfig.NUM_STREAM_THREADS, + Type.INT, + 1, + Importance.LOW, + "The number of threads to execute stream processing."); } public final File stateDir; @@ -75,6 +80,7 @@ public class ProcessorConfig extends AbstractConfig { public final int bufferedRecordsPerPartition; public final long stateCleanupDelay; public final long totalRecordsToProcess; + public final int numStreamThreads; public ProcessorConfig(Properties processor) { super(CONFIG, processor); @@ -85,6 +91,7 @@ public ProcessorConfig(Properties processor) { this.bufferedRecordsPerPartition = this.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); this.stateCleanupDelay = this.getLong(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG); this.totalRecordsToProcess = this.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); + this.numStreamThreads = this.getInt(StreamingConfig.NUM_STREAM_THREADS); } } diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index 269bad6db2779..475117951b099 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -2,29 +2,44 @@ import io.confluent.streaming.*; import io.confluent.streaming.util.MinTimestampTracker; +import io.confluent.streaming.util.ParallelExecutor; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import java.util.ArrayDeque; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; /** * Created by yasuhiro on 6/23/15. */ -public class StreamSynchronizer { +public class StreamSynchronizer implements ParallelExecutor.Task { + + public static class Status { + private AtomicBoolean pollRequired = new AtomicBoolean(); + + public void pollRequired(boolean flag) { + pollRequired.set(flag); + } + + public boolean pollRequired() { + return pollRequired.get(); + } + } public final String name; private final Ingestor ingestor; private final Chooser chooser; private final TimestampExtractor timestampExtractor; - private final Map> stash = new HashMap>(); + private final Map> stash = new HashMap<>(); private final int desiredUnprocessed; private final Map consumedOffsets; private final PunctuationQueue punctuationQueue = new PunctuationQueue(); + private final ArrayDeque> newRecordBuffer = new ArrayDeque<>(); private long streamTime = -1; - private boolean pollRequired = false; private volatile int buffered = 0; StreamSynchronizer(String name, @@ -37,7 +52,7 @@ public class StreamSynchronizer { this.chooser = chooser; this.timestampExtractor = timestampExtractor; this.desiredUnprocessed = desiredUnprocessedPerPartition; - this.consumedOffsets = new HashMap(); + this.consumedOffsets = new HashMap<>(); } @SuppressWarnings("unchecked") @@ -53,9 +68,17 @@ public void addPartition(TopicPartition partition, Receiver rece } } - @SuppressWarnings("unchecked") public void addRecords(TopicPartition partition, Iterator> iterator) { synchronized (this) { + newRecordBuffer.addLast(new NewRecords<>(partition, iterator)); + } + } + + private void ingestNewRecords() { + for (NewRecords newRecords : newRecordBuffer) { + TopicPartition partition = newRecords.partition; + Iterator> iterator = newRecords.iterator; + RecordQueue recordQueue = stash.get(partition); if (recordQueue != null) { boolean wasEmpty = recordQueue.isEmpty(); @@ -63,7 +86,7 @@ public void addRecords(TopicPartition partition, Iterator> while (iterator.hasNext()) { ConsumerRecord record = iterator.next(); long timestamp = timestampExtractor.extract(record.topic(), record.key(), record.value()); - recordQueue.add(new StampedRecord(record, timestamp)); + recordQueue.add(new StampedRecord<>(record, timestamp)); buffered++; } @@ -78,22 +101,19 @@ public void addRecords(TopicPartition partition, Iterator> } } - public boolean requiresPoll() { - return pollRequired; - } - public PunctuationScheduler getPunctuationScheduler(Processor processor) { return new PunctuationSchedulerImpl(punctuationQueue, processor); } @SuppressWarnings("unchecked") - public void process() { + public void process(Object context) { + Status status = (Status) context; synchronized (this) { - pollRequired = false; + ingestNewRecords(); RecordQueue recordQueue = chooser.next(); if (recordQueue == null) { - pollRequired = true; + status.pollRequired(true); return; } @@ -101,12 +121,14 @@ public void process() { if (recordQueue.size() == this.desiredUnprocessed) { ingestor.unpause(recordQueue.partition(), recordQueue.offset()); - pollRequired = true; } long trackedTimestamp = recordQueue.trackedTimestamp(); StampedRecord record = recordQueue.next(); + if (recordQueue.size() < this.desiredUnprocessed) + status.pollRequired(true); + if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; recordQueue.receiver.receive(record.key(), record.value(), record.timestamp, streamTime); @@ -117,8 +139,6 @@ public void process() { buffered--; punctuationQueue.mayPunctuate(streamTime); - - return; } } @@ -139,4 +159,13 @@ protected RecordQueue createRecordQueue(TopicPartition partition, Receiver return new RecordQueue(partition, receiver, new MinTimestampTracker>()); } + private static class NewRecords { + final TopicPartition partition; + final Iterator> iterator; + + NewRecords(TopicPartition partition, Iterator> iterator) { + this.partition = partition; + this.iterator = iterator; + } + } } diff --git a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java new file mode 100644 index 0000000000000..2cb658bf3aad9 --- /dev/null +++ b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java @@ -0,0 +1,135 @@ +package io.confluent.streaming.util; + +import java.util.ArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.LockSupport; + +/** + * A lightweight parallel executor + */ +public class ParallelExecutor { + + /** + * A parallel task must implement this interface + */ + public interface Task { + /** + * Executes a task + * @param context an application specific context object for a task + */ + void process(Object context); + } + + private final WorkerThread[] workerThreads; + private final AtomicInteger taskIndex = new AtomicInteger(0); + private volatile ArrayList tasks = new ArrayList(); + private volatile Object context; + private volatile CountDownLatch latch; + private volatile boolean running = true; + private volatile Exception exception; + + public ParallelExecutor(int parallelDegree) { + parallelDegree = Math.max(parallelDegree, 1); + workerThreads = new WorkerThread[parallelDegree - 1]; + for (int i = 0; i < workerThreads.length; i++) { + workerThreads[i] = new WorkerThread(); + workerThreads[i].start(); + } + } + + /** + * Executes tasks in parallel. While this method is executing, other execute call will be blocked. + * @param tasks a list of tasks executed in parallel + * @param context a context object passed to tasks + * @throws Exception an exception thrown by a failed task + */ + public void execute(ArrayList tasks, Object context) throws Exception { + synchronized (this) { + try { + int numTasks = tasks.size(); + exception = null; + if (numTasks > 0) { + this.tasks = tasks; + this.context = context; + this.latch = new CountDownLatch(numTasks); + + taskIndex.set(numTasks); + wakeUpWorkers(Math.min(numTasks - 1, workerThreads.length)); + + // the calling thread also picks up tasks + if (taskIndex.get() > 0) doRun(); + + while (true) { + try { + latch.await(); + break; + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + } + if (exception != null) throw exception; + } + finally { + this.tasks = null; + this.context = null; + this.latch = null; + this.exception = null; + } + } + } + + /** + * Shuts this parallel executor down + */ + public void shutdown() { + synchronized (this) { + running = false; + // wake up all workers + wakeUpWorkers(workerThreads.length); + } + } + + private void doRun() { + int index = taskIndex.decrementAndGet(); + if (index >= 0) { + try { + tasks.get(index).process(context); + } + catch (Exception ex) { + exception = ex; + } + finally { + latch.countDown(); + } + } + } + + private void wakeUpWorkers(int numWorkers) { + for (int i = 0; i < numWorkers; i++) + LockSupport.unpark(workerThreads[i]); + } + + private class WorkerThread extends Thread { + + WorkerThread() { + super(); + setDaemon(true); + } + + @Override + public void run() { + while (running) { + if (taskIndex.get() > 0) { + doRun(); + } + else { + // no more work. park this thread. + LockSupport.park(); + } + } + } + } + +} diff --git a/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java b/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java new file mode 100644 index 0000000000000..1bafbfe5aa8c8 --- /dev/null +++ b/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java @@ -0,0 +1,128 @@ +package io.confluent.streaming.util; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ParallelExecutorTest { + + @Test + public void testExecutingShortTaskList() throws Exception { + ParallelExecutor parallelExecutor = new ParallelExecutor(10); + ArrayList taskList = new ArrayList<>(); + Counter counter = new Counter(); + + for (int i = 0; i < 5; i++) { + taskList.add(new TestTask()); + } + + parallelExecutor.execute(taskList, counter); + + for (TestTask task : taskList) { + assertEquals(task.executionCount, 1); + } + assertEquals(counter.get(), taskList.size()); + + parallelExecutor.execute(taskList, counter); + + for (TestTask task : taskList) { + assertEquals(task.executionCount, 2); + } + assertEquals(counter.get(), taskList.size() * 2); + } + + @Test + public void testExecutingLongTaskList() throws Exception { + ParallelExecutor parallelExecutor = new ParallelExecutor(10); + ArrayList taskList = new ArrayList<>(); + Counter counter = new Counter(); + + for (int i = 0; i < 20; i++) { + taskList.add(new TestTask()); + } + + parallelExecutor.execute(taskList, counter); + + for (TestTask task : taskList) { + assertEquals(task.executionCount, 1); + } + assertEquals(counter.get(), taskList.size()); + + parallelExecutor.execute(taskList, counter); + + for (TestTask task : taskList) { + assertEquals(task.executionCount, 2); + } + assertEquals(counter.get(), taskList.size() * 2); + } + + @Test + public void testException() { + ParallelExecutor parallelExecutor = new ParallelExecutor(10); + ArrayList taskList = new ArrayList<>(); + Counter counter = new Counter(); + + for (int i = 0; i < 20; i++) { + if (i == 15) { + taskList.add(new TestTask() { + @Override + public void process(Object context) { + throw new TestException(); + } + }); + } + else { + taskList.add(new TestTask()); + } + } + + Exception exception = null; + try { + parallelExecutor.execute(taskList, counter); + } + catch (Exception ex) { + exception = ex; + } + + assertEquals(counter.get(), taskList.size() - 1); + assertFalse(exception == null); + assertTrue(exception instanceof TestException); + } + + private static class TestTask implements ParallelExecutor.Task { + public volatile int executionCount = 0; + + @Override + public void process(Object context) { + Counter counter = (Counter) context; + try { + Thread.sleep(20); + executionCount++; + } + catch (Exception ex) { + // ignore + } + counter.incr(); + } + } + + private static class Counter { + private AtomicInteger totalExecutionCount = new AtomicInteger(); + + public void incr() { + totalExecutionCount.incrementAndGet(); + } + + public int get() { + return totalExecutionCount.get(); + } + } + + private static class TestException extends RuntimeException { + } +} From 8e8e6e44a4aa6ff94359534e7a042cf863788039 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 10 Jul 2015 12:44:33 -0700 Subject: [PATCH 047/275] removed some generics --- .../confluent/streaming/KafkaStreaming.java | 63 ++++++++++++++++--- .../confluent/streaming/StreamingConfig.java | 6 +- .../io/confluent/streaming/SyncGroup.java | 10 +-- .../streaming/TimestampExtractor.java | 13 +++- .../confluent/streaming/internal/Chooser.java | 6 +- .../streaming/internal/Ingestor.java | 2 +- .../streaming/internal/IngestorImpl.java | 33 +++++----- .../streaming/internal/KStreamBranch.java | 6 +- .../internal/KStreamContextImpl.java | 34 +++++----- .../streaming/internal/KStreamFilter.java | 4 +- .../streaming/internal/KStreamFlatMap.java | 4 +- .../internal/KStreamFlatMapValues.java | 4 +- .../streaming/internal/KStreamImpl.java | 16 ++--- .../streaming/internal/KStreamJoin.java | 18 +++--- .../streaming/internal/KStreamMap.java | 4 +- .../streaming/internal/KStreamMapValues.java | 4 +- .../streaming/internal/KStreamSource.java | 2 +- .../internal/KStreamWindowedImpl.java | 4 +- .../streaming/internal/Receiver.java | 4 +- .../streaming/internal/RecordQueue.java | 17 +++-- .../streaming/internal/RoundRobinChooser.java | 10 +-- .../streaming/internal/StampedRecord.java | 8 +-- .../internal/StreamSynchronizer.java | 45 ++++++++++--- .../streaming/internal/TimeBasedChooser.java | 16 ++--- .../streaming/internal/KStreamBranchTest.java | 10 +-- .../streaming/internal/KStreamFilterTest.java | 10 +-- .../internal/KStreamFlatMapTest.java | 10 +-- .../internal/KStreamFlatMapValuesTest.java | 10 +-- .../streaming/internal/KStreamJoinTest.java | 10 +-- .../streaming/internal/KStreamMapTest.java | 10 +-- .../internal/KStreamMapValuesTest.java | 10 +-- .../streaming/internal/KStreamSourceTest.java | 10 +-- .../internal/KStreamWindowedTest.java | 10 +-- .../streaming/testutil/MockIngestor.java | 2 +- 34 files changed, 250 insertions(+), 175 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 8f807dd8e4d34..711ee1636f900 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -21,7 +21,10 @@ import io.confluent.streaming.internal.ProcessorConfig; import io.confluent.streaming.internal.IngestorImpl; import io.confluent.streaming.internal.StreamSynchronizer; +<<<<<<< HEAD import io.confluent.streaming.util.ParallelExecutor; +======= +>>>>>>> removed some generics import io.confluent.streaming.util.Util; import org.apache.kafka.clients.consumer.*; import org.apache.kafka.clients.producer.KafkaProducer; @@ -84,13 +87,17 @@ public class KafkaStreaming implements Runnable { private final Class jobClass; private final Set topics; +<<<<<<< HEAD private final Map> syncGroups = new HashMap<>(); private final ArrayList> streamSynchronizers = new ArrayList<>(); private final ParallelExecutor parallelExecutor; +======= + private final Map> streamSynchronizersForPartition = new HashMap<>(); +>>>>>>> removed some generics private final Map kstreamContexts = new HashMap<>(); protected final Producer producer; protected final Consumer consumer; - private final IngestorImpl ingestor; + private final IngestorImpl ingestor; private final StreamingConfig streamingConfig; private final ProcessorConfig config; private final Metrics metrics; @@ -135,10 +142,17 @@ protected KafkaStreaming(Class jobClass, this.requestingCommit = new ArrayList<>(); this.config = new ProcessorConfig(config.config()); this.ingestor = +<<<<<<< HEAD new IngestorImpl<>(this.consumer, (Deserializer) config.keyDeserializer(), (Deserializer) config.valueDeserializer(), this.config.pollTimeMs); +======= + new IngestorImpl(this.consumer, + (Deserializer) config.keyDeserializer(), + (Deserializer) config.valueDeserializer(), + this.config.pollTimeMs); +>>>>>>> removed some generics this.running = true; this.lastCommit = 0; this.nextStateCleaning = Long.MAX_VALUE; @@ -181,12 +195,23 @@ private void shutdown() { log.info("Shutting down container"); commitAll(time.milliseconds()); +<<<<<<< HEAD for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { try { streamSynchronizer.close(); } catch(Exception e) { log.error("Error while closing stream synchronizers: ", e); +======= + for (Map.Entry> entry : streamSynchronizersForPartition.entrySet()) { + for (StreamSynchronizer streamSynchronizer : entry.getValue()) { + try { + streamSynchronizer.close(); + } + catch(Exception e) { + log.error("Error while closing stream synchronizers: ", e); + } +>>>>>>> removed some generics } } @@ -222,8 +247,17 @@ private void runLoop() { status.pollRequired(false); } +<<<<<<< HEAD parallelExecutor.execute(streamSynchronizers, status); +======= + for (Map.Entry> entry : streamSynchronizersForPartition.entrySet()) { + for (StreamSynchronizer streamSynchronizer : entry.getValue()) { + streamSynchronizer.process(); + pollRequired = pollRequired || streamSynchronizer.requiresPoll(); + } + } +>>>>>>> removed some generics maybeCommit(); maybeCleanState(); } @@ -263,12 +297,23 @@ private void commitAll(long now) { context.flush(); // check co-ordinator } +<<<<<<< HEAD for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { try { commit.putAll(streamSynchronizer.consumedOffsets()); } catch(Exception e) { log.error("Error while closing processor: ", e); +======= + for (Map.Entry> entry : streamSynchronizersForPartition.entrySet()) { + for (StreamSynchronizer streamSynchronizer : entry.getValue()) { + try { + commit.putAll(streamSynchronizer.consumedOffsets()); + } + catch(Exception e) { + log.error("Error while closing processor: ", e); + } +>>>>>>> removed some generics } } @@ -283,8 +328,8 @@ private void commitRequesting(long now) { KStreamContextImpl context = kstreamContexts.get(id); context.flush(); - for (SyncGroup syncGroup : syncGroups.get(id)) { - commit.putAll(syncGroup.streamSynchronizer.consumedOffsets()); // TODO: can this be async? + for (StreamSynchronizer streamSynchronizer : streamSynchronizersForPartition.get(id)) { + commit.putAll(streamSynchronizer.consumedOffsets()); // TODO: can this be async? } } consumer.commit(commit, CommitType.SYNC); @@ -353,11 +398,15 @@ public void shutdown(Coordinator.RequestScope scope) { throw new KafkaException(e); } +<<<<<<< HEAD Collection syncGroups = kstreamContext.syncGroups(); this.syncGroups.put(id, syncGroups); for (SyncGroup syncGroup : syncGroups) { streamSynchronizers.add(syncGroup.streamSynchronizer); } +======= + streamSynchronizersForPartition.put(id, kstreamContext.streamSynchronizers()); +>>>>>>> removed some generics } } @@ -369,11 +418,11 @@ private void removePartitions(Collection assignment) { commitAll(time.milliseconds()); // remove all partitions for (TopicPartition partition : assignment) { - Collection syncGroups = this.syncGroups.remove(partition.partition()); - if (syncGroups != null) { + Collection streamSynchronizers = this.streamSynchronizersForPartition.remove(partition.partition()); + if (streamSynchronizers != null) { log.info("Removing synchronization groups {}", partition.partition()); - for (SyncGroup syncGroup : syncGroups) - syncGroup.streamSynchronizer.close(); + for (StreamSynchronizer streamSynchronizer : streamSynchronizers) + streamSynchronizer.close(); } } for (TopicPartition partition : assignment) { diff --git a/src/main/java/io/confluent/streaming/StreamingConfig.java b/src/main/java/io/confluent/streaming/StreamingConfig.java index 1ae31691ab600..2aea659038be0 100644 --- a/src/main/java/io/confluent/streaming/StreamingConfig.java +++ b/src/main/java/io/confluent/streaming/StreamingConfig.java @@ -62,7 +62,7 @@ public class StreamingConfig { private Serializer valSerializer; private Deserializer keyDeserializer; private Deserializer valDeserializer; - private TimestampExtractor timestampExtractor; + private TimestampExtractor timestampExtractor; public StreamingConfig(Properties config) { this.config = config; @@ -124,11 +124,11 @@ public Deserializer valueDeserializer() { return this.valDeserializer; } - public void timestampExtractor(TimestampExtractor timestampExtractor) { + public void timestampExtractor(TimestampExtractor timestampExtractor) { this.timestampExtractor = timestampExtractor; } - public TimestampExtractor timestampExtractor() { + public TimestampExtractor timestampExtractor() { return this.timestampExtractor; } } diff --git a/src/main/java/io/confluent/streaming/SyncGroup.java b/src/main/java/io/confluent/streaming/SyncGroup.java index c7bc6fb846b0d..8a7f6a8747b27 100644 --- a/src/main/java/io/confluent/streaming/SyncGroup.java +++ b/src/main/java/io/confluent/streaming/SyncGroup.java @@ -5,14 +5,8 @@ /** * SyncGroup represents a group of streams synchronized together. */ -public class SyncGroup { +public interface SyncGroup { - public final String name; - public final StreamSynchronizer streamSynchronizer; - - public SyncGroup(String name, StreamSynchronizer streamSynchronizer) { - this.name = name; - this.streamSynchronizer = streamSynchronizer; - } + String name(); } diff --git a/src/main/java/io/confluent/streaming/TimestampExtractor.java b/src/main/java/io/confluent/streaming/TimestampExtractor.java index b1e1cdab5ffa7..e270fa2bdf0fd 100644 --- a/src/main/java/io/confluent/streaming/TimestampExtractor.java +++ b/src/main/java/io/confluent/streaming/TimestampExtractor.java @@ -1,10 +1,17 @@ package io.confluent.streaming; /** - * Created by yasuhiro on 6/22/15. + * An interface that allows the KStream framework to extract a timestamp from a key-value pair */ -public interface TimestampExtractor { +public interface TimestampExtractor { - long extract(String topic, K key, V value); + /** + * Extracts a timestamp from a key-value pair from a topic + * @param topic the topic name + * @param key the key object + * @param value the value object + * @return + */ + long extract(String topic, Object key, Object value); } diff --git a/src/main/java/io/confluent/streaming/internal/Chooser.java b/src/main/java/io/confluent/streaming/internal/Chooser.java index 37a4a88b14c2d..aafe92cddeabb 100644 --- a/src/main/java/io/confluent/streaming/internal/Chooser.java +++ b/src/main/java/io/confluent/streaming/internal/Chooser.java @@ -3,11 +3,11 @@ /** * Created by yasuhiro on 6/25/15. */ -public interface Chooser { +public interface Chooser { - void add(RecordQueue queue); + void add(RecordQueue queue); - RecordQueue next(); + RecordQueue next(); void close(); diff --git a/src/main/java/io/confluent/streaming/internal/Ingestor.java b/src/main/java/io/confluent/streaming/internal/Ingestor.java index 5080c4f91f2e6..75cc901867124 100644 --- a/src/main/java/io/confluent/streaming/internal/Ingestor.java +++ b/src/main/java/io/confluent/streaming/internal/Ingestor.java @@ -17,7 +17,7 @@ public interface Ingestor { int numPartitions(String topic); - void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition); + void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition); void removeStreamSynchronizerForPartition(TopicPartition partition); diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 9e28a6e68e302..5d95f85541cba 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -11,22 +11,21 @@ import java.util.*; -public class IngestorImpl implements Ingestor { +public class IngestorImpl implements Ingestor { private static final Logger log = LoggerFactory.getLogger(IngestorImpl.class); private final Consumer consumer; - private final Set unpaused = new HashSet(); - private final Set toBePaused = new HashSet(); - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; + private final Set unpaused = new HashSet<>(); + private final Set toBePaused = new HashSet<>(); + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; private final long pollTimeMs; - private final Map> streamSynchronizers = - new HashMap>(); + private final Map streamSynchronizers = new HashMap<>(); public IngestorImpl(Consumer consumer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, + Deserializer keyDeserializer, + Deserializer valueDeserializer, long pollTimeMs) { this.consumer = consumer; this.keyDeserializer = keyDeserializer; @@ -58,7 +57,7 @@ public void poll(long timeoutMs) { } for (TopicPartition partition : unpaused) { - StreamSynchronizer streamSynchronizer = streamSynchronizers.get(partition); + StreamSynchronizer streamSynchronizer = streamSynchronizers.get(partition); if (streamSynchronizer != null) streamSynchronizer.addRecords(partition, new DeserializingIterator(records.records(partition).iterator())); @@ -92,9 +91,9 @@ public int numPartitions(String topic) { @SuppressWarnings("unchecked") @Override - public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition) { + public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition) { synchronized (this) { - streamSynchronizers.put(partition, (StreamSynchronizer) streamSynchronizer); + streamSynchronizers.put(partition, streamSynchronizer); unpaused.add(partition); } } @@ -114,16 +113,16 @@ public void clear() { streamSynchronizers.clear(); } - private class DeserializingIterator extends FilteredIterator, ConsumerRecord> { + private class DeserializingIterator extends FilteredIterator, ConsumerRecord> { DeserializingIterator(Iterator> inner) { super(inner); } - protected ConsumerRecord filter(ConsumerRecord record) { - K key = keyDeserializer.deserialize(record.topic(), record.key()); - V value = valueDeserializer.deserialize(record.topic(), record.value()); - return new ConsumerRecord(record.topic(), record.partition(), record.offset(), key, value); + protected ConsumerRecord filter(ConsumerRecord record) { + Object key = keyDeserializer.deserialize(record.topic(), record.key()); + Object value = valueDeserializer.deserialize(record.topic(), record.value()); + return new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), key, value); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 7446944955d38..1d3b3d842cd5c 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -8,7 +8,7 @@ /** * Created by yasuhiro on 6/18/15. */ -class KStreamBranch implements Receiver { +class KStreamBranch implements Receiver { private final Predicate[] predicates; final KStreamSource[] branches; @@ -23,11 +23,11 @@ class KStreamBranch implements Receiver { } @Override - public void receive(K key, V value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { for (int i = 0; i < predicates.length; i++) { Predicate predicate = predicates[i]; - if (predicate.apply(key, value)) { + if (predicate.apply((K)key, (V)value)) { branches[i].receive(key, value, timestamp, streamTime); return; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 5b31efc0ac4a4..af809ab923328 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -33,8 +33,8 @@ public class KStreamContextImpl implements KStreamContext { private final Coordinator coordinator; private final HashMap> sourceStreams = new HashMap>(); private final HashMap partitioningInfos = new HashMap(); - private final TimestampExtractor timestampExtractor; - private final HashMap syncGroups = new HashMap(); + private final TimestampExtractor timestampExtractor; + private final HashMap streamSynchronizerMap = new HashMap<>(); private final StreamingConfig streamingConfig; private final ProcessorConfig processorConfig; private final Metrics metrics; @@ -65,7 +65,7 @@ public KStreamContextImpl(int id, this.streamingConfig = streamingConfig; this.processorConfig = processorConfig; - this.timestampExtractor = (TimestampExtractor)this.streamingConfig.timestampExtractor(); + this.timestampExtractor = this.streamingConfig.timestampExtractor(); if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractor is missing"); this.stateDir = new File(processorConfig.stateDir, Integer.toString(id)); @@ -126,8 +126,9 @@ public Deserializer valueDeserializer() { sourceStreams.put(topic, stream); TopicPartition partition = new TopicPartition(topic, id); - syncGroup.streamSynchronizer.addPartition(partition, (Receiver)stream); - ingestor.addStreamSynchronizerForPartition(syncGroup.streamSynchronizer, partition); + StreamSynchronizer streamSynchronizer = (StreamSynchronizer)syncGroup; + streamSynchronizer.addPartition(partition, stream); + ingestor.addStreamSynchronizerForPartition(streamSynchronizer, partition); } else { if (stream.partitioningInfo.syncGroup == syncGroup) @@ -170,26 +171,25 @@ public Metrics metrics() { @Override public SyncGroup syncGroup(String name) { - return syncGroup(name, new TimeBasedChooser()); + return syncGroup(name, new TimeBasedChooser()); } @Override public SyncGroup roundRobinSyncGroup(String name) { - return syncGroup(name, new RoundRobinChooser()); + return syncGroup(name, new RoundRobinChooser()); } - private SyncGroup syncGroup(String name, Chooser chooser) { + private SyncGroup syncGroup(String name, Chooser chooser) { int desiredUnprocessedPerPartition = processorConfig.bufferedRecordsPerPartition; synchronized (this) { - SyncGroup syncGroup = syncGroups.get(name); - if (syncGroup == null) { - StreamSynchronizer streamSynchronizer = - new StreamSynchronizer(name, ingestor, chooser, timestampExtractor, desiredUnprocessedPerPartition); - syncGroup = new SyncGroup(name, streamSynchronizer); - syncGroups.put(name, syncGroup); + StreamSynchronizer streamSynchronizer = streamSynchronizerMap.get(name); + if (streamSynchronizer == null) { + streamSynchronizer = + new StreamSynchronizer(name, ingestor, chooser, timestampExtractor, desiredUnprocessedPerPartition); + streamSynchronizerMap.put(name, streamSynchronizer); } - return syncGroup; + return (SyncGroup)streamSynchronizer; } } @@ -202,8 +202,8 @@ public void restore(StorageEngine engine) throws Exception { } - public Collection syncGroups() { - return syncGroups.values(); + public Collection streamSynchronizers() { + return streamSynchronizerMap.values(); } public void init(Consumer restoreConsumer) throws IOException { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index dbf43e7531c9e..ea0b2f5deab8d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -15,9 +15,9 @@ class KStreamFilter extends KStreamImpl { } @Override - public void receive(K key, V value, long timestamp,long streamTime) { + public void receive(Object key, Object value, long timestamp,long streamTime) { synchronized(this) { - if (predicate.apply(key, value)) { + if (predicate.apply((K)key, (V)value)) { forward(key, value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 6e20483b44055..988cae0b6a4de 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -17,9 +17,9 @@ class KStreamFlatMap extends KStreamImpl { } @Override - public void receive(K1 key, V1 value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { - KeyValue> newPair = mapper.apply(key, value); + KeyValue> newPair = mapper.apply((K1)key, (V1)value); for (V v : newPair.value) { forward(newPair.key, v, timestamp, streamTime); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 7cc413ccbe609..a25ac25678dce 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -15,9 +15,9 @@ class KStreamFlatMapValues extends KStreamImpl { } @Override - public void receive(K key, V1 value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { - Iterable newValues = mapper.apply(value); + Iterable newValues = mapper.apply((V1)value); for (V v : newValues) { forward(key, v, timestamp, streamTime); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 3504fbf7fb35a..98e31bad3349a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -8,9 +8,9 @@ /** * Created by yasuhiro on 6/17/15. */ -abstract class KStreamImpl implements KStream, Receiver { +abstract class KStreamImpl implements KStream, Receiver { - private final ArrayList> nextReceivers = new ArrayList>(1); + private final ArrayList nextReceivers = new ArrayList<>(1); final PartitioningInfo partitioningInfo; final KStreamContextImpl context; @@ -96,22 +96,22 @@ public void punctuate(long streamTime) {} @Override public void process(final Processor processor) { - Receiver receiver = new Receiver() { - public void receive(K key, V value, long timestamp, long streamTime) { - processor.apply(key, value); + Receiver receiver = new Receiver() { + public void receive(Object key, Object value, long timestamp, long streamTime) { + processor.apply((K)key, (V)value); } }; registerReceiver(receiver); - PunctuationScheduler scheduler = partitioningInfo.syncGroup.streamSynchronizer.getPunctuationScheduler(processor); + PunctuationScheduler scheduler = ((StreamSynchronizer)partitioningInfo.syncGroup).getPunctuationScheduler(processor); processor.init(scheduler); } - void registerReceiver(Receiver receiver) { + void registerReceiver(Receiver receiver) { nextReceivers.add(receiver); } - protected void forward(K key, V value, long timestamp, long streamTime) { + protected void forward(Object key, Object value, long timestamp, long streamTime) { int numReceivers = nextReceivers.size(); for (int i = 0; i < numReceivers; i++) { nextReceivers.get(i).receive(key, value, timestamp, streamTime); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index a00db2cea1f4d..bbdcde3b338c6 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -17,7 +17,7 @@ private static abstract class Finder { private final Finder finder1; private final Finder finder2; private final ValueJoiner joiner; - final Receiver receiverForOtherStream; + final Receiver receiverForOtherStream; KStreamJoin(final Window window1, final Window window2, boolean prior, ValueJoiner joiner, PartitioningInfo partitioningInfo, KStreamContextImpl context) { super(partitioningInfo, context); @@ -53,24 +53,24 @@ Iterator find(K key, long timestamp) { } @Override - public void receive(K key, V1 value, long timestamp, long streamTime) { - Iterator iter = finder2.find(key, timestamp); + public void receive(Object key, Object value, long timestamp, long streamTime) { + Iterator iter = finder2.find((K)key, timestamp); if (iter != null) { while (iter.hasNext()) { - doJoin(key, value, iter.next(), timestamp, streamTime); + doJoin((K)key, (V1)value, iter.next(), timestamp, streamTime); } } } - private Receiver getReceiverForOther() { - return new Receiver() { + private Receiver getReceiverForOther() { + return new Receiver() { @Override - public void receive(K key, V2 value2, long timestamp, long streamTime) { - Iterator iter = finder1.find(key, timestamp); + public void receive(Object key, Object value2, long timestamp, long streamTime) { + Iterator iter = finder1.find((K)key, timestamp); if (iter != null) { while (iter.hasNext()) { - doJoin(key, iter.next(), value2, timestamp, streamTime); + doJoin((K)key, iter.next(), (V2)value2, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index 20e83ca69538c..eea1ce0349eb4 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -17,9 +17,9 @@ class KStreamMap extends KStreamImpl { } @Override - public void receive(K1 key, V1 value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized (this) { - KeyValue newPair = mapper.apply(key, value); + KeyValue newPair = mapper.apply((K1)key, (V1)value); forward(newPair.key, newPair.value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index 2c99205eddc91..3f814333d103d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -15,9 +15,9 @@ class KStreamMapValues extends KStreamImpl { } @Override - public void receive(K key, V1 value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized (this) { - V newValue = mapper.apply(value); + V newValue = mapper.apply((V1)value); forward(key, newValue, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 7652ac15dcfcb..d83db90f96f21 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -10,7 +10,7 @@ class KStreamSource extends KStreamImpl { } @Override - public void receive(K key, V value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { forward(key, value, timestamp, streamTime); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 0b9f92e4aec78..d4c5def938559 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -15,9 +15,9 @@ public class KStreamWindowedImpl extends KStreamImpl implement } @Override - public void receive(K key, V value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { - window.put(key, value, timestamp); + window.put((K)key, (V)value, timestamp); forward(key, value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 90997088cb42d..1cad1f810acdf 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -3,8 +3,8 @@ /** * Created by yasuhiro on 6/17/15. */ -public interface Receiver { +public interface Receiver { - void receive(K key, V value, long timestamp, long streamTime); + void receive(Object key, Object value, long timestamp, long streamTime); } diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java index 86b2abc913929..3e223b441bfa0 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -6,20 +6,19 @@ import org.apache.kafka.common.TopicPartition; import java.util.ArrayDeque; -import java.util.Deque; /** * Created by yasuhiro on 6/25/15. */ -public class RecordQueue { +public class RecordQueue { - private final Deque> queue = new ArrayDeque>(); - public final Receiver receiver; + private final ArrayDeque queue = new ArrayDeque<>(); + public final Receiver receiver; private final TopicPartition partition; - private TimestampTracker> timestampTracker; + private TimestampTracker> timestampTracker; private long offset; - public RecordQueue(TopicPartition partition, Receiver receiver, TimestampTracker> timestampTracker) { + public RecordQueue(TopicPartition partition, Receiver receiver, TimestampTracker> timestampTracker) { this.partition = partition; this.receiver = receiver; this.timestampTracker = timestampTracker; @@ -29,14 +28,14 @@ public TopicPartition partition() { return partition; } - public void add(StampedRecord record) { + public void add(StampedRecord record) { queue.addLast(record); timestampTracker.addStampedElement(record); offset = record.offset(); } - public StampedRecord next() { - StampedRecord elem = queue.getFirst(); + public StampedRecord next() { + StampedRecord elem = queue.getFirst(); if (elem == null) return null; diff --git a/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java b/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java index 3409a0ac9513a..b754bf490e02d 100644 --- a/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java +++ b/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java @@ -5,21 +5,21 @@ /** * Created by yasuhiro on 6/25/15. */ -public class RoundRobinChooser implements Chooser { +public class RoundRobinChooser implements Chooser { - private final ArrayDeque> deque; + private final ArrayDeque deque; public RoundRobinChooser() { - deque = new ArrayDeque>(); + deque = new ArrayDeque<>(); } @Override - public void add(RecordQueue queue) { + public void add(RecordQueue queue) { deque.offer(queue); } @Override - public RecordQueue next() { + public RecordQueue next() { return deque.poll(); } diff --git a/src/main/java/io/confluent/streaming/internal/StampedRecord.java b/src/main/java/io/confluent/streaming/internal/StampedRecord.java index 1ed961c455061..4cdf5d9d915a0 100644 --- a/src/main/java/io/confluent/streaming/internal/StampedRecord.java +++ b/src/main/java/io/confluent/streaming/internal/StampedRecord.java @@ -3,17 +3,17 @@ import io.confluent.streaming.util.Stamped; import org.apache.kafka.clients.consumer.ConsumerRecord; -public class StampedRecord extends Stamped> { +public class StampedRecord extends Stamped> { - StampedRecord(ConsumerRecord record, long timestamp) { + StampedRecord(ConsumerRecord record, long timestamp) { super(record, timestamp); } - public K key() { + public Object key() { return value.key(); } - public V value() { + public Object value() { return value.value(); } diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index 475117951b099..505ab4f1de7ce 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -15,6 +15,7 @@ /** * Created by yasuhiro on 6/23/15. */ +<<<<<<< HEAD public class StreamSynchronizer implements ParallelExecutor.Task { public static class Status { @@ -34,6 +35,15 @@ public boolean pollRequired() { private final Chooser chooser; private final TimestampExtractor timestampExtractor; private final Map> stash = new HashMap<>(); +======= +public class StreamSynchronizer implements SyncGroup { + + public final String name; + private final Ingestor ingestor; + private final Chooser chooser; + private final TimestampExtractor timestampExtractor; + private final Map stash = new HashMap<>(); +>>>>>>> removed some generics private final int desiredUnprocessed; private final Map consumedOffsets; private final PunctuationQueue punctuationQueue = new PunctuationQueue(); @@ -44,8 +54,8 @@ public boolean pollRequired() { StreamSynchronizer(String name, Ingestor ingestor, - Chooser chooser, - TimestampExtractor timestampExtractor, + Chooser chooser, + TimestampExtractor timestampExtractor, int desiredUnprocessedPerPartition) { this.name = name; this.ingestor = ingestor; @@ -53,22 +63,35 @@ public boolean pollRequired() { this.timestampExtractor = timestampExtractor; this.desiredUnprocessed = desiredUnprocessedPerPartition; this.consumedOffsets = new HashMap<>(); +<<<<<<< HEAD +======= + } + + @Override + public String name() { + return name; +>>>>>>> removed some generics } @SuppressWarnings("unchecked") - public void addPartition(TopicPartition partition, Receiver receiver) { + public void addPartition(TopicPartition partition, Receiver receiver) { synchronized (this) { - RecordQueue recordQueue = stash.get(partition); + RecordQueue recordQueue = stash.get(partition); if (recordQueue == null) { - stash.put(partition, createRecordQueue(partition, (Receiver) receiver)); + stash.put(partition, createRecordQueue(partition, receiver)); } else { throw new IllegalStateException("duplicate partition"); } } } +<<<<<<< HEAD public void addRecords(TopicPartition partition, Iterator> iterator) { +======= + @SuppressWarnings("unchecked") + public void addRecords(TopicPartition partition, Iterator> iterator) { +>>>>>>> removed some generics synchronized (this) { newRecordBuffer.addLast(new NewRecords<>(partition, iterator)); } @@ -84,9 +107,13 @@ private void ingestNewRecords() { boolean wasEmpty = recordQueue.isEmpty(); while (iterator.hasNext()) { - ConsumerRecord record = iterator.next(); + ConsumerRecord record = iterator.next(); long timestamp = timestampExtractor.extract(record.topic(), record.key(), record.value()); +<<<<<<< HEAD recordQueue.add(new StampedRecord<>(record, timestamp)); +======= + recordQueue.add(new StampedRecord(record, timestamp)); +>>>>>>> removed some generics buffered++; } @@ -124,7 +151,7 @@ public void process(Object context) { } long trackedTimestamp = recordQueue.trackedTimestamp(); - StampedRecord record = recordQueue.next(); + StampedRecord record = recordQueue.next(); if (recordQueue.size() < this.desiredUnprocessed) status.pollRequired(true); @@ -155,8 +182,8 @@ public void close() { stash.clear(); } - protected RecordQueue createRecordQueue(TopicPartition partition, Receiver receiver) { - return new RecordQueue(partition, receiver, new MinTimestampTracker>()); + protected RecordQueue createRecordQueue(TopicPartition partition, Receiver receiver) { + return new RecordQueue(partition, receiver, new MinTimestampTracker>()); } private static class NewRecords { diff --git a/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java b/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java index ccb96c9709035..924d548abb408 100644 --- a/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java +++ b/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java @@ -6,13 +6,13 @@ /** * Created by yasuhiro on 6/25/15. */ -public class TimeBasedChooser implements Chooser { +public class TimeBasedChooser implements Chooser { - private final PriorityQueue> pq; + private final PriorityQueue pq; public TimeBasedChooser() { - this(new Comparator>() { - public int compare(RecordQueue queue1, RecordQueue queue2) { + this(new Comparator() { + public int compare(RecordQueue queue1, RecordQueue queue2) { long time1 = queue1.trackedTimestamp(); long time2 = queue2.trackedTimestamp(); @@ -23,17 +23,17 @@ public int compare(RecordQueue queue1, RecordQueue queue2) { }); } - private TimeBasedChooser(Comparator> comparator) { - pq = new PriorityQueue>(3, comparator); + private TimeBasedChooser(Comparator comparator) { + pq = new PriorityQueue<>(3, comparator); } @Override - public void add(RecordQueue queue) { + public void add(RecordQueue queue) { pq.offer(queue); } @Override - public RecordQueue next() { + public RecordQueue next() { return pq.poll(); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index 767f4c74bec78..85c2b513f320a 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -13,19 +13,19 @@ public class KStreamBranchTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { return 0L; } }, 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); @SuppressWarnings("unchecked") @Test diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index cb630a611d263..0edb0cac689d3 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -11,19 +11,19 @@ public class KStreamFilterTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { return 0L; } }, 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); private Predicate isMultipleOfThree = new Predicate() { @Override diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 685662fb2f9f6..6a138acf72637 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -13,19 +13,19 @@ public class KStreamFlatMapTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { return 0L; } }, 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); @Test public void testFlatMap() { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index bed3fde82ba11..1562c367ad684 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -13,19 +13,19 @@ public class KStreamFlatMapValuesTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { return 0L; } }, 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); @Test public void testFlatMapValues() { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 1b196908eaa44..be1d5b784988e 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -15,19 +15,19 @@ public class KStreamJoinTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { return 0L; } }, 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); private ValueJoiner joiner = new ValueJoiner() { @Override diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 27f677f001814..d3f84e33788fd 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -11,19 +11,19 @@ public class KStreamMapTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { return 0L; } }, 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); @Test public void testMap() { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 19832849f83d8..771726116381b 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -13,19 +13,19 @@ public class KStreamMapValuesTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { return 0L; } }, 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); @Test public void testFlatMapValues() { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 4c1e9d5ce72ee..4b81a663d5a0c 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -12,19 +12,19 @@ public class KStreamSourceTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { return 0L; } }, 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); @Test public void testKStreamSource() { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 87849177c7971..a47ea68d55f87 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -13,19 +13,19 @@ public class KStreamWindowedTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( "group", ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, String key, String value) { + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { return 0L; } }, 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(new SyncGroup("group", streamSynchronizer), 1); + private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); @Test public void testWindowedStream() { diff --git a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java index dcfb6435540ea..ee33cae64876c 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java +++ b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java @@ -27,7 +27,7 @@ public int numPartitions(String topic) { } @Override - public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition) { + public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition) { } @Override From 1cf270659d6218c118371b9bd53d8c1b7891951f Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 10 Jul 2015 14:54:00 -0700 Subject: [PATCH 048/275] StreamSynchronizer test and fix --- .../streaming/internal/RecordQueue.java | 2 +- .../internal/StreamSynchronizer.java | 1 + .../internal/StreamSynchronizerTest.java | 159 ++++++++++++++++++ .../streaming/testutil/MockIngestor.java | 16 ++ 4 files changed, 177 insertions(+), 1 deletion(-) create mode 100644 src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java index 3e223b441bfa0..5dbdfdd5bb151 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -35,7 +35,7 @@ public void add(StampedRecord record) { } public StampedRecord next() { - StampedRecord elem = queue.getFirst(); + StampedRecord elem = queue.pollFirst(); if (elem == null) return null; diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index 505ab4f1de7ce..24f676ba5d997 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -126,6 +126,7 @@ private void ingestNewRecords() { } } } + newRecordBuffer.clear(); } public PunctuationScheduler getPunctuationScheduler(Processor processor) { diff --git a/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java b/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java new file mode 100644 index 0000000000000..09a6a36d40b81 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java @@ -0,0 +1,159 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.TimestampExtractor; +import io.confluent.streaming.testutil.MockIngestor; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class StreamSynchronizerTest { + + private static class MockReceiver implements Receiver { + + public int numReceived = 0; + public ArrayList keys = new ArrayList<>(); + public ArrayList values = new ArrayList<>(); + public ArrayList timestamps = new ArrayList<>(); + public ArrayList streamTimes = new ArrayList<>(); + + @Override + public void receive(Object key, Object value, long timestamp, long streamTime) { + this.numReceived++; + this.keys.add(key); + this.values.add(value); + this.timestamps.add(timestamp); + this.streamTimes.add(streamTime); + } + + } + + @Test + public void testAddPartition() { + + MockIngestor mockIngestor = new MockIngestor(); + + StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + "group", + mockIngestor, + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { + if (topic.equals("topic1")) + return ((Integer)key).longValue(); + else + return ((Integer)key).longValue() / 10L + 5L; + } + }, + 3 + ); + + TopicPartition partition1 = new TopicPartition("topic1", 1); + TopicPartition partition2 = new TopicPartition("topic2", 1); + MockReceiver receiver1 = new MockReceiver(); + MockReceiver receiver2 = new MockReceiver(); + MockReceiver receiver3 = new MockReceiver(); + + streamSynchronizer.addPartition(partition1, receiver1); + mockIngestor.addStreamSynchronizerForPartition(streamSynchronizer, partition1); + + streamSynchronizer.addPartition(partition2, receiver2); + mockIngestor.addStreamSynchronizerForPartition(streamSynchronizer, partition2); + + Exception exception = null; + try { + streamSynchronizer.addPartition(partition1, receiver3); + } catch (Exception ex) { + exception = ex; + } + assertTrue(exception != null); + + mockIngestor.addRecords(partition1, records( + new ConsumerRecord(partition1.topic(), partition1.partition(), 1, new Integer(10), "A"), + new ConsumerRecord(partition1.topic(), partition1.partition(), 2, new Integer(20), "AA") + )); + + mockIngestor.addRecords(partition2, records( + new ConsumerRecord(partition2.topic(), partition2.partition(), 1, new Integer(300), "B"), + new ConsumerRecord(partition2.topic(), partition2.partition(), 2, new Integer(400), "BB"), + new ConsumerRecord(partition2.topic(), partition2.partition(), 3, new Integer(500), "BBB"), + new ConsumerRecord(partition2.topic(), partition2.partition(), 4, new Integer(600), "BBBB") + )); + + StreamSynchronizer.Status status = new StreamSynchronizer.Status(); + + streamSynchronizer.process(status); + assertEquals(receiver1.numReceived, 1); + assertEquals(receiver2.numReceived, 0); + + assertTrue(status.pollRequired()); + + assertEquals(mockIngestor.paused.size(), 1); + assertTrue(mockIngestor.paused.contains(partition2)); + + mockIngestor.addRecords(partition1, records( + new ConsumerRecord(partition1.topic(), partition1.partition(), 3, new Integer(30), "AAA"), + new ConsumerRecord(partition1.topic(), partition1.partition(), 4, new Integer(40), "AAAA"), + new ConsumerRecord(partition1.topic(), partition1.partition(), 5, new Integer(50), "AAAAA") + )); + + streamSynchronizer.process(status); + assertEquals(receiver1.numReceived, 2); + assertEquals(receiver2.numReceived, 0); + + assertEquals(mockIngestor.paused.size(), 2); + assertTrue(mockIngestor.paused.contains(partition1)); + assertTrue(mockIngestor.paused.contains(partition2)); + + streamSynchronizer.process(status); + assertEquals(receiver1.numReceived, 3); + assertEquals(receiver2.numReceived, 0); + + streamSynchronizer.process(status); + assertEquals(receiver1.numReceived, 3); + assertEquals(receiver2.numReceived, 1); + + assertEquals(mockIngestor.paused.size(), 1); + assertTrue(mockIngestor.paused.contains(partition2)); + + streamSynchronizer.process(status); + assertEquals(receiver1.numReceived, 4); + assertEquals(receiver2.numReceived, 1); + + assertEquals(mockIngestor.paused.size(), 1); + + streamSynchronizer.process(status); + assertEquals(receiver1.numReceived, 4); + assertEquals(receiver2.numReceived, 2); + + assertEquals(mockIngestor.paused.size(), 0); + + streamSynchronizer.process(status); + assertEquals(receiver1.numReceived, 5); + assertEquals(receiver2.numReceived, 2); + + streamSynchronizer.process(status); + assertEquals(receiver1.numReceived, 5); + assertEquals(receiver2.numReceived, 3); + + streamSynchronizer.process(status); + assertEquals(receiver1.numReceived, 5); + assertEquals(receiver2.numReceived, 4); + + assertEquals(mockIngestor.paused.size(), 0); + + streamSynchronizer.process(status); + assertEquals(receiver1.numReceived, 5); + assertEquals(receiver2.numReceived, 4); + } + + private List> records(ConsumerRecord... recs) { + return Arrays.asList(recs); + } +} diff --git a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java index ee33cae64876c..f88a30b394f28 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java +++ b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java @@ -2,9 +2,18 @@ import io.confluent.streaming.internal.Ingestor; import io.confluent.streaming.internal.StreamSynchronizer; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import java.util.HashMap; +import java.util.HashSet; + public class MockIngestor implements Ingestor { + + private HashMap streamSynchronizers = new HashMap<>(); + + public HashSet paused = new HashSet<>(); + @Override public void poll() { } @@ -15,10 +24,12 @@ public void poll(long timeoutMs) { @Override public void pause(TopicPartition partition) { + paused.add(partition); } @Override public void unpause(TopicPartition partition, long offset) { + paused.remove(partition); } @Override @@ -28,10 +39,15 @@ public int numPartitions(String topic) { @Override public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition) { + streamSynchronizers.put(partition, streamSynchronizer); } @Override public void removeStreamSynchronizerForPartition(TopicPartition partition) { + streamSynchronizers.remove(partition); } + public void addRecords(TopicPartition partition, Iterable> records) { + streamSynchronizers.get(partition).addRecords(partition, records.iterator()); + } } From 370e6d5241898d42e2e0eacccbd78fbaa5900560 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 10 Jul 2015 14:59:26 -0700 Subject: [PATCH 049/275] allow deserializer override at KStream construction --- .../confluent/streaming/KStreamContext.java | 12 +++- .../confluent/streaming/StreamingConfig.java | 11 +++- .../streaming/internal/IngestorImpl.java | 2 +- .../internal/KStreamContextImpl.java | 62 ++++++++++++++++--- .../streaming/internal/KStreamImpl.java | 2 +- .../streaming/internal/KStreamSource.java | 10 +++ .../internal/ProcessorStateManager.java | 4 ++ .../streaming/internal/RecordQueue.java | 6 +- .../internal/StreamSynchronizer.java | 42 ++++++++++--- 9 files changed, 122 insertions(+), 29 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 287b7ad8543d6..ee1cae948ae14 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -49,17 +49,25 @@ public interface KStreamContext { /** * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. * @param topic the topic name + * @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 * @return KStream */ - KStream from(String topic); + KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer); /** * Creates a KStream instance for the specified topic. The stream is added to the specified synchronization group. * @param topic the topic name * @param syncGroup the 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 * @return KStream */ - KStream from(String topic, SyncGroup syncGroup); + KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer); /** * Returns a RecordCollector which takes binary (byte array) key and value. diff --git a/src/main/java/io/confluent/streaming/StreamingConfig.java b/src/main/java/io/confluent/streaming/StreamingConfig.java index 2aea659038be0..87feaba2d15d1 100644 --- a/src/main/java/io/confluent/streaming/StreamingConfig.java +++ b/src/main/java/io/confluent/streaming/StreamingConfig.java @@ -66,9 +66,14 @@ public class StreamingConfig { public StreamingConfig(Properties config) { this.config = config; - config.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - config.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); - config.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); + this.config.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + this.config.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); + this.config.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); + } + + @Override + public StreamingConfig clone() { + return new StreamingConfig(this.config); } public void addContextObject(String key, Object value) { diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 5d95f85541cba..129d5e96945c2 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -60,7 +60,7 @@ public void poll(long timeoutMs) { StreamSynchronizer streamSynchronizer = streamSynchronizers.get(partition); if (streamSynchronizer != null) - streamSynchronizer.addRecords(partition, new DeserializingIterator(records.records(partition).iterator())); + streamSynchronizer.addRecords(partition, records.records(partition).iterator()); else log.warn("unused topic: " + partition.topic()); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index af809ab923328..588bfc1ae942b 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -42,7 +42,6 @@ public class KStreamContextImpl implements KStreamContext { private final ProcessorStateManager stateMgr; private Consumer restoreConsumer; - @SuppressWarnings("unchecked") public KStreamContextImpl(int id, KStreamJob job, Set topics, @@ -52,14 +51,33 @@ public KStreamContextImpl(int id, StreamingConfig streamingConfig, ProcessorConfig processorConfig, Metrics metrics) { + + this(id, job, topics, ingestor, + new RecordCollectors.SimpleRecordCollector(producer), + coordinator, streamingConfig, processorConfig, + new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id))), + metrics); + } + + @SuppressWarnings("unchecked") + public KStreamContextImpl(int id, + KStreamJob job, + Set topics, + Ingestor ingestor, + RecordCollectors.SimpleRecordCollector simpleCollector, + Coordinator coordinator, + StreamingConfig streamingConfig, + ProcessorConfig processorConfig, + ProcessorStateManager stateMgr, + Metrics metrics) { this.id = id; this.job = job; this.topics = topics; this.ingestor = ingestor; - this.simpleCollector = new RecordCollectors.SimpleRecordCollector(producer); + this.simpleCollector = simpleCollector; this.collector = new RecordCollectors.SerializingRecordCollector( - simpleCollector, (Serializer) streamingConfig.keySerializer(), (Serializer) streamingConfig.valueSerializer()); + simpleCollector, (Serializer) streamingConfig.keySerializer(), (Serializer) streamingConfig.valueSerializer()); this.coordinator = coordinator; this.streamingConfig = streamingConfig; @@ -68,8 +86,8 @@ public KStreamContextImpl(int id, this.timestampExtractor = this.streamingConfig.timestampExtractor(); if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractor is missing"); - this.stateDir = new File(processorConfig.stateDir, Integer.toString(id)); - this.stateMgr = new ProcessorStateManager(id, stateDir); + this.stateMgr = stateMgr; + this.stateDir = this.stateMgr.baseDir(); this.metrics = metrics; } @@ -99,13 +117,13 @@ public Deserializer valueDeserializer() { } @Override - public KStream from(String topic) { - return from(topic, syncGroup(DEFAULT_SYNCHRONIZATION_GROUP)); + public KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer) { + return from(topic, syncGroup(DEFAULT_SYNCHRONIZATION_GROUP), keyDeserializer, valDeserializer); } @Override @SuppressWarnings("unchecked") - public KStream from(String topic, SyncGroup syncGroup) { + public KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer) { if (syncGroup == null) throw new NullPointerException(); synchronized (this) { @@ -122,7 +140,24 @@ public Deserializer valueDeserializer() { partitioningInfos.put(topic, partitioningInfo); } - stream = new KStreamSource(partitioningInfo, this); + // override the deserializer classes if specified + if (keyDeserializer == null && valDeserializer == null) { + stream = new KStreamSource(partitioningInfo, this); + } else { + StreamingConfig newConfig = this.streamingConfig.clone(); + if (keyDeserializer != null) + newConfig.keyDeserializer(keyDeserializer); + if (valDeserializer != null) + newConfig.valueDeserializer(valDeserializer); + + KStreamContextImpl newContext = new KStreamContextImpl( + this.id, this.job, this.topics, this.ingestor, + this.simpleCollector, this.coordinator, + newConfig, this.processorConfig, + this.stateMgr, this.metrics); + stream = new KStreamSource(partitioningInfo, newContext); + } + sourceStreams.put(topic, stream); TopicPartition partition = new TopicPartition(topic, id); @@ -133,6 +168,13 @@ public Deserializer valueDeserializer() { else { if (stream.partitioningInfo.syncGroup == syncGroup) throw new IllegalStateException("topic is already assigned a different synchronization group"); + + // with this constraint we will not allow users to create KStream with different deser from the same topic, + // this constraint may better be relaxed later. + if (keyDeserializer != null && !keyDeserializer.getClass().equals(this.keyDeserializer().getClass())) + throw new IllegalStateException("another source stream with the same topic but different key deserializer is already created"); + if (valDeserializer != null && !valDeserializer.getClass().equals(this.valueDeserializer().getClass())) + throw new IllegalStateException("another source stream with the same topic but different value deserializer is already created"); } return stream; @@ -189,7 +231,7 @@ private SyncGroup syncGroup(String name, Chooser chooser) { new StreamSynchronizer(name, ingestor, chooser, timestampExtractor, desiredUnprocessedPerPartition); streamSynchronizerMap.put(name, streamSynchronizer); } - return (SyncGroup)streamSynchronizer; + return streamSynchronizer; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 98e31bad3349a..ce26edfd27472 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -70,7 +70,7 @@ public KStream[] branch(Predicate... predicates) { @Override public KStream through(String topic) { process(this.getSendProcessor(topic)); - return (KStream)context.from(topic); + return (KStream) context.from(topic, null, null); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index d83db90f96f21..70acf24a12d53 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -1,12 +1,22 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; + /** * Created by yasuhiro on 6/17/15. */ class KStreamSource extends KStreamImpl { + private KStreamContext context; + KStreamSource(PartitioningInfo partitioningInfo, KStreamContextImpl context) { super(partitioningInfo, context); + + this.context = context; + } + + public KStreamContext context() { + return this.context; } @Override diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index a2a0d114d4294..d416fe8f26bfa 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -48,6 +48,10 @@ public ProcessorStateManager(int id, File baseDir) { this.restoredOffsets = new HashMap(); } + public File baseDir() { + return this.baseDir; + } + public void init() throws IOException { OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); this.checkpointedOffsets.putAll(checkpoint.read()); diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java index 5dbdfdd5bb151..b05a0c9f0f692 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -13,14 +13,14 @@ public class RecordQueue { private final ArrayDeque queue = new ArrayDeque<>(); - public final Receiver receiver; + public final KStreamSource source; private final TopicPartition partition; private TimestampTracker> timestampTracker; private long offset; - public RecordQueue(TopicPartition partition, Receiver receiver, TimestampTracker> timestampTracker) { + public RecordQueue(TopicPartition partition, KStreamSource source, TimestampTracker> timestampTracker) { this.partition = partition; - this.receiver = receiver; + this.source = source; this.timestampTracker = timestampTracker; } diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index 24f676ba5d997..e3bd5a06077f3 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -5,6 +5,7 @@ import io.confluent.streaming.util.ParallelExecutor; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; import java.util.ArrayDeque; import java.util.HashMap; @@ -74,12 +75,12 @@ public String name() { } @SuppressWarnings("unchecked") - public void addPartition(TopicPartition partition, Receiver receiver) { + public void addPartition(TopicPartition partition, KStreamSource source) { synchronized (this) { RecordQueue recordQueue = stash.get(partition); if (recordQueue == null) { - stash.put(partition, createRecordQueue(partition, receiver)); + stash.put(partition, createRecordQueue(partition, source)); } else { throw new IllegalStateException("duplicate partition"); } @@ -90,23 +91,32 @@ public void addPartition(TopicPartition partition, Receiver receiver) { public void addRecords(TopicPartition partition, Iterator> iterator) { ======= @SuppressWarnings("unchecked") +<<<<<<< HEAD public void addRecords(TopicPartition partition, Iterator> iterator) { >>>>>>> removed some generics +======= + public void addRecords(TopicPartition partition, Iterator> iterator) { +>>>>>>> allow deserializer override at KStream construction synchronized (this) { - newRecordBuffer.addLast(new NewRecords<>(partition, iterator)); + newRecordBuffer.addLast(new NewRecords(partition, iterator)); } } private void ingestNewRecords() { for (NewRecords newRecords : newRecordBuffer) { TopicPartition partition = newRecords.partition; +<<<<<<< HEAD Iterator> iterator = newRecords.iterator; +======= + Iterator> iterator = newRecords.iterator; +>>>>>>> allow deserializer override at KStream construction RecordQueue recordQueue = stash.get(partition); if (recordQueue != null) { boolean wasEmpty = recordQueue.isEmpty(); while (iterator.hasNext()) { +<<<<<<< HEAD ConsumerRecord record = iterator.next(); long timestamp = timestampExtractor.extract(record.topic(), record.key(), record.value()); <<<<<<< HEAD @@ -114,6 +124,20 @@ private void ingestNewRecords() { ======= recordQueue.add(new StampedRecord(record, timestamp)); >>>>>>> removed some generics +======= + ConsumerRecord record = iterator.next(); + + // deserialize the raw record, extract the timestamp and put into the queue + Deserializer keyDeserializer = recordQueue.source.context().keyDeserializer(); + Deserializer valDeserializer = recordQueue.source.context().valueDeserializer(); + + Object key = keyDeserializer.deserialize(record.topic(), record.key()); + Object value = valDeserializer.deserialize(record.topic(), record.value()); + ConsumerRecord deserializedRecord = new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), key, value); + + long timestamp = timestampExtractor.extract(record.topic(), key, value); + recordQueue.add(new StampedRecord(deserializedRecord, timestamp)); +>>>>>>> allow deserializer override at KStream construction buffered++; } @@ -159,7 +183,7 @@ public void process(Object context) { if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - recordQueue.receiver.receive(record.key(), record.value(), record.timestamp, streamTime); + recordQueue.source.receive(record.key(), record.value(), record.timestamp, streamTime); consumedOffsets.put(recordQueue.partition(), record.offset()); if (recordQueue.size() > 0) chooser.add(recordQueue); @@ -183,15 +207,15 @@ public void close() { stash.clear(); } - protected RecordQueue createRecordQueue(TopicPartition partition, Receiver receiver) { - return new RecordQueue(partition, receiver, new MinTimestampTracker>()); + protected RecordQueue createRecordQueue(TopicPartition partition, KStreamSource source) { + return new RecordQueue(partition, source, new MinTimestampTracker>()); } - private static class NewRecords { + private static class NewRecords { final TopicPartition partition; - final Iterator> iterator; + final Iterator> iterator; - NewRecords(TopicPartition partition, Iterator> iterator) { + NewRecords(TopicPartition partition, Iterator> iterator) { this.partition = partition; this.iterator = iterator; } From c5b0a42723ddfa49e686e9040b20c9a9b5028f86 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 10 Jul 2015 15:07:07 -0700 Subject: [PATCH 050/275] removed used DeserializingIterator --- .../streaming/internal/IngestorImpl.java | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 129d5e96945c2..4203369235389 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -112,19 +112,4 @@ public void clear() { toBePaused.clear(); streamSynchronizers.clear(); } - - private class DeserializingIterator extends FilteredIterator, ConsumerRecord> { - - DeserializingIterator(Iterator> inner) { - super(inner); - } - - protected ConsumerRecord filter(ConsumerRecord record) { - Object key = keyDeserializer.deserialize(record.topic(), record.key()); - Object value = valueDeserializer.deserialize(record.topic(), record.value()); - return new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), key, value); - } - - } - } From ad64ddeac74733835f1981c18a57735c98a2a6e7 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 10 Jul 2015 15:20:48 -0700 Subject: [PATCH 051/275] add default deserializer interface --- .../io/confluent/streaming/KStreamContext.java | 16 ++++++++++++++++ .../streaming/internal/KStreamContextImpl.java | 10 ++++++++++ 2 files changed, 26 insertions(+) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index ee1cae948ae14..506795147181c 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -46,6 +46,22 @@ public interface KStreamContext { */ Deserializer valueDeserializer(); + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * @param topic the topic name + * @return KStream + */ + KStream from(String topic); + + /** + * Creates a KStream instance for the specified topic. The stream is added to the specified synchronization group. + * @param topic the topic name + * @param syncGroup the synchronization group + * @return KStream + */ + KStream from(String topic, SyncGroup syncGroup); + + /** * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. * @param topic the topic name diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 588bfc1ae942b..f61b41f0964d1 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -116,6 +116,16 @@ public Deserializer valueDeserializer() { return streamingConfig.valueDeserializer(); } + @Override + public KStream from(String topic) { + return from(topic, syncGroup(DEFAULT_SYNCHRONIZATION_GROUP), null, null); + } + + @Override + public KStream from(String topic, SyncGroup syncGroup) { + return from(topic, syncGroup, null, null); + } + @Override public KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer) { return from(topic, syncGroup(DEFAULT_SYNCHRONIZATION_GROUP), keyDeserializer, valDeserializer); From 4bab92e2aab6cd6928e2be1337beefa5e9942961 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 10 Jul 2015 16:44:23 -0700 Subject: [PATCH 052/275] javadoc --- .../streaming/TimestampExtractor.java | 2 +- .../streaming/internal/RecordQueue.java | 37 ++++++++++++++++++- .../internal/StreamSynchronizer.java | 36 +++++++++++++++++- 3 files changed, 72 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/confluent/streaming/TimestampExtractor.java b/src/main/java/io/confluent/streaming/TimestampExtractor.java index e270fa2bdf0fd..ab700a1893460 100644 --- a/src/main/java/io/confluent/streaming/TimestampExtractor.java +++ b/src/main/java/io/confluent/streaming/TimestampExtractor.java @@ -10,7 +10,7 @@ public interface TimestampExtractor { * @param topic the topic name * @param key the key object * @param value the value object - * @return + * @return timestamp */ long extract(String topic, Object key, Object value); diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java index b05a0c9f0f692..e744fb1d3ac4f 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -8,7 +8,8 @@ import java.util.ArrayDeque; /** - * Created by yasuhiro on 6/25/15. + * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). It is intended to be used in + * {@link StreamSynchronizer}. */ public class RecordQueue { @@ -18,22 +19,44 @@ public class RecordQueue { private TimestampTracker> timestampTracker; private long offset; +<<<<<<< HEAD public RecordQueue(TopicPartition partition, KStreamSource source, TimestampTracker> timestampTracker) { +======= + /** + * Creates a new instance of RecordQueue + * @param partition partition + * @param receiver the receiver of the stream of this partition + * @param timestampTracker TimestampTracker + */ + public RecordQueue(TopicPartition partition, Receiver receiver, TimestampTracker> timestampTracker) { +>>>>>>> javadoc this.partition = partition; this.source = source; this.timestampTracker = timestampTracker; } + /** + * Returns the partition with which this queue is associated + * @return TopicPartition + */ public TopicPartition partition() { return partition; } + /** + * Adds a StampedRecord to the queue + * @param record StampedRecord + */ public void add(StampedRecord record) { queue.addLast(record); timestampTracker.addStampedElement(record); offset = record.offset(); } + /** + * Returns the next record fro the queue + * @return StampedRecord + */ public StampedRecord next() { StampedRecord elem = queue.pollFirst(); @@ -44,14 +67,26 @@ public StampedRecord next() { return elem; } + /** + * Returns the highest offset in the queue + * @return offset + */ public long offset() { return offset; } + /** + * Returns the number of records in the queue + * @return the number of records + */ public int size() { return queue.size(); } + /** + * Tests if the queue is empty + * @return true if the queue is empty, otherwise false + */ public boolean isEmpty() { return queue.isEmpty(); } diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index e3bd5a06077f3..0687c3de21d6f 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -14,7 +14,7 @@ import java.util.concurrent.atomic.AtomicBoolean; /** - * Created by yasuhiro on 6/23/15. + * StreamSynchronizer tries to synchronize the progress of streams from different topics in the same {@link SyncGroup}. */ <<<<<<< HEAD public class StreamSynchronizer implements ParallelExecutor.Task { @@ -53,6 +53,14 @@ public class StreamSynchronizer implements SyncGroup { private long streamTime = -1; private volatile int buffered = 0; + /** + * Creates StreamSynchronizer + * @param name the name of {@link SyncGroup} + * @param ingestor the instance of {@link Ingestor} + * @param chooser the instance of {@link Chooser} + * @param timestampExtractor the instance of {@link TimestampExtractor} + * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic + */ StreamSynchronizer(String name, Ingestor ingestor, Chooser chooser, @@ -74,6 +82,11 @@ public String name() { >>>>>>> removed some generics } + /** + * Adds a partition and its receiver to this stream synchronizer + * @param partition the partition + * @param receiver the receiver + */ @SuppressWarnings("unchecked") public void addPartition(TopicPartition partition, KStreamSource source) { synchronized (this) { @@ -87,9 +100,17 @@ public void addPartition(TopicPartition partition, KStreamSource source) { } } +<<<<<<< HEAD <<<<<<< HEAD public void addRecords(TopicPartition partition, Iterator> iterator) { ======= +======= + /** + * Adds records + * @param partition the partition + * @param iterator the iterator of records + */ +>>>>>>> javadoc @SuppressWarnings("unchecked") <<<<<<< HEAD public void addRecords(TopicPartition partition, Iterator> iterator) { @@ -153,10 +174,19 @@ private void ingestNewRecords() { newRecordBuffer.clear(); } + /** + * Returns a PunctuationScheduler + * @param processor the processor requesting scheduler + * @return PunctuationScheduler + */ public PunctuationScheduler getPunctuationScheduler(Processor processor) { return new PunctuationSchedulerImpl(punctuationQueue, processor); } + /** + * Processes one record + * @param context an application specific context object for a task + */ @SuppressWarnings("unchecked") public void process(Object context) { Status status = (Status) context; @@ -194,6 +224,10 @@ public void process(Object context) { } } + /** + * Returns consumed offsets + * @return the map of partition to consumed offset + */ public Map consumedOffsets() { return this.consumedOffsets; } From 9fbd1eaa5296e32767a3e1d290cab03f5d799481 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sat, 11 Jul 2015 17:41:21 -0700 Subject: [PATCH 053/275] extend sendTo and through with ser-de overriden enabled --- .../java/io/confluent/streaming/KStream.java | 66 ++++++++++++++ .../confluent/streaming/KStreamContext.java | 6 -- .../streaming/internal/KStreamBranch.java | 3 +- .../internal/KStreamContextImpl.java | 15 ++-- .../streaming/internal/KStreamFilter.java | 5 +- .../streaming/internal/KStreamFlatMap.java | 5 +- .../internal/KStreamFlatMapValues.java | 5 +- .../streaming/internal/KStreamImpl.java | 65 +++++++++++--- .../streaming/internal/KStreamJoin.java | 5 +- .../streaming/internal/KStreamMap.java | 5 +- .../streaming/internal/KStreamMapValues.java | 5 +- .../streaming/internal/KStreamSource.java | 12 +-- .../internal/KStreamWindowedImpl.java | 12 ++- .../streaming/internal/RecordQueue.java | 10 ++- .../internal/StreamSynchronizer.java | 20 +++-- .../internal/MockKStreamContext.java | 80 +++++++++++++++++ .../internal/StreamSynchronizerTest.java | 87 +++++++++++-------- .../streaming/testutil/MockIngestor.java | 2 +- 18 files changed, 308 insertions(+), 100 deletions(-) create mode 100644 src/test/java/io/confluent/streaming/internal/MockKStreamContext.java diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index 34f361c75ec87..bef68e44a1e29 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -1,10 +1,18 @@ package io.confluent.streaming; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + /** * KStream is an abstraction of a stream of key-value pairs. */ public interface KStream { + /** + * Returns the KStreamContext used to create this stream + */ + KStreamContext context(); + /** * Creates a new stream consists of all elements of this stream which satisfy a predicate * @param predicate the instance of Predicate @@ -72,18 +80,76 @@ public interface KStream { /** * Sends key-value to a topic, also creates a new stream from the topic. + * The created stream is added to the default synchronization group. * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). * @param topic the topic name * @return KStream */ KStream through(String topic); + /** + * Sends key-value to a topic, also creates a new stream from the topic. + * The created stream is added to the specified synchronization group. + * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic, syncGroup). + * @param topic the topic name + * @param syncGroup the synchronization group + * @return KStream + */ + KStream through(String topic, SyncGroup syncGroup); + + /** + * Sends key-value to a topic, also creates a new stream from the topic. + * The created stream is added to the default synchronization group. + * This is equivalent to calling sendTo(topic) and KStreamContext.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 + * + * @return KStream + */ + KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + + /** + * Sends key-value to a topic, also creates a new stream from the topic. + * The created stream is added to the specific synchronization group. + * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). + * @param topic the topic name + * @param syncGroup the synchronization group + * @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 + * + * @return KStream + */ + KStream through(String topic, SyncGroup syncGroup, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + /** * Sends key-value to a topic. * @param topic the topic name */ void sendTo(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 sendTo(String topic, Serializer keySerializer, Serializer valSerializer); + /** * Processes all elements in this stream by applying a processor. * @param processor the instance of Processor diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 506795147181c..5945750ecb242 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -85,12 +85,6 @@ public interface KStreamContext { */ KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer); - /** - * Returns a RecordCollector which takes binary (byte array) key and value. - * @return RecordCollector - */ - RecordCollector simpleRecordCollector(); - /** * Returns a RecordCollector which applies the serializer to key and value. * @return RecordCollector diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 1d3b3d842cd5c..b38d6bf525228 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -1,5 +1,6 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Predicate; import java.lang.reflect.Array; @@ -14,7 +15,7 @@ class KStreamBranch implements Receiver { final KStreamSource[] branches; @SuppressWarnings("unchecked") - KStreamBranch(Predicate[] predicates, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + KStreamBranch(Predicate[] predicates, PartitioningInfo partitioningInfo, KStreamContext context) { this.predicates = Arrays.copyOf(predicates, predicates.length); this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index f61b41f0964d1..0e6200b6c1537 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -76,8 +76,8 @@ public KStreamContextImpl(int id, this.ingestor = ingestor; this.simpleCollector = simpleCollector; - this.collector = new RecordCollectors.SerializingRecordCollector( - simpleCollector, (Serializer) streamingConfig.keySerializer(), (Serializer) streamingConfig.valueSerializer()); + this.collector = new RecordCollectors.SerializingRecordCollector( + simpleCollector, streamingConfig.keySerializer(), streamingConfig.valueSerializer()); this.coordinator = coordinator; this.streamingConfig = streamingConfig; @@ -91,6 +91,8 @@ public KStreamContextImpl(int id, this.metrics = metrics; } + public RecordCollectors.SimpleRecordCollector simpleRecordCollector() { return this.simpleCollector; } + @Override public int id() { return id; @@ -179,8 +181,8 @@ public Deserializer valueDeserializer() { if (stream.partitioningInfo.syncGroup == syncGroup) throw new IllegalStateException("topic is already assigned a different synchronization group"); - // with this constraint we will not allow users to create KStream with different deser from the same topic, - // this constraint may better be relaxed later. + // TODO: with this constraint we will not allow users to create KStream with different + // deser from the same topic, this constraint may better be relaxed later. if (keyDeserializer != null && !keyDeserializer.getClass().equals(this.keyDeserializer().getClass())) throw new IllegalStateException("another source stream with the same topic but different key deserializer is already created"); if (valDeserializer != null && !valDeserializer.getClass().equals(this.valueDeserializer().getClass())) @@ -191,11 +193,6 @@ public Deserializer valueDeserializer() { } } - @Override - public RecordCollector simpleRecordCollector() { - return simpleCollector; - } - @Override public RecordCollector recordCollector() { return collector; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index ea0b2f5deab8d..9ebac87ae9e3e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -1,15 +1,16 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Predicate; /** * Created by yasuhiro on 6/17/15. */ -class KStreamFilter extends KStreamImpl { +class KStreamFilter extends KStreamImpl { private final Predicate predicate; - KStreamFilter(Predicate predicate, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + KStreamFilter(Predicate predicate, PartitioningInfo partitioningInfo, KStreamContext context) { super(partitioningInfo, context); this.predicate = predicate; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 988cae0b6a4de..97b06cf041d25 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -1,5 +1,6 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; import io.confluent.streaming.SyncGroup; @@ -7,11 +8,11 @@ /** * Created by yasuhiro on 6/17/15. */ -class KStreamFlatMap extends KStreamImpl { +class KStreamFlatMap extends KStreamImpl { private final KeyValueMapper, K1, V1> mapper; - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, SyncGroup syncGroup, KStreamContextImpl context) { + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, SyncGroup syncGroup, KStreamContext context) { super(PartitioningInfo.unjoinable(syncGroup), context); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index a25ac25678dce..87a7a9e6dda04 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -1,15 +1,16 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.ValueMapper; /** * Created by yasuhiro on 6/17/15. */ -class KStreamFlatMapValues extends KStreamImpl { +class KStreamFlatMapValues extends KStreamImpl { private final ValueMapper, V1> mapper; - KStreamFlatMapValues(ValueMapper, V1> mapper, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + KStreamFlatMapValues(ValueMapper, V1> mapper, PartitioningInfo partitioningInfo, KStreamContext context) { super(partitioningInfo, context); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index ce26edfd27472..9fb60b4878474 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -1,24 +1,41 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.*; +import io.confluent.streaming.KStream; +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamWindowed; +import io.confluent.streaming.KeyValueMapper; +import io.confluent.streaming.Predicate; +import io.confluent.streaming.Processor; +import io.confluent.streaming.PunctuationScheduler; +import io.confluent.streaming.RecordCollector; +import io.confluent.streaming.SyncGroup; +import io.confluent.streaming.ValueMapper; +import io.confluent.streaming.Window; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; import java.util.ArrayList; /** * Created by yasuhiro on 6/17/15. */ -abstract class KStreamImpl implements KStream, Receiver { +abstract class KStreamImpl implements KStream, Receiver { private final ArrayList nextReceivers = new ArrayList<>(1); final PartitioningInfo partitioningInfo; - final KStreamContextImpl context; + final KStreamContext context; - protected KStreamImpl(PartitioningInfo partitioningInfo, KStreamContextImpl context) { + protected KStreamImpl(PartitioningInfo partitioningInfo, KStreamContext context) { this.partitioningInfo = partitioningInfo; this.context = context; } + @Override + public KStreamContext context() { + return this.context; + } + @Override public KStream filter(Predicate predicate) { return chain(new KStreamFilter(predicate, partitioningInfo, context)); @@ -69,18 +86,46 @@ public KStream[] branch(Predicate... predicates) { @SuppressWarnings("unchecked") @Override public KStream through(String topic) { - process(this.getSendProcessor(topic)); - return (KStream) context.from(topic, null, null); + return through(topic, null); + } + + @SuppressWarnings("unchecked") + @Override + public KStream through(String topic, SyncGroup syncGroup) { + return through(topic, syncGroup, null, null, null, null); + } + + @SuppressWarnings("unchecked") + @Override + public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { + return through(topic, context.syncGroup(context.DEFAULT_SYNCHRONIZATION_GROUP), keySerializer, valSerializer, keyDeserializer, valDeserializer); + } + + @SuppressWarnings("unchecked") + @Override + public KStream through(String topic, SyncGroup syncGroup, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { + process(this.getSendProcessor(topic, keySerializer, valSerializer)); + return (KStream) context.from(topic, syncGroup, keyDeserializer, valDeserializer); } @Override public void sendTo(String topic) { - process(this.getSendProcessor(topic)); + process(this.getSendProcessor(topic, null, null)); } + @Override + public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); } + @SuppressWarnings("unchecked") - private Processor getSendProcessor(final String topic) { - final RecordCollector collector = (RecordCollector) context.recordCollector(); + private Processor getSendProcessor(final String topic, Serializer keySerializer, Serializer valSerializer) { + final RecordCollector collector; + if (keySerializer == null && valSerializer == null) + collector = (RecordCollector) context.recordCollector(); + else + collector = (RecordCollector) new RecordCollectors.SerializingRecordCollector( + ((KStreamContextImpl)context).simpleRecordCollector(), + keySerializer == null ? context.keySerializer() : keySerializer, + valSerializer == null ? context.valueSerializer() : valSerializer); return new Processor() { @Override @@ -118,7 +163,7 @@ protected void forward(Object key, Object value, long timestamp, long streamTime } } - protected KStream chain(KStreamImpl kstream) { + protected KStream chain(KStreamImpl kstream) { synchronized(this) { nextReceivers.add(kstream); return kstream; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index bbdcde3b338c6..8119af9ff0d13 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -1,5 +1,6 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; @@ -8,7 +9,7 @@ /** * Created by yasuhiro on 6/17/15. */ -class KStreamJoin extends KStreamImpl { +class KStreamJoin extends KStreamImpl { private static abstract class Finder { abstract Iterator find(K key, long timestamp); @@ -19,7 +20,7 @@ private static abstract class Finder { private final ValueJoiner joiner; final Receiver receiverForOtherStream; - KStreamJoin(final Window window1, final Window window2, boolean prior, ValueJoiner joiner, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + KStreamJoin(final Window window1, final Window window2, boolean prior, ValueJoiner joiner, PartitioningInfo partitioningInfo, KStreamContext context) { super(partitioningInfo, context); if (prior) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index eea1ce0349eb4..e93a71cd1f806 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -1,5 +1,6 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; import io.confluent.streaming.SyncGroup; @@ -7,11 +8,11 @@ /** * Created by yasuhiro on 6/17/15. */ -class KStreamMap extends KStreamImpl { +class KStreamMap extends KStreamImpl { private final KeyValueMapper mapper; - KStreamMap(KeyValueMapper mapper, SyncGroup syncGroup, KStreamContextImpl context) { + KStreamMap(KeyValueMapper mapper, SyncGroup syncGroup, KStreamContext context) { super(PartitioningInfo.unjoinable(syncGroup), context); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index 3f814333d103d..925271da647ea 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -1,15 +1,16 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.ValueMapper; /** * Created by yasuhiro on 6/17/15. */ -class KStreamMapValues extends KStreamImpl { +class KStreamMapValues extends KStreamImpl { private final ValueMapper mapper; - KStreamMapValues(ValueMapper mapper, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + KStreamMapValues(ValueMapper mapper, PartitioningInfo partitioningInfo, KStreamContext context) { super(partitioningInfo, context); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 70acf24a12d53..7b8765665c17a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -5,18 +5,10 @@ /** * Created by yasuhiro on 6/17/15. */ -class KStreamSource extends KStreamImpl { +class KStreamSource extends KStreamImpl { - private KStreamContext context; - - KStreamSource(PartitioningInfo partitioningInfo, KStreamContextImpl context) { + KStreamSource(PartitioningInfo partitioningInfo, KStreamContext context) { super(partitioningInfo, context); - - this.context = context; - } - - public KStreamContext context() { - return this.context; } @Override diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index d4c5def938559..33343152123c2 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -1,15 +1,21 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.*; + +import io.confluent.streaming.KStream; +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamWindowed; +import io.confluent.streaming.NotCopartitionedException; +import io.confluent.streaming.ValueJoiner; +import io.confluent.streaming.Window; /** * Created by yasuhiro on 6/18/15. */ -public class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { +public class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { final Window window; - KStreamWindowedImpl(Window window, PartitioningInfo partitioningInfo, KStreamContextImpl context) { + KStreamWindowedImpl(Window window, PartitioningInfo partitioningInfo, KStreamContext context) { super(partitioningInfo, context); this.window = window; } diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java index e744fb1d3ac4f..a5d846bc857ef 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -1,6 +1,6 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.util.Stamped; +import io.confluent.streaming.KStream; import io.confluent.streaming.util.TimestampTracker; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; @@ -14,11 +14,12 @@ public class RecordQueue { private final ArrayDeque queue = new ArrayDeque<>(); - public final KStreamSource source; + public final KStreamImpl stream; private final TopicPartition partition; private TimestampTracker> timestampTracker; private long offset; +<<<<<<< HEAD <<<<<<< HEAD public RecordQueue(TopicPartition partition, KStreamSource source, TimestampTracker> timestampTracker) { ======= @@ -30,8 +31,11 @@ public RecordQueue(TopicPartition partition, KStreamSource source, TimestampTrac */ public RecordQueue(TopicPartition partition, Receiver receiver, TimestampTracker> timestampTracker) { >>>>>>> javadoc +======= + public RecordQueue(TopicPartition partition, KStreamImpl stream, TimestampTracker> timestampTracker) { +>>>>>>> extend sendTo and through with ser-de overriden enabled this.partition = partition; - this.source = source; + this.stream = stream; this.timestampTracker = timestampTracker; } diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index 0687c3de21d6f..3271faf2a28ac 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -1,6 +1,9 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.*; +import io.confluent.streaming.Processor; +import io.confluent.streaming.PunctuationScheduler; +import io.confluent.streaming.SyncGroup; +import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.util.MinTimestampTracker; import io.confluent.streaming.util.ParallelExecutor; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -88,12 +91,12 @@ public String name() { * @param receiver the receiver */ @SuppressWarnings("unchecked") - public void addPartition(TopicPartition partition, KStreamSource source) { + public void addPartition(TopicPartition partition, KStreamImpl stream) { synchronized (this) { RecordQueue recordQueue = stash.get(partition); if (recordQueue == null) { - stash.put(partition, createRecordQueue(partition, source)); + stash.put(partition, createRecordQueue(partition, stream)); } else { throw new IllegalStateException("duplicate partition"); } @@ -123,6 +126,7 @@ public void addRecords(TopicPartition partition, Iterator newRecords : newRecordBuffer) { TopicPartition partition = newRecords.partition; @@ -149,8 +153,8 @@ private void ingestNewRecords() { ConsumerRecord record = iterator.next(); // deserialize the raw record, extract the timestamp and put into the queue - Deserializer keyDeserializer = recordQueue.source.context().keyDeserializer(); - Deserializer valDeserializer = recordQueue.source.context().valueDeserializer(); + Deserializer keyDeserializer = recordQueue.stream.context().keyDeserializer(); + Deserializer valDeserializer = recordQueue.stream.context().valueDeserializer(); Object key = keyDeserializer.deserialize(record.topic(), record.key()); Object value = valDeserializer.deserialize(record.topic(), record.value()); @@ -213,7 +217,7 @@ public void process(Object context) { if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - recordQueue.source.receive(record.key(), record.value(), record.timestamp, streamTime); + recordQueue.stream.receive(record.key(), record.value(), record.timestamp, streamTime); consumedOffsets.put(recordQueue.partition(), record.offset()); if (recordQueue.size() > 0) chooser.add(recordQueue); @@ -241,8 +245,8 @@ public void close() { stash.clear(); } - protected RecordQueue createRecordQueue(TopicPartition partition, KStreamSource source) { - return new RecordQueue(partition, source, new MinTimestampTracker>()); + protected RecordQueue createRecordQueue(TopicPartition partition, KStreamImpl stream) { + return new RecordQueue(partition, stream, new MinTimestampTracker>()); } private static class NewRecords { diff --git a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java b/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java new file mode 100644 index 0000000000000..25ad36f2544a2 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java @@ -0,0 +1,80 @@ +package io.confluent.streaming.internal; + + +import io.confluent.streaming.KStream; +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.RecordCollector; +import io.confluent.streaming.StorageEngine; +import io.confluent.streaming.SyncGroup; +import io.confluent.streaming.Coordinator; +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.Map; + +/** + * Created by guozhang on 7/11/15. + */ +public class MockKStreamContext implements KStreamContext { + + Serializer serializer; + Deserializer deserializer; + + public MockKStreamContext(Serializer serializer, Deserializer deserializer) { + this.serializer = serializer; + this.deserializer = deserializer; + } + + @Override + public int id() { return -1; } + + @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 KStream from(String topic) { throw new UnsupportedOperationException("from() not supported."); } + + @Override + public KStream from(String topic, SyncGroup syncGroup) { throw new UnsupportedOperationException("from() not supported."); } + + @Override + public KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer) { throw new UnsupportedOperationException("from() not supported."); } + + @Override + public KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer) { throw new UnsupportedOperationException("from() not supported."); } + + @Override + public RecordCollector recordCollector() { throw new UnsupportedOperationException("recordCollector() not supported."); } + + @Override + public Coordinator coordinator() { throw new UnsupportedOperationException("coordinator() not supported."); } + + @Override + public Map getContext() { throw new UnsupportedOperationException("getContext() not supported."); } + + @Override + public File stateDir() { throw new UnsupportedOperationException("stateDir() not supported."); } + + @Override + public Metrics metrics() { throw new UnsupportedOperationException("metrics() not supported."); } + + @Override + public SyncGroup syncGroup(String name) { throw new UnsupportedOperationException("syncGroup() not supported."); } + + @Override + public SyncGroup roundRobinSyncGroup(String name) { throw new UnsupportedOperationException("roundRobinSyncGroup() not supported."); } + + @Override + public void restore(StorageEngine engine) throws Exception { throw new UnsupportedOperationException("restore() not supported."); } +} diff --git a/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java b/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java index 09a6a36d40b81..b279ac22b5c48 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java @@ -4,6 +4,10 @@ import io.confluent.streaming.testutil.MockIngestor; 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.junit.Test; import java.util.ArrayList; @@ -15,7 +19,10 @@ public class StreamSynchronizerTest { - private static class MockReceiver implements Receiver { + private static Serializer serializer = new IntegerSerializer(); + private static Deserializer deserializer = new IntegerDeserializer(); + + private static class MockKStreamImpl extends KStreamImpl { public int numReceived = 0; public ArrayList keys = new ArrayList<>(); @@ -23,6 +30,10 @@ private static class MockReceiver implements Receiver { public ArrayList timestamps = new ArrayList<>(); public ArrayList streamTimes = new ArrayList<>(); + public MockKStreamImpl() { + super(null, new MockKStreamContext(serializer, deserializer)); + } + @Override public void receive(Object key, Object value, long timestamp, long streamTime) { this.numReceived++; @@ -56,41 +67,43 @@ public long extract(String topic, Object key, Object value) { TopicPartition partition1 = new TopicPartition("topic1", 1); TopicPartition partition2 = new TopicPartition("topic2", 1); - MockReceiver receiver1 = new MockReceiver(); - MockReceiver receiver2 = new MockReceiver(); - MockReceiver receiver3 = new MockReceiver(); + MockKStreamImpl stream1 = new MockKStreamImpl(); + MockKStreamImpl stream2 = new MockKStreamImpl(); + MockKStreamImpl stream3 = new MockKStreamImpl(); - streamSynchronizer.addPartition(partition1, receiver1); + streamSynchronizer.addPartition(partition1, stream1); mockIngestor.addStreamSynchronizerForPartition(streamSynchronizer, partition1); - streamSynchronizer.addPartition(partition2, receiver2); + streamSynchronizer.addPartition(partition2, stream2); mockIngestor.addStreamSynchronizerForPartition(streamSynchronizer, partition2); Exception exception = null; try { - streamSynchronizer.addPartition(partition1, receiver3); + streamSynchronizer.addPartition(partition1, stream3); } catch (Exception ex) { exception = ex; } assertTrue(exception != null); + byte[] recordValue = serializer.serialize(null, new Integer(10)); + mockIngestor.addRecords(partition1, records( - new ConsumerRecord(partition1.topic(), partition1.partition(), 1, new Integer(10), "A"), - new ConsumerRecord(partition1.topic(), partition1.partition(), 2, new Integer(20), "AA") + new ConsumerRecord(partition1.topic(), partition1.partition(), 1, serializer.serialize(partition1.topic(), new Integer(10)), recordValue), + new ConsumerRecord(partition1.topic(), partition1.partition(), 2, serializer.serialize(partition1.topic(), new Integer(20)), recordValue) )); mockIngestor.addRecords(partition2, records( - new ConsumerRecord(partition2.topic(), partition2.partition(), 1, new Integer(300), "B"), - new ConsumerRecord(partition2.topic(), partition2.partition(), 2, new Integer(400), "BB"), - new ConsumerRecord(partition2.topic(), partition2.partition(), 3, new Integer(500), "BBB"), - new ConsumerRecord(partition2.topic(), partition2.partition(), 4, new Integer(600), "BBBB") + new ConsumerRecord(partition2.topic(), partition2.partition(), 1, serializer.serialize(partition1.topic(), new Integer(300)), recordValue), + new ConsumerRecord(partition2.topic(), partition2.partition(), 2, serializer.serialize(partition1.topic(), new Integer(400)), recordValue), + new ConsumerRecord(partition2.topic(), partition2.partition(), 3, serializer.serialize(partition1.topic(), new Integer(500)), recordValue), + new ConsumerRecord(partition2.topic(), partition2.partition(), 4, serializer.serialize(partition1.topic(), new Integer(600)), recordValue) )); StreamSynchronizer.Status status = new StreamSynchronizer.Status(); streamSynchronizer.process(status); - assertEquals(receiver1.numReceived, 1); - assertEquals(receiver2.numReceived, 0); + assertEquals(stream1.numReceived, 1); + assertEquals(stream2.numReceived, 0); assertTrue(status.pollRequired()); @@ -98,62 +111,62 @@ public long extract(String topic, Object key, Object value) { assertTrue(mockIngestor.paused.contains(partition2)); mockIngestor.addRecords(partition1, records( - new ConsumerRecord(partition1.topic(), partition1.partition(), 3, new Integer(30), "AAA"), - new ConsumerRecord(partition1.topic(), partition1.partition(), 4, new Integer(40), "AAAA"), - new ConsumerRecord(partition1.topic(), partition1.partition(), 5, new Integer(50), "AAAAA") + new ConsumerRecord(partition1.topic(), partition1.partition(), 3, serializer.serialize(partition1.topic(), new Integer(30)), recordValue), + new ConsumerRecord(partition1.topic(), partition1.partition(), 4, serializer.serialize(partition1.topic(), new Integer(40)), recordValue), + new ConsumerRecord(partition1.topic(), partition1.partition(), 5, serializer.serialize(partition1.topic(), new Integer(50)), recordValue) )); streamSynchronizer.process(status); - assertEquals(receiver1.numReceived, 2); - assertEquals(receiver2.numReceived, 0); + assertEquals(stream1.numReceived, 2); + assertEquals(stream2.numReceived, 0); assertEquals(mockIngestor.paused.size(), 2); assertTrue(mockIngestor.paused.contains(partition1)); assertTrue(mockIngestor.paused.contains(partition2)); streamSynchronizer.process(status); - assertEquals(receiver1.numReceived, 3); - assertEquals(receiver2.numReceived, 0); + assertEquals(stream1.numReceived, 3); + assertEquals(stream2.numReceived, 0); streamSynchronizer.process(status); - assertEquals(receiver1.numReceived, 3); - assertEquals(receiver2.numReceived, 1); + assertEquals(stream1.numReceived, 3); + assertEquals(stream2.numReceived, 1); assertEquals(mockIngestor.paused.size(), 1); assertTrue(mockIngestor.paused.contains(partition2)); streamSynchronizer.process(status); - assertEquals(receiver1.numReceived, 4); - assertEquals(receiver2.numReceived, 1); + assertEquals(stream1.numReceived, 4); + assertEquals(stream2.numReceived, 1); assertEquals(mockIngestor.paused.size(), 1); streamSynchronizer.process(status); - assertEquals(receiver1.numReceived, 4); - assertEquals(receiver2.numReceived, 2); + assertEquals(stream1.numReceived, 4); + assertEquals(stream2.numReceived, 2); assertEquals(mockIngestor.paused.size(), 0); streamSynchronizer.process(status); - assertEquals(receiver1.numReceived, 5); - assertEquals(receiver2.numReceived, 2); + assertEquals(stream1.numReceived, 5); + assertEquals(stream2.numReceived, 2); streamSynchronizer.process(status); - assertEquals(receiver1.numReceived, 5); - assertEquals(receiver2.numReceived, 3); + assertEquals(stream1.numReceived, 5); + assertEquals(stream2.numReceived, 3); streamSynchronizer.process(status); - assertEquals(receiver1.numReceived, 5); - assertEquals(receiver2.numReceived, 4); + assertEquals(stream1.numReceived, 5); + assertEquals(stream2.numReceived, 4); assertEquals(mockIngestor.paused.size(), 0); streamSynchronizer.process(status); - assertEquals(receiver1.numReceived, 5); - assertEquals(receiver2.numReceived, 4); + assertEquals(stream1.numReceived, 5); + assertEquals(stream2.numReceived, 4); } - private List> records(ConsumerRecord... recs) { + private List> records(ConsumerRecord... recs) { return Arrays.asList(recs); } } diff --git a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java index f88a30b394f28..0abacfd1b92fe 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java +++ b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java @@ -47,7 +47,7 @@ public void removeStreamSynchronizerForPartition(TopicPartition partition) { streamSynchronizers.remove(partition); } - public void addRecords(TopicPartition partition, Iterable> records) { + public void addRecords(TopicPartition partition, Iterable> records) { streamSynchronizers.get(partition).addRecords(partition, records.iterator()); } } From 120bb139c7a24bc19a9586ae02df5d0e6eb57324 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 13 Jul 2015 10:20:53 -0700 Subject: [PATCH 054/275] more fix --- src/main/java/io/confluent/streaming/KStream.java | 5 +++++ .../java/io/confluent/streaming/internal/RecordQueue.java | 4 ++++ .../io/confluent/streaming/internal/StreamSynchronizer.java | 2 +- 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index bef68e44a1e29..2d4badf5913af 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -10,6 +10,7 @@ public interface KStream { /** * Returns the KStreamContext used to create this stream + * @return KStreamContext */ KStreamContext context(); @@ -110,6 +111,8 @@ public interface KStream { * if not specified the default deserializer defined in the configs will be used * @param valDeserializer value deserializer used to create the new KStream, * if not specified the default deserializer defined in the configs will be used + * @param the key type of the new stream + * @param the value type of the new stream * * @return KStream */ @@ -129,6 +132,8 @@ public interface KStream { * if not specified the default deserializer defined in the configs will be used * @param valDeserializer value deserializer used to create the new KStream, * if not specified the default deserializer defined in the configs will be used + * @param the key type of the new stream + * @param the value type of the new stream * * @return KStream */ diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java index a5d846bc857ef..30539620feb6d 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -26,7 +26,11 @@ public RecordQueue(TopicPartition partition, KStreamSource source, TimestampTrac /** * Creates a new instance of RecordQueue * @param partition partition +<<<<<<< HEAD * @param receiver the receiver of the stream of this partition +======= + * @param stream the instance of KStreamImpl that receives records +>>>>>>> more fix * @param timestampTracker TimestampTracker */ public RecordQueue(TopicPartition partition, Receiver receiver, TimestampTracker> timestampTracker) { diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index 3271faf2a28ac..7e2dbeb602345 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -88,7 +88,7 @@ public String name() { /** * Adds a partition and its receiver to this stream synchronizer * @param partition the partition - * @param receiver the receiver + * @param stream the instance of KStreamImpl */ @SuppressWarnings("unchecked") public void addPartition(TopicPartition partition, KStreamImpl stream) { From 99757e6e8c4d3d6149431e27d2aebd0480f0c514 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 13 Jul 2015 13:09:47 -0700 Subject: [PATCH 055/275] fix interface template definitions --- src/main/java/io/confluent/streaming/KStream.java | 7 ++++--- src/main/java/io/confluent/streaming/KStreamContext.java | 4 ++-- .../confluent/streaming/internal/KStreamContextImpl.java | 6 +++--- .../java/io/confluent/streaming/internal/KStreamImpl.java | 8 ++++---- 4 files changed, 13 insertions(+), 12 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index 2d4badf5913af..88430380ebc91 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -1,5 +1,6 @@ package io.confluent.streaming; +import com.sun.javafx.geom.Vec2d; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; @@ -116,7 +117,7 @@ public interface KStream { * * @return KStream */ - KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); /** * Sends key-value to a topic, also creates a new stream from the topic. @@ -137,7 +138,7 @@ public interface KStream { * * @return KStream */ - KStream through(String topic, SyncGroup syncGroup, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + KStream through(String topic, SyncGroup syncGroup, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); /** * Sends key-value to a topic. @@ -153,7 +154,7 @@ public interface KStream { * @param valSerializer value serializer used to send key-value pairs, * if not specified the default serializer defined in the configs will be used */ - void sendTo(String topic, Serializer keySerializer, Serializer valSerializer); + void sendTo(String topic, Serializer keySerializer, Serializer valSerializer); /** * Processes all elements in this stream by applying a processor. diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 5945750ecb242..c1b23bc398e20 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -71,7 +71,7 @@ public interface KStreamContext { * if not specified the default deserializer defined in the configs will be used * @return KStream */ - KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer); + KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer); /** * Creates a KStream instance for the specified topic. The stream is added to the specified synchronization group. @@ -83,7 +83,7 @@ public interface KStreamContext { * if not specified the default deserializer defined in the configs will be used * @return KStream */ - KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer); + KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer); /** * Returns a RecordCollector which applies the serializer to key and value. diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 0e6200b6c1537..0609e71bffa36 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -129,13 +129,13 @@ public Deserializer valueDeserializer() { } @Override - public KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer) { + public KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer) { return from(topic, syncGroup(DEFAULT_SYNCHRONIZATION_GROUP), keyDeserializer, valDeserializer); } @Override @SuppressWarnings("unchecked") - public KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer) { + public KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer) { if (syncGroup == null) throw new NullPointerException(); synchronized (this) { @@ -189,7 +189,7 @@ public Deserializer valueDeserializer() { throw new IllegalStateException("another source stream with the same topic but different value deserializer is already created"); } - return stream; + return (KStream) stream; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 9fb60b4878474..2b5e76756fece 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -97,15 +97,15 @@ public KStream through(String topic, SyncGroup syncGroup) { @SuppressWarnings("unchecked") @Override - public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { + public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { return through(topic, context.syncGroup(context.DEFAULT_SYNCHRONIZATION_GROUP), keySerializer, valSerializer, keyDeserializer, valDeserializer); } @SuppressWarnings("unchecked") @Override - public KStream through(String topic, SyncGroup syncGroup, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { + public KStream through(String topic, SyncGroup syncGroup, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); - return (KStream) context.from(topic, syncGroup, keyDeserializer, valDeserializer); + return context.from(topic, syncGroup, keyDeserializer, valDeserializer); } @Override @@ -114,7 +114,7 @@ public void sendTo(String topic) { } @Override - public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); } + public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); } @SuppressWarnings("unchecked") private Processor getSendProcessor(final String topic, Serializer keySerializer, Serializer valSerializer) { From 93bad7c6644a13c8245c7959375bc1d682c0b173 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 13 Jul 2015 14:37:03 -0700 Subject: [PATCH 056/275] clear interrupted status after park --- .../java/io/confluent/streaming/util/ParallelExecutor.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java index 2cb658bf3aad9..2f18f184c86f7 100644 --- a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java +++ b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java @@ -58,7 +58,7 @@ public void execute(ArrayList tasks, Object context) throws Exce wakeUpWorkers(Math.min(numTasks - 1, workerThreads.length)); // the calling thread also picks up tasks - if (taskIndex.get() > 0) doRun(); + if (taskIndex.get() > 0) doProcess(); while (true) { try { @@ -91,7 +91,7 @@ public void shutdown() { } } - private void doRun() { + private void doProcess() { int index = taskIndex.decrementAndGet(); if (index >= 0) { try { @@ -122,11 +122,12 @@ private class WorkerThread extends Thread { public void run() { while (running) { if (taskIndex.get() > 0) { - doRun(); + doProcess(); } else { // no more work. park this thread. LockSupport.park(); + Thread.interrupted(); } } } From 9c132834ebc29725698bdfc0b05b4b05467ceb99 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 13 Jul 2015 15:53:06 -0700 Subject: [PATCH 057/275] enable multiple topics in StreamContext.from --- .../confluent/streaming/KStreamContext.java | 23 +++-- .../io/confluent/streaming/Processor.java | 4 +- .../streaming/internal/KStreamBranch.java | 8 +- .../internal/KStreamContextImpl.java | 98 +++++++++++++------ .../streaming/internal/KStreamFilter.java | 8 +- .../streaming/internal/KStreamFlatMap.java | 6 +- .../internal/KStreamFlatMapValues.java | 8 +- .../streaming/internal/KStreamImpl.java | 42 ++++---- .../streaming/internal/KStreamJoin.java | 11 ++- .../streaming/internal/KStreamMap.java | 6 +- .../streaming/internal/KStreamMapValues.java | 8 +- .../streaming/internal/KStreamMetadata.java | 36 +++++++ .../streaming/internal/KStreamSource.java | 10 +- .../internal/KStreamWindowedImpl.java | 12 +-- .../streaming/internal/PartitioningInfo.java | 15 +-- .../streaming/internal/Receiver.java | 2 +- .../streaming/internal/StampedRecord.java | 2 + .../internal/StreamSynchronizer.java | 2 +- .../streaming/internal/KStreamBranchTest.java | 13 ++- .../streaming/internal/KStreamFilterTest.java | 14 ++- .../internal/KStreamFlatMapTest.java | 9 +- .../internal/KStreamFlatMapValuesTest.java | 10 +- .../streaming/internal/KStreamJoinTest.java | 46 +++++---- .../streaming/internal/KStreamMapTest.java | 10 +- .../internal/KStreamMapValuesTest.java | 10 +- .../streaming/internal/KStreamSourceTest.java | 10 +- .../internal/KStreamWindowedTest.java | 11 ++- .../internal/MockKStreamContext.java | 8 +- .../internal/StreamSynchronizerTest.java | 2 +- .../streaming/testutil/TestProcessor.java | 2 +- 30 files changed, 278 insertions(+), 168 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamMetadata.java diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index c1b23bc398e20..4fd3087c33a4a 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -46,44 +46,47 @@ public interface KStreamContext { */ Deserializer valueDeserializer(); + // TODO: support regex topic matching in from() calls, for example: + // context.from("Topic*PageView") + /** - * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. - * @param topic the topic name + * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. + * @param topics the topic name * @return KStream */ - KStream from(String topic); + KStream from(String... topics); /** - * Creates a KStream instance for the specified topic. The stream is added to the specified synchronization group. - * @param topic the topic name + * Creates a KStream instance for the specified topics. The stream is added to the specified synchronization group. * @param syncGroup the synchronization group + * @param topics the topic name * @return KStream */ - KStream from(String topic, SyncGroup syncGroup); + KStream from(SyncGroup syncGroup, String... topics); /** * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. - * @param topic the topic name * @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 name * @return KStream */ - KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer); + KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); /** * Creates a KStream instance for the specified topic. The stream is added to the specified synchronization group. - * @param topic the topic name * @param syncGroup the 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 name * @return KStream */ - KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer); + KStream from(SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); /** * Returns a RecordCollector which applies the serializer to key and value. diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index b77514020bc3c..2756180490592 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -3,9 +3,9 @@ /** * Created by yasuhiro on 6/17/15. */ -public interface Processor { +public interface Processor { - void apply(K key, V value); + void apply(String topic, K key, V value); void init(PunctuationScheduler punctuationScheduler); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index b38d6bf525228..92824f2978a75 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -15,21 +15,21 @@ class KStreamBranch implements Receiver { final KStreamSource[] branches; @SuppressWarnings("unchecked") - KStreamBranch(Predicate[] predicates, PartitioningInfo partitioningInfo, KStreamContext context) { + KStreamBranch(Predicate[] predicates, KStreamMetadata streamMetadata, KStreamContext context) { this.predicates = Arrays.copyOf(predicates, predicates.length); this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { - branches[i] = new KStreamSource(partitioningInfo, context); + branches[i] = new KStreamSource(streamMetadata, context); } } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized(this) { for (int i = 0; i < predicates.length; i++) { Predicate predicate = predicates[i]; if (predicate.apply((K)key, (V)value)) { - branches[i].receive(key, value, timestamp, streamTime); + branches[i].receive(topic, key, value, timestamp, streamTime); return; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 0609e71bffa36..a2ebe5e48c967 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -1,6 +1,15 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.*; +import io.confluent.streaming.Coordinator; +import io.confluent.streaming.KStream; +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamException; +import io.confluent.streaming.KStreamJob; +import io.confluent.streaming.RecordCollector; +import io.confluent.streaming.StorageEngine; +import io.confluent.streaming.StreamingConfig; +import io.confluent.streaming.SyncGroup; +import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.util.Util; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.producer.Producer; @@ -13,7 +22,11 @@ import java.io.File; import java.io.IOException; -import java.util.*; +import java.util.Collection; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Map; +import java.util.Set; /** * Created by yasuhiro on 6/19/15. @@ -31,8 +44,8 @@ public class KStreamContextImpl implements KStreamContext { private final RecordCollector collector; private final Coordinator coordinator; - private final HashMap> sourceStreams = new HashMap>(); - private final HashMap partitioningInfos = new HashMap(); + private final HashMap> sourceStreams = new HashMap<>(); + private final HashMap partitioningInfos = new HashMap<>(); private final TimestampExtractor timestampExtractor; private final HashMap streamSynchronizerMap = new HashMap<>(); private final StreamingConfig streamingConfig; @@ -119,42 +132,68 @@ public Deserializer valueDeserializer() { } @Override - public KStream from(String topic) { - return from(topic, syncGroup(DEFAULT_SYNCHRONIZATION_GROUP), null, null); + public KStream from(String... topics) { + return from(syncGroup(DEFAULT_SYNCHRONIZATION_GROUP), null, null, topics); } @Override - public KStream from(String topic, SyncGroup syncGroup) { - return from(topic, syncGroup, null, null); + public KStream from(SyncGroup syncGroup, String... topics) { + return from(syncGroup, null, null, topics); } @Override - public KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer) { - return from(topic, syncGroup(DEFAULT_SYNCHRONIZATION_GROUP), keyDeserializer, valDeserializer); + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + return from(syncGroup(DEFAULT_SYNCHRONIZATION_GROUP), keyDeserializer, valDeserializer, topics); } @Override @SuppressWarnings("unchecked") - public KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer) { + public KStream from(SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { if (syncGroup == null) throw new NullPointerException(); + KStreamSource stream = null; + synchronized (this) { - if (!topics.contains(topic)) - throw new IllegalArgumentException("topic not subscribed: " + topic); + // iterate over the topics and check if the stream has already been created for them + for (String topic : topics) { + if (!this.topics.contains(topic)) + throw new IllegalArgumentException("topic not subscribed: " + topic); + + KStreamSource streamForTopic = sourceStreams.get(topic); - KStreamSource stream = sourceStreams.get(topic); + if (stream == null) { + if (streamForTopic != null) + stream = streamForTopic; + } else { + if (streamForTopic != null) { + if (!stream.equals(streamForTopic)) + throw new IllegalArgumentException("another stream created with the same topic " + topic); + } else { + sourceStreams.put(topic, stream); + } + } + } + // if there is no stream for any of the topics, create one if (stream == null) { - PartitioningInfo partitioningInfo = partitioningInfos.get(topic); + // create stream metadata + Map topicPartitionInfos = new HashMap<>(); + for (String topic : topics) { + PartitioningInfo partitioningInfo = this.partitioningInfos.get(topic); + + if (partitioningInfo == null) { + partitioningInfo = new PartitioningInfo(ingestor.numPartitions(topic)); + this.partitioningInfos.put(topic, partitioningInfo); + } - if (partitioningInfo == null) { - partitioningInfo = new PartitioningInfo(syncGroup, ingestor.numPartitions(topic)); - partitioningInfos.put(topic, partitioningInfo); + topicPartitionInfos.put(topic, partitioningInfo); } + KStreamMetadata streamMetadata = new KStreamMetadata(syncGroup, topicPartitionInfos); + // override the deserializer classes if specified if (keyDeserializer == null && valDeserializer == null) { - stream = new KStreamSource(partitioningInfo, this); + stream = new KStreamSource(streamMetadata, this); } else { StreamingConfig newConfig = this.streamingConfig.clone(); if (keyDeserializer != null) @@ -167,18 +206,21 @@ public KStream from(String topic, SyncGroup syncGroup, Deserializer this.simpleCollector, this.coordinator, newConfig, this.processorConfig, this.stateMgr, this.metrics); - stream = new KStreamSource(partitioningInfo, newContext); + stream = new KStreamSource(streamMetadata, newContext); } - sourceStreams.put(topic, stream); + // update source stream map + for (String topic : topics) { + if (sourceStreams.containsKey(topic)) + sourceStreams.put(topic, stream); - TopicPartition partition = new TopicPartition(topic, id); - StreamSynchronizer streamSynchronizer = (StreamSynchronizer)syncGroup; - streamSynchronizer.addPartition(partition, stream); - ingestor.addStreamSynchronizerForPartition(streamSynchronizer, partition); - } - else { - if (stream.partitioningInfo.syncGroup == syncGroup) + TopicPartition partition = new TopicPartition(topic, id); + StreamSynchronizer streamSynchronizer = (StreamSynchronizer)syncGroup; + streamSynchronizer.addPartition(partition, stream); + ingestor.addStreamSynchronizerForPartition(streamSynchronizer, partition); + } + } else { + if (stream.metadata.syncGroup == syncGroup) throw new IllegalStateException("topic is already assigned a different synchronization group"); // TODO: with this constraint we will not allow users to create KStream with different diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 9ebac87ae9e3e..9112028eefa9a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -10,16 +10,16 @@ class KStreamFilter extends KStreamImpl { private final Predicate predicate; - KStreamFilter(Predicate predicate, PartitioningInfo partitioningInfo, KStreamContext context) { - super(partitioningInfo, context); + KStreamFilter(Predicate predicate, KStreamMetadata metadata, KStreamContext context) { + super(metadata, context); this.predicate = predicate; } @Override - public void receive(Object key, Object value, long timestamp,long streamTime) { + public void receive(String topic, Object key, Object value, long timestamp,long streamTime) { synchronized(this) { if (predicate.apply((K)key, (V)value)) { - forward(key, value, timestamp, streamTime); + forward(topic, key, value, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 97b06cf041d25..e99f84319692d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -13,16 +13,16 @@ class KStreamFlatMap extends KStreamImpl { private final KeyValueMapper, K1, V1> mapper; KStreamFlatMap(KeyValueMapper, K1, V1> mapper, SyncGroup syncGroup, KStreamContext context) { - super(PartitioningInfo.unjoinable(syncGroup), context); + super(KStreamMetadata.unjoinable(syncGroup), context); this.mapper = mapper; } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized(this) { KeyValue> newPair = mapper.apply((K1)key, (V1)value); for (V v : newPair.value) { - forward(newPair.key, v, timestamp, streamTime); + forward(topic, newPair.key, v, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 87a7a9e6dda04..19e7497eeacbf 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -10,17 +10,17 @@ class KStreamFlatMapValues extends KStreamImpl { private final ValueMapper, V1> mapper; - KStreamFlatMapValues(ValueMapper, V1> mapper, PartitioningInfo partitioningInfo, KStreamContext context) { - super(partitioningInfo, context); + KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamMetadata streamMetadata, KStreamContext context) { + super(streamMetadata, context); this.mapper = mapper; } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized(this) { Iterable newValues = mapper.apply((V1)value); for (V v : newValues) { - forward(key, v, timestamp, streamTime); + forward(topic, key, v, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 2b5e76756fece..00e5e9e2b5d5d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -23,11 +23,11 @@ abstract class KStreamImpl implements KStream, Receiver { private final ArrayList nextReceivers = new ArrayList<>(1); - final PartitioningInfo partitioningInfo; + final KStreamMetadata metadata; final KStreamContext context; - protected KStreamImpl(PartitioningInfo partitioningInfo, KStreamContext context) { - this.partitioningInfo = partitioningInfo; + protected KStreamImpl(KStreamMetadata metadata, KStreamContext context) { + this.metadata = metadata; this.context = context; } @@ -38,7 +38,7 @@ public KStreamContext context() { @Override public KStream filter(Predicate predicate) { - return chain(new KStreamFilter(predicate, partitioningInfo, context)); + return chain(new KStreamFilter(predicate, metadata, context)); } @Override @@ -53,32 +53,36 @@ public boolean apply(K key, V value) { @Override public KStream map(KeyValueMapper mapper) { - return chain(new KStreamMap(mapper, partitioningInfo.syncGroup, context)); + return chain(new KStreamMap(mapper, metadata.syncGroup, context)); } @Override public KStream mapValues(ValueMapper mapper) { - return chain(new KStreamMapValues(mapper, partitioningInfo, context)); + return chain(new KStreamMapValues(mapper, metadata, context)); } @Override public KStream flatMap(KeyValueMapper, K, V> mapper) { - return chain(new KStreamFlatMap(mapper, partitioningInfo.syncGroup, context)); + return chain(new KStreamFlatMap(mapper, metadata.syncGroup, context)); } @Override public KStream flatMapValues(ValueMapper, V> mapper) { - return chain(new KStreamFlatMapValues(mapper, partitioningInfo, context)); + return chain(new KStreamFlatMapValues(mapper, metadata, context)); } @Override public KStreamWindowed with(Window window) { - return (KStreamWindowed)chain(new KStreamWindowedImpl(window, partitioningInfo, context)); + // TODO: we can extend to allow construction of windowable stream with a multiple topics + if (metadata.topicPartitionInfos.size() != 1) + throw new IllegalStateException("Do not support windowable stream construction with multipel topics: " + metadata.topicPartitionInfos.keySet()); + + return (KStreamWindowed)chain(new KStreamWindowedImpl(window, metadata, context)); } @Override public KStream[] branch(Predicate... predicates) { - KStreamBranch branch = new KStreamBranch(predicates, partitioningInfo, context); + KStreamBranch branch = new KStreamBranch(predicates, metadata, context); registerReceiver(branch); return branch.branches; } @@ -105,7 +109,7 @@ public KStream through(String topic, Serializer keySerialize @Override public KStream through(String topic, SyncGroup syncGroup, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); - return context.from(topic, syncGroup, keyDeserializer, valDeserializer); + return context.from(syncGroup, keyDeserializer, valDeserializer, topic); } @Override @@ -117,7 +121,7 @@ public void sendTo(String topic) { public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); } @SuppressWarnings("unchecked") - private Processor getSendProcessor(final String topic, Serializer keySerializer, Serializer valSerializer) { + private Processor getSendProcessor(final String sendTopic, Serializer keySerializer, Serializer valSerializer) { final RecordCollector collector; if (keySerializer == null && valSerializer == null) collector = (RecordCollector) context.recordCollector(); @@ -129,8 +133,8 @@ private Processor getSendProcessor(final String topic, Serializer() { @Override - public void apply(K key, V value) { - collector.send(new ProducerRecord(topic, key, value)); + public void apply(String topic, K key, V value) { + collector.send(new ProducerRecord(sendTopic, key, value)); } @Override public void init(PunctuationScheduler scheduler) {} @@ -142,13 +146,13 @@ public void punctuate(long streamTime) {} @Override public void process(final Processor processor) { Receiver receiver = new Receiver() { - public void receive(Object key, Object value, long timestamp, long streamTime) { - processor.apply((K)key, (V)value); + public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { + processor.apply(topic, (K)key, (V)value); } }; registerReceiver(receiver); - PunctuationScheduler scheduler = ((StreamSynchronizer)partitioningInfo.syncGroup).getPunctuationScheduler(processor); + PunctuationScheduler scheduler = ((StreamSynchronizer)metadata.syncGroup).getPunctuationScheduler(processor); processor.init(scheduler); } @@ -156,10 +160,10 @@ void registerReceiver(Receiver receiver) { nextReceivers.add(receiver); } - protected void forward(Object key, Object value, long timestamp, long streamTime) { + protected void forward(String topic, Object key, Object value, long timestamp, long streamTime) { int numReceivers = nextReceivers.size(); for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).receive(key, value, timestamp, streamTime); + nextReceivers.get(i).receive(topic, key, value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index 8119af9ff0d13..5133524402ced 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -20,8 +20,8 @@ private static abstract class Finder { private final ValueJoiner joiner; final Receiver receiverForOtherStream; - KStreamJoin(final Window window1, final Window window2, boolean prior, ValueJoiner joiner, PartitioningInfo partitioningInfo, KStreamContext context) { - super(partitioningInfo, context); + KStreamJoin(final Window window1, final Window window2, boolean prior, ValueJoiner joiner, KStreamMetadata streamMetadata, KStreamContext context) { + super(streamMetadata, context); if (prior) { this.finder1 = new Finder() { @@ -54,7 +54,7 @@ Iterator find(K key, long timestamp) { } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { Iterator iter = finder2.find((K)key, timestamp); if (iter != null) { while (iter.hasNext()) { @@ -67,7 +67,7 @@ private Receiver getReceiverForOther() { return new Receiver() { @Override - public void receive(Object key, Object value2, long timestamp, long streamTime) { + public void receive(String topic, Object key, Object value2, long timestamp, long streamTime) { Iterator iter = finder1.find((K)key, timestamp); if (iter != null) { while (iter.hasNext()) { @@ -78,8 +78,9 @@ public void receive(Object key, Object value2, long timestamp, long streamTime) }; } + // TODO: use the "outer-stream" topic as the resulted join stream topic private void doJoin(K key, V1 value1, V2 value2, long timestamp, long streamTime) { - forward(key, joiner.apply(value1, value2), timestamp, streamTime); + forward(this.metadata.topicPartitionInfos.keySet().iterator().next(), key, joiner.apply(value1, value2), timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index e93a71cd1f806..fac7f9da5429a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -13,15 +13,15 @@ class KStreamMap extends KStreamImpl { private final KeyValueMapper mapper; KStreamMap(KeyValueMapper mapper, SyncGroup syncGroup, KStreamContext context) { - super(PartitioningInfo.unjoinable(syncGroup), context); + super(KStreamMetadata.unjoinable(syncGroup), context); this.mapper = mapper; } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized (this) { KeyValue newPair = mapper.apply((K1)key, (V1)value); - forward(newPair.key, newPair.value, timestamp, streamTime); + forward(topic, newPair.key, newPair.value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index 925271da647ea..cc3db9de4a8e8 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -10,16 +10,16 @@ class KStreamMapValues extends KStreamImpl { private final ValueMapper mapper; - KStreamMapValues(ValueMapper mapper, PartitioningInfo partitioningInfo, KStreamContext context) { - super(partitioningInfo, context); + KStreamMapValues(ValueMapper mapper, KStreamMetadata metadata, KStreamContext context) { + super(metadata, context); this.mapper = mapper; } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized (this) { V newValue = mapper.apply((V1)value); - forward(key, newValue, timestamp, streamTime); + forward(topic, key, newValue, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java b/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java new file mode 100644 index 0000000000000..71948a6f97491 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java @@ -0,0 +1,36 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.SyncGroup; + +import java.util.Collections; +import java.util.Map; + +/** + * Created by guozhang on 7/13/15. + */ +public class KStreamMetadata { + + public static KStreamMetadata unjoinable(SyncGroup syncGroup) { + // TODO: how to define the topic name for flat functions? + return new KStreamMetadata(syncGroup, Collections.singletonMap("FlatTopic", new PartitioningInfo(-1))); + } + + public final SyncGroup syncGroup; + public final Map topicPartitionInfos; + + KStreamMetadata(SyncGroup syncGroup, Map topicPartitionInfos) { + this.syncGroup = syncGroup; + this.topicPartitionInfos = topicPartitionInfos; + } + + boolean isJoinCompatibleWith(KStreamMetadata other) { + // the two streams should only be joinable if they only contain one topic-partition each + if (this.topicPartitionInfos.size() != 1 || other.topicPartitionInfos.size() != 1) + return false; + else { + return syncGroup == other.syncGroup + && this.topicPartitionInfos.values().iterator().next().numPartitions >= 0 + && this.topicPartitionInfos.values().iterator().next().numPartitions == other.topicPartitionInfos.values().iterator().next().numPartitions; + } + } +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 7b8765665c17a..4723a77d5019c 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -2,19 +2,21 @@ import io.confluent.streaming.KStreamContext; +import java.util.List; + /** * Created by yasuhiro on 6/17/15. */ class KStreamSource extends KStreamImpl { - KStreamSource(PartitioningInfo partitioningInfo, KStreamContext context) { - super(partitioningInfo, context); + KStreamSource(KStreamMetadata streamMetadata, KStreamContext context) { + super(streamMetadata, context); } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized(this) { - forward(key, value, timestamp, streamTime); + forward(topic, key, value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 33343152123c2..8e4bccba6d814 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -15,16 +15,16 @@ public class KStreamWindowedImpl extends KStreamImpl implements KStr final Window window; - KStreamWindowedImpl(Window window, PartitioningInfo partitioningInfo, KStreamContext context) { - super(partitioningInfo, context); + KStreamWindowedImpl(Window window, KStreamMetadata streamMetadata, KStreamContext context) { + super(streamMetadata, context); this.window = window; } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized(this) { window.put((K)key, (V)value, timestamp); - forward(key, value, timestamp, streamTime); + forward(topic, key, value, timestamp, streamTime); } } @@ -42,10 +42,10 @@ private KStream join(KStreamWindowed other, boolean prior KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; - if (!partitioningInfo.isJoinCompatibleWith(otherImpl.partitioningInfo)) throw new NotCopartitionedException(); + if (!this.metadata.isJoinCompatibleWith(otherImpl.metadata)) throw new NotCopartitionedException(); KStreamJoin stream = - new KStreamJoin(this.window, otherImpl.window, prior, processor, partitioningInfo, context); + new KStreamJoin(this.window, otherImpl.window, prior, processor, this.metadata, context); otherImpl.registerReceiver(stream.receiverForOtherStream); return chain(stream); diff --git a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java b/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java index 30de31ffa819c..5088cf1109b7a 100644 --- a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java +++ b/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java @@ -1,26 +1,13 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.SyncGroup; - /** * Created by yasuhiro on 6/19/15. */ class PartitioningInfo { - public static PartitioningInfo unjoinable(SyncGroup syncGroup) { - return new PartitioningInfo(syncGroup, -1); - } - - public final SyncGroup syncGroup; public final int numPartitions; - PartitioningInfo(SyncGroup syncGroup, int numPartitions) { - this.syncGroup = syncGroup; + PartitioningInfo(int numPartitions) { this.numPartitions = numPartitions; } - - boolean isJoinCompatibleWith(PartitioningInfo other) { - return syncGroup == other.syncGroup && numPartitions >= 0 && numPartitions == other.numPartitions; - } - } diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 1cad1f810acdf..2c2332068a51d 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -5,6 +5,6 @@ */ public interface Receiver { - void receive(Object key, Object value, long timestamp, long streamTime); + void receive(String topic, Object key, Object value, long timestamp, long streamTime); } diff --git a/src/main/java/io/confluent/streaming/internal/StampedRecord.java b/src/main/java/io/confluent/streaming/internal/StampedRecord.java index 4cdf5d9d915a0..0b7e668466016 100644 --- a/src/main/java/io/confluent/streaming/internal/StampedRecord.java +++ b/src/main/java/io/confluent/streaming/internal/StampedRecord.java @@ -9,6 +9,8 @@ public class StampedRecord extends Stamped> { super(record, timestamp); } + public String topic() { return value.topic(); } + public Object key() { return value.key(); } diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index 7e2dbeb602345..d19a11690417d 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -217,7 +217,7 @@ public void process(Object context) { if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - recordQueue.stream.receive(record.key(), record.value(), record.timestamp, streamTime); + recordQueue.stream.receive(record.topic(), record.key(), record.value(), record.timestamp, streamTime); consumedOffsets.put(recordQueue.partition(), record.offset()); if (recordQueue.size() > 0) chooser.add(recordQueue); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index 85c2b513f320a..1f7f95026ffb5 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -6,6 +6,7 @@ import org.junit.Test; import java.lang.reflect.Array; +import java.util.Collections; import static org.junit.Assert.assertEquals; @@ -25,7 +26,9 @@ public long extract(String topic, Object key, Object value) { 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); @SuppressWarnings("unchecked") @Test @@ -56,7 +59,7 @@ public boolean apply(Integer key, String value) { KStream[] branches; TestProcessor[] processors; - stream = new KStreamSource(partitioningInfo, null); + stream = new KStreamSource(streamMetadata, null); branches = stream.branch(isEven, isMultipleOfThree, isOdd); assertEquals(3, branches.length); @@ -68,14 +71,14 @@ public boolean apply(Integer key, String value) { } for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(3, processors[0].processed.size()); assertEquals(1, processors[1].processed.size()); assertEquals(3, processors[2].processed.size()); - stream = new KStreamSource(partitioningInfo, null); + stream = new KStreamSource(streamMetadata, null); branches = stream.branch(isEven, isOdd, isMultipleOfThree); assertEquals(3, branches.length); @@ -87,7 +90,7 @@ public boolean apply(Integer key, String value) { } for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(3, processors[0].processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 0edb0cac689d3..0c26f2517afde 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -5,6 +5,8 @@ import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; +import java.util.Collections; + import static org.junit.Assert.assertEquals; public class KStreamFilterTest { @@ -23,7 +25,9 @@ public long extract(String topic, Object key, Object value) { 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); private Predicate isMultipleOfThree = new Predicate() { @Override @@ -40,11 +44,11 @@ public void testFilter() { TestProcessor processor; processor = new TestProcessor(); - stream = new KStreamSource(partitioningInfo, null); + stream = new KStreamSource(streamMetadata, null); stream.filter(isMultipleOfThree).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(2, processor.processed.size()); @@ -58,11 +62,11 @@ public void testFilterOut() { TestProcessor processor; processor = new TestProcessor(); - stream = new KStreamSource(partitioningInfo, null); + stream = new KStreamSource(streamMetadata, null); stream.filterOut(isMultipleOfThree).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(5, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 6a138acf72637..579d891217bd4 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -6,6 +6,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Collections; import static org.junit.Assert.assertEquals; @@ -25,7 +26,9 @@ public long extract(String topic, Object key, Object value) { 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testFlatMap() { @@ -48,11 +51,11 @@ public KeyValue> apply(Integer key, String value) { TestProcessor processor; processor = new TestProcessor(); - stream = new KStreamSource(partitioningInfo, null); + stream = new KStreamSource(streamMetadata, null); stream.flatMap(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(6, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 1562c367ad684..8e274e8181d7f 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -6,6 +6,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Collections; import static org.junit.Assert.assertEquals; @@ -25,7 +26,10 @@ public long extract(String topic, Object key, Object value) { 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); + @Test public void testFlatMapValues() { @@ -47,11 +51,11 @@ public Iterable apply(String value) { TestProcessor processor; processor = new TestProcessor(); - stream = new KStreamSource(partitioningInfo, null); + stream = new KStreamSource(streamMetadata, null); stream.flatMapValues(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(8, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index be1d5b784988e..92e59dbdd4603 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -7,6 +7,8 @@ import io.confluent.streaming.util.Util; import org.junit.Test; +import java.util.Collections; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -27,7 +29,9 @@ public long extract(String topic, Object key, Object value) { 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -79,8 +83,8 @@ public void testJoin() { String[] expected; processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); + stream1 = new KStreamSource(streamMetadata, null); + stream2 = new KStreamSource(streamMetadata, null); windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -97,7 +101,7 @@ public void testJoin() { // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); + stream1.receive(topicName, expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); } assertEquals(0, processor.processed.size()); @@ -105,7 +109,7 @@ public void testJoin() { // push two items to the other stream. the main stream's window has two items for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); + stream2.receive(topicName, expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); } assertEquals(2, processor.processed.size()); @@ -121,7 +125,7 @@ public void testJoin() { // push all items to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); + stream1.receive(topicName, expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); } assertEquals(2, processor.processed.size()); @@ -138,7 +142,7 @@ public void testJoin() { // push all items to the other stream. this should produce 6 items for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); + stream2.receive(topicName, expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); } assertEquals(6, processor.processed.size()); @@ -163,8 +167,8 @@ public void testJoinPrior() { String[] expected; processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); + stream1 = new KStreamSource(streamMetadata, null); + stream2 = new KStreamSource(streamMetadata, null); windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -181,7 +185,7 @@ public void testJoinPrior() { // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i, 0L); + stream1.receive(topicName, expectedKeys[i], "X" + expectedKeys[i], i, 0L); } assertEquals(0, processor.processed.size()); @@ -190,7 +194,7 @@ public void testJoinPrior() { // no corresponding item in the main window has a newer timestamp for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i + 1, 0L); + stream2.receive(topicName, expectedKeys[i], "Y" + expectedKeys[i], i + 1, 0L); } assertEquals(0, processor.processed.size()); @@ -200,7 +204,7 @@ public void testJoinPrior() { // push all items with newer timestamps to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i + 2, 0L); + stream1.receive(topicName, expectedKeys[i], "X" + expectedKeys[i], i + 2, 0L); } assertEquals(2, processor.processed.size()); @@ -217,7 +221,7 @@ public void testJoinPrior() { // push all items with older timestamps to the other stream. this should produce six items for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i, 0L); + stream2.receive(topicName, expectedKeys[i], "Y" + expectedKeys[i], i, 0L); } assertEquals(6, processor.processed.size()); @@ -241,8 +245,8 @@ public void testMap() { TestProcessor processor; processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); + stream1 = new KStreamSource(streamMetadata, null); + stream2 = new KStreamSource(streamMetadata, null); mapped1 = stream1.map(keyValueMapper); mapped2 = stream2.map(keyValueMapper); @@ -296,8 +300,8 @@ public void testFlatMap() { TestProcessor processor; processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); + stream1 = new KStreamSource(streamMetadata, null); + stream2 = new KStreamSource(streamMetadata, null); mapped1 = stream1.flatMap(keyValueMapper2); mapped2 = stream2.flatMap(keyValueMapper2); @@ -351,8 +355,8 @@ public void testMapValues() { TestProcessor processor; processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); + stream1 = new KStreamSource(streamMetadata, null); + stream2 = new KStreamSource(streamMetadata, null); mapped1 = stream1.mapValues(valueMapper); mapped2 = stream2.mapValues(valueMapper); @@ -406,8 +410,8 @@ public void testFlatMapValues() { TestProcessor processor; processor = new TestProcessor(); - stream1 = new KStreamSource(partitioningInfo, null); - stream2 = new KStreamSource(partitioningInfo, null); + stream1 = new KStreamSource(streamMetadata, null); + stream2 = new KStreamSource(streamMetadata, null); mapped1 = stream1.flatMapValues(valueMapper2); mapped2 = stream2.flatMapValues(valueMapper2); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index d3f84e33788fd..c7d258534520d 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -5,6 +5,8 @@ import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; +import java.util.Collections; + import static org.junit.Assert.assertEquals; public class KStreamMapTest { @@ -23,7 +25,9 @@ public long extract(String topic, Object key, Object value) { 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testMap() { @@ -42,11 +46,11 @@ public KeyValue apply(Integer key, String value) { TestProcessor processor; processor = new TestProcessor(); - stream = new KStreamSource(partitioningInfo, null); + stream = new KStreamSource(streamMetadata, null); stream.map(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(4, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 771726116381b..81a68b547e4fc 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -7,6 +7,8 @@ import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; +import java.util.Collections; + import static org.junit.Assert.assertEquals; public class KStreamMapValuesTest { @@ -25,7 +27,9 @@ public long extract(String topic, Object key, Object value) { 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testFlatMapValues() { @@ -44,11 +48,11 @@ public Integer apply(String value) { TestProcessor processor; processor = new TestProcessor(); - stream = new KStreamSource(partitioningInfo, null); + stream = new KStreamSource(streamMetadata, null); stream.mapValues(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L, 0L); + stream.receive(topicName, expectedKeys[i], Integer.toString(expectedKeys[i]), 0L, 0L); } assertEquals(4, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 4b81a663d5a0c..46b0cf5209e41 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -8,6 +8,8 @@ import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; +import java.util.Collections; + public class KStreamSourceTest { private Ingestor ingestor = new MockIngestor(); @@ -24,21 +26,23 @@ public long extract(String topic, Object key, Object value) { 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testKStreamSource() { TestProcessor processor = new TestProcessor(); - KStreamSource stream = new KStreamSource(partitioningInfo, null); + KStreamSource stream = new KStreamSource(streamMetadata, null); stream.process(processor); final String[] expectedKeys = new String[] { "k1", "k2", "k3" }; final String[] expectedValues = new String[] { "v1", "v2", "v3" }; for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], expectedValues[i], 0L, 0L); + stream.receive(topicName, expectedKeys[i], expectedValues[i], 0L, 0L); } assertEquals(3, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index a47ea68d55f87..ad5f3a591a7be 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -5,6 +5,7 @@ import io.confluent.streaming.testutil.UnlimitedWindow; import org.junit.Test; +import java.util.Collections; import java.util.Iterator; import static org.junit.Assert.assertEquals; @@ -25,7 +26,9 @@ public long extract(String topic, Object key, Object value) { 10 ); - private PartitioningInfo partitioningInfo = new PartitioningInfo(streamSynchronizer, 1); + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testWindowedStream() { @@ -37,7 +40,7 @@ public void testWindowedStream() { String[] expected; window = new UnlimitedWindow(); - stream = new KStreamSource(partitioningInfo, null); + stream = new KStreamSource(streamMetadata, null); stream.with(window); boolean exceptionRaised = false; @@ -45,7 +48,7 @@ public void testWindowedStream() { // two items in the window for (int i = 0; i < 2; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(1, countItem(window.find(0, 0L))); @@ -56,7 +59,7 @@ public void testWindowedStream() { // previous two items + all items, thus two are duplicates, in the window for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); + stream.receive(topicName, expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); } assertEquals(2, countItem(window.find(0, 0L))); diff --git a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java b/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java index 25ad36f2544a2..bc3b16fd5fd1e 100644 --- a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java @@ -43,16 +43,16 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer public Deserializer valueDeserializer() { return deserializer; } @Override - public KStream from(String topic) { throw new UnsupportedOperationException("from() not supported."); } + public KStream from(String... topic) { throw new UnsupportedOperationException("from() not supported."); } @Override - public KStream from(String topic, SyncGroup syncGroup) { throw new UnsupportedOperationException("from() not supported."); } + public KStream from(SyncGroup syncGroup, String... topic) { throw new UnsupportedOperationException("from() not supported."); } @Override - public KStream from(String topic, Deserializer keyDeserializer, Deserializer valDeserializer) { throw new UnsupportedOperationException("from() not supported."); } + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topic) { throw new UnsupportedOperationException("from() not supported."); } @Override - public KStream from(String topic, SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer) { throw new UnsupportedOperationException("from() not supported."); } + public KStream from(SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topic) { throw new UnsupportedOperationException("from() not supported."); } @Override public RecordCollector recordCollector() { throw new UnsupportedOperationException("recordCollector() not supported."); } diff --git a/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java b/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java index b279ac22b5c48..dfaa6a5a035b8 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java @@ -35,7 +35,7 @@ public MockKStreamImpl() { } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { this.numReceived++; this.keys.add(key); this.values.add(value); diff --git a/src/test/java/io/confluent/streaming/testutil/TestProcessor.java b/src/test/java/io/confluent/streaming/testutil/TestProcessor.java index aa8fcbbcaf242..83cebfad26aaf 100644 --- a/src/test/java/io/confluent/streaming/testutil/TestProcessor.java +++ b/src/test/java/io/confluent/streaming/testutil/TestProcessor.java @@ -10,7 +10,7 @@ public class TestProcessor implements Processor { public final ArrayList punctuated = new ArrayList(); @Override - public void apply(K key, V value) { + public void apply(String topic, K key, V value) { processed.add(key + ":" + value); } From f30ea1c183d96fb924f443c6bc855cccc5b1a441 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 13 Jul 2015 16:16:54 -0700 Subject: [PATCH 058/275] remove used imports --- src/main/java/io/confluent/streaming/KStream.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index 88430380ebc91..ef4a9a41a5498 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -1,6 +1,5 @@ package io.confluent.streaming; -import com.sun.javafx.geom.Vec2d; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; From 3ae1adc74cbe53bb286b27fabdb631f3a98a63ea Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 13 Jul 2015 16:30:57 -0700 Subject: [PATCH 059/275] simplify onPartitionsRevoked action --- .../confluent/streaming/KafkaStreaming.java | 32 +++++++------------ .../streaming/internal/Ingestor.java | 2 -- .../streaming/internal/IngestorImpl.java | 9 ------ 3 files changed, 11 insertions(+), 32 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 711ee1636f900..21a151c06db99 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -416,29 +416,19 @@ public void shutdown(Coordinator.RequestScope scope) { private void removePartitions(Collection assignment) { commitAll(time.milliseconds()); - // remove all partitions - for (TopicPartition partition : assignment) { - Collection streamSynchronizers = this.streamSynchronizersForPartition.remove(partition.partition()); - if (streamSynchronizers != null) { - log.info("Removing synchronization groups {}", partition.partition()); - for (StreamSynchronizer streamSynchronizer : streamSynchronizers) - streamSynchronizer.close(); - } + for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { + log.info("Removing synchronization groups {}", streamSynchronizer.name()); + streamSynchronizer.close(); } - for (TopicPartition partition : assignment) { - KStreamContextImpl kstreamContext = kstreamContexts.remove(partition.partition()); - ingestor.removeStreamSynchronizerForPartition(partition); - - if (kstreamContext != null) { - log.info("Removing stream context {}", partition.partition()); - try { - kstreamContext.close(); - } - catch (Exception e) { - throw new KafkaException(e); - } - streamingMetrics.processorDestruction.record(); + for (KStreamContextImpl kstreamContext : kstreamContexts.values()) { + log.info("Removing stream context {}", kstreamContext.id()); + try { + kstreamContext.close(); + } + catch (Exception e) { + throw new KafkaException(e); } + streamingMetrics.processorDestruction.record(); } streamSynchronizers.clear(); ingestor.clear(); diff --git a/src/main/java/io/confluent/streaming/internal/Ingestor.java b/src/main/java/io/confluent/streaming/internal/Ingestor.java index 75cc901867124..14e7cc2e35732 100644 --- a/src/main/java/io/confluent/streaming/internal/Ingestor.java +++ b/src/main/java/io/confluent/streaming/internal/Ingestor.java @@ -19,6 +19,4 @@ public interface Ingestor { void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition); - void removeStreamSynchronizerForPartition(TopicPartition partition); - } diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 4203369235389..127a402c68cc9 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -98,15 +98,6 @@ public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchroni } } - @Override - public void removeStreamSynchronizerForPartition(TopicPartition partition) { - synchronized (this) { - streamSynchronizers.remove(partition); - unpaused.remove(partition); - toBePaused.remove(partition); - } - } - public void clear() { unpaused.clear(); toBePaused.clear(); From ac18300ab24361ab60a433e46b607b563085ada2 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 13 Jul 2015 18:07:31 -0700 Subject: [PATCH 060/275] only preserves topic name for KStreamSource and KStreamWindowed --- .../streaming/internal/KStreamFlatMap.java | 2 +- .../internal/KStreamFlatMapValues.java | 2 +- .../streaming/internal/KStreamImpl.java | 9 +++--- .../streaming/internal/KStreamJoin.java | 2 +- .../streaming/internal/KStreamMap.java | 2 +- .../streaming/internal/KStreamMapValues.java | 2 +- .../streaming/internal/KStreamMetadata.java | 32 ++++++++++++++----- .../streaming/internal/KStreamSource.java | 1 + .../internal/KStreamWindowedImpl.java | 1 + 9 files changed, 35 insertions(+), 18 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index e99f84319692d..7da3244d1d749 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -22,7 +22,7 @@ public void receive(String topic, Object key, Object value, long timestamp, long synchronized(this) { KeyValue> newPair = mapper.apply((K1)key, (V1)value); for (V v : newPair.value) { - forward(topic, newPair.key, v, timestamp, streamTime); + forward(KStreamMetadata.UNKNOWN_TOPICNAME, newPair.key, v, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 19e7497eeacbf..3c013b9ef2eed 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -20,7 +20,7 @@ public void receive(String topic, Object key, Object value, long timestamp, long synchronized(this) { Iterable newValues = mapper.apply((V1)value); for (V v : newValues) { - forward(topic, key, v, timestamp, streamTime); + forward(KStreamMetadata.UNKNOWN_TOPICNAME, key, v, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 00e5e9e2b5d5d..f2c018ff0095b 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -73,10 +73,6 @@ public KStream flatMapValues(ValueMapper, V> @Override public KStreamWindowed with(Window window) { - // TODO: we can extend to allow construction of windowable stream with a multiple topics - if (metadata.topicPartitionInfos.size() != 1) - throw new IllegalStateException("Do not support windowable stream construction with multipel topics: " + metadata.topicPartitionInfos.keySet()); - return (KStreamWindowed)chain(new KStreamWindowedImpl(window, metadata, context)); } @@ -147,7 +143,10 @@ public void punctuate(long streamTime) {} public void process(final Processor processor) { Receiver receiver = new Receiver() { public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { - processor.apply(topic, (K)key, (V)value); + if (topic.equals(KStreamMetadata.UNKNOWN_TOPICNAME)) + processor.apply(null, (K)key, (V)value); + else + processor.apply(topic, (K)key, (V)value); } }; registerReceiver(receiver); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index 5133524402ced..5493f8732dc89 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -80,7 +80,7 @@ public void receive(String topic, Object key, Object value2, long timestamp, lon // TODO: use the "outer-stream" topic as the resulted join stream topic private void doJoin(K key, V1 value1, V2 value2, long timestamp, long streamTime) { - forward(this.metadata.topicPartitionInfos.keySet().iterator().next(), key, joiner.apply(value1, value2), timestamp, streamTime); + forward(KStreamMetadata.UNKNOWN_TOPICNAME, key, joiner.apply(value1, value2), timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index fac7f9da5429a..2eefeccf97069 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -21,7 +21,7 @@ class KStreamMap extends KStreamImpl { public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized (this) { KeyValue newPair = mapper.apply((K1)key, (V1)value); - forward(topic, newPair.key, newPair.value, timestamp, streamTime); + forward(KStreamMetadata.UNKNOWN_TOPICNAME, newPair.key, newPair.value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index cc3db9de4a8e8..b5708b3ca096a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -19,7 +19,7 @@ class KStreamMapValues extends KStreamImpl { public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized (this) { V newValue = mapper.apply((V1)value); - forward(topic, key, newValue, timestamp, streamTime); + forward(KStreamMetadata.UNKNOWN_TOPICNAME, key, newValue, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java b/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java index 71948a6f97491..a22238c0d4b2c 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java @@ -10,9 +10,10 @@ */ public class KStreamMetadata { + public static String UNKNOWN_TOPICNAME = "__UNKNOWN_TOPIC__"; + public static KStreamMetadata unjoinable(SyncGroup syncGroup) { - // TODO: how to define the topic name for flat functions? - return new KStreamMetadata(syncGroup, Collections.singletonMap("FlatTopic", new PartitioningInfo(-1))); + return new KStreamMetadata(syncGroup, Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(-1))); } public final SyncGroup syncGroup; @@ -24,13 +25,28 @@ public static KStreamMetadata unjoinable(SyncGroup syncGroup) { } boolean isJoinCompatibleWith(KStreamMetadata other) { - // the two streams should only be joinable if they only contain one topic-partition each - if (this.topicPartitionInfos.size() != 1 || other.topicPartitionInfos.size() != 1) + // the two streams should only be joinable if they are inside the same sync group + // and their contained streams all have the same number of partitions + if (this.syncGroup != other.syncGroup) return false; - else { - return syncGroup == other.syncGroup - && this.topicPartitionInfos.values().iterator().next().numPartitions >= 0 - && this.topicPartitionInfos.values().iterator().next().numPartitions == other.topicPartitionInfos.values().iterator().next().numPartitions; + + int numPartitions = -1; + for (PartitioningInfo partitionInfo : this.topicPartitionInfos.values()) { + if (partitionInfo.numPartitions < 0) { + return false; + } else if (numPartitions >= 0) { + if (partitionInfo.numPartitions != numPartitions) + return false; + } else { + numPartitions = partitionInfo.numPartitions; + } } + + for (PartitioningInfo partitionInfo : other.topicPartitionInfos.values()) { + if (partitionInfo.numPartitions != numPartitions) + return false; + } + + return true; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 4723a77d5019c..958e2f1e76cb8 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -16,6 +16,7 @@ class KStreamSource extends KStreamImpl { @Override public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized(this) { + // KStream needs to forward the topic name since it is directly from the Kafka source forward(topic, key, value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 8e4bccba6d814..8fc87bf3f137b 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -24,6 +24,7 @@ public class KStreamWindowedImpl extends KStreamImpl implements KStr public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { synchronized(this) { window.put((K)key, (V)value, timestamp); + // KStreamWindowed needs to forward the topic name since it may receive directly from KStreamSource forward(topic, key, value, timestamp, streamTime); } } From 4dfa08c7d73e40a52aed3d57e9001f7614dd6dc8 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 14 Jul 2015 09:44:55 -0700 Subject: [PATCH 061/275] clean up ingestor and stream synchronizer --- .../confluent/streaming/KafkaStreaming.java | 17 +++---- .../streaming/internal/IngestorImpl.java | 29 ++++------- .../internal/StreamSynchronizer.java | 21 +------- .../streaming/util/ParallelExecutor.java | 11 ++--- .../internal/StreamSynchronizerTest.java | 25 +++++----- .../streaming/testutil/MockIngestor.java | 6 +-- .../streaming/util/ParallelExecutorTest.java | 48 ++++++++----------- 7 files changed, 56 insertions(+), 101 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 21a151c06db99..43c8f5f8d4048 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -41,7 +41,6 @@ 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.serialization.Deserializer; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -141,6 +140,7 @@ protected KafkaStreaming(Class jobClass, this.streamingMetrics = new KafkaStreamingMetrics(); this.requestingCommit = new ArrayList<>(); this.config = new ProcessorConfig(config.config()); +<<<<<<< HEAD this.ingestor = <<<<<<< HEAD new IngestorImpl<>(this.consumer, @@ -153,6 +153,9 @@ protected KafkaStreaming(Class jobClass, (Deserializer) config.valueDeserializer(), this.config.pollTimeMs); >>>>>>> removed some generics +======= + this.ingestor = new IngestorImpl(this.consumer, this.config.pollTimeMs); +>>>>>>> clean up ingestor and stream synchronizer this.running = true; this.lastCommit = 0; this.nextStateCleaning = Long.MAX_VALUE; @@ -238,17 +241,15 @@ public synchronized void close() { private void runLoop() { try { - StreamSynchronizer.Status status = new StreamSynchronizer.Status(); - status.pollRequired(true); - while (stillRunning()) { - if (status.pollRequired()) { - ingestor.poll(); - status.pollRequired(false); - } + ingestor.poll(); +<<<<<<< HEAD <<<<<<< HEAD parallelExecutor.execute(streamSynchronizers, status); +======= + parallelExecutor.execute(streamSynchronizers); +>>>>>>> clean up ingestor and stream synchronizer ======= for (Map.Entry> entry : streamSynchronizersForPartition.entrySet()) { diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 127a402c68cc9..9ae4b32317ec0 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -1,11 +1,8 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.util.FilteredIterator; import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.Deserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -18,18 +15,12 @@ public class IngestorImpl implements Ingestor { private final Consumer consumer; private final Set unpaused = new HashSet<>(); private final Set toBePaused = new HashSet<>(); - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; private final long pollTimeMs; private final Map streamSynchronizers = new HashMap<>(); public IngestorImpl(Consumer consumer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, long pollTimeMs) { this.consumer = consumer; - this.keyDeserializer = keyDeserializer; - this.valueDeserializer = valueDeserializer; this.pollTimeMs = pollTimeMs; } @@ -45,24 +36,24 @@ public void poll() { @Override public void poll(long timeoutMs) { - ConsumerRecords records; - synchronized (this) { for (TopicPartition partition : toBePaused) { doPause(partition); } toBePaused.clear(); - records = consumer.poll(timeoutMs); - } + if (!unpaused.isEmpty()) { + ConsumerRecords records = consumer.poll(timeoutMs); - for (TopicPartition partition : unpaused) { - StreamSynchronizer streamSynchronizer = streamSynchronizers.get(partition); + for (TopicPartition partition : unpaused) { + StreamSynchronizer streamSynchronizer = streamSynchronizers.get(partition); - if (streamSynchronizer != null) - streamSynchronizer.addRecords(partition, records.records(partition).iterator()); - else - log.warn("unused topic: " + partition.topic()); + if (streamSynchronizer != null) + streamSynchronizer.addRecords(partition, records.records(partition).iterator()); + else + log.warn("unused topic: " + partition.topic()); + } + } } } diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java index d19a11690417d..c74fce0225d7d 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java @@ -22,18 +22,6 @@ <<<<<<< HEAD public class StreamSynchronizer implements ParallelExecutor.Task { - public static class Status { - private AtomicBoolean pollRequired = new AtomicBoolean(); - - public void pollRequired(boolean flag) { - pollRequired.set(flag); - } - - public boolean pollRequired() { - return pollRequired.get(); - } - } - public final String name; private final Ingestor ingestor; private final Chooser chooser; @@ -189,17 +177,15 @@ public PunctuationScheduler getPunctuationScheduler(Processor processor) { /** * Processes one record - * @param context an application specific context object for a task */ @SuppressWarnings("unchecked") - public void process(Object context) { - Status status = (Status) context; + @Override + public void process() { synchronized (this) { ingestNewRecords(); RecordQueue recordQueue = chooser.next(); if (recordQueue == null) { - status.pollRequired(true); return; } @@ -212,9 +198,6 @@ public void process(Object context) { long trackedTimestamp = recordQueue.trackedTimestamp(); StampedRecord record = recordQueue.next(); - if (recordQueue.size() < this.desiredUnprocessed) - status.pollRequired(true); - if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; recordQueue.stream.receive(record.topic(), record.key(), record.value(), record.timestamp, streamTime); diff --git a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java index 2f18f184c86f7..25f5c0b02989b 100644 --- a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java +++ b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java @@ -16,15 +16,13 @@ public class ParallelExecutor { public interface Task { /** * Executes a task - * @param context an application specific context object for a task */ - void process(Object context); + void process(); } private final WorkerThread[] workerThreads; private final AtomicInteger taskIndex = new AtomicInteger(0); private volatile ArrayList tasks = new ArrayList(); - private volatile Object context; private volatile CountDownLatch latch; private volatile boolean running = true; private volatile Exception exception; @@ -41,17 +39,15 @@ public ParallelExecutor(int parallelDegree) { /** * Executes tasks in parallel. While this method is executing, other execute call will be blocked. * @param tasks a list of tasks executed in parallel - * @param context a context object passed to tasks * @throws Exception an exception thrown by a failed task */ - public void execute(ArrayList tasks, Object context) throws Exception { + public void execute(ArrayList tasks) throws Exception { synchronized (this) { try { int numTasks = tasks.size(); exception = null; if (numTasks > 0) { this.tasks = tasks; - this.context = context; this.latch = new CountDownLatch(numTasks); taskIndex.set(numTasks); @@ -73,7 +69,6 @@ public void execute(ArrayList tasks, Object context) throws Exce } finally { this.tasks = null; - this.context = null; this.latch = null; this.exception = null; } @@ -95,7 +90,7 @@ private void doProcess() { int index = taskIndex.decrementAndGet(); if (index >= 0) { try { - tasks.get(index).process(context); + tasks.get(index).process(); } catch (Exception ex) { exception = ex; diff --git a/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java b/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java index dfaa6a5a035b8..7912192913f6e 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java @@ -15,6 +15,7 @@ import java.util.List; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class StreamSynchronizerTest { @@ -99,14 +100,10 @@ public long extract(String topic, Object key, Object value) { new ConsumerRecord(partition2.topic(), partition2.partition(), 4, serializer.serialize(partition1.topic(), new Integer(600)), recordValue) )); - StreamSynchronizer.Status status = new StreamSynchronizer.Status(); - - streamSynchronizer.process(status); + streamSynchronizer.process(); assertEquals(stream1.numReceived, 1); assertEquals(stream2.numReceived, 0); - assertTrue(status.pollRequired()); - assertEquals(mockIngestor.paused.size(), 1); assertTrue(mockIngestor.paused.contains(partition2)); @@ -116,7 +113,7 @@ public long extract(String topic, Object key, Object value) { new ConsumerRecord(partition1.topic(), partition1.partition(), 5, serializer.serialize(partition1.topic(), new Integer(50)), recordValue) )); - streamSynchronizer.process(status); + streamSynchronizer.process(); assertEquals(stream1.numReceived, 2); assertEquals(stream2.numReceived, 0); @@ -124,44 +121,44 @@ public long extract(String topic, Object key, Object value) { assertTrue(mockIngestor.paused.contains(partition1)); assertTrue(mockIngestor.paused.contains(partition2)); - streamSynchronizer.process(status); + streamSynchronizer.process(); assertEquals(stream1.numReceived, 3); assertEquals(stream2.numReceived, 0); - streamSynchronizer.process(status); + streamSynchronizer.process(); assertEquals(stream1.numReceived, 3); assertEquals(stream2.numReceived, 1); assertEquals(mockIngestor.paused.size(), 1); assertTrue(mockIngestor.paused.contains(partition2)); - streamSynchronizer.process(status); + streamSynchronizer.process(); assertEquals(stream1.numReceived, 4); assertEquals(stream2.numReceived, 1); assertEquals(mockIngestor.paused.size(), 1); - streamSynchronizer.process(status); + streamSynchronizer.process(); assertEquals(stream1.numReceived, 4); assertEquals(stream2.numReceived, 2); assertEquals(mockIngestor.paused.size(), 0); - streamSynchronizer.process(status); + streamSynchronizer.process(); assertEquals(stream1.numReceived, 5); assertEquals(stream2.numReceived, 2); - streamSynchronizer.process(status); + streamSynchronizer.process(); assertEquals(stream1.numReceived, 5); assertEquals(stream2.numReceived, 3); - streamSynchronizer.process(status); + streamSynchronizer.process(); assertEquals(stream1.numReceived, 5); assertEquals(stream2.numReceived, 4); assertEquals(mockIngestor.paused.size(), 0); - streamSynchronizer.process(status); + streamSynchronizer.process(); assertEquals(stream1.numReceived, 5); assertEquals(stream2.numReceived, 4); } diff --git a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java index 0abacfd1b92fe..c079e98d85b34 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java +++ b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java @@ -42,12 +42,8 @@ public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchroni streamSynchronizers.put(partition, streamSynchronizer); } - @Override - public void removeStreamSynchronizerForPartition(TopicPartition partition) { - streamSynchronizers.remove(partition); - } - public void addRecords(TopicPartition partition, Iterable> records) { streamSynchronizers.get(partition).addRecords(partition, records.iterator()); } + } diff --git a/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java b/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java index 1bafbfe5aa8c8..84598f6f85d52 100644 --- a/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java +++ b/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java @@ -15,20 +15,20 @@ public class ParallelExecutorTest { public void testExecutingShortTaskList() throws Exception { ParallelExecutor parallelExecutor = new ParallelExecutor(10); ArrayList taskList = new ArrayList<>(); - Counter counter = new Counter(); + AtomicInteger counter = new AtomicInteger(0); for (int i = 0; i < 5; i++) { - taskList.add(new TestTask()); + taskList.add(new TestTask(counter)); } - parallelExecutor.execute(taskList, counter); + parallelExecutor.execute(taskList); for (TestTask task : taskList) { assertEquals(task.executionCount, 1); } assertEquals(counter.get(), taskList.size()); - parallelExecutor.execute(taskList, counter); + parallelExecutor.execute(taskList); for (TestTask task : taskList) { assertEquals(task.executionCount, 2); @@ -40,20 +40,20 @@ public void testExecutingShortTaskList() throws Exception { public void testExecutingLongTaskList() throws Exception { ParallelExecutor parallelExecutor = new ParallelExecutor(10); ArrayList taskList = new ArrayList<>(); - Counter counter = new Counter(); + AtomicInteger counter = new AtomicInteger(0); for (int i = 0; i < 20; i++) { - taskList.add(new TestTask()); + taskList.add(new TestTask(counter)); } - parallelExecutor.execute(taskList, counter); + parallelExecutor.execute(taskList); for (TestTask task : taskList) { assertEquals(task.executionCount, 1); } assertEquals(counter.get(), taskList.size()); - parallelExecutor.execute(taskList, counter); + parallelExecutor.execute(taskList); for (TestTask task : taskList) { assertEquals(task.executionCount, 2); @@ -65,25 +65,25 @@ public void testExecutingLongTaskList() throws Exception { public void testException() { ParallelExecutor parallelExecutor = new ParallelExecutor(10); ArrayList taskList = new ArrayList<>(); - Counter counter = new Counter(); + AtomicInteger counter = new AtomicInteger(0); for (int i = 0; i < 20; i++) { if (i == 15) { - taskList.add(new TestTask() { + taskList.add(new TestTask(counter) { @Override - public void process(Object context) { + public void process() { throw new TestException(); } }); } else { - taskList.add(new TestTask()); + taskList.add(new TestTask(counter)); } } Exception exception = null; try { - parallelExecutor.execute(taskList, counter); + parallelExecutor.execute(taskList); } catch (Exception ex) { exception = ex; @@ -96,10 +96,14 @@ public void process(Object context) { private static class TestTask implements ParallelExecutor.Task { public volatile int executionCount = 0; + private AtomicInteger counter; + + TestTask(AtomicInteger counter) { + this.counter = counter; + } @Override - public void process(Object context) { - Counter counter = (Counter) context; + public void process() { try { Thread.sleep(20); executionCount++; @@ -107,19 +111,7 @@ public void process(Object context) { catch (Exception ex) { // ignore } - counter.incr(); - } - } - - private static class Counter { - private AtomicInteger totalExecutionCount = new AtomicInteger(); - - public void incr() { - totalExecutionCount.incrementAndGet(); - } - - public int get() { - return totalExecutionCount.get(); + counter.incrementAndGet(); } } From bcf30cbb0cbd0cfcf6548b06eb89f867c2a35452 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 14 Jul 2015 18:19:35 -0700 Subject: [PATCH 062/275] examples of Processor and KStreamJob usage --- .../confluent/streaming/KStreamContext.java | 8 ++-- .../io/confluent/streaming/KStreamJob.java | 4 +- .../confluent/streaming/KafkaStreaming.java | 19 +++++++-- .../java/io/confluent/streaming/KeyValue.java | 2 +- .../streaming/ProcessorKStreamJob.java | 24 +++++++++++ .../confluent/streaming/StreamingConfig.java | 5 ++- .../streaming/examples/MapKStreamJob.java | 41 +++++++++++++++++++ .../streaming/examples/PrintProcessor.java | 39 ++++++++++++++++++ .../internal/KStreamContextImpl.java | 11 ++++- .../streaming/internal/ProcessorConfig.java | 17 +++++--- 10 files changed, 152 insertions(+), 18 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/ProcessorKStreamJob.java create mode 100644 src/main/java/io/confluent/streaming/examples/MapKStreamJob.java create mode 100644 src/main/java/io/confluent/streaming/examples/PrintProcessor.java diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 4fd3087c33a4a..965df5117b6de 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -51,7 +51,7 @@ public interface KStreamContext { /** * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. - * @param topics the topic name + * @param topics the topic names, if empty default to all the topics in the config * @return KStream */ KStream from(String... topics); @@ -59,7 +59,7 @@ public interface KStreamContext { /** * Creates a KStream instance for the specified topics. The stream is added to the specified synchronization group. * @param syncGroup the synchronization group - * @param topics the topic name + * @param topics the topic names, if empty default to all the topics in the config * @return KStream */ KStream from(SyncGroup syncGroup, String... topics); @@ -71,7 +71,7 @@ public interface KStreamContext { * 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 name + * @param topics the topic names, if empty default to all the topics in the config * @return KStream */ KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); @@ -83,7 +83,7 @@ public interface KStreamContext { * 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 name + * @param topics the topic names, if empty default to all the topics in the config * @return KStream */ KStream from(SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java index e61b991c418d8..b6b759e75419e 100644 --- a/src/main/java/io/confluent/streaming/KStreamJob.java +++ b/src/main/java/io/confluent/streaming/KStreamJob.java @@ -7,10 +7,8 @@ public interface KStreamJob { /** - * Returns a set of topics used in this job - * @return Topics + * Creates a job instance */ - Topics topics(); /** * Initializes a stream processing job for a partition. This method is called for each partition. diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 43c8f5f8d4048..7488cf4a7f395 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -48,7 +48,14 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -122,7 +129,7 @@ public void onPartitionsRevoked(Consumer consumer, Collection jobClass, StreamingConfig config) { + public KafkaStreaming(Class jobClass, StreamingConfig config) { this(jobClass, config, null, null); } @@ -132,7 +139,6 @@ protected KafkaStreaming(Class jobClass, Producer producer, Consumer consumer) { this.jobClass = jobClass; - this.topics = extractTopics(jobClass); this.producer = producer == null? new KafkaProducer<>(config.config(), new ByteArraySerializer(), new ByteArraySerializer()): producer; this.consumer = consumer == null? new KafkaConsumer<>(config.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()): consumer; this.streamingConfig = config; @@ -162,6 +168,13 @@ protected KafkaStreaming(Class jobClass, this.recordsProcessed = 0; this.time = new SystemTime(); this.parallelExecutor = new ParallelExecutor(this.config.numStreamThreads); + + try { + this.topics = new HashSet<>(Arrays.asList(this.config.topics.split(","))); + } + catch (Exception e) { + throw new KStreamException("failed to get a topic list from the job", e); + } } /** diff --git a/src/main/java/io/confluent/streaming/KeyValue.java b/src/main/java/io/confluent/streaming/KeyValue.java index b9b26d7c576aa..1ef0d3dea4189 100644 --- a/src/main/java/io/confluent/streaming/KeyValue.java +++ b/src/main/java/io/confluent/streaming/KeyValue.java @@ -8,7 +8,7 @@ public class KeyValue { public final K key; public final V value; - private KeyValue(K key, V value) { + public KeyValue(K key, V value) { this.key = key; this.value = value; } diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java new file mode 100644 index 0000000000000..fbb0d5e292a40 --- /dev/null +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -0,0 +1,24 @@ +package io.confluent.streaming; + +import org.apache.kafka.common.utils.Utils; + +/** + * Created by guozhang on 7/14/15. + */ +public class ProcessorKStreamJob implements KStreamJob { + + public static String PROCESSOR_CLASSNAME = "__PROCESSOR_CLASSNAME__"; + + @SuppressWarnings("unchecked") + @Override + public void init(KStreamContext context) { + Processor processor = (Processor) Utils.newInstance((Class) context.getContext().get(PROCESSOR_CLASSNAME)); + + context.from(null).process(processor); + } + + @Override + public void close() { + // do nothing + } +} diff --git a/src/main/java/io/confluent/streaming/StreamingConfig.java b/src/main/java/io/confluent/streaming/StreamingConfig.java index 87feaba2d15d1..513adce295b99 100644 --- a/src/main/java/io/confluent/streaming/StreamingConfig.java +++ b/src/main/java/io/confluent/streaming/StreamingConfig.java @@ -32,7 +32,10 @@ */ public class StreamingConfig { - /** window.time.ms */ + /** topics */ + public static final String TOPICS_CONFIG = "topics"; + + /** state.dir */ public static final String STATE_DIR_CONFIG = "state.dir"; /** poll.time.ms */ diff --git a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java new file mode 100644 index 0000000000000..e82929536a74f --- /dev/null +++ b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java @@ -0,0 +1,41 @@ +package io.confluent.streaming.examples; + +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamJob; +import io.confluent.streaming.KafkaStreaming; +import io.confluent.streaming.KeyValue; +import io.confluent.streaming.KeyValueMapper; +import io.confluent.streaming.StreamingConfig; + +import java.util.Properties; + +/** + * Created by guozhang on 7/14/15. + */ +public class MapKStreamJob implements KStreamJob { + + public String topic = "bla"; + + @SuppressWarnings("unchecked") + @Override + public void init(KStreamContext context) { + context.from(topic) + .map(new KeyValueMapper() { + @Override + public KeyValue apply(Object key, Object value) { + return new KeyValue(key, new Integer((String)value)); + } + }) + .sendTo("bla-bla"); + } + + @Override + public void close() { + // do nothing + } + + public static void main(String[] args) { + KafkaStreaming kstream = new KafkaStreaming(MapKStreamJob.class, new StreamingConfig(new Properties())); + kstream.run(); + } +} diff --git a/src/main/java/io/confluent/streaming/examples/PrintProcessor.java b/src/main/java/io/confluent/streaming/examples/PrintProcessor.java new file mode 100644 index 0000000000000..72b0d3812ac4a --- /dev/null +++ b/src/main/java/io/confluent/streaming/examples/PrintProcessor.java @@ -0,0 +1,39 @@ +package io.confluent.streaming.examples; + +import io.confluent.streaming.KafkaStreaming; +import io.confluent.streaming.Processor; +import io.confluent.streaming.ProcessorKStreamJob; +import io.confluent.streaming.PunctuationScheduler; +import io.confluent.streaming.StreamingConfig; + +import java.util.Properties; + +/** + * Created by guozhang on 7/14/15. + */ +public class PrintProcessor implements Processor { + + @Override + public void apply(String topic, K key, V value) { + System.out.println(topic + ": [" + key + ", " + value + "]"); + } + + @Override + public void init(PunctuationScheduler punctuationScheduler) { + // do nothing + } + + @Override + public void punctuate(long streamTime) { + // do nothing + } + + public static void main(String[] args) { + // put the Processor class into the context configs + StreamingConfig configs = new StreamingConfig(new Properties()); + configs.addContextObject(ProcessorKStreamJob.PROCESSOR_CLASSNAME, PrintProcessor.class); + + KafkaStreaming kstream = new KafkaStreaming(ProcessorKStreamJob.class, configs); + kstream.run(); + } +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index a2ebe5e48c967..81e33a7e6a4bf 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -23,6 +23,7 @@ import java.io.File; import java.io.IOException; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.Map; @@ -152,10 +153,18 @@ public KStream from(SyncGroup syncGroup, Deserializer keyDeseria if (syncGroup == null) throw new NullPointerException(); KStreamSource stream = null; + Set fromTopics; synchronized (this) { + // if topics not specified, use all the topics be default + if (topics == null) { + fromTopics = this.topics; + } else { + fromTopics = Collections.unmodifiableSet(Util.mkSet(topics)); + } + // iterate over the topics and check if the stream has already been created for them - for (String topic : topics) { + for (String topic : fromTopics) { if (!this.topics.contains(topic)) throw new IllegalArgumentException("topic not subscribed: " + topic); diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java b/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java index 84679e6d65c87..3364094a91b10 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java @@ -31,11 +31,16 @@ public class ProcessorConfig extends AbstractConfig { private static final ConfigDef CONFIG; static { - CONFIG = new ConfigDef().define(StreamingConfig.STATE_DIR_CONFIG, - Type.STRING, - System.getProperty("java.io.tmpdir"), - Importance.MEDIUM, - "") + CONFIG = new ConfigDef().define(StreamingConfig.TOPICS_CONFIG, + Type.STRING, + "", + Importance.HIGH, + "All the possible topic names this job need to interact with") + .define(StreamingConfig.STATE_DIR_CONFIG, + Type.STRING, + System.getProperty("java.io.tmpdir"), + Importance.MEDIUM, + "") .define(StreamingConfig.POLL_TIME_MS_CONFIG, Type.LONG, 100, @@ -73,6 +78,7 @@ public class ProcessorConfig extends AbstractConfig { "The number of threads to execute stream processing."); } + public final String topics; public final File stateDir; public final long pollTimeMs; public final long commitTimeMs; @@ -84,6 +90,7 @@ public class ProcessorConfig extends AbstractConfig { public ProcessorConfig(Properties processor) { super(CONFIG, processor); + this.topics = this.getString(StreamingConfig.TOPICS_CONFIG); this.stateDir = new File(this.getString(StreamingConfig.STATE_DIR_CONFIG)); this.pollTimeMs = this.getLong(StreamingConfig.POLL_TIME_MS_CONFIG); this.commitTimeMs = this.getLong(StreamingConfig.COMMIT_TIME_MS_CONFIG); From 962b2ee3ebe9b5346f26c7283884485124b8a46e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 15 Jul 2015 15:15:49 -0700 Subject: [PATCH 063/275] add collector and coordinator into Processor.apply --- .../io/confluent/streaming/Processor.java | 2 +- .../streaming/examples/MapKStreamJob.java | 45 +++++++++++++++---- .../streaming/examples/PrintProcessor.java | 25 ++++++++++- .../streaming/internal/KStreamImpl.java | 9 ++-- 4 files changed, 67 insertions(+), 14 deletions(-) diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 2756180490592..50d4b881371ea 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -5,7 +5,7 @@ */ public interface Processor { - void apply(String topic, K key, V value); + void apply(String topic, K key, V value, RecordCollector collector, Coordinator coordinator); void init(PunctuationScheduler punctuationScheduler); diff --git a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java index e82929536a74f..17e176e9a6855 100644 --- a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java @@ -1,11 +1,14 @@ package io.confluent.streaming.examples; +import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamJob; import io.confluent.streaming.KafkaStreaming; import io.confluent.streaming.KeyValue; import io.confluent.streaming.KeyValueMapper; +import io.confluent.streaming.Predicate; import io.confluent.streaming.StreamingConfig; +import org.apache.kafka.common.serialization.StringDeserializer; import java.util.Properties; @@ -14,19 +17,45 @@ */ public class MapKStreamJob implements KStreamJob { - public String topic = "bla"; - @SuppressWarnings("unchecked") @Override public void init(KStreamContext context) { - context.from(topic) - .map(new KeyValueMapper() { + + // With overriden de-serializer + KStream stream1 = context.from(new StringDeserializer(), new StringDeserializer(), "topic1"); + + stream1.map(new KeyValueMapper() { + @Override + public KeyValue apply(String key, String value) { + return new KeyValue<>(key, new Integer(value)); + } + }).filter(new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + }).sendTo("topic2"); + + // Without overriden de-serialzier + KStream stream2 = (KStream)context.from("topic2"); + + KStream[] streams = stream2.branch( + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + }, + new Predicate() { @Override - public KeyValue apply(Object key, Object value) { - return new KeyValue(key, new Integer((String)value)); + public boolean apply(String key, Integer value) { + return true; } - }) - .sendTo("bla-bla"); + } + ); + + streams[0].sendTo("topic3"); + streams[1].sendTo("topic4"); } @Override diff --git a/src/main/java/io/confluent/streaming/examples/PrintProcessor.java b/src/main/java/io/confluent/streaming/examples/PrintProcessor.java index 72b0d3812ac4a..d9c69d0b8eb99 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintProcessor.java +++ b/src/main/java/io/confluent/streaming/examples/PrintProcessor.java @@ -1,10 +1,13 @@ package io.confluent.streaming.examples; +import io.confluent.streaming.Coordinator; import io.confluent.streaming.KafkaStreaming; import io.confluent.streaming.Processor; import io.confluent.streaming.ProcessorKStreamJob; import io.confluent.streaming.PunctuationScheduler; +import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StreamingConfig; +import org.apache.kafka.clients.producer.ProducerRecord; import java.util.Properties; @@ -14,8 +17,12 @@ public class PrintProcessor implements Processor { @Override - public void apply(String topic, K key, V value) { + public void apply(String topic, K key, V value, RecordCollector collector, Coordinator coordinator) { System.out.println(topic + ": [" + key + ", " + value + "]"); + + coordinator.commit(Coordinator.RequestScope.CURRENT_TASK); + + collector.send(new ProducerRecord<>("topic", key, value)); } @Override @@ -35,5 +42,21 @@ public static void main(String[] args) { KafkaStreaming kstream = new KafkaStreaming(ProcessorKStreamJob.class, configs); kstream.run(); + + /* + * another possible way is to relax the KafkaStreaming job class type, from + * + + KafkaStreaming(Class job ..) + + to + + KafkaStreaming(Class job ..) + + so that we can skip the KStreamJob wrapper around the processor: + + KafkaStreaming kstream = new KafkaStreaming(PrintProcessor.class, configs); + + */ } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index f2c018ff0095b..65b1b425b64dc 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -1,5 +1,6 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.Coordinator; import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamWindowed; @@ -129,8 +130,8 @@ private Processor getSendProcessor(final String sendTopic, Serializ return new Processor() { @Override - public void apply(String topic, K key, V value) { - collector.send(new ProducerRecord(sendTopic, key, value)); + public void apply(String topic, K key, V value, RecordCollector dummyCollector, Coordinator coordinator) { + collector.send(new ProducerRecord<>(sendTopic, key, value)); } @Override public void init(PunctuationScheduler scheduler) {} @@ -144,9 +145,9 @@ public void process(final Processor processor) { Receiver receiver = new Receiver() { public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { if (topic.equals(KStreamMetadata.UNKNOWN_TOPICNAME)) - processor.apply(null, (K)key, (V)value); + processor.apply(null, (K)key, (V)value, (RecordCollector) context.recordCollector(), context.coordinator()); else - processor.apply(topic, (K)key, (V)value); + processor.apply(topic, (K)key, (V)value, (RecordCollector) context.recordCollector(), context.coordinator()); } }; registerReceiver(receiver); From 61f3cd7c356d9e4e7a9f0cfaacf311aed69df071 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 20 Jul 2015 15:11:22 -0700 Subject: [PATCH 064/275] remove SyncGroup from user facing APIs --- .../java/io/confluent/streaming/KStream.java | 31 ---------- .../confluent/streaming/KStreamContext.java | 33 +++------- .../confluent/streaming/KafkaStreaming.java | 49 ++++++++++++--- .../io/confluent/streaming/Processor.java | 2 +- .../io/confluent/streaming/SyncGroup.java | 12 ---- .../streaming/examples/PrintProcessor.java | 2 +- .../streaming/internal/Ingestor.java | 2 +- .../streaming/internal/IngestorImpl.java | 12 ++-- .../internal/KStreamContextImpl.java | 60 +++++++++---------- .../streaming/internal/KStreamFlatMap.java | 5 +- .../streaming/internal/KStreamImpl.java | 29 +++------ .../streaming/internal/KStreamMap.java | 5 +- .../streaming/internal/KStreamMetadata.java | 14 ++--- .../internal/KStreamWindowedImpl.java | 4 ++ .../streaming/internal/RecordQueue.java | 3 +- .../streaming/internal/StampedRecord.java | 9 ++- ...reamSynchronizer.java => StreamGroup.java} | 52 ++++++++++++---- .../streaming/internal/KStreamBranchTest.java | 4 +- .../streaming/internal/KStreamFilterTest.java | 4 +- .../internal/KStreamFlatMapTest.java | 4 +- .../internal/KStreamFlatMapValuesTest.java | 4 +- .../streaming/internal/KStreamJoinTest.java | 4 +- .../streaming/internal/KStreamMapTest.java | 4 +- .../internal/KStreamMapValuesTest.java | 5 +- .../streaming/internal/KStreamSourceTest.java | 4 +- .../internal/KStreamWindowedTest.java | 4 +- .../internal/MockKStreamContext.java | 11 +--- ...hronizerTest.java => StreamGroupTest.java} | 34 +++++------ .../streaming/testutil/MockIngestor.java | 8 +-- 29 files changed, 198 insertions(+), 216 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/SyncGroup.java rename src/main/java/io/confluent/streaming/internal/{StreamSynchronizer.java => StreamGroup.java} (80%) rename src/test/java/io/confluent/streaming/internal/{StreamSynchronizerTest.java => StreamGroupTest.java} (87%) diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index ef4a9a41a5498..5f31104ff0527 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -88,16 +88,6 @@ public interface KStream { */ KStream through(String topic); - /** - * Sends key-value to a topic, also creates a new stream from the topic. - * The created stream is added to the specified synchronization group. - * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic, syncGroup). - * @param topic the topic name - * @param syncGroup the synchronization group - * @return KStream - */ - KStream through(String topic, SyncGroup syncGroup); - /** * Sends key-value to a topic, also creates a new stream from the topic. * The created stream is added to the default synchronization group. @@ -118,27 +108,6 @@ public interface KStream { */ KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); - /** - * Sends key-value to a topic, also creates a new stream from the topic. - * The created stream is added to the specific synchronization group. - * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). - * @param topic the topic name - * @param syncGroup the synchronization group - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used - * @param keyDeserializer key deserializer used to create the new KStream, - * if not specified the default deserializer defined in the configs will be used - * @param valDeserializer value deserializer used to create the new KStream, - * if not specified the default deserializer defined in the configs will be used - * @param the key type of the new stream - * @param the value type of the new stream - * - * @return KStream - */ - KStream through(String topic, SyncGroup syncGroup, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); - /** * Sends key-value to a topic. * @param topic the topic name diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 965df5117b6de..0394be1876f41 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -1,11 +1,13 @@ package io.confluent.streaming; +import io.confluent.streaming.internal.StreamGroup; 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.Map; +import java.util.concurrent.atomic.AtomicInteger; /** * KStreamContext is the interface that allows an implementation of {@link KStreamJob#init(KStreamContext)} to create KStream instances. @@ -14,7 +16,7 @@ */ public interface KStreamContext { - String DEFAULT_SYNCHRONIZATION_GROUP = "defaultSynchronizationGroup"; + AtomicInteger STREAM_GROUP_INDEX = new AtomicInteger(1); /** * Returns the partition id @@ -56,15 +58,6 @@ public interface KStreamContext { */ KStream from(String... topics); - /** - * Creates a KStream instance for the specified topics. The stream is added to the specified synchronization group. - * @param syncGroup the synchronization group - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream - */ - KStream from(SyncGroup syncGroup, String... topics); - - /** * 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, @@ -76,18 +69,6 @@ public interface KStreamContext { */ KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); - /** - * Creates a KStream instance for the specified topic. The stream is added to the specified synchronization group. - * @param syncGroup the 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 - */ - KStream from(SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); - /** * Returns a RecordCollector which applies the serializer to key and value. * @return RecordCollector @@ -119,18 +100,18 @@ public interface KStreamContext { Metrics metrics(); /** - * Creates a synchronization group with the given name. + * Creates a stream synchronization group with the given name. * @param name the synchronization group name * @return a synchronization group */ - SyncGroup syncGroup(String name); + StreamGroup streamGroup(String name); /** - * Creates a round robin synchronization group with the given name. + * Creates a round robin stream synchronization group with the given name. * @param name the synchronization group name * @return a round robin synchronization group */ - SyncGroup roundRobinSyncGroup(String name); + StreamGroup roundRobinStreamGroup(String name); /** * Restores the specified storage engine. diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 7488cf4a7f395..f018620f2a250 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -20,8 +20,12 @@ import io.confluent.streaming.internal.KStreamContextImpl; import io.confluent.streaming.internal.ProcessorConfig; import io.confluent.streaming.internal.IngestorImpl; +<<<<<<< HEAD import io.confluent.streaming.internal.StreamSynchronizer; <<<<<<< HEAD +======= +import io.confluent.streaming.internal.StreamGroup; +>>>>>>> remove SyncGroup from user facing APIs import io.confluent.streaming.util.ParallelExecutor; ======= >>>>>>> removed some generics @@ -93,9 +97,14 @@ public class KafkaStreaming implements Runnable { private final Class jobClass; private final Set topics; +<<<<<<< HEAD <<<<<<< HEAD private final Map> syncGroups = new HashMap<>(); private final ArrayList> streamSynchronizers = new ArrayList<>(); +======= + private final Map> streamSynchronizersForPartition = new HashMap<>(); + private final ArrayList streamGroups = new ArrayList<>(); +>>>>>>> remove SyncGroup from user facing APIs private final ParallelExecutor parallelExecutor; ======= private final Map> streamSynchronizersForPartition = new HashMap<>(); @@ -211,10 +220,14 @@ private void shutdown() { log.info("Shutting down container"); commitAll(time.milliseconds()); +<<<<<<< HEAD <<<<<<< HEAD for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { +======= + for (StreamGroup streamGroup : streamGroups) { +>>>>>>> remove SyncGroup from user facing APIs try { - streamSynchronizer.close(); + streamGroup.close(); } catch(Exception e) { log.error("Error while closing stream synchronizers: ", e); @@ -234,8 +247,13 @@ private void shutdown() { producer.close(); consumer.close(); parallelExecutor.shutdown(); +<<<<<<< HEAD syncGroups.clear(); streamSynchronizers.clear(); +======= + streamSynchronizersForPartition.clear(); + streamGroups.clear(); +>>>>>>> remove SyncGroup from user facing APIs shutdownComplete.countDown(); log.info("Shut down complete"); } @@ -257,12 +275,16 @@ private void runLoop() { while (stillRunning()) { ingestor.poll(); +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD parallelExecutor.execute(streamSynchronizers, status); ======= parallelExecutor.execute(streamSynchronizers); >>>>>>> clean up ingestor and stream synchronizer +======= + parallelExecutor.execute(streamGroups); +>>>>>>> remove SyncGroup from user facing APIs ======= for (Map.Entry> entry : streamSynchronizersForPartition.entrySet()) { @@ -311,10 +333,14 @@ private void commitAll(long now) { context.flush(); // check co-ordinator } +<<<<<<< HEAD <<<<<<< HEAD for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { +======= + for (StreamGroup streamGroup : streamGroups) { +>>>>>>> remove SyncGroup from user facing APIs try { - commit.putAll(streamSynchronizer.consumedOffsets()); + commit.putAll(streamGroup.consumedOffsets()); } catch(Exception e) { log.error("Error while closing processor: ", e); @@ -342,8 +368,8 @@ private void commitRequesting(long now) { KStreamContextImpl context = kstreamContexts.get(id); context.flush(); - for (StreamSynchronizer streamSynchronizer : streamSynchronizersForPartition.get(id)) { - commit.putAll(streamSynchronizer.consumedOffsets()); // TODO: can this be async? + for (StreamGroup streamGroup : streamSynchronizersForPartition.get(id)) { + commit.putAll(streamGroup.consumedOffsets()); // TODO: can this be async? } } consumer.commit(commit, CommitType.SYNC); @@ -412,11 +438,18 @@ public void shutdown(Coordinator.RequestScope scope) { throw new KafkaException(e); } +<<<<<<< HEAD <<<<<<< HEAD Collection syncGroups = kstreamContext.syncGroups(); this.syncGroups.put(id, syncGroups); for (SyncGroup syncGroup : syncGroups) { streamSynchronizers.add(syncGroup.streamSynchronizer); +======= + Collection streamGroups = kstreamContext.streamSynchronizers(); + this.streamSynchronizersForPartition.put(id, streamGroups); + for (StreamGroup streamGroup : streamGroups) { + streamGroups.add(streamGroup); +>>>>>>> remove SyncGroup from user facing APIs } ======= streamSynchronizersForPartition.put(id, kstreamContext.streamSynchronizers()); @@ -430,9 +463,9 @@ public void shutdown(Coordinator.RequestScope scope) { private void removePartitions(Collection assignment) { commitAll(time.milliseconds()); - for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { - log.info("Removing synchronization groups {}", streamSynchronizer.name()); - streamSynchronizer.close(); + for (StreamGroup streamGroup : streamGroups) { + log.info("Removing synchronization groups {}", streamGroup.name()); + streamGroup.close(); } for (KStreamContextImpl kstreamContext : kstreamContexts.values()) { log.info("Removing stream context {}", kstreamContext.id()); @@ -444,7 +477,7 @@ private void removePartitions(Collection assignment) { } streamingMetrics.processorDestruction.record(); } - streamSynchronizers.clear(); + streamGroups.clear(); ingestor.clear(); } diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 50d4b881371ea..52f76ef0d1fab 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -5,7 +5,7 @@ */ public interface Processor { - void apply(String topic, K key, V value, RecordCollector collector, Coordinator coordinator); + void process(String topic, K key, V value, RecordCollector collector, Coordinator coordinator); void init(PunctuationScheduler punctuationScheduler); diff --git a/src/main/java/io/confluent/streaming/SyncGroup.java b/src/main/java/io/confluent/streaming/SyncGroup.java deleted file mode 100644 index 8a7f6a8747b27..0000000000000 --- a/src/main/java/io/confluent/streaming/SyncGroup.java +++ /dev/null @@ -1,12 +0,0 @@ -package io.confluent.streaming; - -import io.confluent.streaming.internal.StreamSynchronizer; - -/** - * SyncGroup represents a group of streams synchronized together. - */ -public interface SyncGroup { - - String name(); - -} diff --git a/src/main/java/io/confluent/streaming/examples/PrintProcessor.java b/src/main/java/io/confluent/streaming/examples/PrintProcessor.java index d9c69d0b8eb99..89728032f897f 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintProcessor.java +++ b/src/main/java/io/confluent/streaming/examples/PrintProcessor.java @@ -17,7 +17,7 @@ public class PrintProcessor implements Processor { @Override - public void apply(String topic, K key, V value, RecordCollector collector, Coordinator coordinator) { + public void process(String topic, K key, V value, RecordCollector collector, Coordinator coordinator) { System.out.println(topic + ": [" + key + ", " + value + "]"); coordinator.commit(Coordinator.RequestScope.CURRENT_TASK); diff --git a/src/main/java/io/confluent/streaming/internal/Ingestor.java b/src/main/java/io/confluent/streaming/internal/Ingestor.java index 14e7cc2e35732..8f62eba04ad6f 100644 --- a/src/main/java/io/confluent/streaming/internal/Ingestor.java +++ b/src/main/java/io/confluent/streaming/internal/Ingestor.java @@ -17,6 +17,6 @@ public interface Ingestor { int numPartitions(String topic); - void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition); + void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition); } diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 9ae4b32317ec0..6eff2567291ad 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -16,7 +16,7 @@ public class IngestorImpl implements Ingestor { private final Set unpaused = new HashSet<>(); private final Set toBePaused = new HashSet<>(); private final long pollTimeMs; - private final Map streamSynchronizers = new HashMap<>(); + private final Map streamSynchronizers = new HashMap<>(); public IngestorImpl(Consumer consumer, long pollTimeMs) { @@ -46,10 +46,10 @@ public void poll(long timeoutMs) { ConsumerRecords records = consumer.poll(timeoutMs); for (TopicPartition partition : unpaused) { - StreamSynchronizer streamSynchronizer = streamSynchronizers.get(partition); + StreamGroup streamGroup = streamSynchronizers.get(partition); - if (streamSynchronizer != null) - streamSynchronizer.addRecords(partition, records.records(partition).iterator()); + if (streamGroup != null) + streamGroup.addRecords(partition, records.records(partition).iterator()); else log.warn("unused topic: " + partition.topic()); } @@ -82,9 +82,9 @@ public int numPartitions(String topic) { @SuppressWarnings("unchecked") @Override - public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition) { + public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition) { synchronized (this) { - streamSynchronizers.put(partition, streamSynchronizer); + streamSynchronizers.put(partition, streamGroup); unpaused.add(partition); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 81e33a7e6a4bf..cf7981d5505d7 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -8,7 +8,6 @@ import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StorageEngine; import io.confluent.streaming.StreamingConfig; -import io.confluent.streaming.SyncGroup; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.util.Util; import org.apache.kafka.clients.consumer.Consumer; @@ -48,7 +47,7 @@ public class KStreamContextImpl implements KStreamContext { private final HashMap> sourceStreams = new HashMap<>(); private final HashMap partitioningInfos = new HashMap<>(); private final TimestampExtractor timestampExtractor; - private final HashMap streamSynchronizerMap = new HashMap<>(); + private final HashMap streamGroups = new HashMap<>(); private final StreamingConfig streamingConfig; private final ProcessorConfig processorConfig; private final Metrics metrics; @@ -134,23 +133,21 @@ public Deserializer valueDeserializer() { @Override public KStream from(String... topics) { - return from(syncGroup(DEFAULT_SYNCHRONIZATION_GROUP), null, null, topics); + return from(streamGroup(getNextGroupName()), null, null, topics); } @Override - public KStream from(SyncGroup syncGroup, String... topics) { - return from(syncGroup, null, null, topics); + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + return from(streamGroup(getNextGroupName()), keyDeserializer, valDeserializer, topics); } - @Override - public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - return from(syncGroup(DEFAULT_SYNCHRONIZATION_GROUP), keyDeserializer, valDeserializer, topics); + private String getNextGroupName() { + return "StreamGroup-" + STREAM_GROUP_INDEX.getAndIncrement(); } - @Override @SuppressWarnings("unchecked") - public KStream from(SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - if (syncGroup == null) throw new NullPointerException(); + private KStream from(StreamGroup streamGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + if (streamGroup == null) throw new IllegalArgumentException("unspecified stream group"); KStreamSource stream = null; Set fromTopics; @@ -187,7 +184,7 @@ public KStream from(SyncGroup syncGroup, Deserializer keyDeseria if (stream == null) { // create stream metadata Map topicPartitionInfos = new HashMap<>(); - for (String topic : topics) { + for (String topic : fromTopics) { PartitioningInfo partitioningInfo = this.partitioningInfos.get(topic); if (partitioningInfo == null) { @@ -198,7 +195,7 @@ public KStream from(SyncGroup syncGroup, Deserializer keyDeseria topicPartitionInfos.put(topic, partitioningInfo); } - KStreamMetadata streamMetadata = new KStreamMetadata(syncGroup, topicPartitionInfos); + KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, topicPartitionInfos); // override the deserializer classes if specified if (keyDeserializer == null && valDeserializer == null) { @@ -219,17 +216,16 @@ public KStream from(SyncGroup syncGroup, Deserializer keyDeseria } // update source stream map - for (String topic : topics) { - if (sourceStreams.containsKey(topic)) + for (String topic : fromTopics) { + if (!sourceStreams.containsKey(topic)) sourceStreams.put(topic, stream); TopicPartition partition = new TopicPartition(topic, id); - StreamSynchronizer streamSynchronizer = (StreamSynchronizer)syncGroup; - streamSynchronizer.addPartition(partition, stream); - ingestor.addStreamSynchronizerForPartition(streamSynchronizer, partition); + streamGroup.addPartition(partition, stream); + ingestor.addPartitionStreamToGroup(streamGroup, partition); } } else { - if (stream.metadata.syncGroup == syncGroup) + if (stream.metadata.streamGroup == streamGroup) throw new IllegalStateException("topic is already assigned a different synchronization group"); // TODO: with this constraint we will not allow users to create KStream with different @@ -270,26 +266,26 @@ public Metrics metrics() { } @Override - public SyncGroup syncGroup(String name) { - return syncGroup(name, new TimeBasedChooser()); + public StreamGroup streamGroup(String name) { + return streamGroup(name, new TimeBasedChooser()); } @Override - public SyncGroup roundRobinSyncGroup(String name) { - return syncGroup(name, new RoundRobinChooser()); + public StreamGroup roundRobinStreamGroup(String name) { + return streamGroup(name, new RoundRobinChooser()); } - private SyncGroup syncGroup(String name, Chooser chooser) { + private StreamGroup streamGroup(String name, Chooser chooser) { int desiredUnprocessedPerPartition = processorConfig.bufferedRecordsPerPartition; synchronized (this) { - StreamSynchronizer streamSynchronizer = streamSynchronizerMap.get(name); - if (streamSynchronizer == null) { - streamSynchronizer = - new StreamSynchronizer(name, ingestor, chooser, timestampExtractor, desiredUnprocessedPerPartition); - streamSynchronizerMap.put(name, streamSynchronizer); + StreamGroup streamGroup = streamGroups.get(name); + if (streamGroup == null) { + streamGroup = + new StreamGroup(name, ingestor, chooser, timestampExtractor, desiredUnprocessedPerPartition); + streamGroups.put(name, streamGroup); } - return streamSynchronizer; + return streamGroup; } } @@ -302,8 +298,8 @@ public void restore(StorageEngine engine) throws Exception { } - public Collection streamSynchronizers() { - return streamSynchronizerMap.values(); + public Collection streamSynchronizers() { + return streamGroups.values(); } public void init(Consumer restoreConsumer) throws IOException { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 7da3244d1d749..979bc740e7b89 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -3,7 +3,6 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; -import io.confluent.streaming.SyncGroup; /** * Created by yasuhiro on 6/17/15. @@ -12,8 +11,8 @@ class KStreamFlatMap extends KStreamImpl { private final KeyValueMapper, K1, V1> mapper; - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, SyncGroup syncGroup, KStreamContext context) { - super(KStreamMetadata.unjoinable(syncGroup), context); + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, StreamGroup streamGroup, KStreamContext context) { + super(KStreamMetadata.unjoinable(streamGroup), context); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 65b1b425b64dc..cfbe5ddd25b0f 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -9,7 +9,6 @@ import io.confluent.streaming.Processor; import io.confluent.streaming.PunctuationScheduler; import io.confluent.streaming.RecordCollector; -import io.confluent.streaming.SyncGroup; import io.confluent.streaming.ValueMapper; import io.confluent.streaming.Window; import org.apache.kafka.clients.producer.ProducerRecord; @@ -54,7 +53,7 @@ public boolean apply(K key, V value) { @Override public KStream map(KeyValueMapper mapper) { - return chain(new KStreamMap(mapper, metadata.syncGroup, context)); + return chain(new KStreamMap(mapper, metadata.streamGroup, context)); } @Override @@ -64,7 +63,7 @@ public KStream mapValues(ValueMapper mapper) { @Override public KStream flatMap(KeyValueMapper, K, V> mapper) { - return chain(new KStreamFlatMap(mapper, metadata.syncGroup, context)); + return chain(new KStreamFlatMap(mapper, metadata.streamGroup, context)); } @Override @@ -87,26 +86,14 @@ public KStream[] branch(Predicate... predicates) { @SuppressWarnings("unchecked") @Override public KStream through(String topic) { - return through(topic, null); - } - - @SuppressWarnings("unchecked") - @Override - public KStream through(String topic, SyncGroup syncGroup) { - return through(topic, syncGroup, null, null, null, null); + return through(topic, null, null, null, null); } @SuppressWarnings("unchecked") @Override public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { - return through(topic, context.syncGroup(context.DEFAULT_SYNCHRONIZATION_GROUP), keySerializer, valSerializer, keyDeserializer, valDeserializer); - } - - @SuppressWarnings("unchecked") - @Override - public KStream through(String topic, SyncGroup syncGroup, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); - return context.from(syncGroup, keyDeserializer, valDeserializer, topic); + return context.from(keyDeserializer, valDeserializer, topic); } @Override @@ -130,7 +117,7 @@ private Processor getSendProcessor(final String sendTopic, Serializ return new Processor() { @Override - public void apply(String topic, K key, V value, RecordCollector dummyCollector, Coordinator coordinator) { + public void process(String topic, K key, V value, RecordCollector dummyCollector, Coordinator coordinator) { collector.send(new ProducerRecord<>(sendTopic, key, value)); } @Override @@ -145,14 +132,14 @@ public void process(final Processor processor) { Receiver receiver = new Receiver() { public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { if (topic.equals(KStreamMetadata.UNKNOWN_TOPICNAME)) - processor.apply(null, (K)key, (V)value, (RecordCollector) context.recordCollector(), context.coordinator()); + processor.process(null, (K) key, (V) value, (RecordCollector) context.recordCollector(), context.coordinator()); else - processor.apply(topic, (K)key, (V)value, (RecordCollector) context.recordCollector(), context.coordinator()); + processor.process(topic, (K) key, (V) value, (RecordCollector) context.recordCollector(), context.coordinator()); } }; registerReceiver(receiver); - PunctuationScheduler scheduler = ((StreamSynchronizer)metadata.syncGroup).getPunctuationScheduler(processor); + PunctuationScheduler scheduler = (metadata.streamGroup).getPunctuationScheduler(processor); processor.init(scheduler); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index 2eefeccf97069..a9bb7d1bedc64 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -3,7 +3,6 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; -import io.confluent.streaming.SyncGroup; /** * Created by yasuhiro on 6/17/15. @@ -12,8 +11,8 @@ class KStreamMap extends KStreamImpl { private final KeyValueMapper mapper; - KStreamMap(KeyValueMapper mapper, SyncGroup syncGroup, KStreamContext context) { - super(KStreamMetadata.unjoinable(syncGroup), context); + KStreamMap(KeyValueMapper mapper, StreamGroup streamGroup, KStreamContext context) { + super(KStreamMetadata.unjoinable(streamGroup), context); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java b/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java index a22238c0d4b2c..e35ae5db68e4f 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java @@ -1,7 +1,5 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.SyncGroup; - import java.util.Collections; import java.util.Map; @@ -12,22 +10,22 @@ public class KStreamMetadata { public static String UNKNOWN_TOPICNAME = "__UNKNOWN_TOPIC__"; - public static KStreamMetadata unjoinable(SyncGroup syncGroup) { - return new KStreamMetadata(syncGroup, Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(-1))); + public static KStreamMetadata unjoinable(StreamGroup streamGroup) { + return new KStreamMetadata(streamGroup, Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(-1))); } - public final SyncGroup syncGroup; + public StreamGroup streamGroup; public final Map topicPartitionInfos; - KStreamMetadata(SyncGroup syncGroup, Map topicPartitionInfos) { - this.syncGroup = syncGroup; + KStreamMetadata(StreamGroup streamGroup, Map topicPartitionInfos) { + this.streamGroup = streamGroup; this.topicPartitionInfos = topicPartitionInfos; } boolean isJoinCompatibleWith(KStreamMetadata other) { // the two streams should only be joinable if they are inside the same sync group // and their contained streams all have the same number of partitions - if (this.syncGroup != other.syncGroup) + if (this.streamGroup != other.streamGroup) return false; int numPartitions = -1; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 8fc87bf3f137b..89385ac7c7964 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -45,6 +45,10 @@ private KStream join(KStreamWindowed other, boolean prior if (!this.metadata.isJoinCompatibleWith(otherImpl.metadata)) throw new NotCopartitionedException(); + // merge the other stream's group with this group + this.metadata.streamGroup.mergeStreamGroup(otherImpl.metadata.streamGroup); + otherImpl.metadata.streamGroup = this.metadata.streamGroup; + KStreamJoin stream = new KStreamJoin(this.window, otherImpl.window, prior, processor, this.metadata, context); otherImpl.registerReceiver(stream.receiverForOtherStream); diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java index 30539620feb6d..90bf81be37abb 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -1,6 +1,5 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStream; import io.confluent.streaming.util.TimestampTracker; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; @@ -9,7 +8,7 @@ /** * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). It is intended to be used in - * {@link StreamSynchronizer}. + * {@link StreamGroup}. */ public class RecordQueue { diff --git a/src/main/java/io/confluent/streaming/internal/StampedRecord.java b/src/main/java/io/confluent/streaming/internal/StampedRecord.java index 0b7e668466016..86926491bafa3 100644 --- a/src/main/java/io/confluent/streaming/internal/StampedRecord.java +++ b/src/main/java/io/confluent/streaming/internal/StampedRecord.java @@ -3,13 +3,20 @@ import io.confluent.streaming.util.Stamped; import org.apache.kafka.clients.consumer.ConsumerRecord; + +// TODO: making this class exposed to user in the lower-level Processor public class StampedRecord extends Stamped> { StampedRecord(ConsumerRecord record, long timestamp) { super(record, timestamp); } - public String topic() { return value.topic(); } + public String topic() { + if (value.topic().equals(KStreamMetadata.UNKNOWN_TOPICNAME)) + return null; + else + return value.topic(); + } public Object key() { return value.key(); diff --git a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java similarity index 80% rename from src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java rename to src/main/java/io/confluent/streaming/internal/StreamGroup.java index c74fce0225d7d..5e828f08282a3 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -2,7 +2,6 @@ import io.confluent.streaming.Processor; import io.confluent.streaming.PunctuationScheduler; -import io.confluent.streaming.SyncGroup; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.util.MinTimestampTracker; import io.confluent.streaming.util.ParallelExecutor; @@ -14,11 +13,11 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; /** - * StreamSynchronizer tries to synchronize the progress of streams from different topics in the same {@link SyncGroup}. + * A StreamGroup is composed of multiple streams from different topics that need to be synchronized. */ +<<<<<<< HEAD:src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java <<<<<<< HEAD public class StreamSynchronizer implements ParallelExecutor.Task { @@ -29,6 +28,9 @@ public class StreamSynchronizer implements ParallelExecutor.Task { private final Map> stash = new HashMap<>(); ======= public class StreamSynchronizer implements SyncGroup { +======= +public class StreamGroup implements ParallelExecutor.Task { +>>>>>>> remove SyncGroup from user facing APIs:src/main/java/io/confluent/streaming/internal/StreamGroup.java public final String name; private final Ingestor ingestor; @@ -37,6 +39,8 @@ public class StreamSynchronizer implements SyncGroup { private final Map stash = new HashMap<>(); >>>>>>> removed some generics private final int desiredUnprocessed; + + // TODO: merge stash, consumedOffset, and newRecordBuffer into sth. like partition metadata private final Map consumedOffsets; private final PunctuationQueue punctuationQueue = new PunctuationQueue(); private final ArrayDeque> newRecordBuffer = new ArrayDeque<>(); @@ -45,18 +49,18 @@ public class StreamSynchronizer implements SyncGroup { private volatile int buffered = 0; /** - * Creates StreamSynchronizer - * @param name the name of {@link SyncGroup} + * Creates StreamGroup + * @param name the name of group * @param ingestor the instance of {@link Ingestor} * @param chooser the instance of {@link Chooser} * @param timestampExtractor the instance of {@link TimestampExtractor} * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic */ - StreamSynchronizer(String name, - Ingestor ingestor, - Chooser chooser, - TimestampExtractor timestampExtractor, - int desiredUnprocessedPerPartition) { + StreamGroup(String name, + Ingestor ingestor, + Chooser chooser, + TimestampExtractor timestampExtractor, + int desiredUnprocessedPerPartition) { this.name = name; this.ingestor = ingestor; this.chooser = chooser; @@ -67,12 +71,38 @@ public class StreamSynchronizer implements SyncGroup { ======= } - @Override public String name() { return name; >>>>>>> removed some generics } + /** + * Merges a stream group into this group + */ + public void mergeStreamGroup(StreamGroup other) { + // check these groups have the same ingestor + if (!this.ingestor.equals(other.ingestor)) + throw new IllegalArgumentException("groups with different ingestors cannot be merged"); + + // check these group have the same chooser and time extractor types + if (!this.chooser.getClass().equals(other.chooser.getClass())) + throw new IllegalArgumentException("groups with different type of choosers cannot be merged"); + + if (!this.timestampExtractor.getClass().equals(other.timestampExtractor.getClass())) + throw new IllegalArgumentException("groups with different type of time extractors cannot be merged"); + + // add all the other's groups partitions + for (TopicPartition partition : other.stash.keySet()) { + this.stash.put(partition, other.stash.get(partition)); + this.consumedOffsets.put(partition, other.consumedOffsets.get(partition)); + } + + // add all the other's buffered records + for (NewRecords records : other.newRecordBuffer) { + this.newRecordBuffer.addLast(records); + } + } + /** * Adds a partition and its receiver to this stream synchronizer * @param partition the partition diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index 1f7f95026ffb5..27597b87915a4 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -14,7 +14,7 @@ public class KStreamBranchTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamGroup streamGroup = new StreamGroup( "group", ingestor, new TimeBasedChooser(), @@ -28,7 +28,7 @@ public long extract(String topic, Object key, Object value) { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); @SuppressWarnings("unchecked") @Test diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 0c26f2517afde..844d014f6d250 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -13,7 +13,7 @@ public class KStreamFilterTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamGroup streamGroup = new StreamGroup( "group", ingestor, new TimeBasedChooser(), @@ -27,7 +27,7 @@ public long extract(String topic, Object key, Object value) { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); private Predicate isMultipleOfThree = new Predicate() { @Override diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 579d891217bd4..1553e193ede3b 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -14,7 +14,7 @@ public class KStreamFlatMapTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamGroup streamGroup = new StreamGroup( "group", ingestor, new TimeBasedChooser(), @@ -28,7 +28,7 @@ public long extract(String topic, Object key, Object value) { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testFlatMap() { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 8e274e8181d7f..3a85f0f9eccff 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -14,7 +14,7 @@ public class KStreamFlatMapValuesTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamGroup streamGroup = new StreamGroup( "group", ingestor, new TimeBasedChooser(), @@ -28,7 +28,7 @@ public long extract(String topic, Object key, Object value) { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 92e59dbdd4603..8ad53b0365995 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -17,7 +17,7 @@ public class KStreamJoinTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamGroup streamGroup = new StreamGroup( "group", ingestor, new TimeBasedChooser(), @@ -31,7 +31,7 @@ public long extract(String topic, Object key, Object value) { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); private ValueJoiner joiner = new ValueJoiner() { @Override diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index c7d258534520d..03d7db227de8b 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -13,7 +13,7 @@ public class KStreamMapTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamGroup streamGroup = new StreamGroup( "group", ingestor, new TimeBasedChooser(), @@ -27,7 +27,7 @@ public long extract(String topic, Object key, Object value) { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testMap() { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 81a68b547e4fc..9c540a0968285 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -1,6 +1,5 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.SyncGroup; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.ValueMapper; import io.confluent.streaming.testutil.MockIngestor; @@ -15,7 +14,7 @@ public class KStreamMapValuesTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamGroup streamGroup = new StreamGroup( "group", ingestor, new TimeBasedChooser(), @@ -29,7 +28,7 @@ public long extract(String topic, Object key, Object value) { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testFlatMapValues() { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 46b0cf5209e41..2267c3f084ec7 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -14,7 +14,7 @@ public class KStreamSourceTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamGroup streamGroup = new StreamGroup( "group", ingestor, new TimeBasedChooser(), @@ -28,7 +28,7 @@ public long extract(String topic, Object key, Object value) { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testKStreamSource() { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index ad5f3a591a7be..a85e43034a8e4 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -14,7 +14,7 @@ public class KStreamWindowedTest { private Ingestor ingestor = new MockIngestor(); - private StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + private StreamGroup streamGroup = new StreamGroup( "group", ingestor, new TimeBasedChooser(), @@ -28,7 +28,7 @@ public long extract(String topic, Object key, Object value) { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamSynchronizer, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testWindowedStream() { diff --git a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java b/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java index bc3b16fd5fd1e..b0aaae73817e4 100644 --- a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java @@ -5,7 +5,6 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StorageEngine; -import io.confluent.streaming.SyncGroup; import io.confluent.streaming.Coordinator; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; @@ -45,15 +44,9 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer @Override public KStream from(String... topic) { throw new UnsupportedOperationException("from() not supported."); } - @Override - public KStream from(SyncGroup syncGroup, String... topic) { throw new UnsupportedOperationException("from() not supported."); } - @Override public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topic) { throw new UnsupportedOperationException("from() not supported."); } - @Override - public KStream from(SyncGroup syncGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topic) { throw new UnsupportedOperationException("from() not supported."); } - @Override public RecordCollector recordCollector() { throw new UnsupportedOperationException("recordCollector() not supported."); } @@ -70,10 +63,10 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer public Metrics metrics() { throw new UnsupportedOperationException("metrics() not supported."); } @Override - public SyncGroup syncGroup(String name) { throw new UnsupportedOperationException("syncGroup() not supported."); } + public StreamGroup streamGroup(String name) { throw new UnsupportedOperationException("streamGroup() not supported."); } @Override - public SyncGroup roundRobinSyncGroup(String name) { throw new UnsupportedOperationException("roundRobinSyncGroup() not supported."); } + public StreamGroup roundRobinStreamGroup(String name) { throw new UnsupportedOperationException("roundRobinStreamGroup() not supported."); } @Override public void restore(StorageEngine engine) throws Exception { throw new UnsupportedOperationException("restore() not supported."); } diff --git a/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java similarity index 87% rename from src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java rename to src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index 7912192913f6e..2fc827af72d35 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamSynchronizerTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -18,7 +18,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -public class StreamSynchronizerTest { +public class StreamGroupTest { private static Serializer serializer = new IntegerSerializer(); private static Deserializer deserializer = new IntegerDeserializer(); @@ -51,7 +51,7 @@ public void testAddPartition() { MockIngestor mockIngestor = new MockIngestor(); - StreamSynchronizer streamSynchronizer = new StreamSynchronizer( + StreamGroup streamGroup = new StreamGroup( "group", mockIngestor, new TimeBasedChooser(), @@ -72,15 +72,15 @@ public long extract(String topic, Object key, Object value) { MockKStreamImpl stream2 = new MockKStreamImpl(); MockKStreamImpl stream3 = new MockKStreamImpl(); - streamSynchronizer.addPartition(partition1, stream1); - mockIngestor.addStreamSynchronizerForPartition(streamSynchronizer, partition1); + streamGroup.addPartition(partition1, stream1); + mockIngestor.addPartitionStreamToGroup(streamGroup, partition1); - streamSynchronizer.addPartition(partition2, stream2); - mockIngestor.addStreamSynchronizerForPartition(streamSynchronizer, partition2); + streamGroup.addPartition(partition2, stream2); + mockIngestor.addPartitionStreamToGroup(streamGroup, partition2); Exception exception = null; try { - streamSynchronizer.addPartition(partition1, stream3); + streamGroup.addPartition(partition1, stream3); } catch (Exception ex) { exception = ex; } @@ -100,7 +100,7 @@ public long extract(String topic, Object key, Object value) { new ConsumerRecord(partition2.topic(), partition2.partition(), 4, serializer.serialize(partition1.topic(), new Integer(600)), recordValue) )); - streamSynchronizer.process(); + streamGroup.process(); assertEquals(stream1.numReceived, 1); assertEquals(stream2.numReceived, 0); @@ -113,7 +113,7 @@ public long extract(String topic, Object key, Object value) { new ConsumerRecord(partition1.topic(), partition1.partition(), 5, serializer.serialize(partition1.topic(), new Integer(50)), recordValue) )); - streamSynchronizer.process(); + streamGroup.process(); assertEquals(stream1.numReceived, 2); assertEquals(stream2.numReceived, 0); @@ -121,44 +121,44 @@ public long extract(String topic, Object key, Object value) { assertTrue(mockIngestor.paused.contains(partition1)); assertTrue(mockIngestor.paused.contains(partition2)); - streamSynchronizer.process(); + streamGroup.process(); assertEquals(stream1.numReceived, 3); assertEquals(stream2.numReceived, 0); - streamSynchronizer.process(); + streamGroup.process(); assertEquals(stream1.numReceived, 3); assertEquals(stream2.numReceived, 1); assertEquals(mockIngestor.paused.size(), 1); assertTrue(mockIngestor.paused.contains(partition2)); - streamSynchronizer.process(); + streamGroup.process(); assertEquals(stream1.numReceived, 4); assertEquals(stream2.numReceived, 1); assertEquals(mockIngestor.paused.size(), 1); - streamSynchronizer.process(); + streamGroup.process(); assertEquals(stream1.numReceived, 4); assertEquals(stream2.numReceived, 2); assertEquals(mockIngestor.paused.size(), 0); - streamSynchronizer.process(); + streamGroup.process(); assertEquals(stream1.numReceived, 5); assertEquals(stream2.numReceived, 2); - streamSynchronizer.process(); + streamGroup.process(); assertEquals(stream1.numReceived, 5); assertEquals(stream2.numReceived, 3); - streamSynchronizer.process(); + streamGroup.process(); assertEquals(stream1.numReceived, 5); assertEquals(stream2.numReceived, 4); assertEquals(mockIngestor.paused.size(), 0); - streamSynchronizer.process(); + streamGroup.process(); assertEquals(stream1.numReceived, 5); assertEquals(stream2.numReceived, 4); } diff --git a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java index c079e98d85b34..59a2ec9b4b2db 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java +++ b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java @@ -1,7 +1,7 @@ package io.confluent.streaming.testutil; import io.confluent.streaming.internal.Ingestor; -import io.confluent.streaming.internal.StreamSynchronizer; +import io.confluent.streaming.internal.StreamGroup; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; @@ -10,7 +10,7 @@ public class MockIngestor implements Ingestor { - private HashMap streamSynchronizers = new HashMap<>(); + private HashMap streamSynchronizers = new HashMap<>(); public HashSet paused = new HashSet<>(); @@ -38,8 +38,8 @@ public int numPartitions(String topic) { } @Override - public void addStreamSynchronizerForPartition(StreamSynchronizer streamSynchronizer, TopicPartition partition) { - streamSynchronizers.put(partition, streamSynchronizer); + public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition) { + streamSynchronizers.put(partition, streamGroup); } public void addRecords(TopicPartition partition, Iterable> records) { From aa5943d9f38a6bdb8531feb17a9a362697c4249d Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 20 Jul 2015 17:06:36 -0700 Subject: [PATCH 065/275] lint --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index 7b4ae64a314a8..f4baf9888eb10 100644 --- a/pom.xml +++ b/pom.xml @@ -52,6 +52,7 @@ 1.7 1.7 + -Xlint:all From 25fdbde6f0e084859a183a1767a8fdd2392f4817 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 21 Jul 2015 16:48:36 -0700 Subject: [PATCH 066/275] new Processor API --- .../confluent/streaming/KafkaStreaming.java | 37 ++++++++++--- .../io/confluent/streaming/Processor.java | 18 +++++-- .../confluent/streaming/RecordCollector.java | 3 +- .../streaming/examples/PrintProcessor.java | 22 ++++---- .../streaming/internal/Ingestor.java | 4 ++ .../streaming/internal/IngestorImpl.java | 8 +++ .../streaming/internal/KStreamBranch.java | 4 +- .../streaming/internal/KStreamFilter.java | 4 +- .../streaming/internal/KStreamFlatMap.java | 4 +- .../internal/KStreamFlatMapValues.java | 4 +- .../streaming/internal/KStreamImpl.java | 44 +++++++-------- .../streaming/internal/KStreamJoin.java | 6 +-- .../streaming/internal/KStreamMap.java | 4 +- .../streaming/internal/KStreamMapValues.java | 4 +- .../streaming/internal/KStreamMetadata.java | 3 +- .../streaming/internal/KStreamSource.java | 4 +- .../internal/KStreamWindowedImpl.java | 4 +- .../internal/ProcessorContextImpl.java | 53 +++++++++++++++++++ .../streaming/internal/Receiver.java | 2 +- .../streaming/internal/StampedRecord.java | 9 ++-- .../streaming/internal/StreamGroup.java | 19 ++++++- .../streaming/internal/KStreamBranchTest.java | 4 +- .../streaming/internal/KStreamFilterTest.java | 4 +- .../internal/KStreamFlatMapTest.java | 2 +- .../internal/KStreamFlatMapValuesTest.java | 2 +- .../streaming/internal/KStreamJoinTest.java | 16 +++--- .../streaming/internal/KStreamMapTest.java | 2 +- .../internal/KStreamMapValuesTest.java | 2 +- .../streaming/internal/KStreamSourceTest.java | 2 +- .../internal/KStreamWindowedTest.java | 4 +- .../PunctuationSchedulerImplTest.java | 10 ++-- .../streaming/internal/StreamGroupTest.java | 2 +- .../streaming/testutil/MockIngestor.java | 4 ++ .../streaming/testutil/TestProcessor.java | 4 +- 34 files changed, 214 insertions(+), 104 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index f018620f2a250..af3f556cde28f 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -357,9 +357,22 @@ private void commitAll(long now) { } } - producer.flush(); - consumer.commit(commit, CommitType.SYNC); // TODO: can this be async? - streamingMetrics.commitTime.record(time.milliseconds() - lastCommit); + // check if commit is really needed, i.e. if all the offsets are already committed + boolean commitNeeded = false; + for (TopicPartition tp : commit.keySet()) { + if (consumer.committed(tp) != commit.get(tp)) { + commitNeeded = true; + break; + } + } + + if (commitNeeded) { + // TODO: for exactly-once we need to make sure the flush and commit + // are executed atomically whenever it is triggered by user + producer.flush(); + consumer.commit(commit, CommitType.SYNC); // TODO: can this be async? + streamingMetrics.commitTime.record(time.milliseconds() - lastCommit); + } } private void commitRequesting(long now) { @@ -372,9 +385,21 @@ private void commitRequesting(long now) { commit.putAll(streamGroup.consumedOffsets()); // TODO: can this be async? } } - consumer.commit(commit, CommitType.SYNC); - requestingCommit.clear(); - streamingMetrics.commitTime.record(time.milliseconds() - now); + + // check if commit is really needed, i.e. if all the offsets are already committed + boolean commitNeeded = false; + for (TopicPartition tp : commit.keySet()) { + if (consumer.committed(tp) != commit.get(tp)) { + commitNeeded = true; + break; + } + } + + if (commitNeeded) { + consumer.commit(commit, CommitType.SYNC); // TODO: can this be async? + requestingCommit.clear(); + streamingMetrics.commitTime.record(time.milliseconds() - now); + } } /* delete any state dirs that aren't for active contexts */ diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 52f76ef0d1fab..4bf17fbf8ed15 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -1,14 +1,26 @@ package io.confluent.streaming; +import org.apache.kafka.common.serialization.Serializer; + /** * Created by yasuhiro on 6/17/15. */ public interface Processor { - void process(String topic, K key, V value, RecordCollector collector, Coordinator coordinator); + public interface ProcessorContext { - void init(PunctuationScheduler punctuationScheduler); + void send(String topic, Object key, Object value); - void punctuate(long streamTime); + void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); + + void schedule(long timestamp); + + void commit(); + } + void init(ProcessorContext context); + + void process(K key, V value); + + void punctuate(long streamTime); } diff --git a/src/main/java/io/confluent/streaming/RecordCollector.java b/src/main/java/io/confluent/streaming/RecordCollector.java index 36104d705c5c2..b711675e9b68b 100644 --- a/src/main/java/io/confluent/streaming/RecordCollector.java +++ b/src/main/java/io/confluent/streaming/RecordCollector.java @@ -8,5 +8,4 @@ public interface RecordCollector { void send(ProducerRecord record); - -} +} \ No newline at end of file diff --git a/src/main/java/io/confluent/streaming/examples/PrintProcessor.java b/src/main/java/io/confluent/streaming/examples/PrintProcessor.java index 89728032f897f..139f0a264bf0c 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintProcessor.java +++ b/src/main/java/io/confluent/streaming/examples/PrintProcessor.java @@ -1,13 +1,9 @@ package io.confluent.streaming.examples; -import io.confluent.streaming.Coordinator; import io.confluent.streaming.KafkaStreaming; import io.confluent.streaming.Processor; import io.confluent.streaming.ProcessorKStreamJob; -import io.confluent.streaming.PunctuationScheduler; -import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StreamingConfig; -import org.apache.kafka.clients.producer.ProducerRecord; import java.util.Properties; @@ -16,18 +12,20 @@ */ public class PrintProcessor implements Processor { - @Override - public void process(String topic, K key, V value, RecordCollector collector, Coordinator coordinator) { - System.out.println(topic + ": [" + key + ", " + value + "]"); - - coordinator.commit(Coordinator.RequestScope.CURRENT_TASK); + private ProcessorContext context; - collector.send(new ProducerRecord<>("topic", key, value)); + @Override + public void init(ProcessorContext context) { + this.context = context; } @Override - public void init(PunctuationScheduler punctuationScheduler) { - // do nothing + public void process(K key, V value) { + System.out.println("[" + key + ", " + value + "]"); + + context.commit(); + + context.send("topic", key, value); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/Ingestor.java b/src/main/java/io/confluent/streaming/internal/Ingestor.java index 8f62eba04ad6f..d92782c2cd71f 100644 --- a/src/main/java/io/confluent/streaming/internal/Ingestor.java +++ b/src/main/java/io/confluent/streaming/internal/Ingestor.java @@ -2,6 +2,8 @@ import org.apache.kafka.common.TopicPartition; +import java.util.Map; + /** * Created by yasuhiro on 6/30/15. */ @@ -15,6 +17,8 @@ public interface Ingestor { void unpause(TopicPartition partition, long offset); + void commit(Map offsets); + int numPartitions(String topic); void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition); diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 6eff2567291ad..f58e5983a1171 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -1,5 +1,6 @@ package io.confluent.streaming.internal; +import org.apache.kafka.clients.consumer.CommitType; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.TopicPartition; @@ -75,6 +76,13 @@ public void unpause(TopicPartition partition, long lastOffset) { } } + @Override + public void commit(Map offsets) { + synchronized (this) { + consumer.commit(offsets, CommitType.SYNC); + } + } + @Override public int numPartitions(String topic) { return consumer.partitionsFor(topic).size(); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 92824f2978a75..1c67097a07984 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -24,12 +24,12 @@ class KStreamBranch implements Receiver { } @Override - public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { for (int i = 0; i < predicates.length; i++) { Predicate predicate = predicates[i]; if (predicate.apply((K)key, (V)value)) { - branches[i].receive(topic, key, value, timestamp, streamTime); + branches[i].receive(key, value, timestamp, streamTime); return; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 9112028eefa9a..13c155dc3ac89 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -16,10 +16,10 @@ class KStreamFilter extends KStreamImpl { } @Override - public void receive(String topic, Object key, Object value, long timestamp,long streamTime) { + public void receive(Object key, Object value, long timestamp,long streamTime) { synchronized(this) { if (predicate.apply((K)key, (V)value)) { - forward(topic, key, value, timestamp, streamTime); + forward(key, value, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 979bc740e7b89..ad29c43ca7f61 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -17,11 +17,11 @@ class KStreamFlatMap extends KStreamImpl { } @Override - public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { KeyValue> newPair = mapper.apply((K1)key, (V1)value); for (V v : newPair.value) { - forward(KStreamMetadata.UNKNOWN_TOPICNAME, newPair.key, v, timestamp, streamTime); + forward(newPair.key, v, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 3c013b9ef2eed..e9222cfe461c6 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -16,11 +16,11 @@ class KStreamFlatMapValues extends KStreamImpl { } @Override - public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { Iterable newValues = mapper.apply((V1)value); for (V v : newValues) { - forward(KStreamMetadata.UNKNOWN_TOPICNAME, key, v, timestamp, streamTime); + forward(key, v, timestamp, streamTime); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index cfbe5ddd25b0f..f4c9e0c254149 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -11,7 +11,6 @@ import io.confluent.streaming.RecordCollector; import io.confluent.streaming.ValueMapper; import io.confluent.streaming.Window; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; @@ -102,26 +101,25 @@ public void sendTo(String topic) { } @Override - public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); } + public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { + process(this.getSendProcessor(topic, keySerializer, valSerializer)); + } @SuppressWarnings("unchecked") - private Processor getSendProcessor(final String sendTopic, Serializer keySerializer, Serializer valSerializer) { - final RecordCollector collector; - if (keySerializer == null && valSerializer == null) - collector = (RecordCollector) context.recordCollector(); - else - collector = (RecordCollector) new RecordCollectors.SerializingRecordCollector( - ((KStreamContextImpl)context).simpleRecordCollector(), - keySerializer == null ? context.keySerializer() : keySerializer, - valSerializer == null ? context.valueSerializer() : valSerializer); - + private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { return new Processor() { + private ProcessorContext processorContext; + @Override - public void process(String topic, K key, V value, RecordCollector dummyCollector, Coordinator coordinator) { - collector.send(new ProducerRecord<>(sendTopic, key, value)); + public void init(ProcessorContext processorContext) { + this.processorContext = processorContext; } @Override - public void init(PunctuationScheduler scheduler) {} + public void process(K key, V value) { + this.processorContext.send(sendTopic, key, value, + (Serializer) keySerializer, + (Serializer) valSerializer); + } @Override public void punctuate(long streamTime) {} }; @@ -129,28 +127,24 @@ public void punctuate(long streamTime) {} @Override public void process(final Processor processor) { + processor.init(new ProcessorContextImpl((KStreamContextImpl) this.context, this.metadata.streamGroup, this.metadata.streamGroup.getPunctuationScheduler(processor))); + Receiver receiver = new Receiver() { - public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { - if (topic.equals(KStreamMetadata.UNKNOWN_TOPICNAME)) - processor.process(null, (K) key, (V) value, (RecordCollector) context.recordCollector(), context.coordinator()); - else - processor.process(topic, (K) key, (V) value, (RecordCollector) context.recordCollector(), context.coordinator()); + public void receive(Object key, Object value, long timestamp, long streamTime) { + processor.process((K) key, (V) value); } }; registerReceiver(receiver); - - PunctuationScheduler scheduler = (metadata.streamGroup).getPunctuationScheduler(processor); - processor.init(scheduler); } void registerReceiver(Receiver receiver) { nextReceivers.add(receiver); } - protected void forward(String topic, Object key, Object value, long timestamp, long streamTime) { + protected void forward(Object key, Object value, long timestamp, long streamTime) { int numReceivers = nextReceivers.size(); for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).receive(topic, key, value, timestamp, streamTime); + nextReceivers.get(i).receive(key, value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index 5493f8732dc89..8d724e72a9571 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -54,7 +54,7 @@ Iterator find(K key, long timestamp) { } @Override - public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { Iterator iter = finder2.find((K)key, timestamp); if (iter != null) { while (iter.hasNext()) { @@ -67,7 +67,7 @@ private Receiver getReceiverForOther() { return new Receiver() { @Override - public void receive(String topic, Object key, Object value2, long timestamp, long streamTime) { + public void receive(Object key, Object value2, long timestamp, long streamTime) { Iterator iter = finder1.find((K)key, timestamp); if (iter != null) { while (iter.hasNext()) { @@ -80,7 +80,7 @@ public void receive(String topic, Object key, Object value2, long timestamp, lon // TODO: use the "outer-stream" topic as the resulted join stream topic private void doJoin(K key, V1 value1, V2 value2, long timestamp, long streamTime) { - forward(KStreamMetadata.UNKNOWN_TOPICNAME, key, joiner.apply(value1, value2), timestamp, streamTime); + forward(key, joiner.apply(value1, value2), timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index a9bb7d1bedc64..753d808576490 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -17,10 +17,10 @@ class KStreamMap extends KStreamImpl { } @Override - public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized (this) { KeyValue newPair = mapper.apply((K1)key, (V1)value); - forward(KStreamMetadata.UNKNOWN_TOPICNAME, newPair.key, newPair.value, timestamp, streamTime); + forward(newPair.key, newPair.value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index b5708b3ca096a..4f9f257ea0869 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -16,10 +16,10 @@ class KStreamMapValues extends KStreamImpl { } @Override - public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized (this) { V newValue = mapper.apply((V1)value); - forward(KStreamMetadata.UNKNOWN_TOPICNAME, key, newValue, timestamp, streamTime); + forward(key, newValue, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java b/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java index e35ae5db68e4f..8520dffb4758a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java @@ -9,9 +9,10 @@ public class KStreamMetadata { public static String UNKNOWN_TOPICNAME = "__UNKNOWN_TOPIC__"; + public static int UNKNOWN_PARTITION = -1; public static KStreamMetadata unjoinable(StreamGroup streamGroup) { - return new KStreamMetadata(streamGroup, Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(-1))); + return new KStreamMetadata(streamGroup, Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(UNKNOWN_PARTITION))); } public StreamGroup streamGroup; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 958e2f1e76cb8..04a53c2a5e194 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -14,10 +14,10 @@ class KStreamSource extends KStreamImpl { } @Override - public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { // KStream needs to forward the topic name since it is directly from the Kafka source - forward(topic, key, value, timestamp, streamTime); + forward(key, value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 89385ac7c7964..09b0d9cdaa735 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -21,11 +21,11 @@ public class KStreamWindowedImpl extends KStreamImpl implements KStr } @Override - public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { window.put((K)key, (V)value, timestamp); // KStreamWindowed needs to forward the topic name since it may receive directly from KStreamSource - forward(topic, key, value, timestamp, streamTime); + forward(key, value, timestamp, streamTime); } } diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java new file mode 100644 index 0000000000000..32a24995d3af9 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java @@ -0,0 +1,53 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.Processor; +import io.confluent.streaming.PunctuationScheduler; +import io.confluent.streaming.RecordCollector; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serializer; + +/** + * Created by guozhang on 7/21/15. + */ +public class ProcessorContextImpl implements Processor.ProcessorContext { + + private final KStreamContextImpl context; + private final StreamGroup streamGroup; + private final PunctuationScheduler scheduler; + + public ProcessorContextImpl(KStreamContextImpl context, + StreamGroup streamGroup, + PunctuationScheduler scheduler) { + + this.context = context; + this.scheduler = scheduler; + this.streamGroup = streamGroup; + } + + @Override + public void send(String topic, Object key, Object value) { + this.context.recordCollector().send(new ProducerRecord(topic, key, value)); + } + + @Override + public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { + if (keySerializer == null || valSerializer == null) + throw new IllegalStateException("key and value serializers must be specified"); + + final RecordCollector collector = new RecordCollectors.SerializingRecordCollector( + context.simpleRecordCollector(), keySerializer, valSerializer); + + collector.send(new ProducerRecord(topic, key, value)); + } + + @Override + public void commit() { + this.streamGroup.commitOffset(); + } + + @Override + public void schedule(long timestamp) { + scheduler.schedule(timestamp); + } +} diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 2c2332068a51d..1cad1f810acdf 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -5,6 +5,6 @@ */ public interface Receiver { - void receive(String topic, Object key, Object value, long timestamp, long streamTime); + void receive(Object key, Object value, long timestamp, long streamTime); } diff --git a/src/main/java/io/confluent/streaming/internal/StampedRecord.java b/src/main/java/io/confluent/streaming/internal/StampedRecord.java index 86926491bafa3..8f4de9c0b6668 100644 --- a/src/main/java/io/confluent/streaming/internal/StampedRecord.java +++ b/src/main/java/io/confluent/streaming/internal/StampedRecord.java @@ -11,12 +11,9 @@ public class StampedRecord extends Stamped> { super(record, timestamp); } - public String topic() { - if (value.topic().equals(KStreamMetadata.UNKNOWN_TOPICNAME)) - return null; - else - return value.topic(); - } + public String topic() { return value.topic(); } + + public int partition() { return value.partition(); } public Object key() { return value.key(); diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index 5e828f08282a3..7effaba4b61aa 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -10,6 +10,7 @@ import org.apache.kafka.common.serialization.Deserializer; import java.util.ArrayDeque; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -32,7 +33,7 @@ public class StreamSynchronizer implements SyncGroup { public class StreamGroup implements ParallelExecutor.Task { >>>>>>> remove SyncGroup from user facing APIs:src/main/java/io/confluent/streaming/internal/StreamGroup.java - public final String name; + private final String name; private final Ingestor ingestor; private final Chooser chooser; private final TimestampExtractor timestampExtractor; @@ -46,6 +47,7 @@ public class StreamGroup implements ParallelExecutor.Task { private final ArrayDeque> newRecordBuffer = new ArrayDeque<>(); private long streamTime = -1; + private boolean commitRequested = false; private volatile int buffered = 0; /** @@ -230,9 +232,15 @@ public void process() { if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - recordQueue.stream.receive(record.topic(), record.key(), record.value(), record.timestamp, streamTime); + recordQueue.stream.receive(record.key(), record.value(), record.timestamp, streamTime); consumedOffsets.put(recordQueue.partition(), record.offset()); + // TODO: local state flush and downstream producer flush + // need to be done altogether with offset commit atomically + if (commitRequested) ingestor.commit(Collections.singletonMap( + new TopicPartition(record.topic(), record.partition()), + record.offset())); + if (recordQueue.size() > 0) chooser.add(recordQueue); buffered--; @@ -249,6 +257,13 @@ public Map consumedOffsets() { return this.consumedOffsets; } + /** + * Request committing the current record's offset + */ + public void commitOffset() { + this.commitRequested = true; + } + public int buffered() { return buffered; } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index 27597b87915a4..d40d659c8f0ff 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -71,7 +71,7 @@ public boolean apply(Integer key, String value) { } for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(3, processors[0].processed.size()); @@ -90,7 +90,7 @@ public boolean apply(Integer key, String value) { } for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(3, processors[0].processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 844d014f6d250..18d60d3ada2aa 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -48,7 +48,7 @@ public void testFilter() { stream.filter(isMultipleOfThree).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(2, processor.processed.size()); @@ -66,7 +66,7 @@ public void testFilterOut() { stream.filterOut(isMultipleOfThree).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(5, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 1553e193ede3b..5367979414d12 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -55,7 +55,7 @@ public KeyValue> apply(Integer key, String value) { stream.flatMap(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(6, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 3a85f0f9eccff..09f064f2f5680 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -55,7 +55,7 @@ public Iterable apply(String value) { stream.flatMapValues(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(8, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 8ad53b0365995..eff28fe7f2172 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -101,7 +101,7 @@ public void testJoin() { // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - stream1.receive(topicName, expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); } assertEquals(0, processor.processed.size()); @@ -109,7 +109,7 @@ public void testJoin() { // push two items to the other stream. the main stream's window has two items for (int i = 0; i < 2; i++) { - stream2.receive(topicName, expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); } assertEquals(2, processor.processed.size()); @@ -125,7 +125,7 @@ public void testJoin() { // push all items to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(topicName, expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); } assertEquals(2, processor.processed.size()); @@ -142,7 +142,7 @@ public void testJoin() { // push all items to the other stream. this should produce 6 items for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(topicName, expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); } assertEquals(6, processor.processed.size()); @@ -185,7 +185,7 @@ public void testJoinPrior() { // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - stream1.receive(topicName, expectedKeys[i], "X" + expectedKeys[i], i, 0L); + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i, 0L); } assertEquals(0, processor.processed.size()); @@ -194,7 +194,7 @@ public void testJoinPrior() { // no corresponding item in the main window has a newer timestamp for (int i = 0; i < 2; i++) { - stream2.receive(topicName, expectedKeys[i], "Y" + expectedKeys[i], i + 1, 0L); + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i + 1, 0L); } assertEquals(0, processor.processed.size()); @@ -204,7 +204,7 @@ public void testJoinPrior() { // push all items with newer timestamps to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(topicName, expectedKeys[i], "X" + expectedKeys[i], i + 2, 0L); + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i + 2, 0L); } assertEquals(2, processor.processed.size()); @@ -221,7 +221,7 @@ public void testJoinPrior() { // push all items with older timestamps to the other stream. this should produce six items for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(topicName, expectedKeys[i], "Y" + expectedKeys[i], i, 0L); + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i, 0L); } assertEquals(6, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 03d7db227de8b..e8ca7a8d20182 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -50,7 +50,7 @@ public KeyValue apply(Integer key, String value) { stream.map(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(4, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 9c540a0968285..f977f4892a6d8 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -51,7 +51,7 @@ public Integer apply(String value) { stream.mapValues(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(topicName, expectedKeys[i], Integer.toString(expectedKeys[i]), 0L, 0L); + stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L, 0L); } assertEquals(4, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 2267c3f084ec7..e3b59bd3940b1 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -42,7 +42,7 @@ public void testKStreamSource() { final String[] expectedValues = new String[] { "v1", "v2", "v3" }; for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(topicName, expectedKeys[i], expectedValues[i], 0L, 0L); + stream.receive(expectedKeys[i], expectedValues[i], 0L, 0L); } assertEquals(3, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index a85e43034a8e4..2f4f0a474098e 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -48,7 +48,7 @@ public void testWindowedStream() { // two items in the window for (int i = 0; i < 2; i++) { - stream.receive(topicName, expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } assertEquals(1, countItem(window.find(0, 0L))); @@ -59,7 +59,7 @@ public void testWindowedStream() { // previous two items + all items, thus two are duplicates, in the window for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(topicName, expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); } assertEquals(2, countItem(window.find(0, 0L))); diff --git a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java index 94358aed7887d..69409c016551c 100644 --- a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java +++ b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java @@ -22,8 +22,8 @@ public void testScheduling() { assertEquals(0, proc1.punctuated.size()); assertEquals(0, proc2.punctuated.size()); - proc1.init(sched1); - proc2.init(sched2); + proc1.init(new ProcessorContextImpl(null, null, sched1)); + proc2.init(new ProcessorContextImpl(null, null, sched2)); sched1.schedule(500); sched2.schedule(1000); @@ -70,8 +70,8 @@ public void testCanceling() { assertEquals(0, proc1.punctuated.size()); assertEquals(0, proc2.punctuated.size()); - proc1.init(sched1); - proc2.init(sched2); + proc1.init(new ProcessorContextImpl(null, null, sched1)); + proc2.init(new ProcessorContextImpl(null, null, sched2)); sched1.schedule(500); sched2.schedule(1000); @@ -107,7 +107,7 @@ public void testDuplicateScheduling() { assertEquals(0, proc1.punctuated.size()); - proc1.init(sched1); + proc1.init(new ProcessorContextImpl(null, null, sched1)); sched1.schedule(500); diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index 2fc827af72d35..775f67623baa5 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -36,7 +36,7 @@ public MockKStreamImpl() { } @Override - public void receive(String topic, Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp, long streamTime) { this.numReceived++; this.keys.add(key); this.values.add(value); diff --git a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java index 59a2ec9b4b2db..13ccbaefc46f1 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java +++ b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java @@ -7,6 +7,7 @@ import java.util.HashMap; import java.util.HashSet; +import java.util.Map; public class MockIngestor implements Ingestor { @@ -32,6 +33,9 @@ public void unpause(TopicPartition partition, long offset) { paused.remove(partition); } + @Override + public void commit(Map offsets) { /* do nothing */} + @Override public int numPartitions(String topic) { return 1; diff --git a/src/test/java/io/confluent/streaming/testutil/TestProcessor.java b/src/test/java/io/confluent/streaming/testutil/TestProcessor.java index 83cebfad26aaf..8a69e2b36b1e7 100644 --- a/src/test/java/io/confluent/streaming/testutil/TestProcessor.java +++ b/src/test/java/io/confluent/streaming/testutil/TestProcessor.java @@ -10,12 +10,12 @@ public class TestProcessor implements Processor { public final ArrayList punctuated = new ArrayList(); @Override - public void apply(String topic, K key, V value) { + public void process(K key, V value) { processed.add(key + ":" + value); } @Override - public void init(PunctuationScheduler punctuationScheduler) { + public void init(ProcessorContext context) { } @Override From 3258ae05d96fa600c7f60fa42ea8607afb6f8de7 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 21 Jul 2015 16:58:00 -0700 Subject: [PATCH 067/275] do not clone kstream context --- .../streaming/internal/KStreamBranch.java | 3 +- .../internal/KStreamContextImpl.java | 28 ++++++------------- .../streaming/internal/KStreamFilter.java | 1 + .../streaming/internal/KStreamFlatMap.java | 1 + .../internal/KStreamFlatMapValues.java | 1 + .../streaming/internal/KStreamImpl.java | 1 + .../streaming/internal/KStreamJoin.java | 4 ++- .../streaming/internal/KStreamMap.java | 1 + .../streaming/internal/KStreamMapValues.java | 1 + .../streaming/internal/KStreamSource.java | 13 +++++++-- .../internal/KStreamWindowedImpl.java | 1 + .../streaming/internal/RecordQueue.java | 6 +++- .../streaming/internal/StreamGroup.java | 8 +++--- .../streaming/util/ParallelExecutor.java | 2 +- .../streaming/internal/StreamGroupTest.java | 10 +++---- 15 files changed, 47 insertions(+), 34 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 1c67097a07984..2474f82422d7e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -19,10 +19,11 @@ class KStreamBranch implements Receiver { this.predicates = Arrays.copyOf(predicates, predicates.length); this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { - branches[i] = new KStreamSource(streamMetadata, context); + branches[i] = new KStreamSource<>(streamMetadata, context); } } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index cf7981d5505d7..bffcbd333af45 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -149,7 +149,7 @@ private String getNextGroupName() { private KStream from(StreamGroup streamGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { if (streamGroup == null) throw new IllegalArgumentException("unspecified stream group"); - KStreamSource stream = null; + KStreamSource stream = null; Set fromTopics; synchronized (this) { @@ -165,7 +165,7 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe if (!this.topics.contains(topic)) throw new IllegalArgumentException("topic not subscribed: " + topic); - KStreamSource streamForTopic = sourceStreams.get(topic); + KStreamSource streamForTopic = (KStreamSource) sourceStreams.get(topic); if (stream == null) { if (streamForTopic != null) @@ -198,22 +198,12 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, topicPartitionInfos); // override the deserializer classes if specified - if (keyDeserializer == null && valDeserializer == null) { - stream = new KStreamSource(streamMetadata, this); - } else { - StreamingConfig newConfig = this.streamingConfig.clone(); - if (keyDeserializer != null) - newConfig.keyDeserializer(keyDeserializer); - if (valDeserializer != null) - newConfig.valueDeserializer(valDeserializer); - - KStreamContextImpl newContext = new KStreamContextImpl( - this.id, this.job, this.topics, this.ingestor, - this.simpleCollector, this.coordinator, - newConfig, this.processorConfig, - this.stateMgr, this.metrics); - stream = new KStreamSource(streamMetadata, newContext); - } + stream = new KStreamSource<>( + streamMetadata, + this, + (Deserializer) (keyDeserializer == null ? keyDeserializer() : keyDeserializer), + (Deserializer) (valDeserializer == null ? valueDeserializer() : valDeserializer) + ); // update source stream map for (String topic : fromTopics) { @@ -236,7 +226,7 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe throw new IllegalStateException("another source stream with the same topic but different value deserializer is already created"); } - return (KStream) stream; + return stream; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 13c155dc3ac89..80eba04e525a1 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -15,6 +15,7 @@ class KStreamFilter extends KStreamImpl { this.predicate = predicate; } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp,long streamTime) { synchronized(this) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index ad29c43ca7f61..77d356af10953 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -16,6 +16,7 @@ class KStreamFlatMap extends KStreamImpl { this.mapper = mapper; } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index e9222cfe461c6..9f6585ac32ffb 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -15,6 +15,7 @@ class KStreamFlatMapValues extends KStreamImpl { this.mapper = mapper; } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index f4c9e0c254149..9f3b9ff44b0f5 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -125,6 +125,7 @@ public void punctuate(long streamTime) {} }; } + @SuppressWarnings("unchecked") @Override public void process(final Processor processor) { processor.init(new ProcessorContextImpl((KStreamContextImpl) this.context, this.metadata.streamGroup, this.metadata.streamGroup.getPunctuationScheduler(processor))); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index 8d724e72a9571..c0b20b8bd089c 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -53,6 +53,7 @@ Iterator find(K key, long timestamp) { this.receiverForOtherStream = getReceiverForOther(); } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { Iterator iter = finder2.find((K)key, timestamp); @@ -63,9 +64,10 @@ public void receive(Object key, Object value, long timestamp, long streamTime) { } } - private Receiver getReceiverForOther() { + private Receiver getReceiverForOther() { return new Receiver() { + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value2, long timestamp, long streamTime) { Iterator iter = finder1.find((K)key, timestamp); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index 753d808576490..6a6797a7415ed 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -16,6 +16,7 @@ class KStreamMap extends KStreamImpl { this.mapper = mapper; } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized (this) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index 4f9f257ea0869..c76eaee8db921 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -15,6 +15,7 @@ class KStreamMapValues extends KStreamImpl { this.mapper = mapper; } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized (this) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 04a53c2a5e194..ca79ffe978a3d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -1,16 +1,25 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; - -import java.util.List; +import org.apache.kafka.common.serialization.Deserializer; /** * Created by yasuhiro on 6/17/15. */ class KStreamSource extends KStreamImpl { + public final Deserializer keyDeserializer; + public final Deserializer valueDeserializer; + KStreamSource(KStreamMetadata streamMetadata, KStreamContext context) { + this(streamMetadata, context, null, null); + } + + KStreamSource(KStreamMetadata streamMetadata, KStreamContext context, Deserializer < K > keyDeserializer, Deserializer < V > valueDeserializer) { super(streamMetadata, context); + + this.keyDeserializer = keyDeserializer; + this.valueDeserializer = valueDeserializer; } @Override diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 09b0d9cdaa735..3070a6fb647d0 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -20,6 +20,7 @@ public class KStreamWindowedImpl extends KStreamImpl implements KStr this.window = window; } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/src/main/java/io/confluent/streaming/internal/RecordQueue.java index 90bf81be37abb..f05deb7dcb1ca 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/src/main/java/io/confluent/streaming/internal/RecordQueue.java @@ -13,7 +13,7 @@ public class RecordQueue { private final ArrayDeque queue = new ArrayDeque<>(); - public final KStreamImpl stream; + public final KStreamSource stream; private final TopicPartition partition; private TimestampTracker> timestampTracker; private long offset; @@ -32,11 +32,15 @@ public RecordQueue(TopicPartition partition, KStreamSource source, TimestampTrac >>>>>>> more fix * @param timestampTracker TimestampTracker */ +<<<<<<< HEAD public RecordQueue(TopicPartition partition, Receiver receiver, TimestampTracker> timestampTracker) { >>>>>>> javadoc ======= public RecordQueue(TopicPartition partition, KStreamImpl stream, TimestampTracker> timestampTracker) { >>>>>>> extend sendTo and through with ser-de overriden enabled +======= + public RecordQueue(TopicPartition partition, KStreamSource stream, TimestampTracker> timestampTracker) { +>>>>>>> do not clone kstream context this.partition = partition; this.stream = stream; this.timestampTracker = timestampTracker; diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index 7effaba4b61aa..035118f71afde 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -111,7 +111,7 @@ public void mergeStreamGroup(StreamGroup other) { * @param stream the instance of KStreamImpl */ @SuppressWarnings("unchecked") - public void addPartition(TopicPartition partition, KStreamImpl stream) { + public void addPartition(TopicPartition partition, KStreamSource stream) { synchronized (this) { RecordQueue recordQueue = stash.get(partition); @@ -173,8 +173,8 @@ private void ingestNewRecords() { ConsumerRecord record = iterator.next(); // deserialize the raw record, extract the timestamp and put into the queue - Deserializer keyDeserializer = recordQueue.stream.context().keyDeserializer(); - Deserializer valDeserializer = recordQueue.stream.context().valueDeserializer(); + Deserializer keyDeserializer = recordQueue.stream.keyDeserializer; + Deserializer valDeserializer = recordQueue.stream.valueDeserializer; Object key = keyDeserializer.deserialize(record.topic(), record.key()); Object value = valDeserializer.deserialize(record.topic(), record.value()); @@ -273,7 +273,7 @@ public void close() { stash.clear(); } - protected RecordQueue createRecordQueue(TopicPartition partition, KStreamImpl stream) { + protected RecordQueue createRecordQueue(TopicPartition partition, KStreamSource stream) { return new RecordQueue(partition, stream, new MinTimestampTracker>()); } diff --git a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java index 25f5c0b02989b..8c12c7791bae3 100644 --- a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java +++ b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java @@ -22,7 +22,7 @@ public interface Task { private final WorkerThread[] workerThreads; private final AtomicInteger taskIndex = new AtomicInteger(0); - private volatile ArrayList tasks = new ArrayList(); + private volatile ArrayList tasks = new ArrayList<>(); private volatile CountDownLatch latch; private volatile boolean running = true; private volatile Exception exception; diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index 775f67623baa5..283e032a2b6e2 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -23,7 +23,7 @@ public class StreamGroupTest { private static Serializer serializer = new IntegerSerializer(); private static Deserializer deserializer = new IntegerDeserializer(); - private static class MockKStreamImpl extends KStreamImpl { + private static class MockKStreamSource extends KStreamSource { public int numReceived = 0; public ArrayList keys = new ArrayList<>(); @@ -31,7 +31,7 @@ private static class MockKStreamImpl extends KStreamImpl { public ArrayList timestamps = new ArrayList<>(); public ArrayList streamTimes = new ArrayList<>(); - public MockKStreamImpl() { + public MockKStreamSource() { super(null, new MockKStreamContext(serializer, deserializer)); } @@ -68,9 +68,9 @@ public long extract(String topic, Object key, Object value) { TopicPartition partition1 = new TopicPartition("topic1", 1); TopicPartition partition2 = new TopicPartition("topic2", 1); - MockKStreamImpl stream1 = new MockKStreamImpl(); - MockKStreamImpl stream2 = new MockKStreamImpl(); - MockKStreamImpl stream3 = new MockKStreamImpl(); + MockKStreamSource stream1 = new MockKStreamSource(); + MockKStreamSource stream2 = new MockKStreamSource(); + MockKStreamSource stream3 = new MockKStreamSource(); streamGroup.addPartition(partition1, stream1); mockIngestor.addPartitionStreamToGroup(streamGroup, partition1); From 64ad8472d761c03d1ae90c173135da2a9df5634d Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 21 Jul 2015 18:02:00 -0700 Subject: [PATCH 068/275] fix indent --- src/main/java/io/confluent/streaming/internal/KStreamJoin.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index c0b20b8bd089c..023e51da7c262 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -64,7 +64,7 @@ public void receive(Object key, Object value, long timestamp, long streamTime) { } } - private Receiver getReceiverForOther() { + private Receiver getReceiverForOther() { return new Receiver() { @SuppressWarnings("unchecked") From 0f174c45a48a88d6edc9d38de2a37c6ab5fa670b Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 22 Jul 2015 10:45:56 -0700 Subject: [PATCH 069/275] RecordCollector api --- .../confluent/streaming/KStreamContext.java | 4 +- .../confluent/streaming/RecordCollector.java | 10 ++- .../io/confluent/streaming/StorageEngine.java | 2 +- .../internal/KStreamContextImpl.java | 38 ++------- .../streaming/internal/KStreamImpl.java | 2 +- .../streaming/internal/KStreamSource.java | 5 +- .../internal/ProcessorContextImpl.java | 9 +-- .../internal/ProcessorStateManager.java | 8 +- .../internal/RecordCollectorImpl.java | 67 ++++++++++++++++ .../streaming/internal/RecordCollectors.java | 77 ------------------- .../streaming/kv/InMemoryKeyValueStore.java | 10 +-- .../kv/internals/MeteredKeyValueStore.java | 2 +- .../streaming/internal/KStreamBranchTest.java | 9 ++- .../streaming/internal/KStreamFilterTest.java | 8 +- .../internal/KStreamFlatMapTest.java | 4 +- .../internal/KStreamFlatMapValuesTest.java | 4 +- .../streaming/internal/KStreamJoinTest.java | 36 ++++----- .../streaming/internal/KStreamMapTest.java | 4 +- .../internal/KStreamMapValuesTest.java | 4 +- .../streaming/internal/KStreamSourceTest.java | 4 +- .../internal/KStreamWindowedTest.java | 4 +- .../internal/MockKStreamContext.java | 2 +- .../PunctuationSchedulerImplTest.java | 4 +- 23 files changed, 143 insertions(+), 174 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java delete mode 100644 src/main/java/io/confluent/streaming/internal/RecordCollectors.java diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 0394be1876f41..b8ba1c2a50001 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -70,10 +70,10 @@ public interface KStreamContext { KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); /** - * Returns a RecordCollector which applies the serializer to key and value. + * Returns a RecordCollector * @return RecordCollector */ - RecordCollector recordCollector(); + RecordCollector recordCollector(); /** * Returns {@link Coordinator}. diff --git a/src/main/java/io/confluent/streaming/RecordCollector.java b/src/main/java/io/confluent/streaming/RecordCollector.java index b711675e9b68b..7a0e9d7be0241 100644 --- a/src/main/java/io/confluent/streaming/RecordCollector.java +++ b/src/main/java/io/confluent/streaming/RecordCollector.java @@ -1,11 +1,15 @@ package io.confluent.streaming; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serializer; /** * Created by yasuhiro on 6/19/15. */ -public interface RecordCollector { +public interface RecordCollector { - void send(ProducerRecord record); -} \ No newline at end of file + void send(ProducerRecord record); + + void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer); + +} diff --git a/src/main/java/io/confluent/streaming/StorageEngine.java b/src/main/java/io/confluent/streaming/StorageEngine.java index 0f6ab6e3e2f21..1dab83aa4850a 100644 --- a/src/main/java/io/confluent/streaming/StorageEngine.java +++ b/src/main/java/io/confluent/streaming/StorageEngine.java @@ -47,7 +47,7 @@ public interface StorageEngine { * @param checkpointedOffset The offset of the last save * @param logEndOffset The last offset in the changelog */ - void registerAndRestore(RecordCollector collector, + void registerAndRestore(RecordCollector collector, Consumer consumer, TopicPartition partition, long checkpointedOffset, diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index bffcbd333af45..c69c6c1264c1e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -40,8 +40,7 @@ public class KStreamContextImpl implements KStreamContext { private final Set topics; private final Ingestor ingestor; - private final RecordCollectors.SimpleRecordCollector simpleCollector; - private final RecordCollector collector; + private final RecordCollectorImpl collector; private final Coordinator coordinator; private final HashMap> sourceStreams = new HashMap<>(); @@ -55,42 +54,23 @@ public class KStreamContextImpl implements KStreamContext { private final ProcessorStateManager stateMgr; private Consumer restoreConsumer; - public KStreamContextImpl(int id, - KStreamJob job, - Set topics, - Ingestor ingestor, - Producer producer, - Coordinator coordinator, - StreamingConfig streamingConfig, - ProcessorConfig processorConfig, - Metrics metrics) { - - this(id, job, topics, ingestor, - new RecordCollectors.SimpleRecordCollector(producer), - coordinator, streamingConfig, processorConfig, - new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id))), - metrics); - } - @SuppressWarnings("unchecked") public KStreamContextImpl(int id, KStreamJob job, Set topics, Ingestor ingestor, - RecordCollectors.SimpleRecordCollector simpleCollector, + Producer producer, Coordinator coordinator, StreamingConfig streamingConfig, ProcessorConfig processorConfig, - ProcessorStateManager stateMgr, Metrics metrics) { this.id = id; this.job = job; this.topics = topics; this.ingestor = ingestor; - this.simpleCollector = simpleCollector; - this.collector = new RecordCollectors.SerializingRecordCollector( - simpleCollector, streamingConfig.keySerializer(), streamingConfig.valueSerializer()); + this.collector = + new RecordCollectorImpl(producer, (Serializer)streamingConfig.keySerializer(), (Serializer)streamingConfig.valueSerializer()); this.coordinator = coordinator; this.streamingConfig = streamingConfig; @@ -99,13 +79,11 @@ public KStreamContextImpl(int id, this.timestampExtractor = this.streamingConfig.timestampExtractor(); if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractor is missing"); - this.stateMgr = stateMgr; + this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id))); this.stateDir = this.stateMgr.baseDir(); this.metrics = metrics; } - public RecordCollectors.SimpleRecordCollector simpleRecordCollector() { return this.simpleCollector; } - @Override public int id() { return id; @@ -231,7 +209,7 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe } @Override - public RecordCollector recordCollector() { + public RecordCollector recordCollector() { return collector; } @@ -284,7 +262,7 @@ private StreamGroup streamGroup(String name, Chooser chooser) { public void restore(StorageEngine engine) throws Exception { if (restoreConsumer == null) throw new IllegalStateException(); - stateMgr.registerAndRestore(simpleCollector, restoreConsumer, engine); + stateMgr.registerAndRestore(collector, restoreConsumer, engine); } @@ -317,7 +295,7 @@ public void flush() { } public void close() throws Exception { - stateMgr.close(simpleCollector.offsets()); + stateMgr.close(collector.offsets()); job.close(); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 9f3b9ff44b0f5..25261ec361298 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -128,7 +128,7 @@ public void punctuate(long streamTime) {} @SuppressWarnings("unchecked") @Override public void process(final Processor processor) { - processor.init(new ProcessorContextImpl((KStreamContextImpl) this.context, this.metadata.streamGroup, this.metadata.streamGroup.getPunctuationScheduler(processor))); + processor.init(new ProcessorContextImpl(this.context, this.metadata.streamGroup, this.metadata.streamGroup.getPunctuationScheduler(processor))); Receiver receiver = new Receiver() { public void receive(Object key, Object value, long timestamp, long streamTime) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index ca79ffe978a3d..55562f06c6a3c 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -11,11 +11,12 @@ class KStreamSource extends KStreamImpl { public final Deserializer keyDeserializer; public final Deserializer valueDeserializer; + @SuppressWarnings("unchecked") KStreamSource(KStreamMetadata streamMetadata, KStreamContext context) { - this(streamMetadata, context, null, null); + this(streamMetadata, context, (Deserializer) context.keyDeserializer(), (Deserializer) context.valueDeserializer()); } - KStreamSource(KStreamMetadata streamMetadata, KStreamContext context, Deserializer < K > keyDeserializer, Deserializer < V > valueDeserializer) { + KStreamSource(KStreamMetadata streamMetadata, KStreamContext context, Deserializer keyDeserializer, Deserializer valueDeserializer) { super(streamMetadata, context); this.keyDeserializer = keyDeserializer; diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java index 32a24995d3af9..d31be6397f76c 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java @@ -12,11 +12,11 @@ */ public class ProcessorContextImpl implements Processor.ProcessorContext { - private final KStreamContextImpl context; + private final KStreamContext context; private final StreamGroup streamGroup; private final PunctuationScheduler scheduler; - public ProcessorContextImpl(KStreamContextImpl context, + public ProcessorContextImpl(KStreamContext context, StreamGroup streamGroup, PunctuationScheduler scheduler) { @@ -35,10 +35,7 @@ public void send(String topic, Object key, Object value, Serializer keyS if (keySerializer == null || valSerializer == null) throw new IllegalStateException("key and value serializers must be specified"); - final RecordCollector collector = new RecordCollectors.SerializingRecordCollector( - context.simpleRecordCollector(), keySerializer, valSerializer); - - collector.send(new ProducerRecord(topic, key, value)); + context.recordCollector().send(new ProducerRecord(topic, key, value), keySerializer, valSerializer); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index d416fe8f26bfa..a50195216c44a 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -43,9 +43,9 @@ public class ProcessorStateManager { public ProcessorStateManager(int id, File baseDir) { this.id = id; this.baseDir = baseDir; - this.stores = new HashMap(); - this.checkpointedOffsets = new HashMap(); - this.restoredOffsets = new HashMap(); + this.stores = new HashMap<>(); + this.checkpointedOffsets = new HashMap<>(); + this.restoredOffsets = new HashMap<>(); } public File baseDir() { @@ -58,7 +58,7 @@ public void init() throws IOException { checkpoint.delete(); } - public void registerAndRestore(RecordCollector collector, Consumer consumer, StorageEngine engine) { + public void registerAndRestore(RecordCollector collector, Consumer consumer, StorageEngine engine) { if (engine.name().equals(CHECKPOINT_FILE_NAME)) throw new IllegalArgumentException("Illegal store name: " + CHECKPOINT_FILE_NAME); diff --git a/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java b/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java new file mode 100644 index 0000000000000..21317eaeb9780 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java @@ -0,0 +1,67 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.RecordCollector; +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; + +/** + * Created by yasuhiro on 6/26/15. + */ +public class RecordCollectorImpl implements RecordCollector { + + private static final Logger log = LoggerFactory.getLogger(RecordCollectorImpl.class); + + private final Producer producer; + private final Map offsets; + private final Callback callback = new Callback(){ + public void onCompletion(RecordMetadata metadata, Exception exception) { + if(exception == null) { + TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); + offsets.put(tp, metadata.offset()); + } else { + log.error("Error sending record: ", exception); + } + } + }; + private final Serializer keySerializer; + private final Serializer valueSerializer; + + + public RecordCollectorImpl(Producer producer, Serializer keySerializer, Serializer valueSerializer) { + this.producer = producer; + this.offsets = new HashMap<>(); + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + } + + @Override + public void send(ProducerRecord record) { + 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); + } + + @Override + public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); + byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); + this.producer.send(new ProducerRecord<>(record.topic(), keyBytes, valBytes), callback); + } + + /** + * The last ack'd offset from the producer + * @return the map from TopicPartition to offset + */ + public Map offsets() { + return this.offsets; + } +} diff --git a/src/main/java/io/confluent/streaming/internal/RecordCollectors.java b/src/main/java/io/confluent/streaming/internal/RecordCollectors.java deleted file mode 100644 index ab0162ea84c5a..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/RecordCollectors.java +++ /dev/null @@ -1,77 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.RecordCollector; -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; - -/** - * Created by yasuhiro on 6/26/15. - */ -public class RecordCollectors { - - private static final Logger log = LoggerFactory.getLogger(RecordCollectors.class); - - public static class SimpleRecordCollector implements RecordCollector { - - private final Producer producer; - private final Map offsets; - private final Callback callback = new Callback(){ - public void onCompletion(RecordMetadata metadata, Exception exception) { - if(exception == null) { - TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); - offsets.put(tp, metadata.offset()); - } else { - log.error("Error sending record: ", exception); - } - } - }; - - public SimpleRecordCollector(Producer producer) { - this.producer = producer; - this.offsets = new HashMap(); - } - - public void send(ProducerRecord record) { - // TODO: need to compute partition - this.producer.send(record, callback); - } - - /** - * The last ack'd offset from the producer - * @return the map from TopicPartition to offset - */ - public Map offsets() { - return this.offsets; - } - } - - public static class SerializingRecordCollector implements RecordCollector { - - private final RecordCollector collector; - private final Serializer keySerializer; - private final Serializer valueSerializer; - - public SerializingRecordCollector(RecordCollector collector, Serializer keySerializer, Serializer valueSerializer) { - super(); - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - this.collector = collector; - } - - public void send(ProducerRecord record) { - byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); - byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); - collector.send(new ProducerRecord(record.topic(), keyBytes, valBytes)); - } - } - -} diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index 8a51f61ebfc3a..7f863bf229cf9 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -37,7 +37,7 @@ private static class MemoryStore implements KeyValueStore, StorageEn private final Serializer valueSerializer; private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; - private RecordCollector collector; + private RecordCollector collector; @SuppressWarnings("unchecked") public MemoryStore(String name, KStreamContext context) { @@ -95,7 +95,7 @@ public KeyValueIterator all() { } @Override - public void registerAndRestore(RecordCollector collector, + public void registerAndRestore(RecordCollector collector, Consumer consumer, TopicPartition partition, long checkpointedOffset, @@ -118,9 +118,7 @@ public void flush() { if(this.collector != null) { for (K k : this.dirty) { V v = this.store.get(k); - byte[] key = this.keySerializer.serialize(this.topic, k); - byte[] value = this.valueSerializer.serialize(this.topic, v); - this.collector.send(new ProducerRecord(this.topic, this.partition, key, value)); + this.collector.send(new ProducerRecord<>(this.topic, this.partition, k, v), this.keySerializer, this.valueSerializer); } this.dirty.clear(); } @@ -146,7 +144,7 @@ public boolean hasNext() { @Override public Entry next() { Map.Entry entry = iter.next(); - return new Entry(entry.getKey(), entry.getValue()); + return new Entry<>(entry.getKey(), entry.getValue()); } @Override diff --git a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java index e1543b0803d80..c1cbd2bcd76a5 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java @@ -73,7 +73,7 @@ public String name() { } @Override - public void registerAndRestore(RecordCollector collector, + public void registerAndRestore(RecordCollector collector, Consumer consumer, TopicPartition partition, long checkpointedOffset, diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index d40d659c8f0ff..4321c813c45e4 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -55,18 +55,19 @@ public boolean apply(Integer key, String value) { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; + KStreamContext context = new MockKStreamContext(null, null); KStreamSource stream; KStream[] branches; TestProcessor[] processors; - stream = new KStreamSource(streamMetadata, null); + stream = new KStreamSource<>(streamMetadata, context); branches = stream.branch(isEven, isMultipleOfThree, isOdd); assertEquals(3, branches.length); processors = (TestProcessor[]) Array.newInstance(TestProcessor.class, branches.length); for (int i = 0; i < branches.length; i++) { - processors[i] = new TestProcessor(); + processors[i] = new TestProcessor<>(); branches[i].process(processors[i]); } @@ -78,14 +79,14 @@ public boolean apply(Integer key, String value) { assertEquals(1, processors[1].processed.size()); assertEquals(3, processors[2].processed.size()); - stream = new KStreamSource(streamMetadata, null); + stream = new KStreamSource<>(streamMetadata, context); branches = stream.branch(isEven, isOdd, isMultipleOfThree); assertEquals(3, branches.length); processors = (TestProcessor[]) Array.newInstance(TestProcessor.class, branches.length); for (int i = 0; i < branches.length; i++) { - processors[i] = new TestProcessor(); + processors[i] = new TestProcessor<>(); branches[i].process(processors[i]); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 18d60d3ada2aa..332bf0dd11ee5 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -43,8 +43,8 @@ public void testFilter() { KStreamSource stream; TestProcessor processor; - processor = new TestProcessor(); - stream = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream = new KStreamSource<>(streamMetadata, null, null, null); stream.filter(isMultipleOfThree).process(processor); for (int i = 0; i < expectedKeys.length; i++) { @@ -61,8 +61,8 @@ public void testFilterOut() { KStreamSource stream; TestProcessor processor; - processor = new TestProcessor(); - stream = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream = new KStreamSource<>(streamMetadata, null, null, null); stream.filterOut(isMultipleOfThree).process(processor); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 5367979414d12..a882148bfad0f 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -50,8 +50,8 @@ public KeyValue> apply(Integer key, String value) { KStreamSource stream; TestProcessor processor; - processor = new TestProcessor(); - stream = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream = new KStreamSource<>(streamMetadata, null, null, null); stream.flatMap(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 09f064f2f5680..2069102766198 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -50,8 +50,8 @@ public Iterable apply(String value) { KStreamSource stream; TestProcessor processor; - processor = new TestProcessor(); - stream = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream = new KStreamSource<>(streamMetadata, null, null, null); stream.flatMapValues(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index eff28fe7f2172..d3c64fd26795a 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -82,9 +82,9 @@ public void testJoin() { TestProcessor processor; String[] expected; - processor = new TestProcessor(); - stream1 = new KStreamSource(streamMetadata, null); - stream2 = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream1 = new KStreamSource<>(streamMetadata, null, null, null); + stream2 = new KStreamSource<>(streamMetadata, null, null, null); windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -166,9 +166,9 @@ public void testJoinPrior() { TestProcessor processor; String[] expected; - processor = new TestProcessor(); - stream1 = new KStreamSource(streamMetadata, null); - stream2 = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream1 = new KStreamSource<>(streamMetadata, null, null, null); + stream2 = new KStreamSource<>(streamMetadata, null, null, null); windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -244,9 +244,9 @@ public void testMap() { KStreamWindowed windowed2; TestProcessor processor; - processor = new TestProcessor(); - stream1 = new KStreamSource(streamMetadata, null); - stream2 = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream1 = new KStreamSource<>(streamMetadata, null, null, null); + stream2 = new KStreamSource<>(streamMetadata, null, null, null); mapped1 = stream1.map(keyValueMapper); mapped2 = stream2.map(keyValueMapper); @@ -299,9 +299,9 @@ public void testFlatMap() { KStreamWindowed windowed2; TestProcessor processor; - processor = new TestProcessor(); - stream1 = new KStreamSource(streamMetadata, null); - stream2 = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream1 = new KStreamSource<>(streamMetadata, null, null, null); + stream2 = new KStreamSource<>(streamMetadata, null, null, null); mapped1 = stream1.flatMap(keyValueMapper2); mapped2 = stream2.flatMap(keyValueMapper2); @@ -354,9 +354,9 @@ public void testMapValues() { KStreamWindowed windowed2; TestProcessor processor; - processor = new TestProcessor(); - stream1 = new KStreamSource(streamMetadata, null); - stream2 = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream1 = new KStreamSource<>(streamMetadata, null, null, null); + stream2 = new KStreamSource<>(streamMetadata, null, null, null); mapped1 = stream1.mapValues(valueMapper); mapped2 = stream2.mapValues(valueMapper); @@ -409,9 +409,9 @@ public void testFlatMapValues() { KStreamWindowed windowed2; TestProcessor processor; - processor = new TestProcessor(); - stream1 = new KStreamSource(streamMetadata, null); - stream2 = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream1 = new KStreamSource<>(streamMetadata, null, null, null); + stream2 = new KStreamSource<>(streamMetadata, null, null, null); mapped1 = stream1.flatMapValues(valueMapper2); mapped2 = stream2.flatMapValues(valueMapper2); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index e8ca7a8d20182..90f1eeceb72ac 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -45,8 +45,8 @@ public KeyValue apply(Integer key, String value) { KStreamSource stream; TestProcessor processor; - processor = new TestProcessor(); - stream = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream = new KStreamSource<>(streamMetadata, null, null, null); stream.map(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index f977f4892a6d8..e7d532b35587e 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -46,8 +46,8 @@ public Integer apply(String value) { KStreamSource stream; TestProcessor processor; - processor = new TestProcessor(); - stream = new KStreamSource(streamMetadata, null); + processor = new TestProcessor<>(); + stream = new KStreamSource<>(streamMetadata, null, null, null); stream.mapValues(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index e3b59bd3940b1..7ee576893255d 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -33,9 +33,9 @@ public long extract(String topic, Object key, Object value) { @Test public void testKStreamSource() { - TestProcessor processor = new TestProcessor(); + TestProcessor processor = new TestProcessor<>(); - KStreamSource stream = new KStreamSource(streamMetadata, null); + KStreamSource stream = new KStreamSource<>(streamMetadata, null, null, null); stream.process(processor); final String[] expectedKeys = new String[] { "k1", "k2", "k3" }; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 2f4f0a474098e..024d2847d5924 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -39,8 +39,8 @@ public void testWindowedStream() { Window window; String[] expected; - window = new UnlimitedWindow(); - stream = new KStreamSource(streamMetadata, null); + window = new UnlimitedWindow<>(); + stream = new KStreamSource<>(streamMetadata, null, null, null); stream.with(window); boolean exceptionRaised = false; diff --git a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java b/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java index b0aaae73817e4..334dc696b5a2a 100644 --- a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java @@ -48,7 +48,7 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topic) { throw new UnsupportedOperationException("from() not supported."); } @Override - public RecordCollector recordCollector() { throw new UnsupportedOperationException("recordCollector() not supported."); } + public RecordCollector recordCollector() { throw new UnsupportedOperationException("recordCollector() not supported."); } @Override public Coordinator coordinator() { throw new UnsupportedOperationException("coordinator() not supported."); } diff --git a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java index 69409c016551c..2d497c9072810 100644 --- a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java +++ b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java @@ -13,10 +13,10 @@ public class PunctuationSchedulerImplTest { public void testScheduling() { PunctuationQueue queue = new PunctuationQueue(); - TestProcessor proc1 = new TestProcessor(); + TestProcessor proc1 = new TestProcessor<>(); PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); - TestProcessor proc2 = new TestProcessor(); + TestProcessor proc2 = new TestProcessor<>(); PunctuationScheduler sched2 = new PunctuationSchedulerImpl(queue, proc2); assertEquals(0, proc1.punctuated.size()); From 4b20d923d2478728bf9a864c1ff28e8d474391b1 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 22 Jul 2015 12:43:55 -0700 Subject: [PATCH 070/275] send(record) delegates to send(record, keySerializer, valSerializer) --- .../io/confluent/streaming/internal/RecordCollectorImpl.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java b/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java index 21317eaeb9780..8e7d227655067 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java @@ -45,9 +45,7 @@ public RecordCollectorImpl(Producer producer, Serializer @Override public void send(ProducerRecord record) { - 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); + send(record, this.keySerializer, this.valueSerializer); } @Override From 72bf9f72e9a4fcacda30930b84322ab418a575e5 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 22 Jul 2015 13:24:21 -0700 Subject: [PATCH 071/275] Small changes on the ProcessorKStreamJob --- .../streaming/ProcessorKStreamJob.java | 2 +- .../streaming/examples/PrintKStreamJob.java | 39 ++++++++++++ .../streaming/examples/PrintProcessor.java | 60 ------------------- 3 files changed, 40 insertions(+), 61 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java delete mode 100644 src/main/java/io/confluent/streaming/examples/PrintProcessor.java diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java index fbb0d5e292a40..d0adb8edfc2e1 100644 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -5,7 +5,7 @@ /** * Created by guozhang on 7/14/15. */ -public class ProcessorKStreamJob implements KStreamJob { +public abstract class ProcessorKStreamJob implements KStreamJob, Processor { public static String PROCESSOR_CLASSNAME = "__PROCESSOR_CLASSNAME__"; diff --git a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java new file mode 100644 index 0000000000000..9428f5ada2527 --- /dev/null +++ b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java @@ -0,0 +1,39 @@ +package io.confluent.streaming.examples; + +import io.confluent.streaming.KafkaStreaming; +import io.confluent.streaming.ProcessorKStreamJob; +import io.confluent.streaming.StreamingConfig; + +import java.util.Properties; + +/** + * Created by guozhang on 7/14/15. + */ +public class PrintKStreamJob extends ProcessorKStreamJob { + + private ProcessorContext context; + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + @Override + public void process(K key, V value) { + System.out.println("[" + key + ", " + value + "]"); + + context.commit(); + + context.send("topic", key, value); + } + + @Override + public void punctuate(long streamTime) { + // do nothing + } + + public static void main(String[] args) { + KafkaStreaming kstream = new KafkaStreaming(PrintKStreamJob.class, new StreamingConfig(new Properties())); + kstream.run(); + } +} diff --git a/src/main/java/io/confluent/streaming/examples/PrintProcessor.java b/src/main/java/io/confluent/streaming/examples/PrintProcessor.java deleted file mode 100644 index 139f0a264bf0c..0000000000000 --- a/src/main/java/io/confluent/streaming/examples/PrintProcessor.java +++ /dev/null @@ -1,60 +0,0 @@ -package io.confluent.streaming.examples; - -import io.confluent.streaming.KafkaStreaming; -import io.confluent.streaming.Processor; -import io.confluent.streaming.ProcessorKStreamJob; -import io.confluent.streaming.StreamingConfig; - -import java.util.Properties; - -/** - * Created by guozhang on 7/14/15. - */ -public class PrintProcessor implements Processor { - - private ProcessorContext context; - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void process(K key, V value) { - System.out.println("[" + key + ", " + value + "]"); - - context.commit(); - - context.send("topic", key, value); - } - - @Override - public void punctuate(long streamTime) { - // do nothing - } - - public static void main(String[] args) { - // put the Processor class into the context configs - StreamingConfig configs = new StreamingConfig(new Properties()); - configs.addContextObject(ProcessorKStreamJob.PROCESSOR_CLASSNAME, PrintProcessor.class); - - KafkaStreaming kstream = new KafkaStreaming(ProcessorKStreamJob.class, configs); - kstream.run(); - - /* - * another possible way is to relax the KafkaStreaming job class type, from - * - - KafkaStreaming(Class job ..) - - to - - KafkaStreaming(Class job ..) - - so that we can skip the KStreamJob wrapper around the processor: - - KafkaStreaming kstream = new KafkaStreaming(PrintProcessor.class, configs); - - */ - } -} From 87d3a8c7dbe826c873cd5c727834e2be6b12c98f Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 22 Jul 2015 14:19:13 -0700 Subject: [PATCH 072/275] compiler warning --- .../internal/ProcessorContextImpl.java | 4 +-- .../streaming/internal/StreamGroupTest.java | 25 ++++++++++--------- 2 files changed, 15 insertions(+), 14 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java index d31be6397f76c..e19da24cf07c2 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java @@ -27,7 +27,7 @@ public ProcessorContextImpl(KStreamContext context, @Override public void send(String topic, Object key, Object value) { - this.context.recordCollector().send(new ProducerRecord(topic, key, value)); + this.context.recordCollector().send(new ProducerRecord<>(topic, key, value)); } @Override @@ -35,7 +35,7 @@ public void send(String topic, Object key, Object value, Serializer keyS if (keySerializer == null || valSerializer == null) throw new IllegalStateException("key and value serializers must be specified"); - context.recordCollector().send(new ProducerRecord(topic, key, value), keySerializer, valSerializer); + context.recordCollector().send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); } @Override diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index 283e032a2b6e2..fcd57a08145dd 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -20,8 +20,8 @@ public class StreamGroupTest { - private static Serializer serializer = new IntegerSerializer(); - private static Deserializer deserializer = new IntegerDeserializer(); + private static Serializer serializer = new IntegerSerializer(); + private static Deserializer deserializer = new IntegerDeserializer(); private static class MockKStreamSource extends KStreamSource { @@ -46,6 +46,7 @@ public void receive(Object key, Object value, long timestamp, long streamTime) { } + @SuppressWarnings("unchecked") @Test public void testAddPartition() { @@ -89,15 +90,15 @@ public long extract(String topic, Object key, Object value) { byte[] recordValue = serializer.serialize(null, new Integer(10)); mockIngestor.addRecords(partition1, records( - new ConsumerRecord(partition1.topic(), partition1.partition(), 1, serializer.serialize(partition1.topic(), new Integer(10)), recordValue), - new ConsumerRecord(partition1.topic(), partition1.partition(), 2, serializer.serialize(partition1.topic(), new Integer(20)), recordValue) + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 1, serializer.serialize(partition1.topic(), new Integer(10)), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 2, serializer.serialize(partition1.topic(), new Integer(20)), recordValue) )); mockIngestor.addRecords(partition2, records( - new ConsumerRecord(partition2.topic(), partition2.partition(), 1, serializer.serialize(partition1.topic(), new Integer(300)), recordValue), - new ConsumerRecord(partition2.topic(), partition2.partition(), 2, serializer.serialize(partition1.topic(), new Integer(400)), recordValue), - new ConsumerRecord(partition2.topic(), partition2.partition(), 3, serializer.serialize(partition1.topic(), new Integer(500)), recordValue), - new ConsumerRecord(partition2.topic(), partition2.partition(), 4, serializer.serialize(partition1.topic(), new Integer(600)), recordValue) + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 1, serializer.serialize(partition1.topic(), new Integer(300)), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 2, serializer.serialize(partition1.topic(), new Integer(400)), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 3, serializer.serialize(partition1.topic(), new Integer(500)), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 4, serializer.serialize(partition1.topic(), new Integer(600)), recordValue) )); streamGroup.process(); @@ -108,9 +109,9 @@ public long extract(String topic, Object key, Object value) { assertTrue(mockIngestor.paused.contains(partition2)); mockIngestor.addRecords(partition1, records( - new ConsumerRecord(partition1.topic(), partition1.partition(), 3, serializer.serialize(partition1.topic(), new Integer(30)), recordValue), - new ConsumerRecord(partition1.topic(), partition1.partition(), 4, serializer.serialize(partition1.topic(), new Integer(40)), recordValue), - new ConsumerRecord(partition1.topic(), partition1.partition(), 5, serializer.serialize(partition1.topic(), new Integer(50)), recordValue) + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 3, serializer.serialize(partition1.topic(), new Integer(30)), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 4, serializer.serialize(partition1.topic(), new Integer(40)), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 5, serializer.serialize(partition1.topic(), new Integer(50)), recordValue) )); streamGroup.process(); @@ -163,7 +164,7 @@ public long extract(String topic, Object key, Object value) { assertEquals(stream2.numReceived, 4); } - private List> records(ConsumerRecord... recs) { + private Iterable> records(ConsumerRecord... recs) { return Arrays.asList(recs); } } From 27be1009b4337b94a3ff8aac63a7fecc28acc603 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 22 Jul 2015 14:32:11 -0700 Subject: [PATCH 073/275] minor fixes --- .../java/io/confluent/streaming/ProcessorKStreamJob.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java index d0adb8edfc2e1..0c89d60bfb14e 100644 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -7,14 +7,10 @@ */ public abstract class ProcessorKStreamJob implements KStreamJob, Processor { - public static String PROCESSOR_CLASSNAME = "__PROCESSOR_CLASSNAME__"; - @SuppressWarnings("unchecked") @Override public void init(KStreamContext context) { - Processor processor = (Processor) Utils.newInstance((Class) context.getContext().get(PROCESSOR_CLASSNAME)); - - context.from(null).process(processor); + context.from(null).process((Processor) this); } @Override From 8ec18bf25a7651cdb7783bf648c4944d2e0ceab0 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 22 Jul 2015 16:04:01 -0700 Subject: [PATCH 074/275] fix commit behavior; add record metadata to processor context --- .../confluent/streaming/KStreamContext.java | 4 +++ .../io/confluent/streaming/Processor.java | 8 ++++++ .../confluent/streaming/RecordCollector.java | 1 + .../internal/KStreamContextImpl.java | 1 + .../streaming/internal/KStreamImpl.java | 3 --- .../internal/ProcessorContextImpl.java | 21 +++++++++++++++- .../internal/RecordCollectorImpl.java | 5 ++++ .../streaming/internal/StreamGroup.java | 25 +++++++++++++++---- .../internal/MockKStreamContext.java | 3 +++ 9 files changed, 62 insertions(+), 9 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index b8ba1c2a50001..006d7fec1c1a5 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -120,4 +120,8 @@ public interface KStreamContext { */ void restore(StorageEngine engine) throws Exception; + /** + * Flush the local state of this context + */ + void flush(); } diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 4bf17fbf8ed15..dccc39fff4939 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -16,6 +16,14 @@ public interface ProcessorContext { void schedule(long timestamp); void commit(); + + String topic(); + + int partition(); + + long offset(); + + long timestamp(); } void init(ProcessorContext context); diff --git a/src/main/java/io/confluent/streaming/RecordCollector.java b/src/main/java/io/confluent/streaming/RecordCollector.java index 7a0e9d7be0241..5e827772c9331 100644 --- a/src/main/java/io/confluent/streaming/RecordCollector.java +++ b/src/main/java/io/confluent/streaming/RecordCollector.java @@ -12,4 +12,5 @@ public interface RecordCollector { void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer); + void flush(); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index c69c6c1264c1e..62a13799a46a6 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -290,6 +290,7 @@ public void init(Consumer restoreConsumer) throws IOException { } } + @Override public void flush() { stateMgr.flush(); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 25261ec361298..a28c514c26400 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -1,14 +1,11 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.Coordinator; import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamWindowed; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.Predicate; import io.confluent.streaming.Processor; -import io.confluent.streaming.PunctuationScheduler; -import io.confluent.streaming.RecordCollector; import io.confluent.streaming.ValueMapper; import io.confluent.streaming.Window; import org.apache.kafka.common.serialization.Deserializer; diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java index e19da24cf07c2..40c10b4537fb5 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java @@ -3,7 +3,6 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; import io.confluent.streaming.PunctuationScheduler; -import io.confluent.streaming.RecordCollector; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; @@ -25,6 +24,26 @@ public ProcessorContextImpl(KStreamContext context, this.streamGroup = streamGroup; } + @Override + public String topic() { + return this.streamGroup.record().topic(); + } + + @Override + public int partition() { + return this.streamGroup.record().partition(); + } + + @Override + public long offset() { + return this.streamGroup.record().offset(); + } + + @Override + public long timestamp() { + return this.streamGroup.record().timestamp; + } + @Override public void send(String topic, Object key, Object value) { this.context.recordCollector().send(new ProducerRecord<>(topic, key, value)); diff --git a/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java b/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java index 8e7d227655067..09e2660dbfb8a 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java @@ -55,6 +55,11 @@ public void send(ProducerRecord record, Serializer keySerializer this.producer.send(new ProducerRecord<>(record.topic(), keyBytes, valBytes), callback); } + @Override + public void flush() { + this.producer.flush(); + } + /** * The last ack'd offset from the producer * @return the map from TopicPartition to offset diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index 035118f71afde..df237b471891b 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -48,6 +48,7 @@ public class StreamGroup implements ParallelExecutor.Task { private long streamTime = -1; private boolean commitRequested = false; + private StampedRecord currRecord = null; private volatile int buffered = 0; /** @@ -78,6 +79,8 @@ public String name() { >>>>>>> removed some generics } + public StampedRecord record() { return currRecord; } + /** * Merges a stream group into this group */ @@ -228,18 +231,30 @@ public void process() { } long trackedTimestamp = recordQueue.trackedTimestamp(); - StampedRecord record = recordQueue.next(); + currRecord = recordQueue.next(); if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - recordQueue.stream.receive(record.key(), record.value(), record.timestamp, streamTime); - consumedOffsets.put(recordQueue.partition(), record.offset()); + recordQueue.stream.receive(currRecord.key(), currRecord.value(), currRecord.timestamp, streamTime); + consumedOffsets.put(recordQueue.partition(), currRecord.offset()); // TODO: local state flush and downstream producer flush // need to be done altogether with offset commit atomically + if (commitRequested) { + // flush local state + recordQueue.stream.context().flush(); + + // flush produced records in the downstream + recordQueue.stream.context().recordCollector().flush(); + + // commit consumed offsets + ingestor.commit(consumedOffsets()); + } + + if (commitRequested) ingestor.commit(Collections.singletonMap( - new TopicPartition(record.topic(), record.partition()), - record.offset())); + new TopicPartition(currRecord.topic(), currRecord.partition()), + currRecord.offset())); if (recordQueue.size() > 0) chooser.add(recordQueue); diff --git a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java b/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java index 334dc696b5a2a..b517ebfeb2b41 100644 --- a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java @@ -70,4 +70,7 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer @Override public void restore(StorageEngine engine) throws Exception { throw new UnsupportedOperationException("restore() not supported."); } + + @Override + public void flush() { throw new UnsupportedOperationException("flush() not supported."); } } From ad217883b7312647b8e3c2219654d1bf579ab9e5 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 22 Jul 2015 16:12:15 -0700 Subject: [PATCH 075/275] use poll(0) for non-blocking poll --- .../java/io/confluent/streaming/KafkaStreaming.java | 12 +++++++++++- .../io/confluent/streaming/internal/Ingestor.java | 2 -- .../confluent/streaming/internal/IngestorImpl.java | 10 +--------- .../confluent/streaming/internal/StreamGroup.java | 13 ++++++++++--- .../confluent/streaming/util/ParallelExecutor.java | 12 +++++++++--- .../confluent/streaming/testutil/MockIngestor.java | 4 ---- .../streaming/util/ParallelExecutorTest.java | 6 ++++-- 7 files changed, 35 insertions(+), 24 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index af3f556cde28f..a1bd8e2e8e6b8 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -155,6 +155,7 @@ protected KafkaStreaming(Class jobClass, this.streamingMetrics = new KafkaStreamingMetrics(); this.requestingCommit = new ArrayList<>(); this.config = new ProcessorConfig(config.config()); +<<<<<<< HEAD <<<<<<< HEAD this.ingestor = <<<<<<< HEAD @@ -171,6 +172,9 @@ protected KafkaStreaming(Class jobClass, ======= this.ingestor = new IngestorImpl(this.consumer, this.config.pollTimeMs); >>>>>>> clean up ingestor and stream synchronizer +======= + this.ingestor = new IngestorImpl(this.consumer); +>>>>>>> use poll(0) for non-blocking poll this.running = true; this.lastCommit = 0; this.nextStateCleaning = Long.MAX_VALUE; @@ -272,11 +276,14 @@ public synchronized void close() { private void runLoop() { try { + boolean readyForNextExecution = false; + while (stillRunning()) { - ingestor.poll(); + ingestor.poll(readyForNextExecution ? 0 : this.config.pollTimeMs); <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD parallelExecutor.execute(streamSynchronizers, status); ======= @@ -285,6 +292,9 @@ private void runLoop() { ======= parallelExecutor.execute(streamGroups); >>>>>>> remove SyncGroup from user facing APIs +======= + readyForNextExecution = parallelExecutor.execute(streamGroups); +>>>>>>> use poll(0) for non-blocking poll ======= for (Map.Entry> entry : streamSynchronizersForPartition.entrySet()) { diff --git a/src/main/java/io/confluent/streaming/internal/Ingestor.java b/src/main/java/io/confluent/streaming/internal/Ingestor.java index d92782c2cd71f..d4a49ec9596a2 100644 --- a/src/main/java/io/confluent/streaming/internal/Ingestor.java +++ b/src/main/java/io/confluent/streaming/internal/Ingestor.java @@ -9,8 +9,6 @@ */ public interface Ingestor { - void poll(); - void poll(long timeoutMs); void pause(TopicPartition partition); diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index f58e5983a1171..ecd63a367f018 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -16,13 +16,10 @@ public class IngestorImpl implements Ingestor { private final Consumer consumer; private final Set unpaused = new HashSet<>(); private final Set toBePaused = new HashSet<>(); - private final long pollTimeMs; private final Map streamSynchronizers = new HashMap<>(); - public IngestorImpl(Consumer consumer, - long pollTimeMs) { + public IngestorImpl(Consumer consumer) { this.consumer = consumer; - this.pollTimeMs = pollTimeMs; } public void init() { @@ -30,11 +27,6 @@ public void init() { unpaused.addAll(consumer.subscriptions()); } - @Override - public void poll() { - poll(pollTimeMs); - } - @Override public void poll(long timeoutMs) { synchronized (this) { diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index df237b471891b..ffe3536b10d76 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -215,13 +215,14 @@ public PunctuationScheduler getPunctuationScheduler(Processor processor) { */ @SuppressWarnings("unchecked") @Override - public void process() { + public boolean process() { synchronized (this) { + boolean readyForNExtExecution = false; ingestNewRecords(); RecordQueue recordQueue = chooser.next(); if (recordQueue == null) { - return; + return false; } if (recordQueue.size() == 0) throw new IllegalStateException("empty record queue"); @@ -256,11 +257,17 @@ public void process() { new TopicPartition(currRecord.topic(), currRecord.partition()), currRecord.offset())); - if (recordQueue.size() > 0) chooser.add(recordQueue); + if (recordQueue.size() > 0) { + readyForNExtExecution = true; + chooser.add(recordQueue); + } + buffered--; punctuationQueue.mayPunctuate(streamTime); + + return readyForNExtExecution; } } diff --git a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java index 8c12c7791bae3..76dbc9b5d8553 100644 --- a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java +++ b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java @@ -16,14 +16,16 @@ public class ParallelExecutor { public interface Task { /** * Executes a task + * @return boolean true if the task are ready for next execution */ - void process(); + boolean process(); } private final WorkerThread[] workerThreads; private final AtomicInteger taskIndex = new AtomicInteger(0); private volatile ArrayList tasks = new ArrayList<>(); private volatile CountDownLatch latch; + private volatile boolean readyForNextExecution = true; private volatile boolean running = true; private volatile Exception exception; @@ -39,13 +41,15 @@ public ParallelExecutor(int parallelDegree) { /** * Executes tasks in parallel. While this method is executing, other execute call will be blocked. * @param tasks a list of tasks executed in parallel + * @return boolean true if all tasks are ready for next execution * @throws Exception an exception thrown by a failed task */ - public void execute(ArrayList tasks) throws Exception { + public boolean execute(ArrayList tasks) throws Exception { synchronized (this) { try { int numTasks = tasks.size(); exception = null; + readyForNextExecution = true; if (numTasks > 0) { this.tasks = tasks; this.latch = new CountDownLatch(numTasks); @@ -72,6 +76,7 @@ public void execute(ArrayList tasks) throws Exception { this.latch = null; this.exception = null; } + return readyForNextExecution; } } @@ -90,7 +95,8 @@ private void doProcess() { int index = taskIndex.decrementAndGet(); if (index >= 0) { try { - tasks.get(index).process(); + if (!tasks.get(index).process()) + this.readyForNextExecution = false; } catch (Exception ex) { exception = ex; diff --git a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java index 13ccbaefc46f1..1958fe9e9f476 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java +++ b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java @@ -15,10 +15,6 @@ public class MockIngestor implements Ingestor { public HashSet paused = new HashSet<>(); - @Override - public void poll() { - } - @Override public void poll(long timeoutMs) { } diff --git a/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java b/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java index 84598f6f85d52..a2ce6c467c69a 100644 --- a/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java +++ b/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java @@ -71,7 +71,7 @@ public void testException() { if (i == 15) { taskList.add(new TestTask(counter) { @Override - public void process() { + public boolean process() { throw new TestException(); } }); @@ -103,7 +103,7 @@ private static class TestTask implements ParallelExecutor.Task { } @Override - public void process() { + public boolean process() { try { Thread.sleep(20); executionCount++; @@ -112,6 +112,8 @@ public void process() { // ignore } counter.incrementAndGet(); + + return true; } } From 7b32e1a7564681b19476b0ad88c793e9a980dca5 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 22 Jul 2015 16:16:54 -0700 Subject: [PATCH 076/275] typo --- .../java/io/confluent/streaming/internal/StreamGroup.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index ffe3536b10d76..7b51799b9337f 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -217,7 +217,7 @@ public PunctuationScheduler getPunctuationScheduler(Processor processor) { @Override public boolean process() { synchronized (this) { - boolean readyForNExtExecution = false; + boolean readyForNextExecution = false; ingestNewRecords(); RecordQueue recordQueue = chooser.next(); @@ -258,7 +258,7 @@ public boolean process() { currRecord.offset())); if (recordQueue.size() > 0) { - readyForNExtExecution = true; + readyForNextExecution = true; chooser.add(recordQueue); } @@ -267,7 +267,7 @@ public boolean process() { punctuationQueue.mayPunctuate(streamTime); - return readyForNExtExecution; + return readyForNextExecution; } } From 90178b0666df559de9c4346f2b027fc5143f0a57 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 22 Jul 2015 16:33:20 -0700 Subject: [PATCH 077/275] address Yasuhiro's comments --- .../streaming/internal/ProcessorContextImpl.java | 12 ++++++++++++ .../io/confluent/streaming/internal/StreamGroup.java | 2 +- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java index 40c10b4537fb5..fefd5fd8c49c0 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java @@ -26,21 +26,33 @@ public ProcessorContextImpl(KStreamContext context, @Override public String topic() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); + return this.streamGroup.record().topic(); } @Override public int partition() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); + return this.streamGroup.record().partition(); } @Override public long offset() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); + return this.streamGroup.record().offset(); } @Override public long timestamp() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); + return this.streamGroup.record().timestamp; } diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index 7b51799b9337f..007ff6c15f960 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -252,7 +252,6 @@ public boolean process() { ingestor.commit(consumedOffsets()); } - if (commitRequested) ingestor.commit(Collections.singletonMap( new TopicPartition(currRecord.topic(), currRecord.partition()), currRecord.offset())); @@ -264,6 +263,7 @@ public boolean process() { buffered--; + currRecord = null; punctuationQueue.mayPunctuate(streamTime); From f1d23d41f67730e62efa82c91f6fe502a84f2d8a Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 23 Jul 2015 09:50:30 -0700 Subject: [PATCH 078/275] pause immediately --- .../streaming/internal/IngestorImpl.java | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index ecd63a367f018..d6f32e13b1f27 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -15,7 +15,6 @@ public class IngestorImpl implements Ingestor { private final Consumer consumer; private final Set unpaused = new HashSet<>(); - private final Set toBePaused = new HashSet<>(); private final Map streamSynchronizers = new HashMap<>(); public IngestorImpl(Consumer consumer) { @@ -30,11 +29,6 @@ public void init() { @Override public void poll(long timeoutMs) { synchronized (this) { - for (TopicPartition partition : toBePaused) { - doPause(partition); - } - toBePaused.clear(); - if (!unpaused.isEmpty()) { ConsumerRecords records = consumer.poll(timeoutMs); @@ -52,12 +46,10 @@ public void poll(long timeoutMs) { @Override public void pause(TopicPartition partition) { - toBePaused.add(partition); - } - - private void doPause(TopicPartition partition) { - consumer.seek(partition, Long.MAX_VALUE); // hack: stop consuming from this partition by setting a big offset - unpaused.remove(partition); + synchronized (this) { + consumer.seek(partition, Long.MAX_VALUE); // hack: stop consuming from this partition by setting a big offset + unpaused.remove(partition); + } } @Override @@ -91,7 +83,6 @@ public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition pa public void clear() { unpaused.clear(); - toBePaused.clear(); streamSynchronizers.clear(); } } From 4b93cc276a6edca4629f682e93f206312b0ee082 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 23 Jul 2015 10:33:25 -0700 Subject: [PATCH 079/275] simplify multi-topic stream --- .../internal/KStreamContextImpl.java | 78 +++++++------------ 1 file changed, 26 insertions(+), 52 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 62a13799a46a6..b3ebaf8fe01b8 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -127,7 +127,6 @@ private String getNextGroupName() { private KStream from(StreamGroup streamGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { if (streamGroup == null) throw new IllegalArgumentException("unspecified stream group"); - KStreamSource stream = null; Set fromTopics; synchronized (this) { @@ -143,65 +142,40 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe if (!this.topics.contains(topic)) throw new IllegalArgumentException("topic not subscribed: " + topic); - KStreamSource streamForTopic = (KStreamSource) sourceStreams.get(topic); - - if (stream == null) { - if (streamForTopic != null) - stream = streamForTopic; - } else { - if (streamForTopic != null) { - if (!stream.equals(streamForTopic)) - throw new IllegalArgumentException("another stream created with the same topic " + topic); - } else { - sourceStreams.put(topic, stream); - } - } + if (sourceStreams.get(topic) != null) + throw new IllegalArgumentException("another stream created with the same topic " + topic); } - // if there is no stream for any of the topics, create one - if (stream == null) { - // create stream metadata - Map topicPartitionInfos = new HashMap<>(); - for (String topic : fromTopics) { - PartitioningInfo partitioningInfo = this.partitioningInfos.get(topic); - - if (partitioningInfo == null) { - partitioningInfo = new PartitioningInfo(ingestor.numPartitions(topic)); - this.partitioningInfos.put(topic, partitioningInfo); - } + // create stream metadata + Map topicPartitionInfos = new HashMap<>(); + for (String topic : fromTopics) { + PartitioningInfo partitioningInfo = this.partitioningInfos.get(topic); - topicPartitionInfos.put(topic, partitioningInfo); + if (partitioningInfo == null) { + partitioningInfo = new PartitioningInfo(ingestor.numPartitions(topic)); + this.partitioningInfos.put(topic, partitioningInfo); } - KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, topicPartitionInfos); + topicPartitionInfos.put(topic, partitioningInfo); + } + KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, topicPartitionInfos); - // override the deserializer classes if specified - stream = new KStreamSource<>( - streamMetadata, - this, - (Deserializer) (keyDeserializer == null ? keyDeserializer() : keyDeserializer), - (Deserializer) (valDeserializer == null ? valueDeserializer() : valDeserializer) - ); + // override the deserializer classes if specified + KStreamSource stream = new KStreamSource<>( + streamMetadata, + this, + (Deserializer) (keyDeserializer == null ? keyDeserializer() : keyDeserializer), + (Deserializer) (valDeserializer == null ? valueDeserializer() : valDeserializer) + ); - // update source stream map - for (String topic : fromTopics) { - if (!sourceStreams.containsKey(topic)) - sourceStreams.put(topic, stream); + // update source stream map + for (String topic : fromTopics) { + if (!sourceStreams.containsKey(topic)) + sourceStreams.put(topic, stream); - TopicPartition partition = new TopicPartition(topic, id); - streamGroup.addPartition(partition, stream); - ingestor.addPartitionStreamToGroup(streamGroup, partition); - } - } else { - if (stream.metadata.streamGroup == streamGroup) - throw new IllegalStateException("topic is already assigned a different synchronization group"); - - // TODO: with this constraint we will not allow users to create KStream with different - // deser from the same topic, this constraint may better be relaxed later. - if (keyDeserializer != null && !keyDeserializer.getClass().equals(this.keyDeserializer().getClass())) - throw new IllegalStateException("another source stream with the same topic but different key deserializer is already created"); - if (valDeserializer != null && !valDeserializer.getClass().equals(this.valueDeserializer().getClass())) - throw new IllegalStateException("another source stream with the same topic but different value deserializer is already created"); + TopicPartition partition = new TopicPartition(topic, id); + streamGroup.addPartition(partition, stream); + ingestor.addPartitionStreamToGroup(streamGroup, partition); } return stream; From 7cb30aa9dc7c431f3947e3b3ea42ead5ce5f99af Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 23 Jul 2015 12:32:47 -0700 Subject: [PATCH 080/275] we can add partition->streamGroup mappings to the ingestor after init --- .../streaming/internal/KStreamContextImpl.java | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index b3ebaf8fe01b8..8afa26226f462 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -142,7 +142,7 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe if (!this.topics.contains(topic)) throw new IllegalArgumentException("topic not subscribed: " + topic); - if (sourceStreams.get(topic) != null) + if (sourceStreams.containsKey(topic)) throw new IllegalArgumentException("another stream created with the same topic " + topic); } @@ -170,12 +170,10 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe // update source stream map for (String topic : fromTopics) { - if (!sourceStreams.containsKey(topic)) - sourceStreams.put(topic, stream); + sourceStreams.put(topic, stream); TopicPartition partition = new TopicPartition(topic, id); streamGroup.addPartition(partition, stream); - ingestor.addPartitionStreamToGroup(streamGroup, partition); } return stream; @@ -254,8 +252,15 @@ public void init(Consumer restoreConsumer) throws IOException { this.restoreConsumer = null; } + // add partition -> stream group mappings to the ingestor + for (Map.Entry> entry : sourceStreams.entrySet()) { + TopicPartition partition = new TopicPartition(entry.getKey(), id); + StreamGroup streamGroup = entry.getValue().metadata.streamGroup; + ingestor.addPartitionStreamToGroup(streamGroup, partition); + } + if (!topics.equals(sourceStreams.keySet())) { - LinkedList unusedTopics = new LinkedList(); + LinkedList unusedTopics = new LinkedList<>(); for (String topic : topics) { if (!sourceStreams.containsKey(topic)) unusedTopics.add(topic); From c6f91475e0dd04a2655e74e1df076dc9f1a03cd1 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 23 Jul 2015 14:40:35 -0700 Subject: [PATCH 081/275] forbid kstream construction outside of init call --- .../confluent/streaming/KStreamContext.java | 7 ++++- .../internal/KStreamContextImpl.java | 9 +++++- .../streaming/internal/KStreamImpl.java | 1 + .../streaming/internal/StreamGroup.java | 12 ++----- .../streaming/internal/KStreamBranchTest.java | 1 + .../streaming/internal/KStreamFilterTest.java | 7 +++-- .../internal/KStreamFlatMapTest.java | 4 ++- .../internal/KStreamFlatMapValuesTest.java | 4 ++- .../streaming/internal/KStreamJoinTest.java | 31 ++++++++++++------- .../streaming/internal/KStreamMapTest.java | 4 ++- .../internal/KStreamMapValuesTest.java | 5 ++- .../streaming/internal/KStreamSourceTest.java | 4 ++- .../internal/KStreamWindowedTest.java | 5 +-- .../streaming/internal/StreamGroupTest.java | 2 +- .../MockKStreamContext.java | 6 +++- 15 files changed, 67 insertions(+), 35 deletions(-) rename src/test/java/io/confluent/streaming/{internal => testutil}/MockKStreamContext.java (94%) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 006d7fec1c1a5..e3e012a3f619b 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -121,7 +121,12 @@ public interface KStreamContext { void restore(StorageEngine engine) throws Exception; /** - * Flush the local state of this context + * Ensures that the context is in the initialization phase where KStream topology can be constructed */ + void ensureInitialization(); + + /** + * Flush the local state of this context + */ void flush(); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 8afa26226f462..81cfb83c34951 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -125,6 +125,8 @@ private String getNextGroupName() { @SuppressWarnings("unchecked") private KStream from(StreamGroup streamGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + ensureInitialization(); + if (streamGroup == null) throw new IllegalArgumentException("unspecified stream group"); Set fromTopics; @@ -232,11 +234,16 @@ private StreamGroup streamGroup(String name, Chooser chooser) { @Override public void restore(StorageEngine engine) throws Exception { - if (restoreConsumer == null) throw new IllegalStateException(); + ensureInitialization(); stateMgr.registerAndRestore(collector, restoreConsumer, engine); } + @Override + public void ensureInitialization() { + if (restoreConsumer != null) + throw new IllegalStateException("context initialization is already finished"); + } public Collection streamSynchronizers() { return streamGroups.values(); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index a28c514c26400..75e51f78fae6a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -23,6 +23,7 @@ abstract class KStreamImpl implements KStream, Receiver { final KStreamContext context; protected KStreamImpl(KStreamMetadata metadata, KStreamContext context) { + context.ensureInitialization(); this.metadata = metadata; this.context = context; } diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index 007ff6c15f960..e03491951ab3f 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -86,7 +86,7 @@ public String name() { */ public void mergeStreamGroup(StreamGroup other) { // check these groups have the same ingestor - if (!this.ingestor.equals(other.ingestor)) + if (ingestor == other.ingestor) throw new IllegalArgumentException("groups with different ingestors cannot be merged"); // check these group have the same chooser and time extractor types @@ -97,15 +97,7 @@ public void mergeStreamGroup(StreamGroup other) { throw new IllegalArgumentException("groups with different type of time extractors cannot be merged"); // add all the other's groups partitions - for (TopicPartition partition : other.stash.keySet()) { - this.stash.put(partition, other.stash.get(partition)); - this.consumedOffsets.put(partition, other.consumedOffsets.get(partition)); - } - - // add all the other's buffered records - for (NewRecords records : other.newRecordBuffer) { - this.newRecordBuffer.addLast(records); - } + this.stash.putAll(stash); } /** diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index 4321c813c45e4..92f0b403e3b6d 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockIngestor; +import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 332bf0dd11ee5..8ef24da8ce7c4 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockIngestor; +import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -40,11 +41,12 @@ public boolean apply(Integer key, String value) { public void testFilter() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; + KStreamContext context = new MockKStreamContext(null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, null, null, null); + stream = new KStreamSource<>(streamMetadata, context, null, null); stream.filter(isMultipleOfThree).process(processor); for (int i = 0; i < expectedKeys.length; i++) { @@ -58,11 +60,12 @@ public void testFilter() { public void testFilterOut() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; + KStreamContext context = new MockKStreamContext(null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, null, null, null); + stream = new KStreamSource<>(streamMetadata, context, null, null); stream.filterOut(isMultipleOfThree).process(processor); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index a882148bfad0f..6438dc431e6c2 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockIngestor; +import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -47,11 +48,12 @@ public KeyValue> apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + KStreamContext context = new MockKStreamContext(null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, null, null, null); + stream = new KStreamSource<>(streamMetadata, context, null, null); stream.flatMap(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 2069102766198..e7b3ff28d6972 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockIngestor; +import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -47,11 +48,12 @@ public Iterable apply(String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + KStreamContext context = new MockKStreamContext(null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, null, null, null); + stream = new KStreamSource<>(streamMetadata, context, null, null); stream.flatMapValues(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index d3c64fd26795a..43a12f12000eb 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockIngestor; +import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import io.confluent.streaming.testutil.UnlimitedWindow; import io.confluent.streaming.util.Util; @@ -82,9 +83,10 @@ public void testJoin() { TestProcessor processor; String[] expected; + KStreamContext context = new MockKStreamContext(null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, null, null, null); - stream2 = new KStreamSource<>(streamMetadata, null, null, null); + stream1 = new KStreamSource<>(streamMetadata, context, null, null); + stream2 = new KStreamSource<>(streamMetadata, context, null, null); windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -166,9 +168,10 @@ public void testJoinPrior() { TestProcessor processor; String[] expected; + KStreamContext context = new MockKStreamContext(null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, null, null, null); - stream2 = new KStreamSource<>(streamMetadata, null, null, null); + stream1 = new KStreamSource<>(streamMetadata, context, null, null); + stream2 = new KStreamSource<>(streamMetadata, context, null, null); windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -244,9 +247,10 @@ public void testMap() { KStreamWindowed windowed2; TestProcessor processor; + KStreamContext context = new MockKStreamContext(null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, null, null, null); - stream2 = new KStreamSource<>(streamMetadata, null, null, null); + stream1 = new KStreamSource<>(streamMetadata, context, null, null); + stream2 = new KStreamSource<>(streamMetadata, context, null, null); mapped1 = stream1.map(keyValueMapper); mapped2 = stream2.map(keyValueMapper); @@ -299,9 +303,10 @@ public void testFlatMap() { KStreamWindowed windowed2; TestProcessor processor; + KStreamContext context = new MockKStreamContext(null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, null, null, null); - stream2 = new KStreamSource<>(streamMetadata, null, null, null); + stream1 = new KStreamSource<>(streamMetadata, context, null, null); + stream2 = new KStreamSource<>(streamMetadata, context, null, null); mapped1 = stream1.flatMap(keyValueMapper2); mapped2 = stream2.flatMap(keyValueMapper2); @@ -354,9 +359,10 @@ public void testMapValues() { KStreamWindowed windowed2; TestProcessor processor; + KStreamContext context = new MockKStreamContext(null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, null, null, null); - stream2 = new KStreamSource<>(streamMetadata, null, null, null); + stream1 = new KStreamSource<>(streamMetadata, context, null, null); + stream2 = new KStreamSource<>(streamMetadata, context, null, null); mapped1 = stream1.mapValues(valueMapper); mapped2 = stream2.mapValues(valueMapper); @@ -409,9 +415,10 @@ public void testFlatMapValues() { KStreamWindowed windowed2; TestProcessor processor; + KStreamContext context = new MockKStreamContext(null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, null, null, null); - stream2 = new KStreamSource<>(streamMetadata, null, null, null); + stream1 = new KStreamSource<>(streamMetadata, context, null, null); + stream2 = new KStreamSource<>(streamMetadata, context, null, null); mapped1 = stream1.flatMapValues(valueMapper2); mapped2 = stream2.flatMapValues(valueMapper2); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 90f1eeceb72ac..1179090334fec 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockIngestor; +import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -42,11 +43,12 @@ public KeyValue apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + KStreamContext context = new MockKStreamContext(null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, null, null, null); + stream = new KStreamSource<>(streamMetadata, context, null, null); stream.map(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index e7d532b35587e..1ad6e3c4d0440 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -1,8 +1,10 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.ValueMapper; import io.confluent.streaming.testutil.MockIngestor; +import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -43,11 +45,12 @@ public Integer apply(String value) { final int[] expectedKeys = new int[] { 1, 10, 100, 1000 }; + KStreamContext context = new MockKStreamContext(null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, null, null, null); + stream = new KStreamSource<>(streamMetadata, context, null, null); stream.mapValues(mapper).process(processor); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 7ee576893255d..74c012465b82d 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -5,6 +5,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockIngestor; +import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -33,9 +34,10 @@ public long extract(String topic, Object key, Object value) { @Test public void testKStreamSource() { + KStreamContext context = new MockKStreamContext(null, null); TestProcessor processor = new TestProcessor<>(); - KStreamSource stream = new KStreamSource<>(streamMetadata, null, null, null); + KStreamSource stream = new KStreamSource<>(streamMetadata, context, null, null); stream.process(processor); final String[] expectedKeys = new String[] { "k1", "k2", "k3" }; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 024d2847d5924..b9b9ee2716c39 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockIngestor; +import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.UnlimitedWindow; import org.junit.Test; @@ -37,10 +38,10 @@ public void testWindowedStream() { KStreamSource stream; Window window; - String[] expected; + KStreamContext context = new MockKStreamContext(null, null); window = new UnlimitedWindow<>(); - stream = new KStreamSource<>(streamMetadata, null, null, null); + stream = new KStreamSource<>(streamMetadata, context, null, null); stream.with(window); boolean exceptionRaised = false; diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index fcd57a08145dd..88cd757692ece 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.testutil.MockIngestor; +import io.confluent.streaming.testutil.MockKStreamContext; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; @@ -12,7 +13,6 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.List; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java similarity index 94% rename from src/test/java/io/confluent/streaming/internal/MockKStreamContext.java rename to src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java index b517ebfeb2b41..58e44ec67dc44 100644 --- a/src/test/java/io/confluent/streaming/internal/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package io.confluent.streaming.testutil; import io.confluent.streaming.KStream; @@ -6,6 +6,7 @@ import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StorageEngine; import io.confluent.streaming.Coordinator; +import io.confluent.streaming.internal.StreamGroup; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; @@ -71,6 +72,9 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer @Override public void restore(StorageEngine engine) throws Exception { throw new UnsupportedOperationException("restore() not supported."); } + @Override + public void ensureInitialization() {} + @Override public void flush() { throw new UnsupportedOperationException("flush() not supported."); } } From 5e1e64db58c0b7747382bb3f4e1ad266f7254cfc Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 23 Jul 2015 15:09:52 -0700 Subject: [PATCH 082/275] simplify Coordinator --- .../io/confluent/streaming/Coordinator.java | 44 ++---------------- .../confluent/streaming/KafkaStreaming.java | 45 ++++--------------- 2 files changed, 12 insertions(+), 77 deletions(-) diff --git a/src/main/java/io/confluent/streaming/Coordinator.java b/src/main/java/io/confluent/streaming/Coordinator.java index 0676199e35b67..0652d3b3f97a6 100644 --- a/src/main/java/io/confluent/streaming/Coordinator.java +++ b/src/main/java/io/confluent/streaming/Coordinator.java @@ -27,55 +27,19 @@ public interface Coordinator { /** - * Requests that the stream processor flush all it's state as well as any buffered output and commit the offsets. - * - *

- * If CURRENT_TASK is given, a checkpoint is only written for the current task. If - * ALL_TASKS_IN_CONTAINER is given, a checkpoint is written for all tasks in the current container. + * Requests that the framework to flush all it's state as well as any buffered output and commit the offsets. * *

* Note that if you also have also configured your job to commit in regular intervals (using the * task.commit.ms property), those time-based commits are not affected by calling this method. Any * commits you request explicitly are in addition to timer-based commits. You can set task.commit.ms=-1 * if you don't want commits to happen automatically. - * - * @param scope Which tasks are being asked to commit. - */ - public void commit(RequestScope scope); - - /** - * Requests that the container should be shut down. - * - *

- * If CURRENT_TASK is given, that indicates a willingness of the current task to shut down. All tasks - * in the container (including the one that requested shutdown) will continue processing messages. Only when every - * task in the container has called shutdown(CURRENT_TASK), the container is shut down. Once a task has - * called shutdown(CURRENT_TASK), it cannot change its mind (i.e. it cannot revoke its willingness to - * shut down). - * - *

- * If ALL_TASKS_IN_CONTAINER is given, the container will shut down immediately after it has finished - * processing the current message. Any buffers of pending writes are flushed, but no further messages will be - * processed in this container. - * - * @param scope The approach we should use for shutting down the container. */ - public void shutdown(RequestScope scope); + void commit(); /** - * A task can make requests to the Samza framework while processing messages, such as - * {@link Coordinator#commit(RequestScope)} and {@link Coordinator#shutdown(RequestScope)}. This enum is used to - * indicate whether those requests apply only to the current task, or to all tasks in the current container. + * Requests that the framework to shut down. */ - public enum RequestScope { - /** - * Indicates that a request applies only to the task making the call. - */ - CURRENT_TASK, + void shutdown(); - /** - * Indicates that a request applies to all tasks in the current container. - */ - ALL_TASKS_IN_CONTAINER; - } } diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index a1bd8e2e8e6b8..c403a47e8b87f 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -118,7 +118,7 @@ public class KafkaStreaming implements Runnable { private final Metrics metrics; private final KafkaStreamingMetrics streamingMetrics; private final Time time; - private final List requestingCommit; + private volatile boolean requestingCommit = false; private final AtomicBoolean started = new AtomicBoolean(false); private volatile boolean running; private CountDownLatch shutdownComplete = new CountDownLatch(1); @@ -153,7 +153,6 @@ protected KafkaStreaming(Class jobClass, this.streamingConfig = config; this.metrics = new Metrics(); this.streamingMetrics = new KafkaStreamingMetrics(); - this.requestingCommit = new ArrayList<>(); this.config = new ProcessorConfig(config.config()); <<<<<<< HEAD <<<<<<< HEAD @@ -329,11 +328,10 @@ private void maybeCommit() { if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < time.milliseconds()) { log.trace("Committing processor instances because the commit interval has elapsed."); commitAll(now); - } else { - if (!requestingCommit.isEmpty()) { - log.trace("Committing processor instances because of user request."); - commitRequesting(now); - } + } else if (requestingCommit) { + requestingCommit = false; + log.trace("Committing processor instances because of user request."); + commitAll(now); } } @@ -385,33 +383,6 @@ private void commitAll(long now) { } } - private void commitRequesting(long now) { - Map commit = new HashMap<>(requestingCommit.size()); - for (Integer id : requestingCommit) { - KStreamContextImpl context = kstreamContexts.get(id); - context.flush(); - - for (StreamGroup streamGroup : streamSynchronizersForPartition.get(id)) { - commit.putAll(streamGroup.consumedOffsets()); // TODO: can this be async? - } - } - - // check if commit is really needed, i.e. if all the offsets are already committed - boolean commitNeeded = false; - for (TopicPartition tp : commit.keySet()) { - if (consumer.committed(tp) != commit.get(tp)) { - commitNeeded = true; - break; - } - } - - if (commitNeeded) { - consumer.commit(commit, CommitType.SYNC); // TODO: can this be async? - requestingCommit.clear(); - streamingMetrics.commitTime.record(time.milliseconds() - now); - } - } - /* delete any state dirs that aren't for active contexts */ private void maybeCleanState() { long now = time.milliseconds(); @@ -451,12 +422,12 @@ private void addPartitions(Collection assignment) { Coordinator coordinator = new Coordinator() { @Override - public void commit(Coordinator.RequestScope scope) { - requestingCommit.add(id); + public void commit() { + requestingCommit = true; } @Override - public void shutdown(Coordinator.RequestScope scope) { + public void shutdown() { running = true; } }; From 63ea4c259e817dc76dc5195f70a76a5eaa1f4c82 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 23 Jul 2015 15:22:15 -0700 Subject: [PATCH 083/275] bug fix --- src/main/java/io/confluent/streaming/internal/StreamGroup.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index e03491951ab3f..64c2e1c776d57 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -86,7 +86,7 @@ public String name() { */ public void mergeStreamGroup(StreamGroup other) { // check these groups have the same ingestor - if (ingestor == other.ingestor) + if (ingestor != other.ingestor) throw new IllegalArgumentException("groups with different ingestors cannot be merged"); // check these group have the same chooser and time extractor types From 2b5663eba5ae3e9006403d2e95ecb54412c5bd5b Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 23 Jul 2015 15:23:36 -0700 Subject: [PATCH 084/275] remove unused member variable --- src/main/java/io/confluent/streaming/KafkaStreaming.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index c403a47e8b87f..463bc4eb37a90 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -98,11 +98,14 @@ public class KafkaStreaming implements Runnable { private final Class jobClass; private final Set topics; <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD private final Map> syncGroups = new HashMap<>(); private final ArrayList> streamSynchronizers = new ArrayList<>(); ======= private final Map> streamSynchronizersForPartition = new HashMap<>(); +======= +>>>>>>> remove unused member variable private final ArrayList streamGroups = new ArrayList<>(); >>>>>>> remove SyncGroup from user facing APIs private final ParallelExecutor parallelExecutor; @@ -250,11 +253,14 @@ private void shutdown() { producer.close(); consumer.close(); parallelExecutor.shutdown(); +<<<<<<< HEAD <<<<<<< HEAD syncGroups.clear(); streamSynchronizers.clear(); ======= streamSynchronizersForPartition.clear(); +======= +>>>>>>> remove unused member variable streamGroups.clear(); >>>>>>> remove SyncGroup from user facing APIs shutdownComplete.countDown(); @@ -452,7 +458,6 @@ public void shutdown() { streamSynchronizers.add(syncGroup.streamSynchronizer); ======= Collection streamGroups = kstreamContext.streamSynchronizers(); - this.streamSynchronizersForPartition.put(id, streamGroups); for (StreamGroup streamGroup : streamGroups) { streamGroups.add(streamGroup); >>>>>>> remove SyncGroup from user facing APIs From 0246618de915ee8b630e0337f776ba5d15d6755e Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 24 Jul 2015 12:01:06 -0700 Subject: [PATCH 085/275] blocking poll only when there is no buffered record at all --- .../io/confluent/streaming/util/ParallelExecutor.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java index 76dbc9b5d8553..a1102e0dc754c 100644 --- a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java +++ b/src/main/java/io/confluent/streaming/util/ParallelExecutor.java @@ -41,7 +41,7 @@ public ParallelExecutor(int parallelDegree) { /** * Executes tasks in parallel. While this method is executing, other execute call will be blocked. * @param tasks a list of tasks executed in parallel - * @return boolean true if all tasks are ready for next execution + * @return boolean true if at least one task is ready for next execution, otherwise false * @throws Exception an exception thrown by a failed task */ public boolean execute(ArrayList tasks) throws Exception { @@ -49,7 +49,7 @@ public boolean execute(ArrayList tasks) throws Exception { try { int numTasks = tasks.size(); exception = null; - readyForNextExecution = true; + readyForNextExecution = false; if (numTasks > 0) { this.tasks = tasks; this.latch = new CountDownLatch(numTasks); @@ -95,8 +95,8 @@ private void doProcess() { int index = taskIndex.decrementAndGet(); if (index >= 0) { try { - if (!tasks.get(index).process()) - this.readyForNextExecution = false; + if (tasks.get(index).process()) + this.readyForNextExecution = true; } catch (Exception ex) { exception = ex; From bfbf9e00645aae683903d926fdf65e54c0e3d00f Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 27 Jul 2015 12:12:06 -0700 Subject: [PATCH 086/275] added KStreamThread --- .../confluent/streaming/KafkaStreaming.java | 318 +++++----------- .../streaming/internal/Ingestor.java | 3 + .../streaming/internal/IngestorImpl.java | 40 +- .../internal/KStreamContextImpl.java | 20 +- .../streaming/internal/KStreamThread.java | 348 ++++++++++++++++++ .../internal/RecordCollectorImpl.java | 9 +- .../streaming/testutil/MockIngestor.java | 6 + 7 files changed, 495 insertions(+), 249 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamThread.java diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 463bc4eb37a90..41270652f9e80 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -17,8 +17,9 @@ package io.confluent.streaming; -import io.confluent.streaming.internal.KStreamContextImpl; +import io.confluent.streaming.internal.KStreamThread; import io.confluent.streaming.internal.ProcessorConfig; +<<<<<<< HEAD import io.confluent.streaming.internal.IngestorImpl; <<<<<<< HEAD import io.confluent.streaming.internal.StreamSynchronizer; @@ -37,31 +38,15 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InterruptException; +======= +>>>>>>> added KStreamThread 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.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; -import java.util.List; -import java.util.Map; import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; /** * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and @@ -95,6 +80,7 @@ public class KafkaStreaming implements Runnable { private static final Logger log = LoggerFactory.getLogger(KafkaStreaming.class); +<<<<<<< HEAD private final Class jobClass; private final Set topics; <<<<<<< HEAD @@ -117,34 +103,17 @@ public class KafkaStreaming implements Runnable { protected final Consumer consumer; private final IngestorImpl ingestor; private final StreamingConfig streamingConfig; +======= +>>>>>>> added KStreamThread private final ProcessorConfig config; - private final Metrics metrics; - private final KafkaStreamingMetrics streamingMetrics; - private final Time time; - private volatile boolean requestingCommit = false; - private final AtomicBoolean started = new AtomicBoolean(false); - private volatile boolean running; - private CountDownLatch shutdownComplete = new CountDownLatch(1); - private long lastCommit; - private long nextStateCleaning; - private long recordsProcessed; - - protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { - @Override - public void onPartitionsAssigned(Consumer consumer, Collection assignment) { - addPartitions(assignment); - } - - @Override - public void onPartitionsRevoked(Consumer consumer, Collection assignment) { - removePartitions(assignment); - } - }; + private final Object lock = new Object(); + private final KStreamThread[] threads; + private final Set topics; + private boolean started = false; + private boolean stopping = false; - public KafkaStreaming(Class jobClass, StreamingConfig config) { - this(jobClass, config, null, null); - } +<<<<<<< HEAD @SuppressWarnings("unchecked") protected KafkaStreaming(Class jobClass, StreamingConfig config, @@ -183,49 +152,19 @@ >>>>>>> use poll(0) for non-blocking poll this.recordsProcessed = 0; this.time = new SystemTime(); this.parallelExecutor = new ParallelExecutor(this.config.numStreamThreads); +======= + public KafkaStreaming(Class jobClass, StreamingConfig streamingConfig) { +>>>>>>> added KStreamThread + this.config = new ProcessorConfig(streamingConfig.config()); try { this.topics = new HashSet<>(Arrays.asList(this.config.topics.split(","))); } catch (Exception e) { - throw new KStreamException("failed to get a topic list from the job", e); - } - } - - /** - * Execute the stream processors - */ - public synchronized void run() { - init(); - try { - runLoop(); - } catch (RuntimeException e) { - log.error("Uncaught error during processing: ", e); - throw e; - } finally { - shutdown(); - } - } - - private void init() { - log.info("Starting container"); - if (started.compareAndSet(false, true)) { - if (!config.stateDir.exists() && !config.stateDir.mkdirs()) - throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); - - for (String topic : topics) - consumer.subscribe(topic); - - log.info("Start-up complete"); - } else { - throw new IllegalStateException("This container was already started"); + throw new KStreamException("failed to get a topic list from the streaming config", e); } - } - - private void shutdown() { - log.info("Shutting down container"); - commitAll(time.milliseconds()); +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { @@ -247,9 +186,21 @@ private void shutdown() { log.error("Error while closing stream synchronizers: ", e); } >>>>>>> removed some generics +======= + Coordinator coordinator = new Coordinator() { + @Override + public void commit() { + throw new UnsupportedOperationException(); } - } + @Override + public void shutdown() { + startShutdown(); +>>>>>>> added KStreamThread + } + }; + +<<<<<<< HEAD producer.close(); consumer.close(); parallelExecutor.shutdown(); @@ -265,11 +216,19 @@ private void shutdown() { >>>>>>> remove SyncGroup from user facing APIs shutdownComplete.countDown(); log.info("Shut down complete"); +======= + Metrics metrics = new Metrics(); + + // TODO: Fix this after the threading model is decided (also fix KStreamThread) + this.threads = new KStreamThread[1]; + threads[0] = new KStreamThread(jobClass, topics, streamingConfig, coordinator, metrics); +>>>>>>> added KStreamThread } /** - * Shutdown this streaming instance. + * Execute the stream processors */ +<<<<<<< HEAD public synchronized void close() { running = false; try { @@ -311,36 +270,22 @@ >>>>>>> use poll(0) for non-blocking poll >>>>>>> removed some generics maybeCommit(); maybeCleanState(); +======= + public void run() { + synchronized (lock) { + log.info("Starting container"); + if (!started) { + if (!config.stateDir.exists() && !config.stateDir.mkdirs()) + throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); + + for (KStreamThread thread : threads) thread.start(); + log.info("Start-up complete"); + } else { + throw new IllegalStateException("This container was already started"); +>>>>>>> added KStreamThread } - } catch (Exception e) { - throw new KafkaException(e); - } - } - - private boolean stillRunning() { - if(!running) { - log.debug("Shutting down at user request."); - return false; - } - if(config.totalRecordsToProcess >= 0 && recordsProcessed >= config.totalRecordsToProcess) { - log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", config.totalRecordsToProcess); - return false; - } - return true; - } - - private void maybeCommit() { - long now = time.milliseconds(); - if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < time.milliseconds()) { - log.trace("Committing processor instances because the commit interval has elapsed."); - commitAll(now); - } else if (requestingCommit) { - requestingCommit = false; - log.trace("Committing processor instances because of user request."); - commitAll(now); - } - } +<<<<<<< HEAD private void commitAll(long now) { Map commit = new HashMap<>(); for (KStreamContextImpl context : kstreamContexts.values()) { @@ -377,65 +322,44 @@ private void commitAll(long now) { if (consumer.committed(tp) != commit.get(tp)) { commitNeeded = true; break; +======= + while (!stopping) { + try { + lock.wait(); + } + catch (InterruptedException ex) { + Thread.interrupted(); + } +>>>>>>> added KStreamThread } } - - if (commitNeeded) { - // TODO: for exactly-once we need to make sure the flush and commit - // are executed atomically whenever it is triggered by user - producer.flush(); - consumer.commit(commit, CommitType.SYNC); // TODO: can this be async? - streamingMetrics.commitTime.record(time.milliseconds() - lastCommit); - } + shutdown(); } - /* delete any state dirs that aren't for active contexts */ - private void maybeCleanState() { - long now = time.milliseconds(); - if(now > nextStateCleaning) { - File[] stateDirs = config.stateDir.listFiles(); - if(stateDirs != null) { - for(File dir: stateDirs) { - try { - Integer id = Integer.parseInt(dir.getName()); - if(!kstreamContexts.keySet().contains(id)) { - log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); - Util.rm(dir); - } - } catch(NumberFormatException e) { - log.warn("Deleting unknown directory in state directory {}.", dir.getAbsolutePath()); - Util.rm(dir); - } - } + private void startShutdown() { + synchronized (lock) { + if (!stopping) { + stopping = true; + lock.notifyAll(); } - nextStateCleaning = Long.MAX_VALUE; } } - private void addPartitions(Collection assignment) { - HashSet partitions = new HashSet<>(assignment); - - ingestor.init(); - - Consumer restoreConsumer = - new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); - - for (TopicPartition partition : partitions) { - final Integer id = partition.partition(); - KStreamContextImpl kstreamContext = kstreamContexts.get(id); - if (kstreamContext == null) { - KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); + private void shutdown() { + synchronized (lock) { + if (stopping) { + log.info("Shutting down the container"); - Coordinator coordinator = new Coordinator() { - @Override - public void commit() { - requestingCommit = true; - } + for (KStreamThread thread : threads) + thread.close(); - @Override - public void shutdown() { - running = true; + for (KStreamThread thread : threads) { + try { + thread.join(); + } catch (InterruptedException ex) { + Thread.interrupted(); } +<<<<<<< HEAD }; kstreamContext = @@ -465,77 +389,21 @@ public void shutdown() { ======= streamSynchronizersForPartition.put(id, kstreamContext.streamSynchronizers()); >>>>>>> removed some generics +======= + } + stopping = false; + log.info("Shutdown complete"); +>>>>>>> added KStreamThread } } - - restoreConsumer.close(); - nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; - } - - private void removePartitions(Collection assignment) { - commitAll(time.milliseconds()); - for (StreamGroup streamGroup : streamGroups) { - log.info("Removing synchronization groups {}", streamGroup.name()); - streamGroup.close(); - } - for (KStreamContextImpl kstreamContext : kstreamContexts.values()) { - log.info("Removing stream context {}", kstreamContext.id()); - try { - kstreamContext.close(); - } - catch (Exception e) { - throw new KafkaException(e); - } - streamingMetrics.processorDestruction.record(); - } - streamGroups.clear(); - ingestor.clear(); - } - - private static Set extractTopics(Class jobClass) { - // extract topics from a jobClass's static member field, topics - try { - Object instance = Utils.newInstance(jobClass); - return ((Topics)instance).topics; - } - catch (Exception e) { - throw new KStreamException("failed to get a topic list from the job", e); - } } - private class KafkaStreamingMetrics { - final Sensor commitTime; - final Sensor processTime; - final Sensor windowTime; - final Sensor processorCreation; - final Sensor processorDestruction; - - public KafkaStreamingMetrics() { - String group = "kafka-streaming"; - - this.commitTime = metrics.sensor("commit-time"); - this.commitTime.add(new MetricName(group, "commit-time-avg-ms"), new Avg()); - this.commitTime.add(new MetricName(group, "commits-time-max-ms"), new Max()); - this.commitTime.add(new MetricName(group, "commits-per-second"), new Rate(new Count())); - - this.processTime = metrics.sensor("process-time"); - this.commitTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); - this.commitTime.add(new MetricName(group, "process-time-max-ms"), new Max()); - this.commitTime.add(new MetricName(group, "process-calls-per-second"), new Rate(new Count())); - - this.windowTime = metrics.sensor("window-time"); - this.windowTime.add(new MetricName(group, "window-time-avg-ms"), new Avg()); - this.windowTime.add(new MetricName(group, "window-time-max-ms"), new Max()); - this.windowTime.add(new MetricName(group, "window-calls-per-second"), new Rate(new Count())); - - this.processorCreation = metrics.sensor("processor-creation"); - this.processorCreation.add(new MetricName(group, "processor-creation"), new Rate(new Count())); - - this.processorDestruction = metrics.sensor("processor-destruction"); - this.processorDestruction.add(new MetricName(group, "processor-destruction"), new Rate(new Count())); - - } - + /** + * Shutdown this streaming instance. + */ + public void close() { + startShutdown(); + shutdown(); } } diff --git a/src/main/java/io/confluent/streaming/internal/Ingestor.java b/src/main/java/io/confluent/streaming/internal/Ingestor.java index d4a49ec9596a2..617bf85bdc205 100644 --- a/src/main/java/io/confluent/streaming/internal/Ingestor.java +++ b/src/main/java/io/confluent/streaming/internal/Ingestor.java @@ -3,12 +3,15 @@ import org.apache.kafka.common.TopicPartition; import java.util.Map; +import java.util.Set; /** * Created by yasuhiro on 6/30/15. */ public interface Ingestor { + Set topics(); + void poll(long timeoutMs); void pause(TopicPartition partition); diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index d6f32e13b1f27..87805b2572672 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -13,12 +13,15 @@ public class IngestorImpl implements Ingestor { private static final Logger log = LoggerFactory.getLogger(IngestorImpl.class); + private final Set topics; private final Consumer consumer; private final Set unpaused = new HashSet<>(); private final Map streamSynchronizers = new HashMap<>(); - public IngestorImpl(Consumer consumer) { + public IngestorImpl(Consumer consumer, Set topics) { this.consumer = consumer; + this.topics = Collections.unmodifiableSet(topics); + for (String topic : this.topics) consumer.subscribe(topic); } public void init() { @@ -26,20 +29,23 @@ public void init() { unpaused.addAll(consumer.subscriptions()); } + @Override + public Set topics() { + return topics; + } + @Override public void poll(long timeoutMs) { synchronized (this) { - if (!unpaused.isEmpty()) { - ConsumerRecords records = consumer.poll(timeoutMs); + ConsumerRecords records = consumer.poll(timeoutMs); - for (TopicPartition partition : unpaused) { - StreamGroup streamGroup = streamSynchronizers.get(partition); + for (TopicPartition partition : unpaused) { + StreamGroup streamGroup = streamSynchronizers.get(partition); - if (streamGroup != null) - streamGroup.addRecords(partition, records.records(partition).iterator()); - else - log.warn("unused topic: " + partition.topic()); - } + if (streamGroup != null) + streamGroup.addRecords(partition, records.records(partition).iterator()); + else + log.warn("unused topic: " + partition.topic()); } } } @@ -85,4 +91,18 @@ public void clear() { unpaused.clear(); streamSynchronizers.clear(); } + + boolean commitNeeded(Map offsets) { + for (TopicPartition tp : offsets.keySet()) { + if (consumer.committed(tp) != offsets.get(tp)) { + return true; + } + } + return false; + } + + void close() { + consumer.close(); + clear(); + } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 81cfb83c34951..596f7344cbcf3 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -37,8 +37,6 @@ public class KStreamContextImpl implements KStreamContext { public final int id; private final KStreamJob job; - private final Set topics; - private final Ingestor ingestor; private final RecordCollectorImpl collector; @@ -59,25 +57,21 @@ public KStreamContextImpl(int id, KStreamJob job, Set topics, Ingestor ingestor, - Producer producer, + RecordCollectorImpl collector, Coordinator coordinator, StreamingConfig streamingConfig, ProcessorConfig processorConfig, Metrics metrics) { this.id = id; this.job = job; - this.topics = topics; this.ingestor = ingestor; - - this.collector = - new RecordCollectorImpl(producer, (Serializer)streamingConfig.keySerializer(), (Serializer)streamingConfig.valueSerializer()); - + this.collector = collector; this.coordinator = coordinator; this.streamingConfig = streamingConfig; this.processorConfig = processorConfig; this.timestampExtractor = this.streamingConfig.timestampExtractor(); - if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractor is missing"); + if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractor is missing"); this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id))); this.stateDir = this.stateMgr.baseDir(); @@ -134,14 +128,14 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe synchronized (this) { // if topics not specified, use all the topics be default if (topics == null) { - fromTopics = this.topics; + fromTopics = ingestor.topics(); } else { fromTopics = Collections.unmodifiableSet(Util.mkSet(topics)); } // iterate over the topics and check if the stream has already been created for them for (String topic : fromTopics) { - if (!this.topics.contains(topic)) + if (!ingestor.topics().contains(topic)) throw new IllegalArgumentException("topic not subscribed: " + topic); if (sourceStreams.containsKey(topic)) @@ -266,9 +260,9 @@ public void init(Consumer restoreConsumer) throws IOException { ingestor.addPartitionStreamToGroup(streamGroup, partition); } - if (!topics.equals(sourceStreams.keySet())) { + if (!ingestor.topics().equals(sourceStreams.keySet())) { LinkedList unusedTopics = new LinkedList<>(); - for (String topic : topics) { + for (String topic : ingestor.topics()) { if (!sourceStreams.containsKey(topic)) unusedTopics.add(topic); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java new file mode 100644 index 0000000000000..7525bbc1024e4 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -0,0 +1,348 @@ +/** + * 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 io.confluent.streaming.internal; + +import io.confluent.streaming.*; +import io.confluent.streaming.util.ParallelExecutor; +import io.confluent.streaming.util.Util; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; +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.errors.InterruptException; +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.serialization.Serializer; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.*; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +public class KStreamThread extends Thread { + + private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); + + private final Class jobClass; + private final Set topics; + private final ArrayList streamGroups = new ArrayList<>(); + private final Coordinator containerCoordinator; + private final ParallelExecutor parallelExecutor; + private final Map kstreamContexts = new HashMap<>(); + private final IngestorImpl ingestor; + private final RecordCollectorImpl collector; + private final StreamingConfig streamingConfig; + private final ProcessorConfig config; + private final Metrics metrics; + private final KafkaStreamingMetrics streamingMetrics; + private final Time time; + private volatile boolean requestingCommit = false; + private volatile boolean running; + private long lastCommit; + private long nextStateCleaning; + private long recordsProcessed; + + protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { + @Override + public void onPartitionsAssigned(Consumer consumer, Collection assignment) { + addPartitions(assignment); + } + + @Override + public void onPartitionsRevoked(Consumer consumer, Collection assignment) { + removePartitions(assignment); + } + }; + + @SuppressWarnings("unchecked") + public KStreamThread(Class jobClass, Set topics, StreamingConfig streamingConfig, Coordinator coordinator, Metrics metrics) { + super(); + this.config = new ProcessorConfig(streamingConfig.config()); + this.containerCoordinator = coordinator; + this.jobClass = jobClass; + this.topics = topics; + this.streamingConfig = streamingConfig; + this.metrics = metrics; + this.streamingMetrics = new KafkaStreamingMetrics(); + + Consumer consumer = new KafkaConsumer<>(streamingConfig.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + this.ingestor = new IngestorImpl(consumer, topics); + + Producer producer = new KafkaProducer<>(streamingConfig.config(), new ByteArraySerializer(), new ByteArraySerializer()); + this.collector = new RecordCollectorImpl(producer, (Serializer)streamingConfig.keySerializer(), (Serializer)streamingConfig.valueSerializer()); + + this.running = true; + this.lastCommit = 0; + this.nextStateCleaning = Long.MAX_VALUE; + this.recordsProcessed = 0; + this.time = new SystemTime(); + + // TODO: Fix this after the threading model is decided (also fix KafkaStreaming) + this.parallelExecutor = new ParallelExecutor(this.config.numStreamThreads); + } + + /** + * Execute the stream processors + */ + public synchronized void run() { + try { + runLoop(); + } catch (RuntimeException e) { + log.error("Uncaught error during processing: ", e); + throw e; + } finally { + shutdown(); + } + } + + private void shutdown() { + log.info("Shutting down a kstream thread"); + commitAll(time.milliseconds()); + + for (StreamGroup streamGroup : streamGroups) { + try { + streamGroup.close(); + } + catch(Exception e) { + log.error("Error while closing stream groups: ", e); + } + } + + collector.close(); + ingestor.close(); + parallelExecutor.shutdown(); + streamGroups.clear(); + log.info("kstream thread shutdown complete"); + } + + /** + * Shutdown this streaming instance. + */ + public synchronized void close() { + running = false; + } + + private void runLoop() { + try { + boolean readyForNextExecution = false; + + while (stillRunning()) { + ingestor.poll(readyForNextExecution ? 0 : this.config.pollTimeMs); + + readyForNextExecution = parallelExecutor.execute(streamGroups); + + maybeCommit(); + maybeCleanState(); + } + } catch (Exception e) { + throw new KafkaException(e); + } + } + + private boolean stillRunning() { + if(!running) { + log.debug("Shutting down at user request."); + return false; + } + if(config.totalRecordsToProcess >= 0 && recordsProcessed >= config.totalRecordsToProcess) { + log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", config.totalRecordsToProcess); + return false; + } + return true; + } + + private void maybeCommit() { + long now = time.milliseconds(); + if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < time.milliseconds()) { + log.trace("Committing processor instances because the commit interval has elapsed."); + commitAll(now); + } else if (requestingCommit) { + requestingCommit = false; + log.trace("Committing processor instances because of user request."); + commitAll(now); + } + } + + private void commitAll(long now) { + Map commit = new HashMap<>(); + for (KStreamContextImpl context : kstreamContexts.values()) { + context.flush(); + // check co-ordinator + } + for (StreamGroup streamGroup : streamGroups) { + try { + commit.putAll(streamGroup.consumedOffsets()); + } + catch(Exception e) { + log.error("Error while closing processor: ", e); + } + } + + // check if commit is really needed, i.e. if all the offsets are already committed + if (ingestor.commitNeeded(commit)) { + // TODO: for exactly-once we need to make sure the flush and commit + // are executed atomically whenever it is triggered by user + collector.flush(); + ingestor.commit(commit); // TODO: can this be async? + streamingMetrics.commitTime.record(time.milliseconds() - lastCommit); + } + } + + /* delete any state dirs that aren't for active contexts */ + private void maybeCleanState() { + long now = time.milliseconds(); + if(now > nextStateCleaning) { + File[] stateDirs = config.stateDir.listFiles(); + if(stateDirs != null) { + for(File dir: stateDirs) { + try { + Integer id = Integer.parseInt(dir.getName()); + if(!kstreamContexts.keySet().contains(id)) { + log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); + Util.rm(dir); + } + } catch(NumberFormatException e) { + log.warn("Deleting unknown directory in state directory {}.", dir.getAbsolutePath()); + Util.rm(dir); + } + } + } + nextStateCleaning = Long.MAX_VALUE; + } + } + + private void addPartitions(Collection assignment) { + HashSet partitions = new HashSet<>(assignment); + + ingestor.init(); + + Consumer restoreConsumer = + new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + + for (TopicPartition partition : partitions) { + final Integer id = partition.partition(); + KStreamContextImpl kstreamContext = kstreamContexts.get(id); + if (kstreamContext == null) { + KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); + + Coordinator coordinator = new Coordinator() { + @Override + public void commit() { + requestingCommit = true; + } + + @Override + public void shutdown() { + containerCoordinator.shutdown(); + } + }; + + kstreamContext = + new KStreamContextImpl(id, job, topics, ingestor, collector, coordinator, streamingConfig, config, metrics); + + kstreamContexts.put(id, kstreamContext); + + try { + kstreamContext.init(restoreConsumer); + } + catch (Exception e) { + throw new KafkaException(e); + } + + Collection streamGroups = kstreamContext.streamSynchronizers(); + for (StreamGroup streamGroup : streamGroups) { + streamGroups.add(streamGroup); + } + } + } + + restoreConsumer.close(); + nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; + } + + private void removePartitions(Collection assignment) { + commitAll(time.milliseconds()); + for (StreamGroup streamGroup : streamGroups) { + log.info("Removing synchronization groups {}", streamGroup.name()); + streamGroup.close(); + } + for (KStreamContextImpl kstreamContext : kstreamContexts.values()) { + log.info("Removing stream context {}", kstreamContext.id()); + try { + kstreamContext.close(); + } + catch (Exception e) { + throw new KafkaException(e); + } + streamingMetrics.processorDestruction.record(); + } + streamGroups.clear(); + ingestor.clear(); + } + + private class KafkaStreamingMetrics { + final Sensor commitTime; + final Sensor processTime; + final Sensor windowTime; + final Sensor processorCreation; + final Sensor processorDestruction; + + public KafkaStreamingMetrics() { + String group = "kafka-streaming"; + + this.commitTime = metrics.sensor("commit-time"); + this.commitTime.add(new MetricName(group, "commit-time-avg-ms"), new Avg()); + this.commitTime.add(new MetricName(group, "commits-time-max-ms"), new Max()); + this.commitTime.add(new MetricName(group, "commits-per-second"), new Rate(new Count())); + + this.processTime = metrics.sensor("process-time"); + this.commitTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); + this.commitTime.add(new MetricName(group, "process-time-max-ms"), new Max()); + this.commitTime.add(new MetricName(group, "process-calls-per-second"), new Rate(new Count())); + + this.windowTime = metrics.sensor("window-time"); + this.windowTime.add(new MetricName(group, "window-time-avg-ms"), new Avg()); + this.windowTime.add(new MetricName(group, "window-time-max-ms"), new Max()); + this.windowTime.add(new MetricName(group, "window-calls-per-second"), new Rate(new Count())); + + this.processorCreation = metrics.sensor("processor-creation"); + this.processorCreation.add(new MetricName(group, "processor-creation"), new Rate(new Count())); + + this.processorDestruction = metrics.sensor("processor-destruction"); + this.processorDestruction.add(new MetricName(group, "processor-destruction"), new Rate(new Count())); + + } + + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java b/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java index 09e2660dbfb8a..e5967bfb69ab0 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java @@ -60,11 +60,18 @@ public void flush() { this.producer.flush(); } + /** + * Closes this RecordCollector + */ + void close() { + producer.close(); + } + /** * The last ack'd offset from the producer * @return the map from TopicPartition to offset */ - public Map offsets() { + Map offsets() { return this.offsets; } } diff --git a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java index 1958fe9e9f476..ee053c167c74a 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java +++ b/src/test/java/io/confluent/streaming/testutil/MockIngestor.java @@ -8,6 +8,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Set; public class MockIngestor implements Ingestor { @@ -15,6 +16,11 @@ public class MockIngestor implements Ingestor { public HashSet paused = new HashSet<>(); + @Override + public Set topics() { + return null; + } + @Override public void poll(long timeoutMs) { } From 9a598bfc483a88c52ef7b4288be57e80ca5a6b3f Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 27 Jul 2015 13:00:56 -0700 Subject: [PATCH 087/275] fix vararg in KStreamContextImpl.from() --- .../java/io/confluent/streaming/ProcessorKStreamJob.java | 4 +--- .../io/confluent/streaming/internal/KStreamContextImpl.java | 5 +++++ 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java index 0c89d60bfb14e..065fe9b238c96 100644 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -1,7 +1,5 @@ package io.confluent.streaming; -import org.apache.kafka.common.utils.Utils; - /** * Created by guozhang on 7/14/15. */ @@ -10,7 +8,7 @@ public abstract class ProcessorKStreamJob implements KStreamJob, Processor @SuppressWarnings("unchecked") @Override public void init(KStreamContext context) { - context.from(null).process((Processor) this); + context.from().process((Processor) this); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 596f7344cbcf3..a74fe5b420642 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -127,8 +127,13 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe synchronized (this) { // if topics not specified, use all the topics be default +<<<<<<< HEAD if (topics == null) { fromTopics = ingestor.topics(); +======= + if (topics == null || topics.length == 0) { + fromTopics = this.topics; +>>>>>>> fix vararg in KStreamContextImpl.from() } else { fromTopics = Collections.unmodifiableSet(Util.mkSet(topics)); } From 5ef2073f90162a9e3133108cdc783ee0efda1b78 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 27 Jul 2015 14:23:43 -0700 Subject: [PATCH 088/275] removed Coordinator --- .../io/confluent/streaming/Coordinator.java | 45 ------------ .../confluent/streaming/KStreamContext.java | 6 -- .../confluent/streaming/KafkaStreaming.java | 69 +++++++++++++------ .../internal/KStreamContextImpl.java | 11 --- .../streaming/internal/KStreamThread.java | 34 +++------ .../testutil/MockKStreamContext.java | 4 -- 6 files changed, 58 insertions(+), 111 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/Coordinator.java diff --git a/src/main/java/io/confluent/streaming/Coordinator.java b/src/main/java/io/confluent/streaming/Coordinator.java deleted file mode 100644 index 0652d3b3f97a6..0000000000000 --- a/src/main/java/io/confluent/streaming/Coordinator.java +++ /dev/null @@ -1,45 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.confluent.streaming; - -/** - * Coordinators are provided by the coordiantor methods of {@link KStreamContext} to allow - * the user code to request actions from the framework. - *

- * This interface may evolve over time. - *

- */ -public interface Coordinator { - - /** - * Requests that the framework to flush all it's state as well as any buffered output and commit the offsets. - * - *

- * Note that if you also have also configured your job to commit in regular intervals (using the - * task.commit.ms property), those time-based commits are not affected by calling this method. Any - * commits you request explicitly are in addition to timer-based commits. You can set task.commit.ms=-1 - * if you don't want commits to happen automatically. - */ - void commit(); - - /** - * Requests that the framework to shut down. - */ - void shutdown(); - -} diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index e3e012a3f619b..480b298a33dda 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -75,12 +75,6 @@ public interface KStreamContext { */ RecordCollector recordCollector(); - /** - * Returns {@link Coordinator}. - * @return Coordinator - */ - Coordinator coordinator(); - /** * Returns an application context registered to {@link StreamingConfig}. * @return an application context diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 41270652f9e80..02a3e878142cd 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -80,6 +80,7 @@ public class KafkaStreaming implements Runnable { private static final Logger log = LoggerFactory.getLogger(KafkaStreaming.class); +<<<<<<< HEAD <<<<<<< HEAD private final Class jobClass; private final Set topics; @@ -105,12 +106,27 @@ public class KafkaStreaming implements Runnable { private final StreamingConfig streamingConfig; ======= >>>>>>> added KStreamThread +======= + // + // Container State Transition + // + // run() startShutdown() shutdown() + // CREATED --------> RUNNING ----------------> STOPPING -----------> STOPPED + // | ^ + // | startShutdown() | + // +--------------------------------------------+ + // + private final int CREATED = 0; + private final int RUNNING = 1; + private final int STOPPING = 2; + private final int STOPPED = 3; + private int state = CREATED; + +>>>>>>> removed Coordinator private final ProcessorConfig config; private final Object lock = new Object(); private final KStreamThread[] threads; private final Set topics; - private boolean started = false; - private boolean stopping = false; <<<<<<< HEAD @@ -166,6 +182,7 @@ public KafkaStreaming(Class jobClass, StreamingConfig stre <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { ======= @@ -217,12 +234,18 @@ public void shutdown() { shutdownComplete.countDown(); log.info("Shut down complete"); ======= +======= +>>>>>>> removed Coordinator Metrics metrics = new Metrics(); // TODO: Fix this after the threading model is decided (also fix KStreamThread) this.threads = new KStreamThread[1]; +<<<<<<< HEAD threads[0] = new KStreamThread(jobClass, topics, streamingConfig, coordinator, metrics); >>>>>>> added KStreamThread +======= + threads[0] = new KStreamThread(jobClass, topics, streamingConfig, metrics); +>>>>>>> removed Coordinator } /** @@ -274,7 +297,7 @@ >>>>>>> use poll(0) for non-blocking poll public void run() { synchronized (lock) { log.info("Starting container"); - if (!started) { + if (state == CREATED) { if (!config.stateDir.exists() && !config.stateDir.mkdirs()) throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); @@ -285,6 +308,7 @@ public void run() { >>>>>>> added KStreamThread } +<<<<<<< HEAD <<<<<<< HEAD private void commitAll(long now) { Map commit = new HashMap<>(); @@ -324,6 +348,10 @@ private void commitAll(long now) { break; ======= while (!stopping) { +======= + state = RUNNING; + while (state == RUNNING) { +>>>>>>> removed Coordinator try { lock.wait(); } @@ -332,22 +360,8 @@ private void commitAll(long now) { } >>>>>>> added KStreamThread } - } - shutdown(); - } - - private void startShutdown() { - synchronized (lock) { - if (!stopping) { - stopping = true; - lock.notifyAll(); - } - } - } - private void shutdown() { - synchronized (lock) { - if (stopping) { + if (state == STOPPING) { log.info("Shutting down the container"); for (KStreamThread thread : threads) @@ -391,7 +405,8 @@ private void shutdown() { >>>>>>> removed some generics ======= } - stopping = false; + state = STOPPED; + lock.notifyAll(); log.info("Shutdown complete"); >>>>>>> added KStreamThread } @@ -402,8 +417,20 @@ private void shutdown() { * Shutdown this streaming instance. */ public void close() { - startShutdown(); - shutdown(); + synchronized (lock) { + if (state == CREATED || state == RUNNING) { + state = STOPPING; + lock.notifyAll(); + } + while (state == STOPPING) { + try { + lock.wait(); + } + catch (InterruptedException ex) { + Thread.interrupted(); + } + } + } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index a74fe5b420642..9242bb2099df8 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -1,6 +1,5 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.Coordinator; import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamException; @@ -11,7 +10,6 @@ import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.util.Util; import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; @@ -40,7 +38,6 @@ public class KStreamContextImpl implements KStreamContext { private final Ingestor ingestor; private final RecordCollectorImpl collector; - private final Coordinator coordinator; private final HashMap> sourceStreams = new HashMap<>(); private final HashMap partitioningInfos = new HashMap<>(); private final TimestampExtractor timestampExtractor; @@ -55,10 +52,8 @@ public class KStreamContextImpl implements KStreamContext { @SuppressWarnings("unchecked") public KStreamContextImpl(int id, KStreamJob job, - Set topics, Ingestor ingestor, RecordCollectorImpl collector, - Coordinator coordinator, StreamingConfig streamingConfig, ProcessorConfig processorConfig, Metrics metrics) { @@ -66,7 +61,6 @@ public KStreamContextImpl(int id, this.job = job; this.ingestor = ingestor; this.collector = collector; - this.coordinator = coordinator; this.streamingConfig = streamingConfig; this.processorConfig = processorConfig; @@ -186,11 +180,6 @@ public RecordCollector recordCollector() { return collector; } - @Override - public Coordinator coordinator() { - return coordinator; - } - @Override public Map getContext() { return streamingConfig.context(); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index 7525bbc1024e4..03e1a65efa07f 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -17,7 +17,8 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.*; +import io.confluent.streaming.KStreamJob; +import io.confluent.streaming.StreamingConfig; import io.confluent.streaming.util.ParallelExecutor; import io.confluent.streaming.util.Util; import org.apache.kafka.clients.consumer.Consumer; @@ -28,7 +29,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; @@ -45,18 +45,19 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.util.*; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; public class KStreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); private final Class jobClass; - private final Set topics; private final ArrayList streamGroups = new ArrayList<>(); - private final Coordinator containerCoordinator; private final ParallelExecutor parallelExecutor; private final Map kstreamContexts = new HashMap<>(); private final IngestorImpl ingestor; @@ -85,12 +86,10 @@ public void onPartitionsRevoked(Consumer consumer, Collection jobClass, Set topics, StreamingConfig streamingConfig, Coordinator coordinator, Metrics metrics) { + public KStreamThread(Class jobClass, Set topics, StreamingConfig streamingConfig, Metrics metrics) { super(); this.config = new ProcessorConfig(streamingConfig.config()); - this.containerCoordinator = coordinator; this.jobClass = jobClass; - this.topics = topics; this.streamingConfig = streamingConfig; this.metrics = metrics; this.streamingMetrics = new KafkaStreamingMetrics(); @@ -197,7 +196,6 @@ private void commitAll(long now) { Map commit = new HashMap<>(); for (KStreamContextImpl context : kstreamContexts.values()) { context.flush(); - // check co-ordinator } for (StreamGroup streamGroup : streamGroups) { try { @@ -255,20 +253,8 @@ private void addPartitions(Collection assignment) { if (kstreamContext == null) { KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); - Coordinator coordinator = new Coordinator() { - @Override - public void commit() { - requestingCommit = true; - } - - @Override - public void shutdown() { - containerCoordinator.shutdown(); - } - }; - kstreamContext = - new KStreamContextImpl(id, job, topics, ingestor, collector, coordinator, streamingConfig, config, metrics); + new KStreamContextImpl(id, job, ingestor, collector, streamingConfig, config, metrics); kstreamContexts.put(id, kstreamContext); diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java index 58e44ec67dc44..e5c40f3a51c20 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java @@ -5,7 +5,6 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StorageEngine; -import io.confluent.streaming.Coordinator; import io.confluent.streaming.internal.StreamGroup; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; @@ -51,9 +50,6 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer @Override public RecordCollector recordCollector() { throw new UnsupportedOperationException("recordCollector() not supported."); } - @Override - public Coordinator coordinator() { throw new UnsupportedOperationException("coordinator() not supported."); } - @Override public Map getContext() { throw new UnsupportedOperationException("getContext() not supported."); } From 180e760e9197ef6718b84678687c9c2b79774956 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 27 Jul 2015 14:26:53 -0700 Subject: [PATCH 089/275] fixed comments --- src/main/java/io/confluent/streaming/KafkaStreaming.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 02a3e878142cd..1393461220886 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -80,6 +80,7 @@ public class KafkaStreaming implements Runnable { private static final Logger log = LoggerFactory.getLogger(KafkaStreaming.class); +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD private final Class jobClass; @@ -116,6 +117,9 @@ public class KafkaStreaming implements Runnable { // | startShutdown() | // +--------------------------------------------+ // +======= + // Container States +>>>>>>> fixed comments private final int CREATED = 0; private final int RUNNING = 1; private final int STOPPING = 2; From fbe00032afc40b3bb87c272cb9ef567e09553175 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 27 Jul 2015 15:31:11 -0700 Subject: [PATCH 090/275] do not subscribe until run() --- .../java/io/confluent/streaming/internal/IngestorImpl.java | 4 ++++ .../java/io/confluent/streaming/internal/KStreamThread.java | 1 + 2 files changed, 5 insertions(+) diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java index 87805b2572672..e292710e567b1 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/src/main/java/io/confluent/streaming/internal/IngestorImpl.java @@ -24,6 +24,10 @@ public IngestorImpl(Consumer consumer, Set topics) { for (String topic : this.topics) consumer.subscribe(topic); } + public void open() { + for (String topic : this.topics) consumer.subscribe(topic); + } + public void init() { unpaused.clear(); unpaused.addAll(consumer.subscriptions()); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index 03e1a65efa07f..58da6b888250c 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -115,6 +115,7 @@ public KStreamThread(Class jobClass, Set topics, S */ public synchronized void run() { try { + ingestor.open(); runLoop(); } catch (RuntimeException e) { log.error("Uncaught error during processing: ", e); From 04354d79f038ddc16e93b68cb1c33b5ccffe7be4 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 27 Jul 2015 15:35:12 -0700 Subject: [PATCH 091/275] wip --- .../confluent/streaming/KafkaStreaming.java | 7 ++- .../io/confluent/streaming/StorageEngine.java | 14 +---- .../internal/KStreamContextImpl.java | 22 +++++--- .../internal/ProcessorStateManager.java | 6 +-- .../streaming/kv/InMemoryKeyValueStore.java | 53 ++++++++----------- .../kv/internals/MeteredKeyValueStore.java | 8 +-- 6 files changed, 50 insertions(+), 60 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 1393461220886..e5ba1423b58d8 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -386,7 +386,7 @@ private void commitAll(long now) { kstreamContexts.put(id, kstreamContext); try { - kstreamContext.init(restoreConsumer); + kstreamContext.init(); } catch (Exception e) { throw new KafkaException(e); @@ -415,6 +415,11 @@ private void commitAll(long now) { >>>>>>> added KStreamThread } } +<<<<<<< HEAD +======= + + nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; +>>>>>>> wip } /** diff --git a/src/main/java/io/confluent/streaming/StorageEngine.java b/src/main/java/io/confluent/streaming/StorageEngine.java index 1dab83aa4850a..d307c99db419d 100644 --- a/src/main/java/io/confluent/streaming/StorageEngine.java +++ b/src/main/java/io/confluent/streaming/StorageEngine.java @@ -39,19 +39,9 @@ public interface StorageEngine { String name(); /** - * Register the given storage engine with the changelog and restore it's state using the given - * consumer instance. - * @param collector The record collector to write records to - * @param consumer The consumer to read with - * @param partition The partition to use as the change log - * @param checkpointedOffset The offset of the last save - * @param logEndOffset The last offset in the changelog + * Restore the store's state */ - void registerAndRestore(RecordCollector collector, - Consumer consumer, - TopicPartition partition, - long checkpointedOffset, - long logEndOffset); + void restore(); /** * Flush any cached data diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 9242bb2099df8..069abe4433735 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -10,8 +10,14 @@ import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.util.Util; import org.apache.kafka.clients.consumer.Consumer; +<<<<<<< HEAD +======= +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.Producer; +>>>>>>> wip import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.slf4j.Logger; @@ -67,6 +73,7 @@ public KStreamContextImpl(int id, this.timestampExtractor = this.streamingConfig.timestampExtractor(); if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractor is missing"); + this.restoreConsumer = new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id))); this.stateDir = this.stateMgr.baseDir(); this.metrics = metrics; @@ -237,15 +244,13 @@ public Collection streamSynchronizers() { return streamGroups.values(); } - public void init(Consumer restoreConsumer) throws IOException { + public Consumer restoreConsumer() { + return restoreConsumer; + } + + public void init() throws IOException { stateMgr.init(); - try { - this.restoreConsumer = restoreConsumer; - job.init(this); - } - finally { - this.restoreConsumer = null; - } + job.init(this); // add partition -> stream group mappings to the ingestor for (Map.Entry> entry : sourceStreams.entrySet()) { @@ -270,6 +275,7 @@ public void flush() { } public void close() throws Exception { + restoreConsumer.close(); stateMgr.close(collector.offsets()); job.close(); } diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index a50195216c44a..e12625091e07c 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -37,15 +37,15 @@ public class ProcessorStateManager { private final int id; private final File baseDir; private final Map stores; - private final Map checkpointedOffsets; private final Map restoredOffsets; + private final Map checkpointedOffsets; public ProcessorStateManager(int id, File baseDir) { this.id = id; this.baseDir = baseDir; this.stores = new HashMap<>(); - this.checkpointedOffsets = new HashMap<>(); this.restoredOffsets = new HashMap<>(); + this.checkpointedOffsets = new HashMap<>(); } public File baseDir() { @@ -83,7 +83,7 @@ public void registerAndRestore(RecordCollector collector, Consumer The key type * @param The value type */ -public class InMemoryKeyValueStore extends MeteredKeyValueStore implements KeyValueStore, StorageEngine { +public class InMemoryKeyValueStore extends MeteredKeyValueStore { public InMemoryKeyValueStore(String name, KStreamContext context) { super(name, "kafka-streams", new MemoryStore(name, context), context.metrics(), new SystemTime()); } - private static class MemoryStore implements KeyValueStore, StorageEngine { + private static class MemoryStore implements KeyValueStore { private final String topic; private final int partition; - private final NavigableMap store; private final Set dirty; private final int maxDirty; - private final Serializer keySerializer; - private final Serializer valueSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; - private RecordCollector collector; + private final NavigableMap map; + private final KStreamContext context; @SuppressWarnings("unchecked") public MemoryStore(String name, KStreamContext context) { this.topic = name; this.partition = context.id(); - this.store = new TreeMap(); + this.map = new TreeMap(); this.dirty = new HashSet(); - this.collector = null; this.maxDirty = 100; - this.keySerializer = (Serializer) context.keySerializer(); - this.valueSerializer = (Serializer) context.valueSerializer(); - this.keyDeserializer = (Deserializer) context.keyDeserializer(); - this.valueDeserializer = (Deserializer) context.valueDeserializer(); + this.context = context; } @Override @@ -60,13 +51,13 @@ public String name() { @Override public V get(K key) { - return this.store.get(key); + return this.map.get(key); } @Override public void put(K key, V value) { - this.store.put(key, value); - if(this.collector != null) { + this.map.put(key, value); + if(context.recordCollector() != null) { this.dirty.add(key); if (this.dirty.size() > this.maxDirty) flush(); @@ -86,24 +77,22 @@ public void delete(K key) { @Override public KeyValueIterator range(K from, K to) { - return new MemoryStoreIterator(this.store.subMap(from, true, to, false).entrySet().iterator()); + return new MemoryStoreIterator(this.map.subMap(from, true, to, false).entrySet().iterator()); } @Override public KeyValueIterator all() { - return new MemoryStoreIterator(this.store.entrySet().iterator()); + return new MemoryStoreIterator(this.map.entrySet().iterator()); } @Override - public void registerAndRestore(RecordCollector collector, - Consumer consumer, - TopicPartition partition, - long checkpointedOffset, - long endOffset) { - this.collector = collector; + public void restore() { + Deserializer keyDeserializer = (Deserializer) context.keyDeserializer(); + Deserializer valueDeserializer = (Deserializer) context.valueDeserializer(); + while (true) { for(ConsumerRecord record: consumer.poll(100)) - this.store.put(keyDeserializer.deserialize(partition.topic(), record.key()), + this.map.put(keyDeserializer.deserialize(partition.topic(), record.key()), valueDeserializer.deserialize(partition.topic(), record.value())); long position = consumer.position(partition); if (position == endOffset) @@ -115,10 +104,14 @@ else if(position > endOffset) @Override public void flush() { - if(this.collector != null) { + RecordCollector collector = context.recordCollector(); + Serializer keySerializer = (Serializer) context.keySerializer(); + Serializer valueSerializer = (Serializer) context.valueSerializer(); + + if(collector != null) { for (K k : this.dirty) { - V v = this.store.get(k); - this.collector.send(new ProducerRecord<>(this.topic, this.partition, k, v), this.keySerializer, this.valueSerializer); + V v = this.map.get(k); + collector.send(new ProducerRecord<>(this.topic, this.partition, k, v), keySerializer, valueSerializer); } this.dirty.clear(); } diff --git a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java index c1cbd2bcd76a5..a45078410db74 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java @@ -73,14 +73,10 @@ public String name() { } @Override - public void registerAndRestore(RecordCollector collector, - Consumer consumer, - TopicPartition partition, - long checkpointedOffset, - long logEndOffset) { + public void restore() { long startNs = time.nanoseconds(); try { - inner.registerAndRestore(collector, consumer, partition, checkpointedOffset, logEndOffset); + inner.restore(); } finally { recordLatency(this.restoreTime, startNs, time.nanoseconds()); } From 506eebc0625d306325f42e90b235df839275054a Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 27 Jul 2015 15:45:08 -0700 Subject: [PATCH 092/275] cleanup --- .../java/io/confluent/streaming/internal/KStreamThread.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index 58da6b888250c..5a9a51706c006 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -67,7 +67,6 @@ public class KStreamThread extends Thread { private final Metrics metrics; private final KafkaStreamingMetrics streamingMetrics; private final Time time; - private volatile boolean requestingCommit = false; private volatile boolean running; private long lastCommit; private long nextStateCleaning; @@ -186,10 +185,6 @@ private void maybeCommit() { if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < time.milliseconds()) { log.trace("Committing processor instances because the commit interval has elapsed."); commitAll(now); - } else if (requestingCommit) { - requestingCommit = false; - log.trace("Committing processor instances because of user request."); - commitAll(now); } } From 0174ebf158b8f79c7182d9981c8e8057511db839 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 27 Jul 2015 16:21:53 -0700 Subject: [PATCH 093/275] fix mergeStreamGroup --- src/main/java/io/confluent/streaming/internal/StreamGroup.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index 64c2e1c776d57..3fad8c0385aac 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -97,7 +97,7 @@ public void mergeStreamGroup(StreamGroup other) { throw new IllegalArgumentException("groups with different type of time extractors cannot be merged"); // add all the other's groups partitions - this.stash.putAll(stash); + this.stash.putAll(other.stash); } /** From dc14dcc23cd8971f942bc04ad9e1b1f56f02e7c0 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 27 Jul 2015 16:37:56 -0700 Subject: [PATCH 094/275] wip --- .../confluent/streaming/KStreamContext.java | 2 +- .../{StorageEngine.java => StateStore.java} | 15 ++--- .../internal/KStreamContextImpl.java | 29 +++++---- .../internal/ProcessorStateManager.java | 65 ++++++++++++------- .../streaming/kv/InMemoryKeyValueStore.java | 28 ++------ .../confluent/streaming/kv/KeyValueStore.java | 4 +- .../kv/internals/MeteredKeyValueStore.java | 8 +-- .../testutil/MockKStreamContext.java | 7 +- 8 files changed, 86 insertions(+), 72 deletions(-) rename src/main/java/io/confluent/streaming/{StorageEngine.java => StateStore.java} (86%) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 480b298a33dda..f0a83a3f67384 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -112,7 +112,7 @@ public interface KStreamContext { * @param engine the storage engine * @throws Exception an exception thrown by the engine */ - void restore(StorageEngine engine) throws Exception; + void restore(StateStore engine) throws Exception; /** * Ensures that the context is in the initialization phase where KStream topology can be constructed diff --git a/src/main/java/io/confluent/streaming/StorageEngine.java b/src/main/java/io/confluent/streaming/StateStore.java similarity index 86% rename from src/main/java/io/confluent/streaming/StorageEngine.java rename to src/main/java/io/confluent/streaming/StateStore.java index d307c99db419d..2075bd5c3d3a4 100644 --- a/src/main/java/io/confluent/streaming/StorageEngine.java +++ b/src/main/java/io/confluent/streaming/StateStore.java @@ -17,8 +17,8 @@ package io.confluent.streaming; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.common.TopicPartition; + +import org.apache.kafka.clients.consumer.ConsumerRecord; /** * A storage engine for managing state maintained by a stream processor. @@ -30,7 +30,7 @@ * as basic lifecycle management. *

*/ -public interface StorageEngine { +public interface StateStore { /** * The name of this store. @@ -38,11 +38,6 @@ public interface StorageEngine { */ String name(); - /** - * Restore the store's state - */ - void restore(); - /** * Flush any cached data */ @@ -53,4 +48,8 @@ public interface StorageEngine { */ void close(); + /** + * Consume one changelog record to restore its recorded state change + */ + void restore(ConsumerRecord record); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 069abe4433735..f8e42a1fcd2bc 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -5,13 +5,16 @@ import io.confluent.streaming.KStreamException; import io.confluent.streaming.KStreamJob; import io.confluent.streaming.RecordCollector; -import io.confluent.streaming.StorageEngine; +import io.confluent.streaming.StateStore; import io.confluent.streaming.StreamingConfig; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.util.Util; +<<<<<<< HEAD import org.apache.kafka.clients.consumer.Consumer; <<<<<<< HEAD ======= +======= +>>>>>>> wip import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.Producer; >>>>>>> wip @@ -53,7 +56,8 @@ public class KStreamContextImpl implements KStreamContext { private final Metrics metrics; private final File stateDir; private final ProcessorStateManager stateMgr; - private Consumer restoreConsumer; + + private boolean initialized = false; @SuppressWarnings("unchecked") public KStreamContextImpl(int id, @@ -73,8 +77,8 @@ public KStreamContextImpl(int id, this.timestampExtractor = this.streamingConfig.timestampExtractor(); if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractor is missing"); - this.restoreConsumer = new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); - this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id))); + this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id)), + new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer())); this.stateDir = this.stateMgr.baseDir(); this.metrics = metrics; } @@ -197,6 +201,10 @@ public File stateDir() { return stateDir; } + public ProcessorStateManager stateMgr() { + return stateMgr; + } + @Override public Metrics metrics() { return metrics; @@ -228,15 +236,15 @@ private StreamGroup streamGroup(String name, Chooser chooser) { @Override - public void restore(StorageEngine engine) throws Exception { + public void restore(StateStore engine) throws Exception { ensureInitialization(); - stateMgr.registerAndRestore(collector, restoreConsumer, engine); + stateMgr.registerAndRestore(collector, engine); } @Override public void ensureInitialization() { - if (restoreConsumer != null) + if (!initialized) throw new IllegalStateException("context initialization is already finished"); } @@ -244,10 +252,6 @@ public Collection streamSynchronizers() { return streamGroups.values(); } - public Consumer restoreConsumer() { - return restoreConsumer; - } - public void init() throws IOException { stateMgr.init(); job.init(this); @@ -267,6 +271,8 @@ public void init() throws IOException { } throw new KStreamException("unused topics: " + Util.mkString(unusedTopics)); } + + initialized = true; } @Override @@ -275,7 +281,6 @@ public void flush() { } public void close() throws Exception { - restoreConsumer.close(); stateMgr.close(collector.offsets()); job.close(); } diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index e12625091e07c..5befcc6d7824c 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -17,10 +17,12 @@ package io.confluent.streaming.internal; import io.confluent.streaming.RecordCollector; -import io.confluent.streaming.StorageEngine; +import io.confluent.streaming.StateStore; import io.confluent.streaming.util.OffsetCheckpoint; import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,14 +38,16 @@ public class ProcessorStateManager { private final int id; private final File baseDir; - private final Map stores; + private final Map stores; + private final Consumer restoreConsumer; private final Map restoredOffsets; private final Map checkpointedOffsets; - public ProcessorStateManager(int id, File baseDir) { + public ProcessorStateManager(int id, File baseDir, Consumer restoreConsumer) { this.id = id; this.baseDir = baseDir; this.stores = new HashMap<>(); + this.restoreConsumer = restoreConsumer; this.restoredOffsets = new HashMap<>(); this.checkpointedOffsets = new HashMap<>(); } @@ -52,40 +56,56 @@ public File baseDir() { return this.baseDir; } + public Consumer restoreConsumer() { + return this.restoreConsumer; + } + public void init() throws IOException { OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); this.checkpointedOffsets.putAll(checkpoint.read()); checkpoint.delete(); } - public void registerAndRestore(RecordCollector collector, Consumer consumer, StorageEngine engine) { - if (engine.name().equals(CHECKPOINT_FILE_NAME)) + public void registerAndRestore(StateStore store) { + if (store.name().equals(CHECKPOINT_FILE_NAME)) throw new IllegalArgumentException("Illegal store name: " + CHECKPOINT_FILE_NAME); - if(this.stores.containsKey(engine.name())) - throw new IllegalArgumentException("Store " + engine.name() + " has already been registered."); + if(this.stores.containsKey(store.name())) + throw new IllegalArgumentException("Store " + store.name() + " has already been registered."); // register store - this.stores.put(engine.name(), engine); + this.stores.put(store.name(), store); - TopicPartition storePartition = new TopicPartition(engine.name(), id); - consumer.subscribe(storePartition); + // subscribe to the store's partition + TopicPartition storePartition = new TopicPartition(store.name(), id); + restoreConsumer.subscribe(storePartition); // calculate the end offset of the partition - consumer.seekToEnd(storePartition); - long partitionEndOffset = consumer.position(storePartition); + // TODO: this is a bit hacky to first seek then position to get the end offset + restoreConsumer.seekToEnd(storePartition); + long endOffset = restoreConsumer.position(storePartition); - // what was the last-written offset when we shutdown last? + // reset the consumer to the written offset when we shutdown last time long checkpointedOffset = 0; - if(checkpointedOffsets.containsKey(storePartition)) + if (checkpointedOffsets.containsKey(storePartition)) checkpointedOffset = checkpointedOffsets.get(storePartition); + restoreConsumer.seek(storePartition, checkpointedOffset); + + // restore its state from changelog records + while (true) { + for(ConsumerRecord record: restoreConsumer.poll(100)) + store.restore(record); + + long position = restoreConsumer.position(storePartition); + if (position == endOffset) + break; + else if(position > endOffset) + throw new IllegalStateException("This should not happen."); + } - // restore - consumer.subscribe(storePartition); - consumer.seekToBeginning(storePartition); - engine.restore(collector, consumer, storePartition, checkpointedOffset, partitionEndOffset); - consumer.unsubscribe(storePartition); - restoredOffsets.put(storePartition, partitionEndOffset); + // record the restored offset for its partition + restoredOffsets.put(storePartition, restoreConsumer.position(storePartition)); + restoreConsumer.unsubscribe(storePartition); } public void cleanup() throws IOException { @@ -101,7 +121,7 @@ public void cleanup() throws IOException { public void flush() { if(!this.stores.isEmpty()) { log.debug("Flushing stores."); - for (StorageEngine engine : this.stores.values()) + for (StateStore engine : this.stores.values()) engine.flush(); } } @@ -109,7 +129,7 @@ public void flush() { public void close(Map ackedOffsets) throws IOException { if(!stores.isEmpty()) { log.debug("Closing stores."); - for (Map.Entry entry : stores.entrySet()) { + for (Map.Entry entry : stores.entrySet()) { log.debug("Closing storage engine {}", entry.getKey()); entry.getValue().flush(); entry.getValue().close(); @@ -127,6 +147,7 @@ public void close(Map ackedOffsets) throws IOException { OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); checkpoint.write(checkpointOffsets); } + restoreConsumer.close(); } } diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index 597f5167de7b8..144bbe0bc048b 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -2,12 +2,10 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.RecordCollector; +import io.confluent.streaming.internal.KStreamContextImpl; import io.confluent.streaming.kv.internals.MeteredKeyValueStore; -import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.SystemTime; @@ -25,7 +23,7 @@ public InMemoryKeyValueStore(String name, KStreamContext context) { super(name, "kafka-streams", new MemoryStore(name, context), context.metrics(), new SystemTime()); } - private static class MemoryStore implements KeyValueStore { + private static class MemoryStore extends KeyValueStore { private final String topic; private final int partition; @@ -85,23 +83,6 @@ public KeyValueIterator all() { return new MemoryStoreIterator(this.map.entrySet().iterator()); } - @Override - public void restore() { - Deserializer keyDeserializer = (Deserializer) context.keyDeserializer(); - Deserializer valueDeserializer = (Deserializer) context.valueDeserializer(); - - while (true) { - for(ConsumerRecord record: consumer.poll(100)) - this.map.put(keyDeserializer.deserialize(partition.topic(), record.key()), - valueDeserializer.deserialize(partition.topic(), record.value())); - long position = consumer.position(partition); - if (position == endOffset) - break; - else if(position > endOffset) - throw new IllegalStateException("This should not happen."); - } - } - @Override public void flush() { RecordCollector collector = context.recordCollector(); @@ -117,6 +98,11 @@ public void flush() { } } + @Override + public void restore(ConsumerRecord record) { + + } + @Override public void close() { flush(); diff --git a/src/main/java/io/confluent/streaming/kv/KeyValueStore.java b/src/main/java/io/confluent/streaming/kv/KeyValueStore.java index 1d03aea1f1304..d13f468477291 100644 --- a/src/main/java/io/confluent/streaming/kv/KeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/KeyValueStore.java @@ -19,7 +19,7 @@ package io.confluent.streaming.kv; -import io.confluent.streaming.StorageEngine; +import io.confluent.streaming.StateStore; import java.util.List; @@ -29,7 +29,7 @@ * @param The key type * @param The value type */ -public interface KeyValueStore extends StorageEngine { +public interface KeyValueStore extends StateStore { /** * Get the value corresponding to this key diff --git a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java index a45078410db74..eb95eadbaef6a 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java @@ -1,12 +1,10 @@ package io.confluent.streaming.kv.internals; -import io.confluent.streaming.RecordCollector; import io.confluent.streaming.kv.Entry; import io.confluent.streaming.kv.KeyValueIterator; import io.confluent.streaming.kv.KeyValueStore; -import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.MeasurableStat; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; @@ -73,10 +71,10 @@ public String name() { } @Override - public void restore() { + public void restore(ConsumerRecord record) { long startNs = time.nanoseconds(); try { - inner.restore(); + inner.restore(record); } finally { recordLatency(this.restoreTime, startNs, time.nanoseconds()); } diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java index e5c40f3a51c20..5bf5cb55f1deb 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java @@ -4,7 +4,12 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.RecordCollector; +<<<<<<< HEAD import io.confluent.streaming.StorageEngine; +======= +import io.confluent.streaming.StateStore; +import io.confluent.streaming.Coordinator; +>>>>>>> wip import io.confluent.streaming.internal.StreamGroup; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; @@ -66,7 +71,7 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer public StreamGroup roundRobinStreamGroup(String name) { throw new UnsupportedOperationException("roundRobinStreamGroup() not supported."); } @Override - public void restore(StorageEngine engine) throws Exception { throw new UnsupportedOperationException("restore() not supported."); } + public void restore(StateStore engine) throws Exception { throw new UnsupportedOperationException("restore() not supported."); } @Override public void ensureInitialization() {} From 8f5c195e3d8a56427627bfde1425a0a1dabe34ee Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 27 Jul 2015 17:26:17 -0700 Subject: [PATCH 095/275] refactored API --- .../confluent/streaming/KStreamContext.java | 6 ++--- .../io/confluent/streaming/StateStore.java | 4 +-- .../internal/KStreamContextImpl.java | 6 ++--- .../internal/ProcessorStateManager.java | 7 +++-- .../streaming/kv/InMemoryKeyValueStore.java | 26 +++++++++++++++---- .../kv/internals/MeteredKeyValueStore.java | 4 +-- .../streaming/kv/internals/RestoreFunc.java | 11 ++++++++ 7 files changed, 45 insertions(+), 19 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index f0a83a3f67384..584329a0fd604 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -1,6 +1,7 @@ package io.confluent.streaming; import io.confluent.streaming.internal.StreamGroup; +import io.confluent.streaming.kv.internals.RestoreFunc; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; @@ -109,10 +110,9 @@ public interface KStreamContext { /** * Restores the specified storage engine. - * @param engine the storage engine - * @throws Exception an exception thrown by the engine + * @param store the storage engine */ - void restore(StateStore engine) throws Exception; + void restore(StateStore store, RestoreFunc restoreFunc); /** * Ensures that the context is in the initialization phase where KStream topology can be constructed diff --git a/src/main/java/io/confluent/streaming/StateStore.java b/src/main/java/io/confluent/streaming/StateStore.java index 2075bd5c3d3a4..ca71decf3142e 100644 --- a/src/main/java/io/confluent/streaming/StateStore.java +++ b/src/main/java/io/confluent/streaming/StateStore.java @@ -49,7 +49,7 @@ public interface StateStore { void close(); /** - * Consume one changelog record to restore its recorded state change + * Restore the state of the storage */ - void restore(ConsumerRecord record); + void restore(); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index f8e42a1fcd2bc..e3587f411fbb8 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -8,6 +8,7 @@ import io.confluent.streaming.StateStore; import io.confluent.streaming.StreamingConfig; import io.confluent.streaming.TimestampExtractor; +import io.confluent.streaming.kv.internals.RestoreFunc; import io.confluent.streaming.util.Util; <<<<<<< HEAD import org.apache.kafka.clients.consumer.Consumer; @@ -234,12 +235,11 @@ private StreamGroup streamGroup(String name, Chooser chooser) { } } - @Override - public void restore(StateStore engine) throws Exception { + public void restore(StateStore store, RestoreFunc restoreFunc) { ensureInitialization(); - stateMgr.registerAndRestore(collector, engine); + stateMgr.registerAndRestore(store, restoreFunc); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index 5befcc6d7824c..b066bcc9ed2d5 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -16,13 +16,12 @@ */ package io.confluent.streaming.internal; -import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StateStore; +import io.confluent.streaming.kv.internals.RestoreFunc; import io.confluent.streaming.util.OffsetCheckpoint; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.Deserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,7 +65,7 @@ public void init() throws IOException { checkpoint.delete(); } - public void registerAndRestore(StateStore store) { + public void registerAndRestore(StateStore store, RestoreFunc restoreFunc) { if (store.name().equals(CHECKPOINT_FILE_NAME)) throw new IllegalArgumentException("Illegal store name: " + CHECKPOINT_FILE_NAME); @@ -94,7 +93,7 @@ public void registerAndRestore(StateStore store) { // restore its state from changelog records while (true) { for(ConsumerRecord record: restoreConsumer.poll(100)) - store.restore(record); + restoreFunc.apply(record.key(), record.value()); long position = restoreConsumer.position(storePartition); if (position == endOffset) diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index 144bbe0bc048b..6f89ff38bf9c7 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -2,14 +2,21 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.RecordCollector; -import io.confluent.streaming.internal.KStreamContextImpl; import io.confluent.streaming.kv.internals.MeteredKeyValueStore; +import io.confluent.streaming.kv.internals.RestoreFunc; 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.utils.SystemTime; -import java.util.*; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import java.util.TreeMap; /** * An in-memory key-value store based on a TreeMap @@ -23,7 +30,7 @@ public InMemoryKeyValueStore(String name, KStreamContext context) { super(name, "kafka-streams", new MemoryStore(name, context), context.metrics(), new SystemTime()); } - private static class MemoryStore extends KeyValueStore { + private static class MemoryStore implements KeyValueStore { private final String topic; private final int partition; @@ -99,8 +106,17 @@ public void flush() { } @Override - public void restore(ConsumerRecord record) { - + public void restore() { + final Deserializer keyDeserializer = (Deserializer) context.keySerializer(); + final Deserializer valDeserializer = (Deserializer) context.valueSerializer(); + + context.restore(this, new RestoreFunc () { + @Override + public void apply(byte[] key, byte[] value) { + map.put(keyDeserializer.deserialize(topic, key), + valDeserializer.deserialize(topic, value)); + } + }); } @Override diff --git a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java index eb95eadbaef6a..c59bc94b9c70f 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java @@ -71,10 +71,10 @@ public String name() { } @Override - public void restore(ConsumerRecord record) { + public void restore() { long startNs = time.nanoseconds(); try { - inner.restore(record); + inner.restore(); } finally { recordLatency(this.restoreTime, startNs, time.nanoseconds()); } diff --git a/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java b/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java new file mode 100644 index 0000000000000..096bbd2c86c1f --- /dev/null +++ b/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java @@ -0,0 +1,11 @@ +package io.confluent.streaming.kv.internals; + +/** + * Created by guozhang on 7/27/15. + */ + +// TODO: this should be removed once we move to Java 8 +public interface RestoreFunc { + + void apply(byte[] key, byte[] value); +} From d6c00da1a020fb41cb7b691c140dbfb9765b3774 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 27 Jul 2015 17:55:45 -0700 Subject: [PATCH 096/275] add an example for using local state store --- .../streaming/ProcessorKStreamJob.java | 6 ++ .../streaming/examples/PrintKStreamJob.java | 8 +-- .../examples/StatefulKStreamJob.java | 57 +++++++++++++++++++ 3 files changed, 67 insertions(+), 4 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java index 065fe9b238c96..aab194114e699 100644 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -5,9 +5,15 @@ */ public abstract class ProcessorKStreamJob implements KStreamJob, Processor { + protected KStreamContext streamContext; + @SuppressWarnings("unchecked") @Override public void init(KStreamContext context) { +<<<<<<< HEAD +======= + this.streamContext = context; +>>>>>>> add an example for using local state store context.from().process((Processor) this); } diff --git a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java index 9428f5ada2527..d8867a26acdbb 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java @@ -11,20 +11,20 @@ */ public class PrintKStreamJob extends ProcessorKStreamJob { - private ProcessorContext context; + private ProcessorContext processorContext; @Override public void init(ProcessorContext context) { - this.context = context; + this.processorContext = context; } @Override public void process(K key, V value) { System.out.println("[" + key + ", " + value + "]"); - context.commit(); + processorContext.commit(); - context.send("topic", key, value); + processorContext.send("topic", key, value); } @Override diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java new file mode 100644 index 0000000000000..958ab45d1d891 --- /dev/null +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -0,0 +1,57 @@ +package io.confluent.streaming.examples; + +import io.confluent.streaming.KafkaStreaming; +import io.confluent.streaming.ProcessorKStreamJob; +import io.confluent.streaming.StreamingConfig; +import io.confluent.streaming.kv.Entry; +import io.confluent.streaming.kv.InMemoryKeyValueStore; +import io.confluent.streaming.kv.KeyValueIterator; +import io.confluent.streaming.kv.KeyValueStore; + +import java.util.Properties; + +/** + * Created by guozhang on 7/27/15. + */ + +public class StatefulKStreamJob extends ProcessorKStreamJob { + + private ProcessorContext processorContext; + private KeyValueStore kvStore; + + @Override + public void init(ProcessorContext context) { + this.processorContext = context; + this.processorContext.schedule(1000); + + this.kvStore = new InMemoryKeyValueStore<>("local-state", this.streamContext); + this.kvStore.restore(); // call restore inside processor.init + } + + @Override + public void process(String key, Integer value) { + Integer oldValue = this.kvStore.get(key); + if (oldValue == null) { + this.kvStore.put(key, value); + } else { + int newValue = oldValue + value; + this.kvStore.put(key, newValue); + } + + processorContext.commit(); + } + + @Override + public void punctuate(long streamTime) { + KeyValueIterator iter = this.kvStore.all(); + while (iter.hasNext()) { + Entry entry = iter.next(); + System.out.println("[" + entry.key() + ", " + entry.value() + "]"); + } + } + + public static void main(String[] args) { + KafkaStreaming kstream = new KafkaStreaming(PrintKStreamJob.class, new StreamingConfig(new Properties())); + kstream.run(); + } +} \ No newline at end of file From 39a45aa0d9dccde10784b8067c0794070d7896a9 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 28 Jul 2015 13:43:33 -0700 Subject: [PATCH 097/275] Address Yasu's comments --- .../streaming/internal/KStreamThread.java | 6 +----- .../internal/ProcessorStateManager.java | 16 +++++++++------- .../streaming/kv/InMemoryKeyValueStore.java | 2 +- .../kv/internals/MeteredKeyValueStore.java | 1 - 4 files changed, 11 insertions(+), 14 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index 5a9a51706c006..0e731d5dbc0cf 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -240,9 +240,6 @@ private void addPartitions(Collection assignment) { ingestor.init(); - Consumer restoreConsumer = - new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); - for (TopicPartition partition : partitions) { final Integer id = partition.partition(); KStreamContextImpl kstreamContext = kstreamContexts.get(id); @@ -255,7 +252,7 @@ private void addPartitions(Collection assignment) { kstreamContexts.put(id, kstreamContext); try { - kstreamContext.init(restoreConsumer); + kstreamContext.init(); } catch (Exception e) { throw new KafkaException(e); @@ -268,7 +265,6 @@ private void addPartitions(Collection assignment) { } } - restoreConsumer.close(); nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; } diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index b066bcc9ed2d5..18c6b1a23f0e4 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -91,15 +91,17 @@ public void registerAndRestore(StateStore store, RestoreFunc restoreFunc) { restoreConsumer.seek(storePartition, checkpointedOffset); // restore its state from changelog records - while (true) { - for(ConsumerRecord record: restoreConsumer.poll(100)) + boolean consumedToEnd = false; + while (!consumedToEnd) { + for(ConsumerRecord record: restoreConsumer.poll(100)) { + if (record.offset() > endOffset) { + consumedToEnd = true; + break; + } + restoreFunc.apply(record.key(), record.value()); + } - long position = restoreConsumer.position(storePartition); - if (position == endOffset) - break; - else if(position > endOffset) - throw new IllegalStateException("This should not happen."); } // record the restored offset for its partition diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index 6f89ff38bf9c7..cd70d60227518 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -4,7 +4,6 @@ import io.confluent.streaming.RecordCollector; import io.confluent.streaming.kv.internals.MeteredKeyValueStore; import io.confluent.streaming.kv.internals.RestoreFunc; -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; @@ -90,6 +89,7 @@ public KeyValueIterator all() { return new MemoryStoreIterator(this.map.entrySet().iterator()); } + @SuppressWarnings("unchecked") @Override public void flush() { RecordCollector collector = context.recordCollector(); diff --git a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java index c59bc94b9c70f..4f31f979e4bfd 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java @@ -3,7 +3,6 @@ import io.confluent.streaming.kv.Entry; import io.confluent.streaming.kv.KeyValueIterator; import io.confluent.streaming.kv.KeyValueStore; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.MeasurableStat; import org.apache.kafka.common.metrics.Metrics; From bad7c833cbc32f60ff51d77c1cd82bfd1e80a256 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 28 Jul 2015 14:03:40 -0700 Subject: [PATCH 098/275] close stream groups in context --- .../confluent/streaming/KafkaStreaming.java | 2 + .../internal/KStreamContextImpl.java | 25 +++++++--- .../streaming/internal/KStreamThread.java | 47 +++++++------------ 3 files changed, 37 insertions(+), 37 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index e5ba1423b58d8..e77b2fc88d1c9 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -176,6 +176,8 @@ >>>>>>> use poll(0) for non-blocking poll public KafkaStreaming(Class jobClass, StreamingConfig streamingConfig) { >>>>>>> added KStreamThread + if (streamingConfig.timestampExtractor() == null) throw new NullPointerException("timestamp extractor is missing"); + this.config = new ProcessorConfig(streamingConfig.config()); try { this.topics = new HashSet<>(Arrays.asList(this.config.topics.split(","))); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index e3587f411fbb8..949bbbba5592e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -55,7 +55,6 @@ public class KStreamContextImpl implements KStreamContext { private final StreamingConfig streamingConfig; private final ProcessorConfig processorConfig; private final Metrics metrics; - private final File stateDir; private final ProcessorStateManager stateMgr; private boolean initialized = false; @@ -74,13 +73,16 @@ public KStreamContextImpl(int id, this.collector = collector; this.streamingConfig = streamingConfig; this.processorConfig = processorConfig; - this.timestampExtractor = this.streamingConfig.timestampExtractor(); +<<<<<<< HEAD if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractor is missing"); this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id)), new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer())); this.stateDir = this.stateMgr.baseDir(); +======= + this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id))); +>>>>>>> close stream groups in context this.metrics = metrics; } @@ -199,7 +201,7 @@ public Map getContext() { @Override public File stateDir() { - return stateDir; + return stateMgr.baseDir(); } public ProcessorStateManager stateMgr() { @@ -248,7 +250,12 @@ public void ensureInitialization() { throw new IllegalStateException("context initialization is already finished"); } - public Collection streamSynchronizers() { + @Override + public void flush() { + stateMgr.flush(); + } + + public Collection streamGroups() { return streamGroups.values(); } @@ -275,13 +282,17 @@ public void init() throws IOException { initialized = true; } - @Override - public void flush() { - stateMgr.flush(); + public void getConsumedOffsets(Map offsets) { + for (StreamGroup streamGroup : streamGroups.values()) + offsets.putAll(streamGroup.consumedOffsets()); } public void close() throws Exception { stateMgr.close(collector.offsets()); + + for (StreamGroup streamGroup : streamGroups.values()) + streamGroup.close(); + job.close(); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index 0e731d5dbc0cf..31c647a874ac5 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -75,12 +75,15 @@ public class KStreamThread extends Thread { protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { @Override public void onPartitionsAssigned(Consumer consumer, Collection assignment) { + ingestor.init(); addPartitions(assignment); } @Override public void onPartitionsRevoked(Consumer consumer, Collection assignment) { - removePartitions(assignment); + commitAll(time.milliseconds()); + removePartitions(); + ingestor.clear(); } }; @@ -113,6 +116,7 @@ public KStreamThread(Class jobClass, Set topics, S * Execute the stream processors */ public synchronized void run() { + log.info("Starting a kstream thread"); try { ingestor.open(); runLoop(); @@ -128,24 +132,15 @@ private void shutdown() { log.info("Shutting down a kstream thread"); commitAll(time.milliseconds()); - for (StreamGroup streamGroup : streamGroups) { - try { - streamGroup.close(); - } - catch(Exception e) { - log.error("Error while closing stream groups: ", e); - } - } - collector.close(); ingestor.close(); parallelExecutor.shutdown(); - streamGroups.clear(); + removePartitions(); log.info("kstream thread shutdown complete"); } /** - * Shutdown this streaming instance. + * Shutdown this streaming thread. */ public synchronized void close() { running = false; @@ -182,7 +177,7 @@ private boolean stillRunning() { private void maybeCommit() { long now = time.milliseconds(); - if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < time.milliseconds()) { + if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < now) { log.trace("Committing processor instances because the commit interval has elapsed."); commitAll(now); } @@ -192,14 +187,7 @@ private void commitAll(long now) { Map commit = new HashMap<>(); for (KStreamContextImpl context : kstreamContexts.values()) { context.flush(); - } - for (StreamGroup streamGroup : streamGroups) { - try { - commit.putAll(streamGroup.consumedOffsets()); - } - catch(Exception e) { - log.error("Error while closing processor: ", e); - } + context.getConsumedOffsets(commit); } // check if commit is really needed, i.e. if all the offsets are already committed @@ -208,7 +196,7 @@ private void commitAll(long now) { // are executed atomically whenever it is triggered by user collector.flush(); ingestor.commit(commit); // TODO: can this be async? - streamingMetrics.commitTime.record(time.milliseconds() - lastCommit); + streamingMetrics.commitTime.record(now - lastCommit); } } @@ -238,7 +226,12 @@ private void maybeCleanState() { private void addPartitions(Collection assignment) { HashSet partitions = new HashSet<>(assignment); +<<<<<<< HEAD ingestor.init(); +======= + Consumer restoreConsumer = + new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); +>>>>>>> close stream groups in context for (TopicPartition partition : partitions) { final Integer id = partition.partition(); @@ -258,7 +251,7 @@ private void addPartitions(Collection assignment) { throw new KafkaException(e); } - Collection streamGroups = kstreamContext.streamSynchronizers(); + Collection streamGroups = kstreamContext.streamGroups(); for (StreamGroup streamGroup : streamGroups) { streamGroups.add(streamGroup); } @@ -268,12 +261,7 @@ private void addPartitions(Collection assignment) { nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; } - private void removePartitions(Collection assignment) { - commitAll(time.milliseconds()); - for (StreamGroup streamGroup : streamGroups) { - log.info("Removing synchronization groups {}", streamGroup.name()); - streamGroup.close(); - } + private void removePartitions() { for (KStreamContextImpl kstreamContext : kstreamContexts.values()) { log.info("Removing stream context {}", kstreamContext.id()); try { @@ -285,7 +273,6 @@ private void removePartitions(Collection assignment) { streamingMetrics.processorDestruction.record(); } streamGroups.clear(); - ingestor.clear(); } private class KafkaStreamingMetrics { From 71c9e5e33b3d48212b36166073ff7c66b9ddf352 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 28 Jul 2015 15:31:13 -0700 Subject: [PATCH 099/275] rename getConsumedOffsets to putConsumedOffsetsTo --- .../io/confluent/streaming/internal/KStreamContextImpl.java | 2 +- .../java/io/confluent/streaming/internal/KStreamThread.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 949bbbba5592e..6e3cbf8e8219e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -282,7 +282,7 @@ public void init() throws IOException { initialized = true; } - public void getConsumedOffsets(Map offsets) { + public void putConsumedOffsetsTo(Map offsets) { for (StreamGroup streamGroup : streamGroups.values()) offsets.putAll(streamGroup.consumedOffsets()); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index 31c647a874ac5..bb59c0261a21f 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -187,7 +187,7 @@ private void commitAll(long now) { Map commit = new HashMap<>(); for (KStreamContextImpl context : kstreamContexts.values()) { context.flush(); - context.getConsumedOffsets(commit); + context.putConsumedOffsetsTo(commit); } // check if commit is really needed, i.e. if all the offsets are already committed From 05035ec3e855abbd4c93f9a9ac95b5b90e9bbd02 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 28 Jul 2015 16:34:26 -0700 Subject: [PATCH 100/275] renamed WindowByTime to SlidingWindow --- .../{WindowByTime.java => SlidingWindow.java} | 25 +++++++++++++++++-- .../java/io/confluent/streaming/Window.java | 2 +- .../streaming/testutil/UnlimitedWindow.java | 16 ++++++++++++ 3 files changed, 40 insertions(+), 3 deletions(-) rename src/main/java/io/confluent/streaming/{WindowByTime.java => SlidingWindow.java} (86%) diff --git a/src/main/java/io/confluent/streaming/WindowByTime.java b/src/main/java/io/confluent/streaming/SlidingWindow.java similarity index 86% rename from src/main/java/io/confluent/streaming/WindowByTime.java rename to src/main/java/io/confluent/streaming/SlidingWindow.java index ba42f3771d845..668cb88644482 100644 --- a/src/main/java/io/confluent/streaming/WindowByTime.java +++ b/src/main/java/io/confluent/streaming/SlidingWindow.java @@ -10,14 +10,16 @@ /** * Created by yasuhiro on 6/18/15. */ -public class WindowByTime implements Window { +public class SlidingWindow implements Window { + private String name; private final long duration; private final int maxCount; private LinkedList list = new LinkedList(); private HashMap>> map = new HashMap>>(); - public WindowByTime(long duration, int maxCount) { + public SlidingWindow(String name, long duration, int maxCount) { + this.name = name; this.duration = duration; this.maxCount = maxCount; } @@ -105,4 +107,23 @@ private void evictExpired(long cutoffTime) { } } + @Override + public String name() { + return name; + } + + @Override + public void flush() { + // TODO + } + + @Override + public void close() { + // TODO + } + + @Override + public void restore() { + // TODO + } } diff --git a/src/main/java/io/confluent/streaming/Window.java b/src/main/java/io/confluent/streaming/Window.java index dc949568d9f0f..d3b500f19ff4b 100644 --- a/src/main/java/io/confluent/streaming/Window.java +++ b/src/main/java/io/confluent/streaming/Window.java @@ -5,7 +5,7 @@ /** * Created by yasuhiro on 6/17/15. */ -public interface Window { +public interface Window extends StateStore { Iterator find(K key, long timestamp); diff --git a/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java b/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java index 98a897dc8a77d..d0973f5d28b39 100644 --- a/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java +++ b/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java @@ -42,4 +42,20 @@ public void put(K key, V value, long timestamp) { list.add(new Stamped>(KeyValue.pair(key, value), timestamp)); } + @Override + public String name() { + return null; + } + + @Override + public void flush() { + } + + @Override + public void close() { + } + + @Override + public void restore() { + } } From 5370b797c2ff1f0164adead8760025ea95968434 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 30 Jul 2015 11:52:00 -0700 Subject: [PATCH 101/275] complete the state flush / restore workflow --- .../io/confluent/streaming/SlidingWindow.java | 6 ++ .../io/confluent/streaming/StateStore.java | 5 ++ .../internal/ProcessorStateManager.java | 55 ++++++++++++------- .../streaming/kv/InMemoryKeyValueStore.java | 9 +++ .../kv/internals/MeteredKeyValueStore.java | 5 ++ .../streaming/kv/internals/RestoreFunc.java | 2 + .../streaming/testutil/UnlimitedWindow.java | 5 ++ 7 files changed, 68 insertions(+), 19 deletions(-) diff --git a/src/main/java/io/confluent/streaming/SlidingWindow.java b/src/main/java/io/confluent/streaming/SlidingWindow.java index 668cb88644482..8dc2e92aa81df 100644 --- a/src/main/java/io/confluent/streaming/SlidingWindow.java +++ b/src/main/java/io/confluent/streaming/SlidingWindow.java @@ -126,4 +126,10 @@ public void close() { public void restore() { // TODO } + + @Override + public boolean persistent() { + // TODO: should not be persistent, right? + return false; + } } diff --git a/src/main/java/io/confluent/streaming/StateStore.java b/src/main/java/io/confluent/streaming/StateStore.java index ca71decf3142e..4beb3d68453c6 100644 --- a/src/main/java/io/confluent/streaming/StateStore.java +++ b/src/main/java/io/confluent/streaming/StateStore.java @@ -52,4 +52,9 @@ public interface StateStore { * Restore the state of the storage */ void restore(); + + /** + * If the storage is persistent + */ + boolean persistent(); } diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index 18c6b1a23f0e4..051a390525516 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -77,6 +77,9 @@ public void registerAndRestore(StateStore store, RestoreFunc restoreFunc) { // subscribe to the store's partition TopicPartition storePartition = new TopicPartition(store.name(), id); + if (!restoreConsumer.subscriptions().isEmpty()) { + throw new IllegalStateException("Restore consumer should have not subscribed to any partitions beforehand"); + } restoreConsumer.subscribe(storePartition); // calculate the end offset of the partition @@ -84,28 +87,34 @@ public void registerAndRestore(StateStore store, RestoreFunc restoreFunc) { restoreConsumer.seekToEnd(storePartition); long endOffset = restoreConsumer.position(storePartition); - // reset the consumer to the written offset when we shutdown last time - long checkpointedOffset = 0; - if (checkpointedOffsets.containsKey(storePartition)) - checkpointedOffset = checkpointedOffsets.get(storePartition); - restoreConsumer.seek(storePartition, checkpointedOffset); + // 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)) { + restoreFunc.load(); + restoreConsumer.seek(storePartition, checkpointedOffsets.get(storePartition)); + } else { + restoreConsumer.seekToBeginning(storePartition); + } - // restore its state from changelog records - boolean consumedToEnd = false; - while (!consumedToEnd) { + // restore its state from changelog records; while restoring the log end offset + // should not change since it is only written by this thread. + while (true) { for(ConsumerRecord record: restoreConsumer.poll(100)) { - if (record.offset() > endOffset) { - consumedToEnd = true; - break; - } - 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 partition - restoredOffsets.put(storePartition, restoreConsumer.position(storePartition)); + // record the restored offset for its change log partition + long newOffset = restoreConsumer.position(storePartition); + restoredOffsets.put(storePartition, newOffset); + + // un-subscribe the change log partition restoreConsumer.unsubscribe(storePartition); } @@ -139,15 +148,23 @@ public void close(Map ackedOffsets) throws IOException { Map checkpointOffsets = new HashMap(restoredOffsets); for(String storeName: stores.keySet()) { TopicPartition part = new TopicPartition(storeName, id); - if(ackedOffsets.containsKey(part)) - // store the last ack'd offset + 1 (the log position after restoration) - checkpointOffsets.put(part, ackedOffsets.get(part) + 1); + + // only checkpoint the offset to the offsets file if it is persistent; + if (stores.get(storeName).persistent()) { + if(ackedOffsets.containsKey(part)) + // store the last ack'd offset + 1 (the log position after restoration) + checkpointOffsets.put(part, ackedOffsets.get(part) + 1); + } else { + checkpointOffsets.remove(part); + } } - // record that shutdown was clean + // write the checkpoint offset file to indicate clean shutdown OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); checkpoint.write(checkpointOffsets); } + + // close the restore consumer restoreConsumer.close(); } diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index cd70d60227518..9d9f9abe82c21 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -53,6 +53,9 @@ public String name() { return this.topic; } + @Override + public boolean persistent() { return false; } + @Override public V get(K key) { return this.map.get(key); @@ -116,6 +119,12 @@ public void apply(byte[] key, byte[] value) { map.put(keyDeserializer.deserialize(topic, key), valDeserializer.deserialize(topic, value)); } + + @Override + public void load() { + // this should not happen since it is in-memory, hence no state to load from disk + throw new IllegalStateException("This should not happen"); + } }); } diff --git a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java index 4f31f979e4bfd..f27617fc76dd8 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java @@ -69,6 +69,11 @@ public String name() { return inner.name(); } + @Override + public boolean persistent() { + return inner.persistent(); + } + @Override public void restore() { long startNs = time.nanoseconds(); diff --git a/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java b/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java index 096bbd2c86c1f..20afad2c50cb7 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java +++ b/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java @@ -8,4 +8,6 @@ public interface RestoreFunc { void apply(byte[] key, byte[] value); + + void load(); } diff --git a/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java b/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java index d0973f5d28b39..04ddcb147a01c 100644 --- a/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java +++ b/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java @@ -58,4 +58,9 @@ public void close() { @Override public void restore() { } + + @Override + public boolean persistent() { + return false; + } } From 9c6f3928d01724583fe3087bbf97d71f4ccd3efa Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 30 Jul 2015 14:07:22 -0700 Subject: [PATCH 102/275] Add the change log topics checking upon registration --- .../confluent/streaming/KStreamContext.java | 6 +++++ .../internal/KStreamContextImpl.java | 13 ++++++---- .../internal/ProcessorStateManager.java | 26 ++++++++++++++++++- .../streaming/kv/InMemoryKeyValueStore.java | 2 ++ 4 files changed, 41 insertions(+), 6 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 584329a0fd604..780f53f771ca1 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -114,6 +114,12 @@ public interface KStreamContext { */ void restore(StateStore store, RestoreFunc restoreFunc); + /** + * Registers the specified storage enging. + * @param store the storage engine + */ + void register(StateStore store); + /** * Ensures that the context is in the initialization phase where KStream topology can be constructed */ diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 6e3cbf8e8219e..cec55e421831f 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -204,10 +204,6 @@ public File stateDir() { return stateMgr.baseDir(); } - public ProcessorStateManager stateMgr() { - return stateMgr; - } - @Override public Metrics metrics() { return metrics; @@ -241,7 +237,14 @@ private StreamGroup streamGroup(String name, Chooser chooser) { public void restore(StateStore store, RestoreFunc restoreFunc) { ensureInitialization(); - stateMgr.registerAndRestore(store, restoreFunc); + stateMgr.restore(store, restoreFunc); + } + + @Override + public void register(StateStore store) { + ensureInitialization(); + + stateMgr.register(store); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index 051a390525516..ae3e45dff2737 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -19,8 +19,10 @@ import io.confluent.streaming.StateStore; import io.confluent.streaming.kv.internals.RestoreFunc; import io.confluent.streaming.util.OffsetCheckpoint; +import kafka.cluster.Partition; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,16 +67,38 @@ public void init() throws IOException { checkpoint.delete(); } - public void registerAndRestore(StateStore store, RestoreFunc restoreFunc) { + public void register(StateStore store) { 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."); + // check that the underlying change log topic exist or not + if (restoreConsumer.listTopics().keySet().contains(store.name())) { + // TODO: if we know the total number of context ids then we can just check if the #.partitions match that number. + // my id must be in the partition list + 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 { + // try to create a topic the the number of partitions + // TODO: this is not possible yet since we do not know the total number of ids. + } + // register store this.stores.put(store.name(), store); + } + public void restore(StateStore store, RestoreFunc restoreFunc) { // subscribe to the store's partition TopicPartition storePartition = new TopicPartition(store.name(), id); if (!restoreConsumer.subscriptions().isEmpty()) { diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index 9d9f9abe82c21..9fd3a0f99a482 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -46,6 +46,8 @@ public MemoryStore(String name, KStreamContext context) { this.dirty = new HashSet(); this.maxDirty = 100; this.context = context; + + this.context.register(this); } @Override From 15f212be1300316620e6c4c08865b11a099718f7 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 30 Jul 2015 14:36:45 -0700 Subject: [PATCH 103/275] minor comment fix --- .../io/confluent/streaming/internal/ProcessorStateManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index ae3e45dff2737..d9f2e6754f5cd 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -90,7 +90,7 @@ public void register(StateStore store) { throw new IllegalStateException("Store " + store.name() + "'s change log does not contain the partition for group " + id); } else { - // try to create a topic the the number of partitions + // try to create the topic with the number of partitions equal to the total number of store instances. // TODO: this is not possible yet since we do not know the total number of ids. } From beac8f333a01efd9f46b10282dc446f9886674db Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 30 Jul 2015 18:35:05 -0700 Subject: [PATCH 104/275] add rocksDB: wip --- pom.xml | 5 + .../streaming/kv/InMemoryKeyValueStore.java | 71 ++--- .../streaming/kv/RocksDBKeyValueStore.java | 260 ++++++++++++++++++ .../kv/internals/LoggedKeyValueStore.java | 141 ++++++++++ .../kv/internals/MeteredKeyValueStore.java | 2 +- 5 files changed, 425 insertions(+), 54 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java create mode 100644 src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java diff --git a/pom.xml b/pom.xml index f4baf9888eb10..d4fb05b23e229 100644 --- a/pom.xml +++ b/pom.xml @@ -41,6 +41,11 @@ slf4j-log4j12 1.7.6 + + org.rocksdb + rocksdbjni + 3.10.1 + diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index 9fd3a0f99a482..a97d5adf652c9 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -1,20 +1,14 @@ package io.confluent.streaming.kv; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.RecordCollector; +import io.confluent.streaming.kv.internals.LoggedKeyValueStore; import io.confluent.streaming.kv.internals.MeteredKeyValueStore; -import io.confluent.streaming.kv.internals.RestoreFunc; -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.utils.SystemTime; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.NavigableMap; -import java.util.Set; import java.util.TreeMap; /** @@ -26,25 +20,27 @@ public class InMemoryKeyValueStore extends MeteredKeyValueStore { public InMemoryKeyValueStore(String name, KStreamContext context) { - super(name, "kafka-streams", new MemoryStore(name, context), context.metrics(), new SystemTime()); + // always wrap the logged store with the metered store + // TODO: this may need to be relaxed in the future + super(name, /* topic name as store name */ + "kafka-streams", + new LoggedKeyValueStore<>(name, /* topic name as store name */ + new MemoryStore(name, context), + context), + context.metrics(), + new SystemTime()); } private static class MemoryStore implements KeyValueStore { - private final String topic; - private final int partition; - private final Set dirty; - private final int maxDirty; + private final String name; private final NavigableMap map; private final KStreamContext context; @SuppressWarnings("unchecked") public MemoryStore(String name, KStreamContext context) { - this.topic = name; - this.partition = context.id(); - this.map = new TreeMap(); - this.dirty = new HashSet(); - this.maxDirty = 100; + this.name = name; + this.map = new TreeMap<>(); this.context = context; this.context.register(this); @@ -52,7 +48,7 @@ public MemoryStore(String name, KStreamContext context) { @Override public String name() { - return this.topic; + return this.name; } @Override @@ -66,11 +62,6 @@ public V get(K key) { @Override public void put(K key, V value) { this.map.put(key, value); - if(context.recordCollector() != null) { - this.dirty.add(key); - if (this.dirty.size() > this.maxDirty) - flush(); - } } @Override @@ -94,45 +85,20 @@ public KeyValueIterator all() { return new MemoryStoreIterator(this.map.entrySet().iterator()); } - @SuppressWarnings("unchecked") @Override public void flush() { - RecordCollector collector = context.recordCollector(); - Serializer keySerializer = (Serializer) context.keySerializer(); - Serializer valueSerializer = (Serializer) context.valueSerializer(); - - if(collector != null) { - for (K k : this.dirty) { - V v = this.map.get(k); - collector.send(new ProducerRecord<>(this.topic, this.partition, k, v), keySerializer, valueSerializer); - } - this.dirty.clear(); - } + // do-nothing since it is in-memory } @Override public void restore() { - final Deserializer keyDeserializer = (Deserializer) context.keySerializer(); - final Deserializer valDeserializer = (Deserializer) context.valueSerializer(); - - context.restore(this, new RestoreFunc () { - @Override - public void apply(byte[] key, byte[] value) { - map.put(keyDeserializer.deserialize(topic, key), - valDeserializer.deserialize(topic, value)); - } - - @Override - public void load() { - // this should not happen since it is in-memory, hence no state to load from disk - throw new IllegalStateException("This should not happen"); - } - }); + // 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() { - flush(); + // do-nothing } private static class MemoryStoreIterator implements KeyValueIterator { @@ -163,5 +129,4 @@ public void close() {} } } - } diff --git a/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java new file mode 100644 index 0000000000000..94e211eed073f --- /dev/null +++ b/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java @@ -0,0 +1,260 @@ +package io.confluent.streaming.kv; + +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.kv.internals.MeteredKeyValueStore; +import org.apache.kafka.common.utils.SystemTime; + +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.TtlDB; +import org.rocksdb.WriteOptions; + +import java.io.File; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; + +/** + * Created by guozhang on 7/30/15. + */ +public class RocksDBKeyValueStore extends MeteredKeyValueStore { + + public RocksDBKeyValueStore(String name, KStreamContext context) { + super(name, "kafka-streams", new RocksDBStore(name, context), context.metrics(), new SystemTime()); + } + + private static class RocksDBStore implements KeyValueStore { + + private static final int TTL_NOT_USED = -1; + + // TODO: these values should be configurable + private static final long WRITE_BUFFER_SIZE = 32 * 1024 * 1024L; + private static final long BLOCK_CACHE_SIZE = 100 * 1024 * 1024L; + private static final long BLOCK_SIZE = 4096L; + private static final int TTL_SECONDS = TTL_NOT_USED; + private static final int MAX_WRITE_BUFFERS = 3; + private static final CompressionType COMPRESSION_TYPE = CompressionType.NO_COMPRESSION; + private static final CompactionStyle COMPACTION_STYLE = CompactionStyle.UNIVERSAL; + private static final String DB_FILE_DIR = "/tmp/rocksdb"; + + + private final RocksDB db; + private final String topic; + private final int partition; + private final KStreamContext context; + + private final Options options; + private final WriteOptions wOptions; + private final FlushOptions fOptions; + + @SuppressWarnings("unchecked") + public RocksDBStore(String name, KStreamContext 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); + + String dbName = this.topic + "." + this.partition; + db = openDB(new File(DB_FILE_DIR, dbName), this.options, TTL_SECONDS); + + this.context.register(this); + } + + private RocksDB openDB(File dir, Options options, int ttl) { + try { + if (ttl == TTL_NOT_USED) { + return RocksDB.open(options, dir.toString()); + } else { + throw new IllegalStateException("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 RuntimeException("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 RuntimeException("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 RuntimeException("Error while executing put " + key.toString() + " from store " + this.topic, e); + } + } + + @Override + public void putAll(List> entries) { + for (Entry entry : entries) + put(entry.key(), entry.value()); + } + + @Override + public void delete(byte[] key) { + put(key, null); + } + + @Override + public KeyValueIterator range(byte[] from, byte[] to) { + return new RocksDBRangeIterator(db.newIterator(), from, to); + } + + @Override + public KeyValueIterator all() { + RocksIterator innerIter = db.newIterator(); + innerIter.seekToFirst(); + return new RocksDbIterator(innerIter); + } + + @Override + public void flush() { + try { + db.flush(fOptions); + } catch (RocksDBException e) { + // TODO: this needs to be handled more accurately + throw new RuntimeException("Error while executing flush from store " + this.topic, e); + } + } + + @Override + public void restore() { + + } + + @Override + public void close() { + flush(); + db.close(); + } + + private static class RocksDbIterator implements KeyValueIterator { + private final RocksIterator iter; + + public RocksDbIterator(RocksIterator iter) { + this.iter = iter; + } + + protected byte[] peekKey() { + return this.getEntry().key(); + } + + protected Entry getEntry() { + return new Entry<>(iter.key(), iter.value()); + } + + @Override + public boolean hasNext() { + return iter.isValid(); + } + + @Override + public Entry next() { + if (!hasNext()) + throw new NoSuchElementException(); + + Entry entry = this.getEntry(); + iter.next(); + + return entry; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("RocksDB iterator does not support remove"); + } + + @Override + public void close() { + } + + } + + private static class LexicographicComparator implements Comparator { + + @Override + public int compare(byte[] left, byte[] right) { + for (int i = 0, j = 0; i < left.length && j < right.length; i++, j++) { + int leftByte = (left[i] & 0xff); + int rightByte = (right[j] & 0xff); + if (leftByte != rightByte) { + return leftByte - rightByte; + } + } + return left.length - right.length; + } + } + + private static class RocksDBRangeIterator extends RocksDbIterator { + // RocksDB's JNI interface does not expose getters/setters that allow the + // comparator to be pluggable, and the default is lexicographic, so it's + // safe to just force lexicographic comparator here for now. + private final Comparator comparator = new LexicographicComparator(); + byte[] to; + + public RocksDBRangeIterator(RocksIterator iter, byte[] from, byte[] to) { + super(iter); + iter.seek(from); + this.to = to; + } + + @Override + public boolean hasNext() { + return super.hasNext() && comparator.compare(super.peekKey(), this.to) < 0; + } + } + + } +} diff --git a/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java new file mode 100644 index 0000000000000..83174bbda890a --- /dev/null +++ b/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java @@ -0,0 +1,141 @@ +package io.confluent.streaming.kv.internals; + +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.RecordCollector; +import io.confluent.streaming.kv.Entry; +import io.confluent.streaming.kv.KeyValueIterator; +import io.confluent.streaming.kv.KeyValueStore; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Created by guozhang on 7/30/15. + */ +public class LoggedKeyValueStore implements KeyValueStore { + + protected final KeyValueStore inner; + + private final String topic; + private final int partition; + private final KStreamContext context; + + private final Set dirty; + private final int maxDirty; + + + public LoggedKeyValueStore(String topic, KeyValueStore inner, KStreamContext context) { + this.inner = inner; + this.context = context; + + this.topic = topic; + this.partition = context.id(); + + this.dirty = new HashSet(); + this.maxDirty = 100; // TODO: this needs to be configurable + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public boolean persistent() { + return inner.persistent(); + } + + @Override + public void restore() { + final Deserializer keyDeserializer = (Deserializer) context.keySerializer(); + final Deserializer valDeserializer = (Deserializer) context.valueSerializer(); + + context.restore(this, new RestoreFunc () { + @Override + public void apply(byte[] key, byte[] value) { + inner.put(keyDeserializer.deserialize(topic, key), + valDeserializer.deserialize(topic, value)); + } + + @Override + public void load() { + inner.restore(); + } + }); + } + + @Override + public V get(K key) { + return inner.get(key); + } + + @Override + public void put(K key, V value) { + inner.put(key, value); + + this.dirty.add(key); + if (this.dirty.size() > this.maxDirty) + log(); + } + + @Override + public void putAll(List> entries) { + inner.putAll(entries); + + for (Entry entry : entries) { + this.dirty.add(entry.key()); + } + + if (this.dirty.size() > this.maxDirty) + log(); + } + + @Override + public void delete(K key) { + inner.delete(key); + + this.dirty.add(key); + if (this.dirty.size() > this.maxDirty) + log(); + + } + + @Override + public KeyValueIterator range(K from, K to) { + return inner.range(from, to); + } + + @Override + public KeyValueIterator all() { + return inner.all(); + } + + @Override + public void close() {} + + @SuppressWarnings("unchecked") + @Override + public void flush() { + // TODO: these two operations should be done atomically + log(); + inner.flush(); + } + + private void log() { + RecordCollector collector = context.recordCollector(); + Serializer keySerializer = (Serializer) context.keySerializer(); + Serializer valueSerializer = (Serializer) context.valueSerializer(); + + if(collector != null) { + for (K k : this.dirty) { + V v = this.inner.get(k); + collector.send(new ProducerRecord<>(this.topic, this.partition, k, v), keySerializer, valueSerializer); + } + this.dirty.clear(); + } + } +} diff --git a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java index f27617fc76dd8..539917e55a6ed 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java @@ -135,7 +135,7 @@ public KeyValueIterator all() { } @Override - public void close() {} + public void close() { inner.close(); } @Override public void flush() { From ee422e7f654bdee836607b42ad3e500c035ac524 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 09:49:43 -0700 Subject: [PATCH 105/275] new api model --- .../java/io/confluent/streaming/KStream.java | 6 - .../confluent/streaming/KStreamContext.java | 70 +++---- .../streaming/KStreamInitializer.java | 62 ++++++ .../io/confluent/streaming/KStreamJob.java | 4 +- .../confluent/streaming/KafkaStreaming.java | 16 +- .../io/confluent/streaming/Processor.java | 3 + .../streaming/ProcessorKStreamJob.java | 7 +- .../io/confluent/streaming/SlidingWindow.java | 4 + .../java/io/confluent/streaming/Window.java | 2 + .../streaming/examples/MapKStreamJob.java | 4 +- .../examples/StatefulKStreamJob.java | 6 +- .../streaming/internal/KStreamBranch.java | 26 ++- .../internal/KStreamContextImpl.java | 179 +++++++++++++----- .../streaming/internal/KStreamFilter.java | 6 +- .../streaming/internal/KStreamFlatMap.java | 10 +- .../internal/KStreamFlatMapValues.java | 6 +- .../streaming/internal/KStreamImpl.java | 43 +++-- .../internal/KStreamInitializerImpl.java | 78 ++++++++ .../streaming/internal/KStreamJoin.java | 24 ++- .../streaming/internal/KStreamMap.java | 10 +- .../streaming/internal/KStreamMapValues.java | 6 +- .../streaming/internal/KStreamMetadata.java | 7 +- .../streaming/internal/KStreamSource.java | 14 +- .../streaming/internal/KStreamThread.java | 38 ++-- .../internal/KStreamWindowedImpl.java | 21 +- .../internal/ProcessorContextImpl.java | 39 ++-- .../streaming/internal/Receiver.java | 4 + .../streaming/internal/StreamGroup.java | 16 +- .../streaming/internal/TopologyAnalyzer.java | 36 ++++ .../streaming/internal/KStreamBranchTest.java | 25 +-- .../streaming/internal/KStreamFilterTest.java | 29 +-- .../internal/KStreamFlatMapTest.java | 23 +-- .../internal/KStreamFlatMapValuesTest.java | 24 +-- .../streaming/internal/KStreamJoinTest.java | 123 ++++++++---- .../streaming/internal/KStreamMapTest.java | 23 +-- .../internal/KStreamMapValuesTest.java | 25 +-- .../streaming/internal/KStreamSourceTest.java | 23 +-- .../internal/KStreamWindowedTest.java | 25 +-- .../PunctuationSchedulerImplTest.java | 10 +- .../streaming/internal/StreamGroupTest.java | 6 +- .../testutil/MockKStreamContext.java | 46 +++-- .../streaming/testutil/UnlimitedWindow.java | 5 + 42 files changed, 699 insertions(+), 435 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/KStreamInitializer.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index 5f31104ff0527..7073f9375eebf 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -8,12 +8,6 @@ */ public interface KStream { - /** - * Returns the KStreamContext used to create this stream - * @return KStreamContext - */ - KStreamContext context(); - /** * Creates a new stream consists of all elements of this stream which satisfy a predicate * @param predicate the instance of Predicate diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 780f53f771ca1..704b4c3352f52 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -1,6 +1,5 @@ package io.confluent.streaming; -import io.confluent.streaming.internal.StreamGroup; import io.confluent.streaming.kv.internals.RestoreFunc; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; @@ -8,7 +7,6 @@ import java.io.File; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; /** * KStreamContext is the interface that allows an implementation of {@link KStreamJob#init(KStreamContext)} to create KStream instances. @@ -17,11 +15,9 @@ */ public interface KStreamContext { - AtomicInteger STREAM_GROUP_INDEX = new AtomicInteger(1); - /** - * Returns the partition id - * @return partition id + * Returns the partition group id + * @return partition group id */ int id(); @@ -49,27 +45,6 @@ public interface KStreamContext { */ Deserializer valueDeserializer(); - // TODO: support regex topic matching in from() calls, for example: - // context.from("Topic*PageView") - - /** - * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream - */ - KStream from(String... topics); - - /** - * 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 - */ - KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); - /** * Returns a RecordCollector * @return RecordCollector @@ -94,20 +69,6 @@ public interface KStreamContext { */ Metrics metrics(); - /** - * Creates a stream synchronization group with the given name. - * @param name the synchronization group name - * @return a synchronization group - */ - StreamGroup streamGroup(String name); - - /** - * Creates a round robin stream synchronization group with the given name. - * @param name the synchronization group name - * @return a round robin synchronization group - */ - StreamGroup roundRobinStreamGroup(String name); - /** * Restores the specified storage engine. * @param store the storage engine @@ -115,6 +76,7 @@ public interface KStreamContext { void restore(StateStore store, RestoreFunc restoreFunc); /** +<<<<<<< HEAD * Registers the specified storage enging. * @param store the storage engine */ @@ -122,11 +84,27 @@ public interface KStreamContext { /** * Ensures that the context is in the initialization phase where KStream topology can be constructed +======= + * Flush the local state of this context +>>>>>>> new api model */ - void ensureInitialization(); - - /** - * Flush the local state of this context - */ void flush(); + + + void send(String topic, Object key, Object value); + + void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); + + PunctuationScheduler getPunctuationScheduler(Processor processor); + + void commit(); + + String topic(); + + int partition(); + + long offset(); + + long timestamp(); + } diff --git a/src/main/java/io/confluent/streaming/KStreamInitializer.java b/src/main/java/io/confluent/streaming/KStreamInitializer.java new file mode 100644 index 0000000000000..43464dc884adc --- /dev/null +++ b/src/main/java/io/confluent/streaming/KStreamInitializer.java @@ -0,0 +1,62 @@ +package io.confluent.streaming; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * KStreamContext is the interface that allows an implementation of {@link KStreamJob#init(KStreamInitializer)} to create KStream instances. + * It also provides access to the system resources for a stream processing job. + * An instance of KStreamContext is created for each partition. + */ +public interface KStreamInitializer { + + AtomicInteger STREAM_GROUP_INDEX = new AtomicInteger(1); + + /** + * 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(); + + // TODO: support regex topic matching in from() calls, for example: + // context.from("Topic*PageView") + + /** + * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + KStream from(String... topics); + + /** + * 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 + */ + KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); + +} diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java index b6b759e75419e..3e97475b8e9ad 100644 --- a/src/main/java/io/confluent/streaming/KStreamJob.java +++ b/src/main/java/io/confluent/streaming/KStreamJob.java @@ -17,14 +17,14 @@ public interface KStreamJob { * For example, *

*
-   *   public init(KStreamContext context) {
+   *   public bind(KStreamContext context) {
    *     KStream<Integer, PageView> pageViewStream = context.from("pageView").mapValues(...);
    *     KStream<Integer, AdClick> adClickStream = context.from("adClick").join(pageViewStream, ...).process(...);
    *   }
    * 
* @param context KStreamContext for this partition */ - void init(KStreamContext context); + void init(KStreamInitializer context); /** * Closes this partition of the stream processing job. diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index e77b2fc88d1c9..a0fe124fc7cab 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -20,6 +20,7 @@ import io.confluent.streaming.internal.KStreamThread; import io.confluent.streaming.internal.ProcessorConfig; <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.internal.IngestorImpl; <<<<<<< HEAD import io.confluent.streaming.internal.StreamSynchronizer; @@ -40,12 +41,13 @@ import org.apache.kafka.common.errors.InterruptException; ======= >>>>>>> added KStreamThread +======= +import io.confluent.streaming.internal.TopologyAnalyzer; +>>>>>>> new api model import org.apache.kafka.common.metrics.Metrics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; -import java.util.HashSet; import java.util.Set; /** @@ -179,12 +181,10 @@ public KafkaStreaming(Class jobClass, StreamingConfig stre if (streamingConfig.timestampExtractor() == null) throw new NullPointerException("timestamp extractor is missing"); this.config = new ProcessorConfig(streamingConfig.config()); - try { - this.topics = new HashSet<>(Arrays.asList(this.config.topics.split(","))); - } - catch (Exception e) { - throw new KStreamException("failed to get a topic list from the streaming config", e); - } + + TopologyAnalyzer topologyAnalyzer = new TopologyAnalyzer(jobClass, streamingConfig); + + this.topics = topologyAnalyzer.topics; <<<<<<< HEAD <<<<<<< HEAD diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index dccc39fff4939..568fb90f1a73d 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -24,6 +24,9 @@ public interface ProcessorContext { long offset(); long timestamp(); + + KStreamContext kstreamContext(); + } void init(ProcessorContext context); diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java index aab194114e699..85aa99670a716 100644 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -5,16 +5,19 @@ */ public abstract class ProcessorKStreamJob implements KStreamJob, Processor { - protected KStreamContext streamContext; - @SuppressWarnings("unchecked") @Override +<<<<<<< HEAD public void init(KStreamContext context) { <<<<<<< HEAD ======= this.streamContext = context; >>>>>>> add an example for using local state store context.from().process((Processor) this); +======= + public void init(KStreamInitializer initializer) { + initializer.from().process((Processor) this); +>>>>>>> new api model } @Override diff --git a/src/main/java/io/confluent/streaming/SlidingWindow.java b/src/main/java/io/confluent/streaming/SlidingWindow.java index 8dc2e92aa81df..48869d5fab809 100644 --- a/src/main/java/io/confluent/streaming/SlidingWindow.java +++ b/src/main/java/io/confluent/streaming/SlidingWindow.java @@ -24,6 +24,10 @@ public SlidingWindow(String name, long duration, int maxCount) { this.maxCount = maxCount; } + @Override + public void init(KStreamContext context) { + } + @Override public Iterator findAfter(K key, final long timestamp) { return find(key, timestamp, timestamp + duration); diff --git a/src/main/java/io/confluent/streaming/Window.java b/src/main/java/io/confluent/streaming/Window.java index d3b500f19ff4b..32eeb7536bcb6 100644 --- a/src/main/java/io/confluent/streaming/Window.java +++ b/src/main/java/io/confluent/streaming/Window.java @@ -7,6 +7,8 @@ */ public interface Window extends StateStore { + void init(KStreamContext context); + Iterator find(K key, long timestamp); Iterator findAfter(K key, long timestamp); diff --git a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java index 17e176e9a6855..6fe0823acd105 100644 --- a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java @@ -1,7 +1,7 @@ package io.confluent.streaming.examples; import io.confluent.streaming.KStream; -import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.KStreamJob; import io.confluent.streaming.KafkaStreaming; import io.confluent.streaming.KeyValue; @@ -19,7 +19,7 @@ public class MapKStreamJob implements KStreamJob { @SuppressWarnings("unchecked") @Override - public void init(KStreamContext context) { + public void init(KStreamInitializer context) { // With overriden de-serializer KStream stream1 = context.from(new StringDeserializer(), new StringDeserializer(), "topic1"); diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index 958ab45d1d891..50429c7d5c227 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -24,8 +24,8 @@ public void init(ProcessorContext context) { this.processorContext = context; this.processorContext.schedule(1000); - this.kvStore = new InMemoryKeyValueStore<>("local-state", this.streamContext); - this.kvStore.restore(); // call restore inside processor.init + this.kvStore = new InMemoryKeyValueStore<>("local-state", context.kstreamContext()); + this.kvStore.restore(); // call restore inside processor.bind } @Override @@ -54,4 +54,4 @@ public static void main(String[] args) { KafkaStreaming kstream = new KafkaStreaming(PrintKStreamJob.class, new StreamingConfig(new Properties())); kstream.run(); } -} \ No newline at end of file +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 2474f82422d7e..fc3d29675349a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -1,6 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.Predicate; import java.lang.reflect.Array; @@ -15,27 +16,32 @@ class KStreamBranch implements Receiver { final KStreamSource[] branches; @SuppressWarnings("unchecked") - KStreamBranch(Predicate[] predicates, KStreamMetadata streamMetadata, KStreamContext context) { + KStreamBranch(Predicate[] predicates, KStreamInitializer initializer) { this.predicates = Arrays.copyOf(predicates, predicates.length); this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { - branches[i] = new KStreamSource<>(streamMetadata, context); + branches[i] = new KStreamSource<>(null, initializer); + } + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + for (KStreamSource stream : branches) { + stream.bind(context, metadata); } } @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { - synchronized(this) { - for (int i = 0; i < predicates.length; i++) { - Predicate predicate = predicates[i]; - if (predicate.apply((K)key, (V)value)) { - branches[i].receive(key, value, timestamp, streamTime); - return; - } + for (int i = 0; i < predicates.length; i++) { + Predicate predicate = predicates[i]; + if (predicate.apply((K)key, (V)value)) { + branches[i].receive(key, value, timestamp, streamTime); + return; } - return; } + return; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index cec55e421831f..1713619db65ab 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -1,12 +1,12 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStream; -import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamException; -import io.confluent.streaming.KStreamJob; +import io.confluent.streaming.Processor; +import io.confluent.streaming.PunctuationScheduler; import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StateStore; import io.confluent.streaming.StreamingConfig; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.kv.internals.RestoreFunc; import io.confluent.streaming.util.Util; @@ -17,8 +17,12 @@ ======= >>>>>>> wip import org.apache.kafka.clients.consumer.KafkaConsumer; +<<<<<<< HEAD import org.apache.kafka.clients.producer.Producer; >>>>>>> wip +======= +import org.apache.kafka.clients.producer.ProducerRecord; +>>>>>>> new api model import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.ByteArrayDeserializer; @@ -44,14 +48,13 @@ public class KStreamContextImpl implements KStreamContext { private static final Logger log = LoggerFactory.getLogger(KStreamContextImpl.class); public final int id; - private final KStreamJob job; - private final Ingestor ingestor; - private final RecordCollectorImpl collector; + final StreamGroup streamGroup; + final Ingestor ingestor; + private final RecordCollectorImpl collector; private final HashMap> sourceStreams = new HashMap<>(); private final HashMap partitioningInfos = new HashMap<>(); private final TimestampExtractor timestampExtractor; - private final HashMap streamGroups = new HashMap<>(); private final StreamingConfig streamingConfig; private final ProcessorConfig processorConfig; private final Metrics metrics; @@ -61,14 +64,12 @@ public class KStreamContextImpl implements KStreamContext { @SuppressWarnings("unchecked") public KStreamContextImpl(int id, - KStreamJob job, - Ingestor ingestor, - RecordCollectorImpl collector, - StreamingConfig streamingConfig, - ProcessorConfig processorConfig, - Metrics metrics) { + Ingestor ingestor, + RecordCollectorImpl collector, + StreamingConfig streamingConfig, + ProcessorConfig processorConfig, + Metrics metrics) { this.id = id; - this.job = job; this.ingestor = ingestor; this.collector = collector; this.streamingConfig = streamingConfig; @@ -84,6 +85,7 @@ public KStreamContextImpl(int id, this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id))); >>>>>>> close stream groups in context this.metrics = metrics; + this.streamGroup = new StreamGroup(this, this.ingestor, new TimeBasedChooser(), this.timestampExtractor, this.processorConfig.bufferedRecordsPerPartition); } @Override @@ -112,6 +114,7 @@ public Deserializer valueDeserializer() { } @Override +<<<<<<< HEAD public KStream from(String... topics) { return from(streamGroup(getNextGroupName()), null, null, topics); } @@ -190,6 +193,8 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe } @Override +======= +>>>>>>> new api model public RecordCollector recordCollector() { return collector; } @@ -210,33 +215,36 @@ public Metrics metrics() { } @Override - public StreamGroup streamGroup(String name) { - return streamGroup(name, new TimeBasedChooser()); + public void restore(StateStore store, RestoreFunc restoreFunc) { + ensureInitialization(); + + stateMgr.registerAndRestore(store, restoreFunc); + } + + public void ensureInitialization() { + if (!initialized) + throw new IllegalStateException("context initialization is already finished"); } @Override - public StreamGroup roundRobinStreamGroup(String name) { - return streamGroup(name, new RoundRobinChooser()); + public void flush() { + stateMgr.flush(); } - private StreamGroup streamGroup(String name, Chooser chooser) { - int desiredUnprocessedPerPartition = processorConfig.bufferedRecordsPerPartition; + @Override + public String topic() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); - synchronized (this) { - StreamGroup streamGroup = streamGroups.get(name); - if (streamGroup == null) { - streamGroup = - new StreamGroup(name, ingestor, chooser, timestampExtractor, desiredUnprocessedPerPartition); - streamGroups.put(name, streamGroup); - } - return streamGroup; - } + return this.streamGroup.record().topic(); } @Override - public void restore(StateStore store, RestoreFunc restoreFunc) { - ensureInitialization(); + public int partition() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); +<<<<<<< HEAD stateMgr.restore(store, restoreFunc); } @@ -245,31 +253,62 @@ public void register(StateStore store) { ensureInitialization(); stateMgr.register(store); +======= + return this.streamGroup.record().partition(); +>>>>>>> new api model } @Override - public void ensureInitialization() { - if (!initialized) - throw new IllegalStateException("context initialization is already finished"); + public long offset() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); + + return this.streamGroup.record().offset(); } @Override - public void flush() { - stateMgr.flush(); + public long timestamp() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); + + return this.streamGroup.record().timestamp; } - public Collection streamGroups() { - return streamGroups.values(); + @Override + public void send(String topic, Object key, Object value) { + collector.send(new ProducerRecord<>(topic, key, value)); } - public void init() throws IOException { + @Override + public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { + if (keySerializer == null || valSerializer == null) + throw new IllegalStateException("key and value serializers must be specified"); + + collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); + } + + @Override + public void commit() { + this.streamGroup.commitOffset(); + } + + @Override + public PunctuationScheduler getPunctuationScheduler(Processor processor) { + return streamGroup.getPunctuationScheduler(processor); + } + + public void init(Collection> streams) throws IOException { stateMgr.init(); - job.init(this); + + for (KStreamSource stream: streams) { + KStreamMetadata metadata = linkStreamToTopics(stream); + + stream.bind(this, metadata); + } // add partition -> stream group mappings to the ingestor for (Map.Entry> entry : sourceStreams.entrySet()) { TopicPartition partition = new TopicPartition(entry.getKey(), id); - StreamGroup streamGroup = entry.getValue().metadata.streamGroup; ingestor.addPartitionStreamToGroup(streamGroup, partition); } @@ -285,18 +324,60 @@ public void init() throws IOException { initialized = true; } - public void putConsumedOffsetsTo(Map offsets) { - for (StreamGroup streamGroup : streamGroups.values()) - offsets.putAll(streamGroup.consumedOffsets()); + private KStreamMetadata linkStreamToTopics(KStreamSource stream) { + ensureInitialization(); + + Set fromTopics; + + synchronized (this) { + // if topics not specified, use all the topics be default + if (stream.topics == null || stream.topics.length == 0) { + fromTopics = ingestor.topics(); + } else { + fromTopics = Collections.unmodifiableSet(Util.mkSet(stream.topics)); + } + + // iterate over the topics and check if the stream has already been created for them + for (String topic : fromTopics) { + if (!ingestor.topics().contains(topic)) + throw new IllegalArgumentException("topic not subscribed: " + topic); + + if (sourceStreams.containsKey(topic)) + throw new IllegalArgumentException("another stream created with the same topic " + topic); + } + + // create stream metadata + Map topicPartitionInfos = new HashMap<>(); + for (String topic : fromTopics) { + PartitioningInfo partitioningInfo = this.partitioningInfos.get(topic); + + if (partitioningInfo == null) { + partitioningInfo = new PartitioningInfo(ingestor.numPartitions(topic)); + this.partitioningInfos.put(topic, partitioningInfo); + } + + topicPartitionInfos.put(topic, partitioningInfo); + } + + // update source stream map + for (String topic : fromTopics) { + sourceStreams.put(topic, stream); + + TopicPartition partition = new TopicPartition(topic, id); + streamGroup.addPartition(partition, stream); + } + + return new KStreamMetadata(topicPartitionInfos); + } + } + + public Map consumedOffsets() { + return streamGroup.consumedOffsets(); } public void close() throws Exception { stateMgr.close(collector.offsets()); - - for (StreamGroup streamGroup : streamGroups.values()) - streamGroup.close(); - - job.close(); + streamGroup.close(); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 80eba04e525a1..2f106cd33d256 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -1,6 +1,6 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.Predicate; /** @@ -10,8 +10,8 @@ class KStreamFilter extends KStreamImpl { private final Predicate predicate; - KStreamFilter(Predicate predicate, KStreamMetadata metadata, KStreamContext context) { - super(metadata, context); + KStreamFilter(Predicate predicate, KStreamInitializer initializer) { + super(initializer); this.predicate = predicate; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 77d356af10953..0a244ceff22fd 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -1,6 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; @@ -11,11 +12,16 @@ class KStreamFlatMap extends KStreamImpl { private final KeyValueMapper, K1, V1> mapper; - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, StreamGroup streamGroup, KStreamContext context) { - super(KStreamMetadata.unjoinable(streamGroup), context); + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamInitializer initializer) { + super(initializer); this.mapper = mapper; } + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + super.bind(context, KStreamMetadata.unjoinable()); + } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 9f6585ac32ffb..2ca0616fc4d48 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -1,6 +1,6 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.ValueMapper; /** @@ -10,8 +10,8 @@ class KStreamFlatMapValues extends KStreamImpl { private final ValueMapper, V1> mapper; - KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamMetadata streamMetadata, KStreamContext context) { - super(streamMetadata, context); + KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamInitializer initializer) { + super(initializer); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 75e51f78fae6a..ebc42867630d3 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -2,6 +2,7 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.KStreamWindowed; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.Predicate; @@ -19,23 +20,25 @@ abstract class KStreamImpl implements KStream, Receiver { private final ArrayList nextReceivers = new ArrayList<>(1); - final KStreamMetadata metadata; - final KStreamContext context; + protected KStreamMetadata metadata; + protected KStreamInitializer initializer; - protected KStreamImpl(KStreamMetadata metadata, KStreamContext context) { - context.ensureInitialization(); - this.metadata = metadata; - this.context = context; + protected KStreamImpl(KStreamInitializer initializer) { + this.initializer = initializer; } @Override - public KStreamContext context() { - return this.context; + public void bind(KStreamContext context, KStreamMetadata metadata) { + this.metadata = metadata; + int numReceivers = nextReceivers.size(); + for (int i = 0; i < numReceivers; i++) { + nextReceivers.get(i).bind(context, metadata); + } } @Override public KStream filter(Predicate predicate) { - return chain(new KStreamFilter(predicate, metadata, context)); + return chain(new KStreamFilter(predicate, initializer)); } @Override @@ -47,35 +50,34 @@ public boolean apply(K key, V value) { }); } - @Override public KStream map(KeyValueMapper mapper) { - return chain(new KStreamMap(mapper, metadata.streamGroup, context)); + return chain(new KStreamMap(mapper, initializer)); } @Override public KStream mapValues(ValueMapper mapper) { - return chain(new KStreamMapValues(mapper, metadata, context)); + return chain(new KStreamMapValues(mapper, initializer)); } @Override public KStream flatMap(KeyValueMapper, K, V> mapper) { - return chain(new KStreamFlatMap(mapper, metadata.streamGroup, context)); + return chain(new KStreamFlatMap(mapper, initializer)); } @Override public KStream flatMapValues(ValueMapper, V> mapper) { - return chain(new KStreamFlatMapValues(mapper, metadata, context)); + return chain(new KStreamFlatMapValues(mapper, initializer)); } @Override public KStreamWindowed with(Window window) { - return (KStreamWindowed)chain(new KStreamWindowedImpl(window, metadata, context)); + return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, initializer)); } @Override public KStream[] branch(Predicate... predicates) { - KStreamBranch branch = new KStreamBranch(predicates, metadata, context); + KStreamBranch branch = new KStreamBranch<>(predicates, initializer); registerReceiver(branch); return branch.branches; } @@ -89,8 +91,8 @@ public KStream through(String topic) { @SuppressWarnings("unchecked") @Override public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { - process(this.getSendProcessor(topic, keySerializer, valSerializer)); - return context.from(keyDeserializer, valDeserializer, topic); + process(this.getSendProcessor(topic, keySerializer, valSerializer)); + return initializer.from(keyDeserializer, valDeserializer, topic); } @Override @@ -126,9 +128,10 @@ public void punctuate(long streamTime) {} @SuppressWarnings("unchecked") @Override public void process(final Processor processor) { - processor.init(new ProcessorContextImpl(this.context, this.metadata.streamGroup, this.metadata.streamGroup.getPunctuationScheduler(processor))); - Receiver receiver = new Receiver() { + public void bind(KStreamContext context, KStreamMetadata metadata) { + processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); + } public void receive(Object key, Object value, long timestamp, long streamTime) { processor.process((K) key, (V) value); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java new file mode 100644 index 0000000000000..f4b58e674c1e9 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java @@ -0,0 +1,78 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KStream; +import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.StreamingConfig; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; + +/** + * Created by yasuhiro on 6/19/15. + */ +public class KStreamInitializerImpl implements KStreamInitializer { + + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + private final ArrayList> streams = new ArrayList<>(); + + KStreamInitializerImpl(StreamingConfig streamingConfig) { + this( + streamingConfig.keySerializer(), + streamingConfig.valueSerializer(), + streamingConfig.keyDeserializer(), + streamingConfig.valueDeserializer() + ); + } + + KStreamInitializerImpl(Serializer keySerializer,Serializer valueSerializer, Deserializer keyDeserializer, Deserializer valueDeserializer) { + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.keyDeserializer = keyDeserializer; + this.valueDeserializer = valueDeserializer; + } + + @Override + public Serializer keySerializer() { + return keySerializer; + } + + @Override + public Serializer valueSerializer() { + return valueSerializer; + } + + @Override + public Deserializer keyDeserializer() { + return keyDeserializer; + } + + @Override + public Deserializer valueDeserializer() { + return valueDeserializer; + } + + @Override + public KStream from(String... topics) { + return from(this.keyDeserializer(), this.valueDeserializer(), topics); + } + + @SuppressWarnings("unchecked") + @Override + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + + KStreamSource stream = new KStreamSource<>(topics, keyDeserializer, valDeserializer, this); + streams.add(stream); + return stream; + } + + Collection> sourceStreams() { + return Collections.unmodifiableCollection(streams); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index 023e51da7c262..59f392b4c93ed 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -1,6 +1,8 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.NotCopartitionedException; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; @@ -19,9 +21,14 @@ private static abstract class Finder { private final Finder finder2; private final ValueJoiner joiner; final Receiver receiverForOtherStream; + private KStreamMetadata thisMetadata; + private KStreamMetadata otherMetadata; - KStreamJoin(final Window window1, final Window window2, boolean prior, ValueJoiner joiner, KStreamMetadata streamMetadata, KStreamContext context) { - super(streamMetadata, context); + KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamInitializer initializer) { + super(initializer); + + final Window window1 = stream1.window; + final Window window2 = stream2.window; if (prior) { this.finder1 = new Finder() { @@ -53,6 +60,14 @@ Iterator find(K key, long timestamp) { this.receiverForOtherStream = getReceiverForOther(); } + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + super.bind(context, metadata); + + thisMetadata = metadata; + if (otherMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) throw new NotCopartitionedException(); + } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { @@ -66,6 +81,11 @@ public void receive(Object key, Object value, long timestamp, long streamTime) { private Receiver getReceiverForOther() { return new Receiver() { + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + otherMetadata = metadata; + if (thisMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) throw new NotCopartitionedException(); + } @SuppressWarnings("unchecked") @Override diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index 6a6797a7415ed..f6aefd7181e4a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -1,6 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; @@ -11,11 +12,16 @@ class KStreamMap extends KStreamImpl { private final KeyValueMapper mapper; - KStreamMap(KeyValueMapper mapper, StreamGroup streamGroup, KStreamContext context) { - super(KStreamMetadata.unjoinable(streamGroup), context); + KStreamMap(KeyValueMapper mapper, KStreamInitializer initializer) { + super(initializer); this.mapper = mapper; } + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + super.bind(context, KStreamMetadata.unjoinable()); + } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index c76eaee8db921..45921508e9fd5 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -1,6 +1,6 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.ValueMapper; /** @@ -10,8 +10,8 @@ class KStreamMapValues extends KStreamImpl { private final ValueMapper mapper; - KStreamMapValues(ValueMapper mapper, KStreamMetadata metadata, KStreamContext context) { - super(metadata, context); + KStreamMapValues(ValueMapper mapper, KStreamInitializer initializer) { + super(initializer); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java b/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java index 8520dffb4758a..e7493ba910a70 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java @@ -11,15 +11,14 @@ public class KStreamMetadata { public static String UNKNOWN_TOPICNAME = "__UNKNOWN_TOPIC__"; public static int UNKNOWN_PARTITION = -1; - public static KStreamMetadata unjoinable(StreamGroup streamGroup) { - return new KStreamMetadata(streamGroup, Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(UNKNOWN_PARTITION))); + public static KStreamMetadata unjoinable() { + return new KStreamMetadata(Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(UNKNOWN_PARTITION))); } public StreamGroup streamGroup; public final Map topicPartitionInfos; - KStreamMetadata(StreamGroup streamGroup, Map topicPartitionInfos) { - this.streamGroup = streamGroup; + KStreamMetadata(Map topicPartitionInfos) { this.topicPartitionInfos = topicPartitionInfos; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 55562f06c6a3c..0ad97cb3b9ff9 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -1,6 +1,6 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; import org.apache.kafka.common.serialization.Deserializer; /** @@ -11,14 +11,16 @@ class KStreamSource extends KStreamImpl { public final Deserializer keyDeserializer; public final Deserializer valueDeserializer; + final String[] topics; + @SuppressWarnings("unchecked") - KStreamSource(KStreamMetadata streamMetadata, KStreamContext context) { - this(streamMetadata, context, (Deserializer) context.keyDeserializer(), (Deserializer) context.valueDeserializer()); + KStreamSource(String[] topics, KStreamInitializer initializer) { + this(topics, (Deserializer) initializer.keyDeserializer(), (Deserializer) initializer.valueDeserializer(), initializer); } - KStreamSource(KStreamMetadata streamMetadata, KStreamContext context, Deserializer keyDeserializer, Deserializer valueDeserializer) { - super(streamMetadata, context); - + KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamInitializer initializer) { + super(initializer); + this.topics = topics; this.keyDeserializer = keyDeserializer; this.valueDeserializer = valueDeserializer; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index bb59c0261a21f..a4c8b29955345 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -17,6 +17,7 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamJob; import io.confluent.streaming.StreamingConfig; import io.confluent.streaming.util.ParallelExecutor; @@ -187,7 +188,7 @@ private void commitAll(long now) { Map commit = new HashMap<>(); for (KStreamContextImpl context : kstreamContexts.values()) { context.flush(); - context.putConsumedOffsetsTo(commit); + commit.putAll(context.consumedOffsets()); } // check if commit is really needed, i.e. if all the offsets are already committed @@ -234,27 +235,30 @@ private void addPartitions(Collection assignment) { >>>>>>> close stream groups in context for (TopicPartition partition : partitions) { - final Integer id = partition.partition(); - KStreamContextImpl kstreamContext = kstreamContexts.get(id); - if (kstreamContext == null) { - KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); + final Integer id = partition.partition(); // TODO: switch this to the group id + KStreamContextImpl context = kstreamContexts.get(id); + if (context == null) { + try { + KStreamInitializerImpl initializer = new KStreamInitializerImpl( + streamingConfig.keySerializer(), + streamingConfig.valueSerializer(), + streamingConfig.keyDeserializer(), + streamingConfig.valueDeserializer() + ); + KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); - kstreamContext = - new KStreamContextImpl(id, job, ingestor, collector, streamingConfig, config, metrics); + job.init(initializer); - kstreamContexts.put(id, kstreamContext); + context = new KStreamContextImpl(id, ingestor, collector, streamingConfig, config, metrics); + context.init(initializer.sourceStreams()); - try { - kstreamContext.init(); + kstreamContexts.put(id, context); } catch (Exception e) { throw new KafkaException(e); } - Collection streamGroups = kstreamContext.streamGroups(); - for (StreamGroup streamGroup : streamGroups) { - streamGroups.add(streamGroup); - } + streamGroups.add(context.streamGroup); } } @@ -262,10 +266,10 @@ private void addPartitions(Collection assignment) { } private void removePartitions() { - for (KStreamContextImpl kstreamContext : kstreamContexts.values()) { - log.info("Removing stream context {}", kstreamContext.id()); + for (KStreamContextImpl context : kstreamContexts.values()) { + log.info("Removing task context {}", context.id()); try { - kstreamContext.close(); + context.close(); } catch (Exception e) { throw new KafkaException(e); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 3070a6fb647d0..6a5070254f7b9 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -3,8 +3,8 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.KStreamWindowed; -import io.confluent.streaming.NotCopartitionedException; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; @@ -15,11 +15,17 @@ public class KStreamWindowedImpl extends KStreamImpl implements KStr final Window window; - KStreamWindowedImpl(Window window, KStreamMetadata streamMetadata, KStreamContext context) { - super(streamMetadata, context); + KStreamWindowedImpl(Window window, KStreamInitializer initializer) { + super(initializer); this.window = window; } + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + super.bind(context, metadata); + window.init(context); + } + @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp, long streamTime) { @@ -44,14 +50,7 @@ private KStream join(KStreamWindowed other, boolean prior KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; - if (!this.metadata.isJoinCompatibleWith(otherImpl.metadata)) throw new NotCopartitionedException(); - - // merge the other stream's group with this group - this.metadata.streamGroup.mergeStreamGroup(otherImpl.metadata.streamGroup); - otherImpl.metadata.streamGroup = this.metadata.streamGroup; - - KStreamJoin stream = - new KStreamJoin(this.window, otherImpl.window, prior, processor, this.metadata, context); + KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, initializer); otherImpl.registerReceiver(stream.receiverForOtherStream); return chain(stream); diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java index fefd5fd8c49c0..2874f8abbfe23 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java @@ -1,8 +1,8 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; import io.confluent.streaming.PunctuationScheduler; +import io.confluent.streaming.KStreamContext; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; @@ -12,70 +12,57 @@ public class ProcessorContextImpl implements Processor.ProcessorContext { private final KStreamContext context; - private final StreamGroup streamGroup; private final PunctuationScheduler scheduler; public ProcessorContextImpl(KStreamContext context, - StreamGroup streamGroup, PunctuationScheduler scheduler) { this.context = context; this.scheduler = scheduler; - this.streamGroup = streamGroup; } @Override public String topic() { - if (this.streamGroup.record() == null) - throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); - - return this.streamGroup.record().topic(); + return context.topic(); } @Override public int partition() { - if (this.streamGroup.record() == null) - throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); - - return this.streamGroup.record().partition(); + return context.partition(); } @Override public long offset() { - if (this.streamGroup.record() == null) - throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); - - return this.streamGroup.record().offset(); + return context.offset(); } @Override public long timestamp() { - if (this.streamGroup.record() == null) - throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); - - return this.streamGroup.record().timestamp; + return context.timestamp(); } @Override public void send(String topic, Object key, Object value) { - this.context.recordCollector().send(new ProducerRecord<>(topic, key, value)); + context.send(topic, key, value); } @Override public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { - if (keySerializer == null || valSerializer == null) - throw new IllegalStateException("key and value serializers must be specified"); - - context.recordCollector().send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); + context.send(topic, key, value, keySerializer, valSerializer); } @Override public void commit() { - this.streamGroup.commitOffset(); + context.commit(); } @Override public void schedule(long timestamp) { scheduler.schedule(timestamp); } + + @Override + public KStreamContext kstreamContext() { + return context; + } } diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 1cad1f810acdf..84ab9e87f6c17 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -1,10 +1,14 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; + /** * Created by yasuhiro on 6/17/15. */ public interface Receiver { + void bind(KStreamContext context, KStreamMetadata metadata); + void receive(Object key, Object value, long timestamp, long streamTime); } diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index 3fad8c0385aac..01a8e02d9a29e 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -1,5 +1,6 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; import io.confluent.streaming.PunctuationScheduler; import io.confluent.streaming.TimestampExtractor; @@ -33,7 +34,7 @@ public class StreamSynchronizer implements SyncGroup { public class StreamGroup implements ParallelExecutor.Task { >>>>>>> remove SyncGroup from user facing APIs:src/main/java/io/confluent/streaming/internal/StreamGroup.java - private final String name; + private final KStreamContext context; private final Ingestor ingestor; private final Chooser chooser; private final TimestampExtractor timestampExtractor; @@ -53,18 +54,18 @@ public class StreamGroup implements ParallelExecutor.Task { /** * Creates StreamGroup - * @param name the name of group + * @param context the task context * @param ingestor the instance of {@link Ingestor} * @param chooser the instance of {@link Chooser} * @param timestampExtractor the instance of {@link TimestampExtractor} * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic */ - StreamGroup(String name, + StreamGroup(KStreamContext context, Ingestor ingestor, Chooser chooser, TimestampExtractor timestampExtractor, int desiredUnprocessedPerPartition) { - this.name = name; + this.context = context; this.ingestor = ingestor; this.chooser = chooser; this.timestampExtractor = timestampExtractor; @@ -74,11 +75,14 @@ public class StreamGroup implements ParallelExecutor.Task { ======= } +<<<<<<< HEAD public String name() { return name; >>>>>>> removed some generics } +======= +>>>>>>> new api model public StampedRecord record() { return currRecord; } /** @@ -235,10 +239,10 @@ public boolean process() { // need to be done altogether with offset commit atomically if (commitRequested) { // flush local state - recordQueue.stream.context().flush(); + context.flush(); // flush produced records in the downstream - recordQueue.stream.context().recordCollector().flush(); + context.recordCollector().flush(); // commit consumed offsets ingestor.commit(consumedOffsets()); diff --git a/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java b/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java new file mode 100644 index 0000000000000..8dee74ed57a57 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java @@ -0,0 +1,36 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KStreamJob; +import io.confluent.streaming.StreamingConfig; +import org.apache.kafka.common.utils.Utils; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * Created by yasuhiro on 7/30/15. + */ +public class TopologyAnalyzer { + + public final Set topics; + public final Collection> streams; + + public TopologyAnalyzer(Class jobClass, StreamingConfig streamingConfig) { + KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); + KStreamInitializerImpl context = new KStreamInitializerImpl(streamingConfig); + + job.init(context); + + this.streams = context.sourceStreams(); + Set topics = new HashSet<>(); + for (KStreamSource stream : this.streams) { + for (String topic : stream.topics) { + topics.add(topic); + } + } + this.topics = Collections.unmodifiableSet(topics); + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index 92f0b403e3b6d..790d1252fc894 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -13,23 +12,9 @@ public class KStreamBranchTest { - private Ingestor ingestor = new MockIngestor(); - - private StreamGroup streamGroup = new StreamGroup( - "group", - ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - return 0L; - } - }, - 10 - ); - private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); @SuppressWarnings("unchecked") @Test @@ -56,12 +41,12 @@ public boolean apply(Integer key, String value) { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); KStreamSource stream; KStream[] branches; TestProcessor[] processors; - stream = new KStreamSource<>(streamMetadata, context); + stream = new KStreamSource<>(null, initializer); branches = stream.branch(isEven, isMultipleOfThree, isOdd); assertEquals(3, branches.length); @@ -80,7 +65,7 @@ public boolean apply(Integer key, String value) { assertEquals(1, processors[1].processed.size()); assertEquals(3, processors[2].processed.size()); - stream = new KStreamSource<>(streamMetadata, context); + stream = new KStreamSource<>(null, initializer); branches = stream.branch(isEven, isOdd, isMultipleOfThree); assertEquals(3, branches.length); @@ -91,6 +76,8 @@ public boolean apply(Integer key, String value) { branches[i].process(processors[i]); } + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 8ef24da8ce7c4..09ad3b88af5d0 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -12,23 +11,9 @@ public class KStreamFilterTest { - private Ingestor ingestor = new MockIngestor(); - - private StreamGroup streamGroup = new StreamGroup( - "group", - ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - return 0L; - } - }, - 10 - ); - private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); private Predicate isMultipleOfThree = new Predicate() { @Override @@ -41,14 +26,16 @@ public boolean apply(Integer key, String value) { public void testFilter() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, context, null, null); + stream = new KStreamSource<>(null, initializer); stream.filter(isMultipleOfThree).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } @@ -60,14 +47,16 @@ public void testFilter() { public void testFilterOut() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, context, null, null); + stream = new KStreamSource<>(null, initializer); stream.filterOut(isMultipleOfThree).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 6438dc431e6c2..6ff9b80a53095 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -13,23 +12,9 @@ public class KStreamFlatMapTest { - private Ingestor ingestor = new MockIngestor(); - - private StreamGroup streamGroup = new StreamGroup( - "group", - ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - return 0L; - } - }, - 10 - ); - private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testFlatMap() { @@ -48,14 +33,16 @@ public KeyValue> apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, context, null, null); + stream = new KStreamSource<>(null, initializer); stream.flatMap(mapper).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index e7b3ff28d6972..44b812d0855be 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -13,24 +12,9 @@ public class KStreamFlatMapValuesTest { - private Ingestor ingestor = new MockIngestor(); - - private StreamGroup streamGroup = new StreamGroup( - "group", - ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - return 0L; - } - }, - 10 - ); - private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); - + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testFlatMapValues() { @@ -48,14 +32,16 @@ public Iterable apply(String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, context, null, null); + stream = new KStreamSource<>(null, initializer); stream.flatMapValues(mapper).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 43a12f12000eb..3c0fdca855de2 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import io.confluent.streaming.testutil.UnlimitedWindow; @@ -16,23 +15,9 @@ public class KStreamJoinTest { - private Ingestor ingestor = new MockIngestor(); - - private StreamGroup streamGroup = new StreamGroup( - "group", - ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - return 0L; - } - }, - 10 - ); - private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -83,10 +68,10 @@ public void testJoin() { TestProcessor processor; String[] expected; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, context, null, null); - stream2 = new KStreamSource<>(streamMetadata, context, null, null); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -94,6 +79,11 @@ public void testJoin() { try { windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -168,10 +158,10 @@ public void testJoinPrior() { TestProcessor processor; String[] expected; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, context, null, null); - stream2 = new KStreamSource<>(streamMetadata, context, null, null); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -179,6 +169,11 @@ public void testJoinPrior() { try { windowed1.joinPrior(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -247,10 +242,10 @@ public void testMap() { KStreamWindowed windowed2; TestProcessor processor; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, context, null, null); - stream2 = new KStreamSource<>(streamMetadata, context, null, null); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); mapped1 = stream1.map(keyValueMapper); mapped2 = stream2.map(keyValueMapper); @@ -262,6 +257,11 @@ public void testMap() { windowed2 = mapped2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -274,6 +274,11 @@ public void testMap() { windowed2 = stream2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -286,6 +291,11 @@ public void testMap() { windowed2 = mapped2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -303,10 +313,10 @@ public void testFlatMap() { KStreamWindowed windowed2; TestProcessor processor; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, context, null, null); - stream2 = new KStreamSource<>(streamMetadata, context, null, null); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); mapped1 = stream1.flatMap(keyValueMapper2); mapped2 = stream2.flatMap(keyValueMapper2); @@ -318,6 +328,11 @@ public void testFlatMap() { windowed2 = mapped2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -330,6 +345,11 @@ public void testFlatMap() { windowed2 = stream2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -342,6 +362,11 @@ public void testFlatMap() { windowed2 = mapped2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -359,10 +384,10 @@ public void testMapValues() { KStreamWindowed windowed2; TestProcessor processor; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, context, null, null); - stream2 = new KStreamSource<>(streamMetadata, context, null, null); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); mapped1 = stream1.mapValues(valueMapper); mapped2 = stream2.mapValues(valueMapper); @@ -374,6 +399,11 @@ public void testMapValues() { windowed2 = mapped2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -386,6 +416,11 @@ public void testMapValues() { windowed2 = stream2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -398,6 +433,11 @@ public void testMapValues() { windowed2 = mapped2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -415,10 +455,10 @@ public void testFlatMapValues() { KStreamWindowed windowed2; TestProcessor processor; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(streamMetadata, context, null, null); - stream2 = new KStreamSource<>(streamMetadata, context, null, null); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); mapped1 = stream1.flatMapValues(valueMapper2); mapped2 = stream2.flatMapValues(valueMapper2); @@ -430,6 +470,11 @@ public void testFlatMapValues() { windowed2 = mapped2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -442,6 +487,11 @@ public void testFlatMapValues() { windowed2 = stream2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } @@ -454,6 +504,11 @@ public void testFlatMapValues() { windowed2 = mapped2.with(new UnlimitedWindow()); windowed1.join(windowed2, joiner).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { exceptionRaised = true; } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 1179090334fec..2597584fb54b7 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -12,23 +11,9 @@ public class KStreamMapTest { - private Ingestor ingestor = new MockIngestor(); - - private StreamGroup streamGroup = new StreamGroup( - "group", - ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - return 0L; - } - }, - 10 - ); - private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testMap() { @@ -43,14 +28,16 @@ public KeyValue apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, context, null, null); + stream = new KStreamSource<>(null, initializer); stream.map(mapper).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 1ad6e3c4d0440..eb4f477144bd4 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -1,9 +1,8 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.TimestampExtractor; +import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.ValueMapper; -import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -14,23 +13,9 @@ public class KStreamMapValuesTest { - private Ingestor ingestor = new MockIngestor(); - - private StreamGroup streamGroup = new StreamGroup( - "group", - ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - return 0L; - } - }, - 10 - ); - private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testFlatMapValues() { @@ -45,14 +30,16 @@ public Integer apply(String value) { final int[] expectedKeys = new int[] { 1, 10, 100, 1000 }; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(streamMetadata, context, null, null); + stream = new KStreamSource<>(null, initializer); stream.mapValues(mapper).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L, 0L); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 74c012465b82d..4de327114e570 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -4,7 +4,6 @@ import io.confluent.streaming.*; -import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -13,36 +12,24 @@ public class KStreamSourceTest { - private Ingestor ingestor = new MockIngestor(); - - private StreamGroup streamGroup = new StreamGroup( - "group", - ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - return 0L; - } - }, - 10 - ); - private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testKStreamSource() { - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); TestProcessor processor = new TestProcessor<>(); - KStreamSource stream = new KStreamSource<>(streamMetadata, context, null, null); + KStreamSource stream = new KStreamSource<>(null, initializer); stream.process(processor); final String[] expectedKeys = new String[] { "k1", "k2", "k3" }; final String[] expectedValues = new String[] { "v1", "v2", "v3" }; + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { stream.receive(expectedKeys[i], expectedValues[i], 0L, 0L); } diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index b9b9ee2716c39..2b37c29b7e22b 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.*; -import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.UnlimitedWindow; import org.junit.Test; @@ -13,23 +12,9 @@ public class KStreamWindowedTest { - private Ingestor ingestor = new MockIngestor(); - - private StreamGroup streamGroup = new StreamGroup( - "group", - ingestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - return 0L; - } - }, - 10 - ); - private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); @Test public void testWindowedStream() { @@ -38,14 +23,18 @@ public void testWindowedStream() { KStreamSource stream; Window window; - KStreamContext context = new MockKStreamContext(null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null) { + }; window = new UnlimitedWindow<>(); - stream = new KStreamSource<>(streamMetadata, context, null, null); + stream = new KStreamSource<>(null, initializer); stream.with(window); boolean exceptionRaised = false; + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + // two items in the window for (int i = 0; i < 2; i++) { diff --git a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java index 2d497c9072810..014e140685268 100644 --- a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java +++ b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java @@ -22,8 +22,8 @@ public void testScheduling() { assertEquals(0, proc1.punctuated.size()); assertEquals(0, proc2.punctuated.size()); - proc1.init(new ProcessorContextImpl(null, null, sched1)); - proc2.init(new ProcessorContextImpl(null, null, sched2)); + proc1.init(new ProcessorContextImpl(null, sched1)); + proc2.init(new ProcessorContextImpl(null, sched2)); sched1.schedule(500); sched2.schedule(1000); @@ -70,8 +70,8 @@ public void testCanceling() { assertEquals(0, proc1.punctuated.size()); assertEquals(0, proc2.punctuated.size()); - proc1.init(new ProcessorContextImpl(null, null, sched1)); - proc2.init(new ProcessorContextImpl(null, null, sched2)); + proc1.init(new ProcessorContextImpl(null, sched1)); + proc2.init(new ProcessorContextImpl(null, sched2)); sched1.schedule(500); sched2.schedule(1000); @@ -107,7 +107,7 @@ public void testDuplicateScheduling() { assertEquals(0, proc1.punctuated.size()); - proc1.init(new ProcessorContextImpl(null, null, sched1)); + proc1.init(new ProcessorContextImpl(null, sched1)); sched1.schedule(500); diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index 88cd757692ece..6030a3046972c 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -23,7 +23,7 @@ public class StreamGroupTest { private static Serializer serializer = new IntegerSerializer(); private static Deserializer deserializer = new IntegerDeserializer(); - private static class MockKStreamSource extends KStreamSource { + private static class MockKStreamSource extends KStreamSource { public int numReceived = 0; public ArrayList keys = new ArrayList<>(); @@ -32,7 +32,7 @@ private static class MockKStreamSource extends KStreamSource { public ArrayList streamTimes = new ArrayList<>(); public MockKStreamSource() { - super(null, new MockKStreamContext(serializer, deserializer)); + super(null, deserializer, deserializer, new KStreamInitializerImpl(serializer, serializer, deserializer, deserializer)); } @Override @@ -53,7 +53,7 @@ public void testAddPartition() { MockIngestor mockIngestor = new MockIngestor(); StreamGroup streamGroup = new StreamGroup( - "group", + new MockKStreamContext(serializer, deserializer), mockIngestor, new TimeBasedChooser(), new TimestampExtractor() { diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java index 5bf5cb55f1deb..e43cb15fb4bb7 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java @@ -1,16 +1,23 @@ package io.confluent.streaming.testutil; -import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.Processor; +import io.confluent.streaming.PunctuationScheduler; import io.confluent.streaming.RecordCollector; <<<<<<< HEAD import io.confluent.streaming.StorageEngine; ======= import io.confluent.streaming.StateStore; +<<<<<<< HEAD import io.confluent.streaming.Coordinator; >>>>>>> wip import io.confluent.streaming.internal.StreamGroup; +======= +import io.confluent.streaming.internal.PunctuationQueue; +import io.confluent.streaming.internal.PunctuationSchedulerImpl; +import io.confluent.streaming.kv.internals.RestoreFunc; +>>>>>>> new api model import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; @@ -25,6 +32,7 @@ public class MockKStreamContext implements KStreamContext { Serializer serializer; Deserializer deserializer; + private final PunctuationQueue punctuationQueue = new PunctuationQueue(); public MockKStreamContext(Serializer serializer, Deserializer deserializer) { this.serializer = serializer; @@ -46,12 +54,6 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer @Override public Deserializer valueDeserializer() { return deserializer; } - @Override - public KStream from(String... topic) { throw new UnsupportedOperationException("from() not supported."); } - - @Override - public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topic) { throw new UnsupportedOperationException("from() not supported."); } - @Override public RecordCollector recordCollector() { throw new UnsupportedOperationException("recordCollector() not supported."); } @@ -65,17 +67,39 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer public Metrics metrics() { throw new UnsupportedOperationException("metrics() not supported."); } @Override - public StreamGroup streamGroup(String name) { throw new UnsupportedOperationException("streamGroup() not supported."); } + public void restore(StateStore store, RestoreFunc func) { throw new UnsupportedOperationException("restore() not supported."); } @Override - public StreamGroup roundRobinStreamGroup(String name) { throw new UnsupportedOperationException("roundRobinStreamGroup() not supported."); } + public void flush() { throw new UnsupportedOperationException("flush() not supported."); } @Override +<<<<<<< HEAD public void restore(StateStore engine) throws Exception { throw new UnsupportedOperationException("restore() not supported."); } +======= + public void send(String topic, Object key, Object value) { throw new UnsupportedOperationException("send() not supported."); } +>>>>>>> new api model @Override - public void ensureInitialization() {} + public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { throw new UnsupportedOperationException("send() not supported."); } @Override - public void flush() { throw new UnsupportedOperationException("flush() not supported."); } + public PunctuationScheduler getPunctuationScheduler(Processor processor) { + return new PunctuationSchedulerImpl(punctuationQueue, processor); + } + + @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() { throw new UnsupportedOperationException("timestamp() not supported."); } + } diff --git a/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java b/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java index 04ddcb147a01c..80a538cbc2b2a 100644 --- a/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java +++ b/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java @@ -1,5 +1,6 @@ package io.confluent.streaming.testutil; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KeyValue; import io.confluent.streaming.Window; import io.confluent.streaming.util.FilteredIterator; @@ -12,6 +13,10 @@ public class UnlimitedWindow implements Window { private LinkedList>> list = new LinkedList>>(); + @Override + public void init(KStreamContext context) { + + } @Override public Iterator find(final K key, long timestamp) { return find(key, Long.MIN_VALUE, timestamp); From 2062d7a2ea41a0a63435b26c714ebf1d4764769f Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 10:46:18 -0700 Subject: [PATCH 106/275] wip --- .../streaming/KStreamInitializer.java | 26 ----------- .../confluent/streaming/KafkaStreaming.java | 2 +- .../internal/KStreamInitializerImpl.java | 46 +------------------ .../streaming/internal/KStreamSource.java | 25 ++++++++-- .../streaming/internal/KStreamThread.java | 7 +-- .../streaming/internal/StreamGroup.java | 4 +- .../streaming/internal/TopologyAnalyzer.java | 5 +- .../streaming/internal/KStreamBranchTest.java | 2 +- .../streaming/internal/KStreamFilterTest.java | 4 +- .../internal/KStreamFlatMapTest.java | 2 +- .../internal/KStreamFlatMapValuesTest.java | 2 +- .../streaming/internal/KStreamJoinTest.java | 12 ++--- .../streaming/internal/KStreamMapTest.java | 2 +- .../internal/KStreamMapValuesTest.java | 2 +- .../streaming/internal/KStreamSourceTest.java | 2 +- .../internal/KStreamWindowedTest.java | 3 +- .../streaming/internal/StreamGroupTest.java | 2 +- 17 files changed, 44 insertions(+), 104 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStreamInitializer.java b/src/main/java/io/confluent/streaming/KStreamInitializer.java index 43464dc884adc..d707306b0eab9 100644 --- a/src/main/java/io/confluent/streaming/KStreamInitializer.java +++ b/src/main/java/io/confluent/streaming/KStreamInitializer.java @@ -12,32 +12,6 @@ */ public interface KStreamInitializer { - AtomicInteger STREAM_GROUP_INDEX = new AtomicInteger(1); - - /** - * 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(); - // TODO: support regex topic matching in from() calls, for example: // context.from("Topic*PageView") diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index a0fe124fc7cab..71a27baaaaee2 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -182,7 +182,7 @@ public KafkaStreaming(Class jobClass, StreamingConfig stre this.config = new ProcessorConfig(streamingConfig.config()); - TopologyAnalyzer topologyAnalyzer = new TopologyAnalyzer(jobClass, streamingConfig); + TopologyAnalyzer topologyAnalyzer = new TopologyAnalyzer(jobClass); this.topics = topologyAnalyzer.topics; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java index f4b58e674c1e9..120b8038e057e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java @@ -2,9 +2,7 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamInitializer; -import io.confluent.streaming.StreamingConfig; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import java.util.ArrayList; import java.util.Collection; @@ -15,57 +13,15 @@ */ public class KStreamInitializerImpl implements KStreamInitializer { - private final Serializer keySerializer; - private final Serializer valueSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; private final ArrayList> streams = new ArrayList<>(); - KStreamInitializerImpl(StreamingConfig streamingConfig) { - this( - streamingConfig.keySerializer(), - streamingConfig.valueSerializer(), - streamingConfig.keyDeserializer(), - streamingConfig.valueDeserializer() - ); - } - - KStreamInitializerImpl(Serializer keySerializer,Serializer valueSerializer, Deserializer keyDeserializer, Deserializer valueDeserializer) { - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - this.keyDeserializer = keyDeserializer; - this.valueDeserializer = valueDeserializer; - } - - @Override - public Serializer keySerializer() { - return keySerializer; - } - - @Override - public Serializer valueSerializer() { - return valueSerializer; - } - - @Override - public Deserializer keyDeserializer() { - return keyDeserializer; - } - - @Override - public Deserializer valueDeserializer() { - return valueDeserializer; - } - @Override public KStream from(String... topics) { - return from(this.keyDeserializer(), this.valueDeserializer(), topics); + return from(null, null, topics); } - @SuppressWarnings("unchecked") @Override public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KStreamSource stream = new KStreamSource<>(topics, keyDeserializer, valDeserializer, this); streams.add(stream); return stream; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 0ad97cb3b9ff9..afac059a268d4 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -1,5 +1,6 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamInitializer; import org.apache.kafka.common.serialization.Deserializer; @@ -8,14 +9,13 @@ */ class KStreamSource extends KStreamImpl { - public final Deserializer keyDeserializer; - public final Deserializer valueDeserializer; + private Deserializer keyDeserializer; + private Deserializer valueDeserializer; final String[] topics; - @SuppressWarnings("unchecked") KStreamSource(String[] topics, KStreamInitializer initializer) { - this(topics, (Deserializer) initializer.keyDeserializer(), (Deserializer) initializer.valueDeserializer(), initializer); + this(topics, null, null, initializer); } KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamInitializer initializer) { @@ -25,6 +25,15 @@ class KStreamSource extends KStreamImpl { this.valueDeserializer = valueDeserializer; } + @SuppressWarnings("unchecked") + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + if (keyDeserializer == null) keyDeserializer = (Deserializer) context.keyDeserializer(); + if (valueDeserializer == null) valueDeserializer = (Deserializer) context.valueDeserializer(); + + super.bind(context, metadata); + } + @Override public void receive(Object key, Object value, long timestamp, long streamTime) { synchronized(this) { @@ -33,4 +42,12 @@ public void receive(Object key, Object value, long timestamp, long streamTime) { } } + public Deserializer keyDeserializer() { + return keyDeserializer; + } + + public Deserializer valueDeserializer() { + return valueDeserializer; + } + } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index a4c8b29955345..313de9b37496c 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -239,12 +239,7 @@ private void addPartitions(Collection assignment) { KStreamContextImpl context = kstreamContexts.get(id); if (context == null) { try { - KStreamInitializerImpl initializer = new KStreamInitializerImpl( - streamingConfig.keySerializer(), - streamingConfig.valueSerializer(), - streamingConfig.keyDeserializer(), - streamingConfig.valueDeserializer() - ); + KStreamInitializerImpl initializer = new KStreamInitializerImpl(); KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); job.init(initializer); diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index 01a8e02d9a29e..fce7393a110bc 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -172,8 +172,8 @@ private void ingestNewRecords() { ConsumerRecord record = iterator.next(); // deserialize the raw record, extract the timestamp and put into the queue - Deserializer keyDeserializer = recordQueue.stream.keyDeserializer; - Deserializer valDeserializer = recordQueue.stream.valueDeserializer; + Deserializer keyDeserializer = recordQueue.stream.keyDeserializer(); + Deserializer valDeserializer = recordQueue.stream.valueDeserializer(); Object key = keyDeserializer.deserialize(record.topic(), record.key()); Object value = valDeserializer.deserialize(record.topic(), record.value()); diff --git a/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java b/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java index 8dee74ed57a57..75c583bbb4d0e 100644 --- a/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java +++ b/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamJob; -import io.confluent.streaming.StreamingConfig; import org.apache.kafka.common.utils.Utils; import java.util.Collection; @@ -17,9 +16,9 @@ public class TopologyAnalyzer { public final Set topics; public final Collection> streams; - public TopologyAnalyzer(Class jobClass, StreamingConfig streamingConfig) { + public TopologyAnalyzer(Class jobClass) { KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); - KStreamInitializerImpl context = new KStreamInitializerImpl(streamingConfig); + KStreamInitializerImpl context = new KStreamInitializerImpl(); job.init(context); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index 790d1252fc894..fc7f8dfa5fa9a 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -41,7 +41,7 @@ public boolean apply(Integer key, String value) { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); KStreamSource stream; KStream[] branches; TestProcessor[] processors; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 09ad3b88af5d0..f6d0ea07c5339 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -26,7 +26,7 @@ public boolean apply(Integer key, String value) { public void testFilter() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); KStreamSource stream; TestProcessor processor; @@ -47,7 +47,7 @@ public void testFilter() { public void testFilterOut() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 6ff9b80a53095..3074c0621485a 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -33,7 +33,7 @@ public KeyValue> apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 44b812d0855be..9d0a21c9bcb4e 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -32,7 +32,7 @@ public Iterable apply(String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 3c0fdca855de2..7b59ae9c45105 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -68,7 +68,7 @@ public void testJoin() { TestProcessor processor; String[] expected; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -158,7 +158,7 @@ public void testJoinPrior() { TestProcessor processor; String[] expected; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -242,7 +242,7 @@ public void testMap() { KStreamWindowed windowed2; TestProcessor processor; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -313,7 +313,7 @@ public void testFlatMap() { KStreamWindowed windowed2; TestProcessor processor; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -384,7 +384,7 @@ public void testMapValues() { KStreamWindowed windowed2; TestProcessor processor; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -455,7 +455,7 @@ public void testFlatMapValues() { KStreamWindowed windowed2; TestProcessor processor; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 2597584fb54b7..ce6736e135345 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -28,7 +28,7 @@ public KeyValue apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index eb4f477144bd4..85a889d23a7e0 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -30,7 +30,7 @@ public Integer apply(String value) { final int[] expectedKeys = new int[] { 1, 10, 100, 1000 }; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 4de327114e570..0db2b43881978 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -19,7 +19,7 @@ public class KStreamSourceTest { @Test public void testKStreamSource() { - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + KStreamInitializer initializer = new KStreamInitializerImpl(); TestProcessor processor = new TestProcessor<>(); KStreamSource stream = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 2b37c29b7e22b..b8ea561efa81d 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -23,8 +23,7 @@ public void testWindowedStream() { KStreamSource stream; Window window; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null) { - }; + KStreamInitializer initializer = new KStreamInitializerImpl(); window = new UnlimitedWindow<>(); stream = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index 6030a3046972c..fb0a65ef1c350 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -32,7 +32,7 @@ private static class MockKStreamSource extends KStreamSource { public ArrayList streamTimes = new ArrayList<>(); public MockKStreamSource() { - super(null, deserializer, deserializer, new KStreamInitializerImpl(serializer, serializer, deserializer, deserializer)); + super(null, deserializer, deserializer, new KStreamInitializerImpl()); } @Override From 12c2a5485bf0de67c2dc1ebd35905537e2ba62f7 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 11:09:55 -0700 Subject: [PATCH 107/275] wip --- .../io/confluent/streaming/testutil/MockKStreamContext.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java index e43cb15fb4bb7..009d3503cf3a2 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java @@ -69,6 +69,8 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer @Override public void restore(StateStore store, RestoreFunc func) { throw new UnsupportedOperationException("restore() not supported."); } + public void register(StateStore store) { throw new UnsupportedOperationException("restore() not supported."); } + @Override public void flush() { throw new UnsupportedOperationException("flush() not supported."); } From 103c18b81ae2010344befd1e02699e78f2dd0faa Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 31 Jul 2015 12:19:34 -0700 Subject: [PATCH 108/275] Address comments, finish RocksDB restore logic --- .../internal/ProcessorStateManager.java | 7 +- .../streaming/kv/InMemoryKeyValueStore.java | 11 ++- .../streaming/kv/RocksDBKeyValueStore.java | 68 +++++++++++++++---- .../kv/internals/LoggedKeyValueStore.java | 3 +- 4 files changed, 67 insertions(+), 22 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index d9f2e6754f5cd..a699592d79107 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -64,6 +64,8 @@ public Consumer restoreConsumer() { public void init() throws IOException { OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); this.checkpointedOffsets.putAll(checkpoint.read()); + + // delete the checkpoint file after finish loading its stored offsets checkpoint.delete(); } @@ -92,6 +94,7 @@ public void register(StateStore store) { } else { // try to create the topic with the number of partitions equal to the total number of store instances. // TODO: this is not possible yet since we do not know the total number of ids. + throw new UnsupportedOperationException("Cannot create change log topic on-the-fly"); } // register store @@ -175,7 +178,7 @@ public void close(Map ackedOffsets) throws IOException { // only checkpoint the offset to the offsets file if it is persistent; if (stores.get(storeName).persistent()) { - if(ackedOffsets.containsKey(part)) + if (ackedOffsets.containsKey(part)) // store the last ack'd offset + 1 (the log position after restoration) checkpointOffsets.put(part, ackedOffsets.get(part) + 1); } else { @@ -183,7 +186,7 @@ public void close(Map ackedOffsets) throws IOException { } } - // write the checkpoint offset file to indicate clean shutdown + // write the checkpoint file before closing, to indicate clean shutdown OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); checkpoint.write(checkpointOffsets); } diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index a97d5adf652c9..bfb27a3953571 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -4,6 +4,7 @@ import io.confluent.streaming.kv.internals.LoggedKeyValueStore; import io.confluent.streaming.kv.internals.MeteredKeyValueStore; import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; import java.util.Iterator; import java.util.List; @@ -20,15 +21,19 @@ public class InMemoryKeyValueStore extends MeteredKeyValueStore { public InMemoryKeyValueStore(String name, KStreamContext context) { + this(name, context, new SystemTime()); + } + + public InMemoryKeyValueStore(String name, KStreamContext context, Time time) { // always wrap the logged store with the metered store // TODO: this may need to be relaxed in the future - super(name, /* topic name as store name */ + super(name, "kafka-streams", - new LoggedKeyValueStore<>(name, /* topic name as store name */ + new LoggedKeyValueStore<>(name, /* store name as topic name */ new MemoryStore(name, context), context), context.metrics(), - new SystemTime()); + time); } private static class MemoryStore implements KeyValueStore { diff --git a/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java index 94e211eed073f..82f34622b0329 100644 --- a/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java @@ -1,9 +1,12 @@ package io.confluent.streaming.kv; import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.kv.internals.LoggedKeyValueStore; import io.confluent.streaming.kv.internals.MeteredKeyValueStore; +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; @@ -12,14 +15,11 @@ import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import org.rocksdb.RocksIterator; -import org.rocksdb.TtlDB; import org.rocksdb.WriteOptions; import java.io.File; import java.util.Comparator; -import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.NoSuchElementException; /** @@ -28,7 +28,19 @@ public class RocksDBKeyValueStore extends MeteredKeyValueStore { public RocksDBKeyValueStore(String name, KStreamContext context) { - super(name, "kafka-streams", new RocksDBStore(name, context), context.metrics(), new SystemTime()); + this(name, context, new SystemTime()); + } + + public RocksDBKeyValueStore(String name, KStreamContext context, Time time) { + // always wrap the logged store with the metered store + // TODO: this may need to be relaxed in the future + super(name, + "kafka-streams", + new LoggedKeyValueStore<>(name, /* store name as topic name */ + new RocksDBStore(name, context), + context), + context.metrics(), + time); } private static class RocksDBStore implements KeyValueStore { @@ -43,10 +55,9 @@ private static class RocksDBStore implements KeyValueStore { 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 = "/tmp/rocksdb"; - + private static final String DB_FILE_DIR = "rocksdb"; + private static final String TMP_FILE_SUFFIX = ".tmp"; - private final RocksDB db; private final String topic; private final int partition; private final KStreamContext context; @@ -55,6 +66,12 @@ private static class RocksDBStore implements KeyValueStore { private final WriteOptions wOptions; private final FlushOptions fOptions; + private final String dbName; + private final String dirName; + private final File tmpFile; + + private RocksDB db; + @SuppressWarnings("unchecked") public RocksDBStore(String name, KStreamContext context) { this.topic = name; @@ -81,8 +98,16 @@ public RocksDBStore(String name, KStreamContext context) { fOptions = new FlushOptions(); fOptions.setWaitForFlush(true); - String dbName = this.topic + "." + this.partition; - db = openDB(new File(DB_FILE_DIR, dbName), this.options, TTL_SECONDS); + dbName = this.topic + "." + this.partition; + dirName = this.context.stateDir() + File.separator + DB_FILE_DIR; + + // rename the file with a tmp suffix to make sure the db instance is created fresh at first + tmpFile = new File(dirName, dbName); + if (tmpFile.exists()) { + if (!tmpFile.renameTo(new File(dirName, dbName + TMP_FILE_SUFFIX))) + throw new KafkaException("Failed to add the tmp suffix to the existing file " + tmpFile.getName()); + } + db = openDB(new File(dirName, dbName), this.options, TTL_SECONDS); this.context.register(this); } @@ -92,13 +117,13 @@ private RocksDB openDB(File dir, Options options, int ttl) { if (ttl == TTL_NOT_USED) { return RocksDB.open(options, dir.toString()); } else { - throw new IllegalStateException("Change log is not supported for store " + this.topic + " since it is TTL based."); + 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 RuntimeException("Error opening store " + this.topic + " at location " + dir.toString(), e); + throw new KafkaException("Error opening store " + this.topic + " at location " + dir.toString(), e); } } @@ -118,7 +143,7 @@ public byte[] get(byte[] key) { return this.db.get(key); } catch (RocksDBException e) { // TODO: this needs to be handled more accurately - throw new RuntimeException("Error while executing get " + key.toString() + " from store " + this.topic, e); + throw new KafkaException("Error while executing get " + key.toString() + " from store " + this.topic, e); } } @@ -132,7 +157,7 @@ public void put(byte[] key, byte[] value) { } } catch (RocksDBException e) { // TODO: this needs to be handled more accurately - throw new RuntimeException("Error while executing put " + key.toString() + " from store " + this.topic, e); + throw new KafkaException("Error while executing put " + key.toString() + " from store " + this.topic, e); } } @@ -165,13 +190,26 @@ public void flush() { db.flush(fOptions); } catch (RocksDBException e) { // TODO: this needs to be handled more accurately - throw new RuntimeException("Error while executing flush from store " + this.topic, e); + throw new KafkaException("Error while executing flush from store " + this.topic, e); } } @Override public void restore() { - + if (tmpFile.exists()) { + // close the db and delete its file + // TODO: this db should not take any writes yet + db.close(); + File file = new File(dirName, dbName); + if (!file.delete()) + throw new KafkaException("Failed to delete the existing file " + file.getName()); + + // rename the tmp file by removing its tmp suffix and reopen the database + if (!tmpFile.renameTo(file)) + throw new KafkaException("Failed to remove the tmp suffix to the existing file " + tmpFile.getName()); + + db = openDB(tmpFile, this.options, TTL_SECONDS); + } } @Override diff --git a/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java index 83174bbda890a..0c96a47b64e67 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java @@ -120,9 +120,8 @@ public void close() {} @SuppressWarnings("unchecked") @Override public void flush() { - // TODO: these two operations should be done atomically - log(); inner.flush(); + log(); } private void log() { From f9f0185c4f7358c071341d5a00efc3fe919b82bf Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 13:40:29 -0700 Subject: [PATCH 109/275] wip --- .../streaming/KStreamInitializer.java | 36 ------ .../io/confluent/streaming/KStreamJob.java | 35 ------ .../confluent/streaming/KStreamTopology.java | 105 ++++++++++++++++++ .../confluent/streaming/KafkaStreaming.java | 22 ++-- .../io/confluent/streaming/Processor.java | 2 + .../streaming/ProcessorKStreamJob.java | 11 +- .../streaming/examples/MapKStreamJob.java | 18 +-- .../streaming/examples/PrintKStreamJob.java | 7 +- .../examples/StatefulKStreamJob.java | 7 +- .../streaming/internal/KStreamBranch.java | 16 ++- .../streaming/internal/KStreamFilter.java | 4 +- .../streaming/internal/KStreamFlatMap.java | 4 +- .../internal/KStreamFlatMapValues.java | 4 +- .../streaming/internal/KStreamImpl.java | 33 +++--- .../internal/KStreamInitializerImpl.java | 34 ------ .../streaming/internal/KStreamJoin.java | 9 +- .../streaming/internal/KStreamMap.java | 4 +- .../streaming/internal/KStreamMapValues.java | 4 +- .../streaming/internal/KStreamSource.java | 19 +++- .../streaming/internal/KStreamThread.java | 17 +-- .../internal/KStreamWindowedImpl.java | 10 +- .../streaming/internal/ProcessorNode.java | 35 ++++++ .../streaming/internal/Receiver.java | 1 + .../streaming/internal/TopologyAnalyzer.java | 35 ------ .../streaming/internal/KStreamBranchTest.java | 3 +- .../streaming/internal/KStreamFilterTest.java | 5 +- .../internal/KStreamFlatMapTest.java | 6 +- .../internal/KStreamFlatMapValuesTest.java | 3 +- .../streaming/internal/KStreamJoinTest.java | 89 +++++++++++---- .../streaming/internal/KStreamMapTest.java | 3 +- .../internal/KStreamMapValuesTest.java | 5 +- .../streaming/internal/KStreamSourceTest.java | 3 +- .../internal/KStreamWindowedTest.java | 3 +- .../streaming/internal/StreamGroupTest.java | 3 +- .../testutil/MockKStreamTopology.java | 12 ++ .../streaming/testutil/TestProcessor.java | 5 + 36 files changed, 360 insertions(+), 252 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/KStreamInitializer.java delete mode 100644 src/main/java/io/confluent/streaming/KStreamJob.java create mode 100644 src/main/java/io/confluent/streaming/KStreamTopology.java delete mode 100644 src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/ProcessorNode.java delete mode 100644 src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java create mode 100644 src/test/java/io/confluent/streaming/testutil/MockKStreamTopology.java diff --git a/src/main/java/io/confluent/streaming/KStreamInitializer.java b/src/main/java/io/confluent/streaming/KStreamInitializer.java deleted file mode 100644 index d707306b0eab9..0000000000000 --- a/src/main/java/io/confluent/streaming/KStreamInitializer.java +++ /dev/null @@ -1,36 +0,0 @@ -package io.confluent.streaming; - -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; - -import java.util.concurrent.atomic.AtomicInteger; - -/** - * KStreamContext is the interface that allows an implementation of {@link KStreamJob#init(KStreamInitializer)} to create KStream instances. - * It also provides access to the system resources for a stream processing job. - * An instance of KStreamContext is created for each partition. - */ -public interface KStreamInitializer { - - // TODO: support regex topic matching in from() calls, for example: - // context.from("Topic*PageView") - - /** - * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream - */ - KStream from(String... topics); - - /** - * 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 - */ - KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); - -} diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java deleted file mode 100644 index 3e97475b8e9ad..0000000000000 --- a/src/main/java/io/confluent/streaming/KStreamJob.java +++ /dev/null @@ -1,35 +0,0 @@ -package io.confluent.streaming; - -/** - * An interface to implement an application logic of a stream processing. - * An instance is created and initialized by the framework for each partition. - */ -public interface KStreamJob { - - /** - * Creates a job instance - */ - - /** - * Initializes a stream processing job for a partition. This method is called for each partition. - * An application constructs a processing logic using KStream API. - *

- * For example, - *

- *
-   *   public bind(KStreamContext context) {
-   *     KStream<Integer, PageView> pageViewStream = context.from("pageView").mapValues(...);
-   *     KStream<Integer, AdClick> adClickStream = context.from("adClick").join(pageViewStream, ...).process(...);
-   *   }
-   * 
- * @param context KStreamContext for this partition - */ - void init(KStreamInitializer context); - - /** - * Closes this partition of the stream processing job. - * An application can perform its special clean up here. - */ - void close(); - -} diff --git a/src/main/java/io/confluent/streaming/KStreamTopology.java b/src/main/java/io/confluent/streaming/KStreamTopology.java new file mode 100644 index 0000000000000..0eb368667a4f4 --- /dev/null +++ b/src/main/java/io/confluent/streaming/KStreamTopology.java @@ -0,0 +1,105 @@ +package io.confluent.streaming; + +import io.confluent.streaming.internal.KStreamSource; +import org.apache.kafka.common.serialization.Deserializer; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; + +/** + * KStreamTopology is the class that allows an implementation of {@link KStreamTopology#topology()} to create KStream instances. + * + */ +public abstract class KStreamTopology { + + private final ArrayList> streams = new ArrayList<>(); + + /** + * Initializes a stream processing topology. This method may be called multiple times. + * An application constructs a processing logic using KStream API. + *

+ * For example, + *

+ *
+   *   KStreamTopology topology = new KStreamTopology() {
+   *     public void topology() {
+   *       KStream<Integer, PageView> pageViewStream = from("pageView").mapValues(...);
+   *       KStream<Integer, AdClick> adClickStream = from("adClick").join(pageViewStream, ...).process(...);
+   *     }
+   *   }
+   *
+   *   KafkaStreaming streaming = new KafkaStreaming(topology, streamingConfig)
+   *   streaming.run();
+   * 
+ */ + public abstract void topology(); + + /** + * Extracts topics used in the KStream topology. This method calls {@link KStreamTopology#topology()} method. + * @return + */ + public final Set topics() { + synchronized (streams) { + try { + streams.clear(); + topology(); + Set topics = new HashSet<>(); + for (KStreamSource stream : streams) { + topics.addAll(stream.topics()); + } + return topics; + } + finally { + streams.clear(); + } + } + } + + /** + * Returns source streams in the KStream topology. This method calls {@link KStreamTopology#topology()} method. + * This method may be called multiple times. + */ + public final Collection> sourceStreams() { + synchronized (streams) { + try { + streams.clear(); + topology(); + return new ArrayList<>(streams); + } + finally { + streams.clear(); + } + } + } + + + // TODO: support regex topic matching in from() calls, for example: + // context.from("Topic*PageView") + + /** + * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(String... topics) { + return from(null, null, topics); + } + + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * @param keyDeserializer key deserializer used to read this source KStream, + * if not specified the default deserializer defined in the configs will be used + * @param valDeserializer value deserializer used to read this source KStream, + * if not specified the default deserializer defined in the configs will be used + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + KStreamSource stream = new KStreamSource<>(topics, keyDeserializer, valDeserializer, this); + streams.add(stream); + return stream; + } + +} diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index 71a27baaaaee2..bde4a5a9714cb 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -21,6 +21,7 @@ import io.confluent.streaming.internal.ProcessorConfig; <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.internal.IngestorImpl; <<<<<<< HEAD import io.confluent.streaming.internal.StreamSynchronizer; @@ -44,6 +45,8 @@ ======= import io.confluent.streaming.internal.TopologyAnalyzer; >>>>>>> new api model +======= +>>>>>>> wip import org.apache.kafka.common.metrics.Metrics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,13 +57,13 @@ * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and * sends output to zero or more output topics. *

- * This processing is done by implementing the {@link KStreamJob} interface to specify the transformation. The + * This processing is defined by extending the {@link KStreamTopology} abstract class to specify the transformation operator topology. The * {@link KafkaStreaming} instance will be responsible for the lifecycle of these processors. It will instantiate and * start one or more of these processors to process the Kafka partitions assigned to this particular instance. *

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

* Internally the {@link KafkaStreaming} instance contains a normal {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer} @@ -73,7 +76,7 @@ * StreamingConfig config = new StreamingConfig(props); * config.processor(ExampleStreamProcessor.class); * config.serialization(new StringSerializer(), new StringDeserializer()); - * KafkaStreaming container = new KafkaStreaming(MyKStreamJob.class, config); + * KafkaStreaming container = new KafkaStreaming(new MyKStreamTopology(), config); * container.run(); * * @@ -135,6 +138,7 @@ public class KafkaStreaming implements Runnable { private final Set topics; +<<<<<<< HEAD <<<<<<< HEAD @SuppressWarnings("unchecked") protected KafkaStreaming(Class jobClass, @@ -177,14 +181,14 @@ >>>>>>> use poll(0) for non-blocking poll ======= public KafkaStreaming(Class jobClass, StreamingConfig streamingConfig) { >>>>>>> added KStreamThread +======= + public KafkaStreaming(KStreamTopology topology, StreamingConfig streamingConfig) { +>>>>>>> wip if (streamingConfig.timestampExtractor() == null) throw new NullPointerException("timestamp extractor is missing"); this.config = new ProcessorConfig(streamingConfig.config()); - - TopologyAnalyzer topologyAnalyzer = new TopologyAnalyzer(jobClass); - - this.topics = topologyAnalyzer.topics; + this.topics = topology.topics(); <<<<<<< HEAD <<<<<<< HEAD @@ -246,12 +250,16 @@ public void shutdown() { // TODO: Fix this after the threading model is decided (also fix KStreamThread) this.threads = new KStreamThread[1]; +<<<<<<< HEAD <<<<<<< HEAD threads[0] = new KStreamThread(jobClass, topics, streamingConfig, coordinator, metrics); >>>>>>> added KStreamThread ======= threads[0] = new KStreamThread(jobClass, topics, streamingConfig, metrics); >>>>>>> removed Coordinator +======= + threads[0] = new KStreamThread(topology, topics, streamingConfig, metrics); +>>>>>>> wip } /** diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 568fb90f1a73d..2e1754ebd76d5 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -34,4 +34,6 @@ public interface ProcessorContext { void process(K key, V value); void punctuate(long streamTime); + + void close(); } diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java index 85aa99670a716..15479bcc81085 100644 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -3,10 +3,11 @@ /** * Created by guozhang on 7/14/15. */ -public abstract class ProcessorKStreamJob implements KStreamJob, Processor { +public abstract class ProcessorKStreamJob extends KStreamTopology implements Processor { @SuppressWarnings("unchecked") @Override +<<<<<<< HEAD <<<<<<< HEAD public void init(KStreamContext context) { <<<<<<< HEAD @@ -18,10 +19,10 @@ public void init(KStreamContext context) { public void init(KStreamInitializer initializer) { initializer.from().process((Processor) this); >>>>>>> new api model +======= + public void topology() { + ((KStream)from()).process(this); +>>>>>>> wip } - @Override - public void close() { - // do nothing - } } diff --git a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java index 6fe0823acd105..cef0b7689b293 100644 --- a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java @@ -1,8 +1,7 @@ package io.confluent.streaming.examples; import io.confluent.streaming.KStream; -import io.confluent.streaming.KStreamInitializer; -import io.confluent.streaming.KStreamJob; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.KafkaStreaming; import io.confluent.streaming.KeyValue; import io.confluent.streaming.KeyValueMapper; @@ -15,14 +14,14 @@ /** * Created by guozhang on 7/14/15. */ -public class MapKStreamJob implements KStreamJob { +public class MapKStreamJob extends KStreamTopology { @SuppressWarnings("unchecked") @Override - public void init(KStreamInitializer context) { + public void topology() { // With overriden de-serializer - KStream stream1 = context.from(new StringDeserializer(), new StringDeserializer(), "topic1"); + KStream stream1 = from(new StringDeserializer(), new StringDeserializer(), "topic1"); stream1.map(new KeyValueMapper() { @Override @@ -37,7 +36,7 @@ public boolean apply(String key, Integer value) { }).sendTo("topic2"); // Without overriden de-serialzier - KStream stream2 = (KStream)context.from("topic2"); + KStream stream2 = (KStream)from("topic2"); KStream[] streams = stream2.branch( new Predicate() { @@ -58,13 +57,8 @@ public boolean apply(String key, Integer value) { streams[1].sendTo("topic4"); } - @Override - public void close() { - // do nothing - } - public static void main(String[] args) { - KafkaStreaming kstream = new KafkaStreaming(MapKStreamJob.class, new StreamingConfig(new Properties())); + KafkaStreaming kstream = new KafkaStreaming(new MapKStreamJob(), new StreamingConfig(new Properties())); kstream.run(); } } diff --git a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java index d8867a26acdbb..5d335a8946391 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java @@ -32,8 +32,13 @@ public void punctuate(long streamTime) { // do nothing } + @Override + public void close() { + // do nothing + } + public static void main(String[] args) { - KafkaStreaming kstream = new KafkaStreaming(PrintKStreamJob.class, new StreamingConfig(new Properties())); + KafkaStreaming kstream = new KafkaStreaming(new PrintKStreamJob(), new StreamingConfig(new Properties())); kstream.run(); } } diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index 50429c7d5c227..d7b98a6e6a095 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -50,8 +50,13 @@ public void punctuate(long streamTime) { } } + @Override + public void close() { + // do nothing + } + public static void main(String[] args) { - KafkaStreaming kstream = new KafkaStreaming(PrintKStreamJob.class, new StreamingConfig(new Properties())); + KafkaStreaming kstream = new KafkaStreaming(new StatefulKStreamJob(), new StreamingConfig(new Properties())); kstream.run(); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index fc3d29675349a..87e8c580a1a9d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.Predicate; import java.lang.reflect.Array; @@ -16,7 +16,7 @@ class KStreamBranch implements Receiver { final KStreamSource[] branches; @SuppressWarnings("unchecked") - KStreamBranch(Predicate[] predicates, KStreamInitializer initializer) { + KStreamBranch(Predicate[] predicates, KStreamTopology initializer) { this.predicates = Arrays.copyOf(predicates, predicates.length); this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { @@ -26,8 +26,8 @@ class KStreamBranch implements Receiver { @Override public void bind(KStreamContext context, KStreamMetadata metadata) { - for (KStreamSource stream : branches) { - stream.bind(context, metadata); + for (KStreamSource branch : branches) { + branch.bind(context, metadata); } } @@ -41,7 +41,13 @@ public void receive(Object key, Object value, long timestamp, long streamTime) { return; } } - return; + } + + @Override + public void close() { + for (KStreamSource branch : branches) { + branch.close(); + } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 2f106cd33d256..4a4571b2c2045 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -1,6 +1,6 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.Predicate; /** @@ -10,7 +10,7 @@ class KStreamFilter extends KStreamImpl { private final Predicate predicate; - KStreamFilter(Predicate predicate, KStreamInitializer initializer) { + KStreamFilter(Predicate predicate, KStreamTopology initializer) { super(initializer); this.predicate = predicate; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 0a244ceff22fd..85f5d35189f07 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; @@ -12,7 +12,7 @@ class KStreamFlatMap extends KStreamImpl { private final KeyValueMapper, K1, V1> mapper; - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamInitializer initializer) { + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology initializer) { super(initializer); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 2ca0616fc4d48..3fe4f36ab44dd 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -1,6 +1,6 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.ValueMapper; /** @@ -10,7 +10,7 @@ class KStreamFlatMapValues extends KStreamImpl { private final ValueMapper, V1> mapper; - KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamInitializer initializer) { + KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology initializer) { super(initializer); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index ebc42867630d3..75230504fc160 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -2,7 +2,7 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.KStreamWindowed; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.Predicate; @@ -20,15 +20,18 @@ abstract class KStreamImpl implements KStream, Receiver { private final ArrayList nextReceivers = new ArrayList<>(1); + protected KStreamTopology initializer; + protected KStreamContext context; protected KStreamMetadata metadata; - protected KStreamInitializer initializer; - protected KStreamImpl(KStreamInitializer initializer) { + protected KStreamImpl(KStreamTopology initializer) { this.initializer = initializer; } @Override public void bind(KStreamContext context, KStreamMetadata metadata) { + if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); + this.context = context; this.metadata = metadata; int numReceivers = nextReceivers.size(); for (int i = 0; i < numReceivers; i++) { @@ -36,6 +39,14 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { } } + @Override + public void close() { + int numReceivers = nextReceivers.size(); + for (int i = 0; i < numReceivers; i++) { + nextReceivers.get(i).close(); + } + } + @Override public KStream filter(Predicate predicate) { return chain(new KStreamFilter(predicate, initializer)); @@ -116,27 +127,19 @@ public void init(ProcessorContext processorContext) { } @Override public void process(K key, V value) { - this.processorContext.send(sendTopic, key, value, - (Serializer) keySerializer, - (Serializer) valSerializer); + this.processorContext.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); } @Override public void punctuate(long streamTime) {} + @Override + public void close() {} }; } @SuppressWarnings("unchecked") @Override public void process(final Processor processor) { - Receiver receiver = new Receiver() { - public void bind(KStreamContext context, KStreamMetadata metadata) { - processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); - } - public void receive(Object key, Object value, long timestamp, long streamTime) { - processor.process((K) key, (V) value); - } - }; - registerReceiver(receiver); + registerReceiver(new ProcessorNode<>(processor)); } void registerReceiver(Receiver receiver) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java deleted file mode 100644 index 120b8038e057e..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java +++ /dev/null @@ -1,34 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.KStream; -import io.confluent.streaming.KStreamInitializer; -import org.apache.kafka.common.serialization.Deserializer; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; - -/** - * Created by yasuhiro on 6/19/15. - */ -public class KStreamInitializerImpl implements KStreamInitializer { - - private final ArrayList> streams = new ArrayList<>(); - - @Override - public KStream from(String... topics) { - return from(null, null, topics); - } - - @Override - public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KStreamSource stream = new KStreamSource<>(topics, keyDeserializer, valDeserializer, this); - streams.add(stream); - return stream; - } - - Collection> sourceStreams() { - return Collections.unmodifiableCollection(streams); - } - -} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index 59f392b4c93ed..1810d984f2377 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.NotCopartitionedException; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; @@ -24,7 +24,7 @@ private static abstract class Finder { private KStreamMetadata thisMetadata; private KStreamMetadata otherMetadata; - KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamInitializer initializer) { + KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology initializer) { super(initializer); final Window window1 = stream1.window; @@ -86,7 +86,6 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { otherMetadata = metadata; if (thisMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) throw new NotCopartitionedException(); } - @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value2, long timestamp, long streamTime) { @@ -97,6 +96,10 @@ public void receive(Object key, Object value2, long timestamp, long streamTime) } } } + @Override + public void close() { + // down stream instances are close when the primary stream is closed + } }; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index f6aefd7181e4a..7e607b634f232 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -1,7 +1,7 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; @@ -12,7 +12,7 @@ class KStreamMap extends KStreamImpl { private final KeyValueMapper mapper; - KStreamMap(KeyValueMapper mapper, KStreamInitializer initializer) { + KStreamMap(KeyValueMapper mapper, KStreamTopology initializer) { super(initializer); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index 45921508e9fd5..ffca04141eda0 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -1,6 +1,6 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.ValueMapper; /** @@ -10,7 +10,7 @@ class KStreamMapValues extends KStreamImpl { private final ValueMapper mapper; - KStreamMapValues(ValueMapper mapper, KStreamInitializer initializer) { + KStreamMapValues(ValueMapper mapper, KStreamTopology initializer) { super(initializer); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index afac059a268d4..331ea29a29c6b 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -1,24 +1,29 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import org.apache.kafka.common.serialization.Deserializer; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + /** * Created by yasuhiro on 6/17/15. */ -class KStreamSource extends KStreamImpl { +public class KStreamSource extends KStreamImpl { private Deserializer keyDeserializer; private Deserializer valueDeserializer; - final String[] topics; + String[] topics; - KStreamSource(String[] topics, KStreamInitializer initializer) { + public KStreamSource(String[] topics, KStreamTopology initializer) { this(topics, null, null, initializer); } - KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamInitializer initializer) { + public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology initializer) { super(initializer); this.topics = topics; this.keyDeserializer = keyDeserializer; @@ -50,4 +55,8 @@ public Deserializer valueDeserializer() { return valueDeserializer; } + public Set topics() { + return new HashSet<>(Arrays.asList(topics)); + } + } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index 313de9b37496c..b91dd763c63e7 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -17,8 +17,7 @@ package io.confluent.streaming.internal; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamJob; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.StreamingConfig; import io.confluent.streaming.util.ParallelExecutor; import io.confluent.streaming.util.Util; @@ -41,7 +40,6 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,7 +55,7 @@ public class KStreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); - private final Class jobClass; + private final KStreamTopology topology; private final ArrayList streamGroups = new ArrayList<>(); private final ParallelExecutor parallelExecutor; private final Map kstreamContexts = new HashMap<>(); @@ -89,10 +87,10 @@ public void onPartitionsRevoked(Consumer consumer, Collection jobClass, Set topics, StreamingConfig streamingConfig, Metrics metrics) { + public KStreamThread(KStreamTopology topology, Set topics, StreamingConfig streamingConfig, Metrics metrics) { super(); this.config = new ProcessorConfig(streamingConfig.config()); - this.jobClass = jobClass; + this.topology = topology; this.streamingConfig = streamingConfig; this.metrics = metrics; this.streamingMetrics = new KafkaStreamingMetrics(); @@ -239,13 +237,8 @@ private void addPartitions(Collection assignment) { KStreamContextImpl context = kstreamContexts.get(id); if (context == null) { try { - KStreamInitializerImpl initializer = new KStreamInitializerImpl(); - KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); - - job.init(initializer); - context = new KStreamContextImpl(id, ingestor, collector, streamingConfig, config, metrics); - context.init(initializer.sourceStreams()); + context.init(topology.sourceStreams()); kstreamContexts.put(id, context); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 6a5070254f7b9..4aa0395d04b93 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -3,7 +3,7 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.KStreamWindowed; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; @@ -15,7 +15,7 @@ public class KStreamWindowedImpl extends KStreamImpl implements KStr final Window window; - KStreamWindowedImpl(Window window, KStreamInitializer initializer) { + KStreamWindowedImpl(Window window, KStreamTopology initializer) { super(initializer); this.window = window; } @@ -36,6 +36,12 @@ public void receive(Object key, Object value, long timestamp, long streamTime) { } } + @Override + public void close() { + window.close(); + super.close(); + } + @Override public KStream join(KStreamWindowed other, ValueJoiner processor) { return join(other, false, processor); diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java new file mode 100644 index 0000000000000..1ed0ff256135c --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java @@ -0,0 +1,35 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.Processor; + +/** + * Created by yasuhiro on 7/31/15. + */ +public class ProcessorNode implements Receiver { + + private final Processor processor; + private KStreamContext context; + + ProcessorNode(Processor processor) { + this.processor = processor; + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); + + this.context = context; + processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); + } + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp, long streamTime) { + processor.process((K) key, (V) value); + } + @Override + public void close() { + processor.close(); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 84ab9e87f6c17..598e7b573860b 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -11,4 +11,5 @@ public interface Receiver { void receive(Object key, Object value, long timestamp, long streamTime); + void close(); } diff --git a/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java b/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java deleted file mode 100644 index 75c583bbb4d0e..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java +++ /dev/null @@ -1,35 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.KStreamJob; -import org.apache.kafka.common.utils.Utils; - -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - -/** - * Created by yasuhiro on 7/30/15. - */ -public class TopologyAnalyzer { - - public final Set topics; - public final Collection> streams; - - public TopologyAnalyzer(Class jobClass) { - KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); - KStreamInitializerImpl context = new KStreamInitializerImpl(); - - job.init(context); - - this.streams = context.sourceStreams(); - Set topics = new HashSet<>(); - for (KStreamSource stream : this.streams) { - for (String topic : stream.topics) { - topics.add(topic); - } - } - this.topics = Collections.unmodifiableSet(topics); - } - -} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index fc7f8dfa5fa9a..07be8729620ba 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -41,7 +42,7 @@ public boolean apply(Integer key, String value) { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); KStreamSource stream; KStream[] branches; TestProcessor[] processors; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index f6d0ea07c5339..23f75a6b43d9b 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -26,7 +27,7 @@ public boolean apply(Integer key, String value) { public void testFilter() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); KStreamSource stream; TestProcessor processor; @@ -47,7 +48,7 @@ public void testFilter() { public void testFilterOut() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 3074c0621485a..5ee1a8735cce1 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -33,12 +34,13 @@ public KeyValue> apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology topology = new MockKStreamTopology(); + KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); - stream = new KStreamSource<>(null, initializer); + stream = new KStreamSource<>(null, topology); stream.flatMap(mapper).process(processor); KStreamContext context = new MockKStreamContext(null, null); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 9d0a21c9bcb4e..5c527d457100e 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -32,7 +33,7 @@ public Iterable apply(String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 7b59ae9c45105..d56c77dc4633a 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; import io.confluent.streaming.testutil.UnlimitedWindow; import io.confluent.streaming.util.Util; @@ -68,7 +69,7 @@ public void testJoin() { TestProcessor processor; String[] expected; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -158,7 +159,7 @@ public void testJoinPrior() { TestProcessor processor; String[] expected; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -242,16 +243,17 @@ public void testMap() { KStreamWindowed windowed2; TestProcessor processor; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.map(keyValueMapper); - mapped2 = stream2.map(keyValueMapper); boolean exceptionRaised; try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.map(keyValueMapper); + mapped2 = stream2.map(keyValueMapper); + exceptionRaised = false; windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = mapped2.with(new UnlimitedWindow()); @@ -269,6 +271,11 @@ public void testMap() { assertTrue(exceptionRaised); try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.map(keyValueMapper); + mapped2 = stream2.map(keyValueMapper); + exceptionRaised = false; windowed1 = mapped1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -286,6 +293,11 @@ public void testMap() { assertTrue(exceptionRaised); try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.map(keyValueMapper); + mapped2 = stream2.map(keyValueMapper); + exceptionRaised = false; windowed1 = mapped1.with(new UnlimitedWindow()); windowed2 = mapped2.with(new UnlimitedWindow()); @@ -313,16 +325,17 @@ public void testFlatMap() { KStreamWindowed windowed2; TestProcessor processor; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.flatMap(keyValueMapper2); - mapped2 = stream2.flatMap(keyValueMapper2); boolean exceptionRaised; try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMap(keyValueMapper2); + mapped2 = stream2.flatMap(keyValueMapper2); + exceptionRaised = false; windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = mapped2.with(new UnlimitedWindow()); @@ -340,6 +353,11 @@ public void testFlatMap() { assertTrue(exceptionRaised); try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMap(keyValueMapper2); + mapped2 = stream2.flatMap(keyValueMapper2); + exceptionRaised = false; windowed1 = mapped1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -357,6 +375,11 @@ public void testFlatMap() { assertTrue(exceptionRaised); try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMap(keyValueMapper2); + mapped2 = stream2.flatMap(keyValueMapper2); + exceptionRaised = false; windowed1 = mapped1.with(new UnlimitedWindow()); windowed2 = mapped2.with(new UnlimitedWindow()); @@ -384,16 +407,17 @@ public void testMapValues() { KStreamWindowed windowed2; TestProcessor processor; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.mapValues(valueMapper); - mapped2 = stream2.mapValues(valueMapper); boolean exceptionRaised; try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.mapValues(valueMapper); + mapped2 = stream2.mapValues(valueMapper); + exceptionRaised = false; windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = mapped2.with(new UnlimitedWindow()); @@ -411,6 +435,11 @@ public void testMapValues() { assertFalse(exceptionRaised); try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.mapValues(valueMapper); + mapped2 = stream2.mapValues(valueMapper); + exceptionRaised = false; windowed1 = mapped1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -428,6 +457,11 @@ public void testMapValues() { assertFalse(exceptionRaised); try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.mapValues(valueMapper); + mapped2 = stream2.mapValues(valueMapper); + exceptionRaised = false; windowed1 = mapped1.with(new UnlimitedWindow()); windowed2 = mapped2.with(new UnlimitedWindow()); @@ -455,16 +489,17 @@ public void testFlatMapValues() { KStreamWindowed windowed2; TestProcessor processor; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.flatMapValues(valueMapper2); - mapped2 = stream2.flatMapValues(valueMapper2); boolean exceptionRaised; try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMapValues(valueMapper2); + mapped2 = stream2.flatMapValues(valueMapper2); + exceptionRaised = false; windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = mapped2.with(new UnlimitedWindow()); @@ -482,6 +517,11 @@ public void testFlatMapValues() { assertFalse(exceptionRaised); try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMapValues(valueMapper2); + mapped2 = stream2.flatMapValues(valueMapper2); + exceptionRaised = false; windowed1 = mapped1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); @@ -499,6 +539,11 @@ public void testFlatMapValues() { assertFalse(exceptionRaised); try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMapValues(valueMapper2); + mapped2 = stream2.flatMapValues(valueMapper2); + exceptionRaised = false; windowed1 = mapped1.with(new UnlimitedWindow()); windowed2 = mapped2.with(new UnlimitedWindow()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index ce6736e135345..44073c0a97dad 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -28,7 +29,7 @@ public KeyValue apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 85a889d23a7e0..50d2725cc1097 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -1,9 +1,10 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamTopology; import io.confluent.streaming.ValueMapper; import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -30,7 +31,7 @@ public Integer apply(String value) { final int[] expectedKeys = new int[] { 1, 10, 100, 1000 }; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 0db2b43881978..8493ebb800834 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -5,6 +5,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; import org.junit.Test; @@ -19,7 +20,7 @@ public class KStreamSourceTest { @Test public void testKStreamSource() { - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); TestProcessor processor = new TestProcessor<>(); KStreamSource stream = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index b8ea561efa81d..35846de99b9a1 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -2,6 +2,7 @@ import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.UnlimitedWindow; import org.junit.Test; @@ -23,7 +24,7 @@ public void testWindowedStream() { KStreamSource stream; Window window; - KStreamInitializer initializer = new KStreamInitializerImpl(); + KStreamTopology initializer = new MockKStreamTopology(); window = new UnlimitedWindow<>(); stream = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index fb0a65ef1c350..9999cfb8970b1 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -3,6 +3,7 @@ import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; @@ -32,7 +33,7 @@ private static class MockKStreamSource extends KStreamSource { public ArrayList streamTimes = new ArrayList<>(); public MockKStreamSource() { - super(null, deserializer, deserializer, new KStreamInitializerImpl()); + super(null, deserializer, deserializer, new MockKStreamTopology()); } @Override diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamTopology.java b/src/test/java/io/confluent/streaming/testutil/MockKStreamTopology.java new file mode 100644 index 0000000000000..e0bd3f5e951a9 --- /dev/null +++ b/src/test/java/io/confluent/streaming/testutil/MockKStreamTopology.java @@ -0,0 +1,12 @@ +package io.confluent.streaming.testutil; + +import io.confluent.streaming.KStreamTopology; + +/** + * Created by yasuhiro on 7/31/15. + */ +public class MockKStreamTopology extends KStreamTopology { + @Override + public void topology() { + } +} diff --git a/src/test/java/io/confluent/streaming/testutil/TestProcessor.java b/src/test/java/io/confluent/streaming/testutil/TestProcessor.java index 8a69e2b36b1e7..87c0764fceb6a 100644 --- a/src/test/java/io/confluent/streaming/testutil/TestProcessor.java +++ b/src/test/java/io/confluent/streaming/testutil/TestProcessor.java @@ -22,4 +22,9 @@ public void init(ProcessorContext context) { public void punctuate(long streamTime) { punctuated.add(streamTime); } + + @Override + public void close() { + } + } From 9964374b1e256e591f22945229f136c3e2b2f607 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 14:37:45 -0700 Subject: [PATCH 110/275] wip --- src/main/java/io/confluent/streaming/internal/Receiver.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 598e7b573860b..1d7479f138a38 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -12,4 +12,5 @@ public interface Receiver { void receive(Object key, Object value, long timestamp, long streamTime); void close(); + } From e2f612c99efcb919d401e8feaa4276d55488c71e Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 16:27:45 -0700 Subject: [PATCH 111/275] fix examples --- .../confluent/streaming/KStreamContext.java | 5 ++--- .../streaming/ProcessorKStreamJob.java | 9 +++++++++ .../java/io/confluent/streaming/Topics.java | 19 ------------------- .../streaming/examples/PrintKStreamJob.java | 6 +++++- .../examples/StatefulKStreamJob.java | 6 +++++- .../streaming/internal/Receiver.java | 2 +- 6 files changed, 22 insertions(+), 25 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/Topics.java diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 704b4c3352f52..cb69884ced49b 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -9,9 +9,8 @@ import java.util.Map; /** - * KStreamContext is the interface that allows an implementation of {@link KStreamJob#init(KStreamContext)} to create KStream instances. - * It also provides access to the system resources for a stream processing job. - * An instance of KStreamContext is created for each partition. + * KStreamContext is access to the system resources for a stream processing job. + * An instance of KStreamContext is created for each partition group. */ public interface KStreamContext { diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java index 15479bcc81085..cdcfa0cfab084 100644 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -5,6 +5,11 @@ */ public abstract class ProcessorKStreamJob extends KStreamTopology implements Processor { + private final String[] topics; + + public ProcessorKStreamJob(String... topics) { + this.topics = topics; + } @SuppressWarnings("unchecked") @Override <<<<<<< HEAD @@ -21,8 +26,12 @@ public void init(KStreamInitializer initializer) { >>>>>>> new api model ======= public void topology() { +<<<<<<< HEAD ((KStream)from()).process(this); >>>>>>> wip +======= + ((KStream)from(topics)).process(this); +>>>>>>> fix examples } } diff --git a/src/main/java/io/confluent/streaming/Topics.java b/src/main/java/io/confluent/streaming/Topics.java deleted file mode 100644 index 55df44f484c6b..0000000000000 --- a/src/main/java/io/confluent/streaming/Topics.java +++ /dev/null @@ -1,19 +0,0 @@ -package io.confluent.streaming; - -import io.confluent.streaming.util.Util; - -import java.util.Collections; -import java.util.Set; - -/** - * A class represents a set of topic names. - */ -public class Topics { - - public final Set topics; - - Topics(String... topics) { - this.topics = Collections.unmodifiableSet(Util.mkSet(topics)); - } - -} diff --git a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java index 5d335a8946391..a34ecbc8becaf 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java @@ -13,6 +13,10 @@ public class PrintKStreamJob extends ProcessorKStreamJob { private ProcessorContext processorContext; + public PrintKStreamJob(String... topics) { + super(topics); + } + @Override public void init(ProcessorContext context) { this.processorContext = context; @@ -38,7 +42,7 @@ public void close() { } public static void main(String[] args) { - KafkaStreaming kstream = new KafkaStreaming(new PrintKStreamJob(), new StreamingConfig(new Properties())); + KafkaStreaming kstream = new KafkaStreaming(new PrintKStreamJob(args), new StreamingConfig(new Properties())); kstream.run(); } } diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index d7b98a6e6a095..bd4261851a2cf 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -19,6 +19,10 @@ public class StatefulKStreamJob extends ProcessorKStreamJob { private ProcessorContext processorContext; private KeyValueStore kvStore; + public StatefulKStreamJob(String... topics) { + super(topics); + } + @Override public void init(ProcessorContext context) { this.processorContext = context; @@ -56,7 +60,7 @@ public void close() { } public static void main(String[] args) { - KafkaStreaming kstream = new KafkaStreaming(new StatefulKStreamJob(), new StreamingConfig(new Properties())); + KafkaStreaming kstream = new KafkaStreaming(new StatefulKStreamJob(args), new StreamingConfig(new Properties())); kstream.run(); } } diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 1d7479f138a38..509cab6fade20 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -12,5 +12,5 @@ public interface Receiver { void receive(Object key, Object value, long timestamp, long streamTime); void close(); - + } From cdbe8455cb2e686984f1a6443dc81d1acfbf0243 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 16:56:57 -0700 Subject: [PATCH 112/275] fix examples --- .../streaming/SingleProcessorTopology.java | 27 +++++++++++++++++++ .../streaming/examples/PrintKStreamJob.java | 16 +++++------ .../examples/StatefulKStreamJob.java | 18 ++++++------- 3 files changed, 44 insertions(+), 17 deletions(-) create mode 100644 src/main/java/io/confluent/streaming/SingleProcessorTopology.java diff --git a/src/main/java/io/confluent/streaming/SingleProcessorTopology.java b/src/main/java/io/confluent/streaming/SingleProcessorTopology.java new file mode 100644 index 0000000000000..59686c3f8db98 --- /dev/null +++ b/src/main/java/io/confluent/streaming/SingleProcessorTopology.java @@ -0,0 +1,27 @@ +package io.confluent.streaming; + +import org.apache.kafka.common.utils.Utils; + +/** + * Created by guozhang on 7/14/15. + */ +public class SingleProcessorTopology extends KStreamTopology { + + private final Class processorClass; + private final String[] topics; + + public SingleProcessorTopology(Class processorClass, String... topics) { + this.processorClass = processorClass; + this.topics = topics; + } + @SuppressWarnings("unchecked") + @Override + public void topology() { + from(topics).process(newProcessor()); + } + + @SuppressWarnings("unchecked") + private Processor newProcessor() { + return (Processor) Utils.newInstance(processorClass); + } +} diff --git a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java index a34ecbc8becaf..a5c9e54177a3c 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java @@ -1,7 +1,8 @@ package io.confluent.streaming.examples; import io.confluent.streaming.KafkaStreaming; -import io.confluent.streaming.ProcessorKStreamJob; +import io.confluent.streaming.Processor; +import io.confluent.streaming.SingleProcessorTopology; import io.confluent.streaming.StreamingConfig; import java.util.Properties; @@ -9,14 +10,10 @@ /** * Created by guozhang on 7/14/15. */ -public class PrintKStreamJob extends ProcessorKStreamJob { +public class PrintKStreamJob implements Processor { private ProcessorContext processorContext; - public PrintKStreamJob(String... topics) { - super(topics); - } - @Override public void init(ProcessorContext context) { this.processorContext = context; @@ -42,7 +39,10 @@ public void close() { } public static void main(String[] args) { - KafkaStreaming kstream = new KafkaStreaming(new PrintKStreamJob(args), new StreamingConfig(new Properties())); - kstream.run(); + KafkaStreaming streaming = new KafkaStreaming( + new SingleProcessorTopology(PrintKStreamJob.class, args), + new StreamingConfig(new Properties()) + ); + streaming.run(); } } diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index bd4261851a2cf..3473607087f28 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -1,7 +1,8 @@ package io.confluent.streaming.examples; import io.confluent.streaming.KafkaStreaming; -import io.confluent.streaming.ProcessorKStreamJob; +import io.confluent.streaming.Processor; +import io.confluent.streaming.SingleProcessorTopology; import io.confluent.streaming.StreamingConfig; import io.confluent.streaming.kv.Entry; import io.confluent.streaming.kv.InMemoryKeyValueStore; @@ -14,22 +15,18 @@ * Created by guozhang on 7/27/15. */ -public class StatefulKStreamJob extends ProcessorKStreamJob { +public class StatefulKStreamJob implements Processor { private ProcessorContext processorContext; private KeyValueStore kvStore; - public StatefulKStreamJob(String... topics) { - super(topics); - } - @Override public void init(ProcessorContext context) { this.processorContext = context; this.processorContext.schedule(1000); this.kvStore = new InMemoryKeyValueStore<>("local-state", context.kstreamContext()); - this.kvStore.restore(); // call restore inside processor.bind + this.kvStore.restore(); // call restore inside processor.init } @Override @@ -60,7 +57,10 @@ public void close() { } public static void main(String[] args) { - KafkaStreaming kstream = new KafkaStreaming(new StatefulKStreamJob(args), new StreamingConfig(new Properties())); - kstream.run(); + KafkaStreaming streaming = new KafkaStreaming( + new SingleProcessorTopology(StatefulKStreamJob.class, args), + new StreamingConfig(new Properties()) + ); + streaming.run(); } } From 3b6b3b4231c16f1a5fe9651a1438f9e9627ca0f9 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 3 Aug 2015 11:43:42 -0700 Subject: [PATCH 113/275] removed ProcessorContext --- .../confluent/streaming/KStreamContext.java | 2 +- .../io/confluent/streaming/Processor.java | 24 +--- .../streaming/PunctuationScheduler.java | 12 -- .../streaming/examples/PrintKStreamJob.java | 11 +- .../examples/StatefulKStreamJob.java | 13 +- .../internal/KStreamContextImpl.java | 23 ++-- .../streaming/internal/KStreamImpl.java | 30 ++--- .../internal/KStreamWindowedImpl.java | 2 +- .../internal/ProcessorContextImpl.java | 68 ---------- .../streaming/internal/ProcessorNode.java | 2 +- .../streaming/internal/PunctuationQueue.java | 24 ++-- .../internal/PunctuationSchedule.java | 23 ++++ .../internal/PunctuationSchedulerImpl.java | 44 ------ .../streaming/internal/StreamGroup.java | 9 +- .../PunctuationSchedulerImplTest.java | 125 ------------------ .../testutil/MockKStreamContext.java | 6 +- .../streaming/testutil/TestProcessor.java | 8 +- 17 files changed, 86 insertions(+), 340 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/PunctuationScheduler.java delete mode 100644 src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/PunctuationSchedule.java delete mode 100644 src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java delete mode 100644 src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index cb69884ced49b..89395b39873d0 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -94,7 +94,7 @@ public interface KStreamContext { void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); - PunctuationScheduler getPunctuationScheduler(Processor processor); + void schedule(Processor processor, long interval); void commit(); diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 2e1754ebd76d5..6ab9f2ddcba19 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -7,29 +7,7 @@ */ public interface Processor { - public interface ProcessorContext { - - void send(String topic, Object key, Object value); - - void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); - - void schedule(long timestamp); - - void commit(); - - String topic(); - - int partition(); - - long offset(); - - long timestamp(); - - KStreamContext kstreamContext(); - - } - - void init(ProcessorContext context); + void init(KStreamContext context); void process(K key, V value); diff --git a/src/main/java/io/confluent/streaming/PunctuationScheduler.java b/src/main/java/io/confluent/streaming/PunctuationScheduler.java deleted file mode 100644 index a79a4b746581c..0000000000000 --- a/src/main/java/io/confluent/streaming/PunctuationScheduler.java +++ /dev/null @@ -1,12 +0,0 @@ -package io.confluent.streaming; - -/** - * An interface that allows {@link Processor} to schedule a notification at a specified stream time. - */ -public interface PunctuationScheduler { - - void schedule(long timestamp); - - void cancel(); - -} diff --git a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java index a5c9e54177a3c..002c71ec70ebd 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java @@ -1,5 +1,6 @@ package io.confluent.streaming.examples; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KafkaStreaming; import io.confluent.streaming.Processor; import io.confluent.streaming.SingleProcessorTopology; @@ -12,20 +13,20 @@ */ public class PrintKStreamJob implements Processor { - private ProcessorContext processorContext; + private KStreamContext context; @Override - public void init(ProcessorContext context) { - this.processorContext = context; + public void init(KStreamContext context) { + this.context = context; } @Override public void process(K key, V value) { System.out.println("[" + key + ", " + value + "]"); - processorContext.commit(); + context.commit(); - processorContext.send("topic", key, value); + context.send("topic", key, value); } @Override diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index 3473607087f28..2ae4c076bd1e9 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -1,5 +1,6 @@ package io.confluent.streaming.examples; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KafkaStreaming; import io.confluent.streaming.Processor; import io.confluent.streaming.SingleProcessorTopology; @@ -17,15 +18,15 @@ public class StatefulKStreamJob implements Processor { - private ProcessorContext processorContext; + private KStreamContext context; private KeyValueStore kvStore; @Override - public void init(ProcessorContext context) { - this.processorContext = context; - this.processorContext.schedule(1000); + public void init(KStreamContext context) { + this.context = context; + this.context.schedule(this, 1000); - this.kvStore = new InMemoryKeyValueStore<>("local-state", context.kstreamContext()); + this.kvStore = new InMemoryKeyValueStore<>("local-state", context); this.kvStore.restore(); // call restore inside processor.init } @@ -39,7 +40,7 @@ public void process(String key, Integer value) { this.kvStore.put(key, newValue); } - processorContext.commit(); + context.commit(); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 1713619db65ab..31e887550a92d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -2,7 +2,6 @@ import io.confluent.streaming.KStreamException; import io.confluent.streaming.Processor; -import io.confluent.streaming.PunctuationScheduler; import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StateStore; import io.confluent.streaming.StreamingConfig; @@ -233,17 +232,18 @@ public void flush() { @Override public String topic() { - if (this.streamGroup.record() == null) + if (streamGroup.record() == null) throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); - return this.streamGroup.record().topic(); + return streamGroup.record().topic(); } @Override public int partition() { - if (this.streamGroup.record() == null) + if (streamGroup.record() == null) throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); +<<<<<<< HEAD <<<<<<< HEAD stateMgr.restore(store, restoreFunc); } @@ -256,6 +256,9 @@ public void register(StateStore store) { ======= return this.streamGroup.record().partition(); >>>>>>> new api model +======= + return streamGroup.record().partition(); +>>>>>>> removed ProcessorContext } @Override @@ -268,10 +271,10 @@ public long offset() { @Override public long timestamp() { - if (this.streamGroup.record() == null) + if (streamGroup.record() == null) throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); - return this.streamGroup.record().timestamp; + return streamGroup.record().timestamp; } @Override @@ -289,15 +292,15 @@ public void send(String topic, Object key, Object value, Serializer keyS @Override public void commit() { - this.streamGroup.commitOffset(); + streamGroup.commitOffset(); } @Override - public PunctuationScheduler getPunctuationScheduler(Processor processor) { - return streamGroup.getPunctuationScheduler(processor); + public void schedule(Processor processor, long interval) { + streamGroup.schedule(processor, interval); } - public void init(Collection> streams) throws IOException { + void init(Collection> streams) throws IOException { stateMgr.init(); for (KStreamSource stream: streams) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 75230504fc160..ae587c18b3e73 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -20,12 +20,12 @@ abstract class KStreamImpl implements KStream, Receiver { private final ArrayList nextReceivers = new ArrayList<>(1); - protected KStreamTopology initializer; + protected KStreamTopology topology; protected KStreamContext context; protected KStreamMetadata metadata; - protected KStreamImpl(KStreamTopology initializer) { - this.initializer = initializer; + protected KStreamImpl(KStreamTopology topology) { + this.topology = topology; } @Override @@ -49,7 +49,7 @@ public void close() { @Override public KStream filter(Predicate predicate) { - return chain(new KStreamFilter(predicate, initializer)); + return chain(new KStreamFilter(predicate, topology)); } @Override @@ -63,32 +63,32 @@ public boolean apply(K key, V value) { @Override public KStream map(KeyValueMapper mapper) { - return chain(new KStreamMap(mapper, initializer)); + return chain(new KStreamMap(mapper, topology)); } @Override public KStream mapValues(ValueMapper mapper) { - return chain(new KStreamMapValues(mapper, initializer)); + return chain(new KStreamMapValues(mapper, topology)); } @Override public KStream flatMap(KeyValueMapper, K, V> mapper) { - return chain(new KStreamFlatMap(mapper, initializer)); + return chain(new KStreamFlatMap(mapper, topology)); } @Override public KStream flatMapValues(ValueMapper, V> mapper) { - return chain(new KStreamFlatMapValues(mapper, initializer)); + return chain(new KStreamFlatMapValues(mapper, topology)); } @Override public KStreamWindowed with(Window window) { - return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, initializer)); + return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, topology)); } @Override public KStream[] branch(Predicate... predicates) { - KStreamBranch branch = new KStreamBranch<>(predicates, initializer); + KStreamBranch branch = new KStreamBranch<>(predicates, topology); registerReceiver(branch); return branch.branches; } @@ -103,7 +103,7 @@ public KStream through(String topic) { @Override public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); - return initializer.from(keyDeserializer, valDeserializer, topic); + return topology.from(keyDeserializer, valDeserializer, topic); } @Override @@ -119,15 +119,15 @@ public void sendTo(String topic, Serializer keySerializer, Serializer valS @SuppressWarnings("unchecked") private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { return new Processor() { - private ProcessorContext processorContext; + private KStreamContext context; @Override - public void init(ProcessorContext processorContext) { - this.processorContext = processorContext; + public void init(KStreamContext context) { + this.context = context; } @Override public void process(K key, V value) { - this.processorContext.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); + this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); } @Override public void punctuate(long streamTime) {} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 4aa0395d04b93..d650f2572f27d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -56,7 +56,7 @@ private KStream join(KStreamWindowed other, boolean prior KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; - KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, initializer); + KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, topology); otherImpl.registerReceiver(stream.receiverForOtherStream); return chain(stream); diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java deleted file mode 100644 index 2874f8abbfe23..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java +++ /dev/null @@ -1,68 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.Processor; -import io.confluent.streaming.PunctuationScheduler; -import io.confluent.streaming.KStreamContext; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.Serializer; - -/** - * Created by guozhang on 7/21/15. - */ -public class ProcessorContextImpl implements Processor.ProcessorContext { - - private final KStreamContext context; - private final PunctuationScheduler scheduler; - - public ProcessorContextImpl(KStreamContext context, - PunctuationScheduler scheduler) { - - this.context = context; - this.scheduler = scheduler; - } - - @Override - public String topic() { - return context.topic(); - } - - @Override - public int partition() { - return context.partition(); - } - - @Override - public long offset() { - return context.offset(); - } - - @Override - public long timestamp() { - return context.timestamp(); - } - - @Override - public void send(String topic, Object key, Object value) { - context.send(topic, key, value); - } - - @Override - public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { - context.send(topic, key, value, keySerializer, valSerializer); - } - - @Override - public void commit() { - context.commit(); - } - - @Override - public void schedule(long timestamp) { - scheduler.schedule(timestamp); - } - - @Override - public KStreamContext kstreamContext() { - return context; - } -} diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java index 1ed0ff256135c..615105fe0e0ac 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java @@ -20,7 +20,7 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); this.context = context; - processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); + processor.init(context); } @SuppressWarnings("unchecked") @Override diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java b/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java index 4f32d8002526d..d975c6f7bf2da 100644 --- a/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java +++ b/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java @@ -1,5 +1,6 @@ package io.confluent.streaming.internal; +import io.confluent.streaming.Processor; import io.confluent.streaming.util.Stamped; import java.util.PriorityQueue; @@ -9,23 +10,14 @@ */ public class PunctuationQueue { - private PriorityQueue> pq = new PriorityQueue>(); + private PriorityQueue pq = new PriorityQueue<>(); - public Stamped schedule(PunctuationSchedulerImpl scheduler, long time) { + public void schedule(PunctuationSchedule sched) { synchronized (pq) { - Stamped stamped = new Stamped(scheduler, time); - pq.add(stamped); - return stamped; + pq.add(sched); } } - public void cancel(Stamped stamped) { - synchronized (pq) { - pq.remove(stamped); - } - - } - public void close() { synchronized (pq) { pq.clear(); @@ -34,12 +26,12 @@ public void close() { public void mayPunctuate(long streamTime) { synchronized (pq) { - Stamped top = pq.peek(); + PunctuationSchedule top = pq.peek(); while (top != null && top.timestamp <= streamTime) { - PunctuationSchedulerImpl scheduler = top.value; + PunctuationSchedule sched = top; pq.poll(); - scheduler.processor.punctuate(streamTime); - scheduler.processed(); + sched.processor().punctuate(streamTime); + pq.add(sched.next()); top = pq.peek(); } diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationSchedule.java b/src/main/java/io/confluent/streaming/internal/PunctuationSchedule.java new file mode 100644 index 0000000000000..ae8317633cd22 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/PunctuationSchedule.java @@ -0,0 +1,23 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.Processor; +import io.confluent.streaming.util.Stamped; + +public class PunctuationSchedule extends Stamped>{ + + final long interval; + + public PunctuationSchedule(Processor processor, long interval) { + super(processor, System.currentTimeMillis() + interval); + this.interval = interval; + } + + public Processor processor() { + return value; + } + + public PunctuationSchedule next() { + return new PunctuationSchedule(value, timestamp + interval); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java b/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java deleted file mode 100644 index 5df8ec4b7a845..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/PunctuationSchedulerImpl.java +++ /dev/null @@ -1,44 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.Processor; -import io.confluent.streaming.PunctuationScheduler; -import io.confluent.streaming.util.Stamped; - -public class PunctuationSchedulerImpl implements PunctuationScheduler { - - private Stamped scheduled; - private final PunctuationQueue queue; - final Processor processor; - - public PunctuationSchedulerImpl(PunctuationQueue queue, Processor processor) { - this.queue = queue; - this.processor = processor; - } - - @Override - public void schedule(long timestamp) { - synchronized (this) { - if (scheduled != null) - throw new IllegalStateException("punctuation is already scheduled"); - - scheduled = queue.schedule(this, timestamp); - } - } - - @Override - public void cancel() { - synchronized (this) { - if (scheduled != null) { - queue.cancel(scheduled); - scheduled = null; - } - } - } - - public void processed() { - synchronized (this) { - scheduled = null; - } - } - -} diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index fce7393a110bc..45b5a54dc3970 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -2,7 +2,6 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; -import io.confluent.streaming.PunctuationScheduler; import io.confluent.streaming.TimestampExtractor; import io.confluent.streaming.util.MinTimestampTracker; import io.confluent.streaming.util.ParallelExecutor; @@ -198,12 +197,12 @@ private void ingestNewRecords() { } /** - * Returns a PunctuationScheduler + * Schedules a punctuation for the processor * @param processor the processor requesting scheduler - * @return PunctuationScheduler + * @param interval the interval in milliseconds */ - public PunctuationScheduler getPunctuationScheduler(Processor processor) { - return new PunctuationSchedulerImpl(punctuationQueue, processor); + public void schedule(Processor processor, long interval) { + punctuationQueue.schedule(new PunctuationSchedule(processor, interval)); } /** diff --git a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java deleted file mode 100644 index 014e140685268..0000000000000 --- a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java +++ /dev/null @@ -1,125 +0,0 @@ -package io.confluent.streaming.internal; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import io.confluent.streaming.PunctuationScheduler; -import io.confluent.streaming.testutil.TestProcessor; -import org.junit.Test; - -public class PunctuationSchedulerImplTest { - - @Test - public void testScheduling() { - PunctuationQueue queue = new PunctuationQueue(); - - TestProcessor proc1 = new TestProcessor<>(); - PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); - - TestProcessor proc2 = new TestProcessor<>(); - PunctuationScheduler sched2 = new PunctuationSchedulerImpl(queue, proc2); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - - proc1.init(new ProcessorContextImpl(null, sched1)); - proc2.init(new ProcessorContextImpl(null, sched2)); - - sched1.schedule(500); - sched2.schedule(1000); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - - queue.mayPunctuate(999); - - assertEquals(1, proc1.punctuated.size()); - assertEquals(999, (long) proc1.punctuated.get(0)); - assertEquals(0, proc2.punctuated.size()); - - proc1.punctuated.clear(); - queue.mayPunctuate(1000); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(1, proc2.punctuated.size()); - assertEquals(1000, (long) proc2.punctuated.get(0)); - - proc2.punctuated.clear(); - queue.mayPunctuate(2000); - assertEquals(0, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - - sched1.schedule(3000); - queue.mayPunctuate(4000); - - assertEquals(1, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - assertEquals(4000, (long) proc1.punctuated.get(0)); - } - - @Test - public void testCanceling() { - PunctuationQueue queue = new PunctuationQueue(); - - TestProcessor proc1 = new TestProcessor(); - PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); - - TestProcessor proc2 = new TestProcessor(); - PunctuationScheduler sched2 = new PunctuationSchedulerImpl(queue, proc2); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - - proc1.init(new ProcessorContextImpl(null, sched1)); - proc2.init(new ProcessorContextImpl(null, sched2)); - - sched1.schedule(500); - sched2.schedule(1000); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - - sched1.cancel(); - - queue.mayPunctuate(1000); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(1, proc2.punctuated.size()); - assertEquals(1000, (long) proc2.punctuated.get(0)); - - sched1.schedule(2000); - sched1.cancel(); - sched1.schedule(3000); - - queue.mayPunctuate(2000); - assertEquals(0, proc1.punctuated.size()); - - queue.mayPunctuate(3000); - assertEquals(1, proc1.punctuated.size()); - } - - @Test - public void testDuplicateScheduling() { - PunctuationQueue queue = new PunctuationQueue(); - - TestProcessor proc1 = new TestProcessor(); - PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); - - assertEquals(0, proc1.punctuated.size()); - - proc1.init(new ProcessorContextImpl(null, sched1)); - - sched1.schedule(500); - - boolean exceptionRaised = false; - try { - sched1.schedule(1000); - } - catch (IllegalStateException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - } - -} diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java index 009d3503cf3a2..7e478764da471 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java @@ -3,7 +3,6 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; -import io.confluent.streaming.PunctuationScheduler; import io.confluent.streaming.RecordCollector; <<<<<<< HEAD import io.confluent.streaming.StorageEngine; @@ -15,7 +14,6 @@ import io.confluent.streaming.internal.StreamGroup; ======= import io.confluent.streaming.internal.PunctuationQueue; -import io.confluent.streaming.internal.PunctuationSchedulerImpl; import io.confluent.streaming.kv.internals.RestoreFunc; >>>>>>> new api model import org.apache.kafka.common.metrics.Metrics; @@ -85,8 +83,8 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { throw new UnsupportedOperationException("send() not supported."); } @Override - public PunctuationScheduler getPunctuationScheduler(Processor processor) { - return new PunctuationSchedulerImpl(punctuationQueue, processor); + public void schedule(Processor processor, long interval) { + throw new UnsupportedOperationException("schedule() not supported"); } @Override diff --git a/src/test/java/io/confluent/streaming/testutil/TestProcessor.java b/src/test/java/io/confluent/streaming/testutil/TestProcessor.java index 87c0764fceb6a..324407d0123ab 100644 --- a/src/test/java/io/confluent/streaming/testutil/TestProcessor.java +++ b/src/test/java/io/confluent/streaming/testutil/TestProcessor.java @@ -1,13 +1,13 @@ package io.confluent.streaming.testutil; +import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; -import io.confluent.streaming.PunctuationScheduler; import java.util.ArrayList; public class TestProcessor implements Processor { - public final ArrayList processed = new ArrayList(); - public final ArrayList punctuated = new ArrayList(); + public final ArrayList processed = new ArrayList<>(); + public final ArrayList punctuated = new ArrayList<>(); @Override public void process(K key, V value) { @@ -15,7 +15,7 @@ public void process(K key, V value) { } @Override - public void init(ProcessorContext context) { + public void init(KStreamContext context) { } @Override From 0cff31b1abd42456676cb131836f97cf2dfea71b Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 3 Aug 2015 14:09:21 -0700 Subject: [PATCH 114/275] remove streamTime from Receiver --- .../streaming/internal/KStreamBranch.java | 4 ++-- .../streaming/internal/KStreamFilter.java | 4 ++-- .../streaming/internal/KStreamFlatMap.java | 4 ++-- .../streaming/internal/KStreamFlatMapValues.java | 4 ++-- .../streaming/internal/KStreamImpl.java | 4 ++-- .../streaming/internal/KStreamJoin.java | 12 ++++++------ .../confluent/streaming/internal/KStreamMap.java | 4 ++-- .../streaming/internal/KStreamMapValues.java | 4 ++-- .../streaming/internal/KStreamSource.java | 4 ++-- .../streaming/internal/KStreamWindowedImpl.java | 4 ++-- .../streaming/internal/ProcessorNode.java | 2 +- .../confluent/streaming/internal/Receiver.java | 2 +- .../streaming/internal/StreamGroup.java | 2 +- .../streaming/internal/KStreamBranchTest.java | 4 ++-- .../streaming/internal/KStreamFilterTest.java | 4 ++-- .../streaming/internal/KStreamFlatMapTest.java | 2 +- .../internal/KStreamFlatMapValuesTest.java | 2 +- .../streaming/internal/KStreamJoinTest.java | 16 ++++++++-------- .../streaming/internal/KStreamMapTest.java | 2 +- .../streaming/internal/KStreamMapValuesTest.java | 2 +- .../streaming/internal/KStreamSourceTest.java | 2 +- .../streaming/internal/KStreamWindowedTest.java | 4 ++-- .../streaming/internal/StreamGroupTest.java | 4 +--- 23 files changed, 47 insertions(+), 49 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 87e8c580a1a9d..87f1619dd8bbd 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -33,11 +33,11 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp) { for (int i = 0; i < predicates.length; i++) { Predicate predicate = predicates[i]; if (predicate.apply((K)key, (V)value)) { - branches[i].receive(key, value, timestamp, streamTime); + branches[i].receive(key, value, timestamp); return; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 4a4571b2c2045..6b5e1c52ff209 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -17,10 +17,10 @@ class KStreamFilter extends KStreamImpl { @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp,long streamTime) { + public void receive(Object key, Object value, long timestamp) { synchronized(this) { if (predicate.apply((K)key, (V)value)) { - forward(key, value, timestamp, streamTime); + forward(key, value, timestamp); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 85f5d35189f07..95533c0cf8a73 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -24,11 +24,11 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp) { synchronized(this) { KeyValue> newPair = mapper.apply((K1)key, (V1)value); for (V v : newPair.value) { - forward(newPair.key, v, timestamp, streamTime); + forward(newPair.key, v, timestamp); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 3fe4f36ab44dd..3f09024a8d1ef 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -17,11 +17,11 @@ class KStreamFlatMapValues extends KStreamImpl { @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp) { synchronized(this) { Iterable newValues = mapper.apply((V1)value); for (V v : newValues) { - forward(key, v, timestamp, streamTime); + forward(key, v, timestamp); } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index ae587c18b3e73..ffec849cb768e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -146,10 +146,10 @@ void registerReceiver(Receiver receiver) { nextReceivers.add(receiver); } - protected void forward(Object key, Object value, long timestamp, long streamTime) { + protected void forward(Object key, Object value, long timestamp) { int numReceivers = nextReceivers.size(); for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).receive(key, value, timestamp, streamTime); + nextReceivers.get(i).receive(key, value, timestamp); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index 1810d984f2377..f7bf3459346d9 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -70,11 +70,11 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp) { Iterator iter = finder2.find((K)key, timestamp); if (iter != null) { while (iter.hasNext()) { - doJoin((K)key, (V1)value, iter.next(), timestamp, streamTime); + doJoin((K)key, (V1)value, iter.next(), timestamp); } } } @@ -88,11 +88,11 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { } @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value2, long timestamp, long streamTime) { + public void receive(Object key, Object value2, long timestamp) { Iterator iter = finder1.find((K)key, timestamp); if (iter != null) { while (iter.hasNext()) { - doJoin((K)key, iter.next(), (V2)value2, timestamp, streamTime); + doJoin((K)key, iter.next(), (V2)value2, timestamp); } } } @@ -104,8 +104,8 @@ public void close() { } // TODO: use the "outer-stream" topic as the resulted join stream topic - private void doJoin(K key, V1 value1, V2 value2, long timestamp, long streamTime) { - forward(key, joiner.apply(value1, value2), timestamp, streamTime); + private void doJoin(K key, V1 value1, V2 value2, long timestamp) { + forward(key, joiner.apply(value1, value2), timestamp); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index 7e607b634f232..dba4448c3f7c5 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -24,10 +24,10 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp) { synchronized (this) { KeyValue newPair = mapper.apply((K1)key, (V1)value); - forward(newPair.key, newPair.value, timestamp, streamTime); + forward(newPair.key, newPair.value, timestamp); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index ffca04141eda0..c67d86c8b467a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -17,10 +17,10 @@ class KStreamMapValues extends KStreamImpl { @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp) { synchronized (this) { V newValue = mapper.apply((V1)value); - forward(key, newValue, timestamp, streamTime); + forward(key, newValue, timestamp); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 331ea29a29c6b..6e4662ed4c791 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -40,10 +40,10 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp) { synchronized(this) { // KStream needs to forward the topic name since it is directly from the Kafka source - forward(key, value, timestamp, streamTime); + forward(key, value, timestamp); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index d650f2572f27d..7e6295f2d667d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -28,11 +28,11 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp) { synchronized(this) { window.put((K)key, (V)value, timestamp); // KStreamWindowed needs to forward the topic name since it may receive directly from KStreamSource - forward(key, value, timestamp, streamTime); + forward(key, value, timestamp); } } diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java index 615105fe0e0ac..ebd7622e1b07d 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java @@ -24,7 +24,7 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { } @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp) { processor.process((K) key, (V) value); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 509cab6fade20..28ac89c71411f 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -9,7 +9,7 @@ public interface Receiver { void bind(KStreamContext context, KStreamMetadata metadata); - void receive(Object key, Object value, long timestamp, long streamTime); + void receive(Object key, Object value, long timestamp); void close(); diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index 45b5a54dc3970..a08c4c06efeee 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -231,7 +231,7 @@ public boolean process() { if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - recordQueue.stream.receive(currRecord.key(), currRecord.value(), currRecord.timestamp, streamTime); + recordQueue.stream.receive(currRecord.key(), currRecord.value(), currRecord.timestamp); consumedOffsets.put(recordQueue.partition(), currRecord.offset()); // TODO: local state flush and downstream producer flush diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index 07be8729620ba..b9f1ffb3c38fe 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -59,7 +59,7 @@ public boolean apply(Integer key, String value) { } for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); } assertEquals(3, processors[0].processed.size()); @@ -80,7 +80,7 @@ public boolean apply(Integer key, String value) { KStreamContext context = new MockKStreamContext(null, null); stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); } assertEquals(3, processors[0].processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 23f75a6b43d9b..784c9cefe793f 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -38,7 +38,7 @@ public void testFilter() { KStreamContext context = new MockKStreamContext(null, null); stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); } assertEquals(2, processor.processed.size()); @@ -59,7 +59,7 @@ public void testFilterOut() { KStreamContext context = new MockKStreamContext(null, null); stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); } assertEquals(5, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 5ee1a8735cce1..c6d06a2ca8e44 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -46,7 +46,7 @@ public KeyValue> apply(Integer key, String value) { KStreamContext context = new MockKStreamContext(null, null); stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); } assertEquals(6, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 5c527d457100e..d6903108df4ef 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -44,7 +44,7 @@ public Iterable apply(String value) { KStreamContext context = new MockKStreamContext(null, null); stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); } assertEquals(8, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index d56c77dc4633a..77a91c1d282df 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -94,7 +94,7 @@ public void testJoin() { // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); } assertEquals(0, processor.processed.size()); @@ -102,7 +102,7 @@ public void testJoin() { // push two items to the other stream. the main stream's window has two items for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); } assertEquals(2, processor.processed.size()); @@ -118,7 +118,7 @@ public void testJoin() { // push all items to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L, 0L); + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); } assertEquals(2, processor.processed.size()); @@ -135,7 +135,7 @@ public void testJoin() { // push all items to the other stream. this should produce 6 items for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); } assertEquals(6, processor.processed.size()); @@ -184,7 +184,7 @@ public void testJoinPrior() { // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i, 0L); + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i); } assertEquals(0, processor.processed.size()); @@ -193,7 +193,7 @@ public void testJoinPrior() { // no corresponding item in the main window has a newer timestamp for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i + 1, 0L); + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i + 1); } assertEquals(0, processor.processed.size()); @@ -203,7 +203,7 @@ public void testJoinPrior() { // push all items with newer timestamps to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i + 2, 0L); + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i + 2); } assertEquals(2, processor.processed.size()); @@ -220,7 +220,7 @@ public void testJoinPrior() { // push all items with older timestamps to the other stream. this should produce six items for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i, 0L); + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i); } assertEquals(6, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 44073c0a97dad..93d3b47c352ec 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -40,7 +40,7 @@ public KeyValue apply(Integer key, String value) { KStreamContext context = new MockKStreamContext(null, null); stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); } assertEquals(4, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 50d2725cc1097..19c90e8c02970 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -42,7 +42,7 @@ public Integer apply(String value) { KStreamContext context = new MockKStreamContext(null, null); stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L, 0L); + stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L); } assertEquals(4, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 8493ebb800834..293c5c9f9dc35 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -32,7 +32,7 @@ public void testKStreamSource() { KStreamContext context = new MockKStreamContext(null, null); stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], expectedValues[i], 0L, 0L); + stream.receive(expectedKeys[i], expectedValues[i], 0L); } assertEquals(3, processor.processed.size()); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 35846de99b9a1..07a61466728ec 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -38,7 +38,7 @@ public void testWindowedStream() { // two items in the window for (int i = 0; i < 2; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); } assertEquals(1, countItem(window.find(0, 0L))); @@ -49,7 +49,7 @@ public void testWindowedStream() { // previous two items + all items, thus two are duplicates, in the window for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L, 0L); + stream.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); } assertEquals(2, countItem(window.find(0, 0L))); diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index 9999cfb8970b1..db54814c6ee72 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -30,19 +30,17 @@ private static class MockKStreamSource extends KStreamSource { public ArrayList keys = new ArrayList<>(); public ArrayList values = new ArrayList<>(); public ArrayList timestamps = new ArrayList<>(); - public ArrayList streamTimes = new ArrayList<>(); public MockKStreamSource() { super(null, deserializer, deserializer, new MockKStreamTopology()); } @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { + public void receive(Object key, Object value, long timestamp) { this.numReceived++; this.keys.add(key); this.values.add(value); this.timestamps.add(timestamp); - this.streamTimes.add(streamTime); } } From 7a75a5dd5a01c09ab90b430f0304e86b20a5cfa0 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 3 Aug 2015 15:29:49 -0700 Subject: [PATCH 115/275] fix parameter name --- src/main/java/io/confluent/streaming/Processor.java | 2 -- .../io/confluent/streaming/internal/KStreamBranch.java | 4 ++-- .../io/confluent/streaming/internal/KStreamFilter.java | 4 ++-- .../io/confluent/streaming/internal/KStreamFlatMap.java | 4 ++-- .../streaming/internal/KStreamFlatMapValues.java | 4 ++-- .../java/io/confluent/streaming/internal/KStreamJoin.java | 4 ++-- .../java/io/confluent/streaming/internal/KStreamMap.java | 4 ++-- .../io/confluent/streaming/internal/KStreamMapValues.java | 4 ++-- .../io/confluent/streaming/internal/KStreamSource.java | 8 ++++---- 9 files changed, 18 insertions(+), 20 deletions(-) diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 6ab9f2ddcba19..dc30cf4ae30be 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -1,7 +1,5 @@ package io.confluent.streaming; -import org.apache.kafka.common.serialization.Serializer; - /** * Created by yasuhiro on 6/17/15. */ diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 87f1619dd8bbd..880676e936e51 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -16,11 +16,11 @@ class KStreamBranch implements Receiver { final KStreamSource[] branches; @SuppressWarnings("unchecked") - KStreamBranch(Predicate[] predicates, KStreamTopology initializer) { + KStreamBranch(Predicate[] predicates, KStreamTopology topology) { this.predicates = Arrays.copyOf(predicates, predicates.length); this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { - branches[i] = new KStreamSource<>(null, initializer); + branches[i] = new KStreamSource<>(null, topology); } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 6b5e1c52ff209..55edcf5c133c0 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -10,8 +10,8 @@ class KStreamFilter extends KStreamImpl { private final Predicate predicate; - KStreamFilter(Predicate predicate, KStreamTopology initializer) { - super(initializer); + KStreamFilter(Predicate predicate, KStreamTopology topology) { + super(topology); this.predicate = predicate; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 95533c0cf8a73..1748f7b136a1e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -12,8 +12,8 @@ class KStreamFlatMap extends KStreamImpl { private final KeyValueMapper, K1, V1> mapper; - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology initializer) { - super(initializer); + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology topology) { + super(topology); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 3f09024a8d1ef..7e0c7e32acdc0 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -10,8 +10,8 @@ class KStreamFlatMapValues extends KStreamImpl { private final ValueMapper, V1> mapper; - KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology initializer) { - super(initializer); + KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology topology) { + super(topology); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index f7bf3459346d9..776815f211594 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -24,8 +24,8 @@ private static abstract class Finder { private KStreamMetadata thisMetadata; private KStreamMetadata otherMetadata; - KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology initializer) { - super(initializer); + KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology topology) { + super(topology); final Window window1 = stream1.window; final Window window2 = stream2.window; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index dba4448c3f7c5..bcb6ca4599bda 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -12,8 +12,8 @@ class KStreamMap extends KStreamImpl { private final KeyValueMapper mapper; - KStreamMap(KeyValueMapper mapper, KStreamTopology initializer) { - super(initializer); + KStreamMap(KeyValueMapper mapper, KStreamTopology topology) { + super(topology); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index c67d86c8b467a..96f4c2e78c070 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -10,8 +10,8 @@ class KStreamMapValues extends KStreamImpl { private final ValueMapper mapper; - KStreamMapValues(ValueMapper mapper, KStreamTopology initializer) { - super(initializer); + KStreamMapValues(ValueMapper mapper, KStreamTopology topology) { + super(topology); this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 6e4662ed4c791..22e8f4989975b 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -19,12 +19,12 @@ public class KStreamSource extends KStreamImpl { String[] topics; - public KStreamSource(String[] topics, KStreamTopology initializer) { - this(topics, null, null, initializer); + public KStreamSource(String[] topics, KStreamTopology topology) { + this(topics, null, null, topology); } - public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology initializer) { - super(initializer); + public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology topology) { + super(topology); this.topics = topics; this.keyDeserializer = keyDeserializer; this.valueDeserializer = valueDeserializer; From 736da1025879e5597e54600a8fe7418f98d42b5c Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 3 Aug 2015 16:14:25 -0700 Subject: [PATCH 116/275] KStream.tranform method for generalized transformation --- .../java/io/confluent/streaming/KStream.java | 6 + .../io/confluent/streaming/Transformer.java | 14 +++ .../streaming/internal/KStreamImpl.java | 9 +- .../streaming/internal/KStreamTransform.java | 41 +++++++ .../internal/KStreamTransformTest.java | 111 ++++++++++++++++++ 5 files changed, 180 insertions(+), 1 deletion(-) create mode 100644 src/main/java/io/confluent/streaming/Transformer.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamTransform.java create mode 100644 src/test/java/io/confluent/streaming/internal/KStreamTransformTest.java diff --git a/src/main/java/io/confluent/streaming/KStream.java b/src/main/java/io/confluent/streaming/KStream.java index 7073f9375eebf..1f90c7f41b93e 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/src/main/java/io/confluent/streaming/KStream.java @@ -124,4 +124,10 @@ public interface KStream { */ void process(Processor processor); + /** + * Transform all elements in this stream by applying a tranformer. + * @param transformer the instance of Transformer + */ + KStream transform(Transformer transformer); + } diff --git a/src/main/java/io/confluent/streaming/Transformer.java b/src/main/java/io/confluent/streaming/Transformer.java new file mode 100644 index 0000000000000..f3bf196853cb2 --- /dev/null +++ b/src/main/java/io/confluent/streaming/Transformer.java @@ -0,0 +1,14 @@ +package io.confluent.streaming; + +/** + * Created by yasuhiro on 6/17/15. + */ +public interface Transformer extends Processor { + + interface Forwarder { + void send(K key, V value, long timestamp); + } + + void forwarder(Forwarder forwarder); + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index ffec849cb768e..af0d4ba670265 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -7,6 +7,7 @@ import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.Predicate; import io.confluent.streaming.Processor; +import io.confluent.streaming.Transformer; import io.confluent.streaming.ValueMapper; import io.confluent.streaming.Window; import org.apache.kafka.common.serialization.Deserializer; @@ -138,10 +139,16 @@ public void close() {} @SuppressWarnings("unchecked") @Override - public void process(final Processor processor) { + public void process(Processor processor) { registerReceiver(new ProcessorNode<>(processor)); } + @SuppressWarnings("unchecked") + @Override + public KStream transform(Transformer transformer) { + return chain(new KStreamTransform<>(transformer, topology)); + } + void registerReceiver(Receiver receiver) { nextReceivers.add(receiver); } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamTransform.java b/src/main/java/io/confluent/streaming/internal/KStreamTransform.java new file mode 100644 index 0000000000000..5cdb44068c748 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamTransform.java @@ -0,0 +1,41 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamTopology; +import io.confluent.streaming.KeyValue; +import io.confluent.streaming.KeyValueMapper; +import io.confluent.streaming.Transformer; + +/** + * Created by yasuhiro on 6/17/15. + */ +class KStreamTransform extends KStreamImpl implements Transformer.Forwarder { + + private final Transformer transformer; + + KStreamTransform(Transformer transformer, KStreamTopology topology) { + super(topology); + this.transformer = transformer; + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + transformer.init(context); + transformer.forwarder(this); + + super.bind(context, KStreamMetadata.unjoinable()); + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + synchronized (this) { + transformer.process((K1) key, (V1) value); + } + } + + @Override + public void send(K key, V value, long timestamp) { + forward(key, value, timestamp); + } +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamTransformTest.java b/src/test/java/io/confluent/streaming/internal/KStreamTransformTest.java new file mode 100644 index 0000000000000..ff6ed8123627e --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/KStreamTransformTest.java @@ -0,0 +1,111 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamTopology; +import io.confluent.streaming.Predicate; +import io.confluent.streaming.Transformer; +import io.confluent.streaming.testutil.MockKStreamContext; +import io.confluent.streaming.testutil.MockKStreamTopology; +import io.confluent.streaming.testutil.TestProcessor; +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; + +public class KStreamTransformTest { + + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + + @Test + public void testTransform() { + final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; + + KStreamTopology topology = new MockKStreamTopology(); + KStreamSource stream; + TestProcessor processor; + Transformer transformer; + + processor = new TestProcessor<>(); + transformer = new Transformer() { + KStreamContext context; + Forwarder forwarder; + public void init(KStreamContext context) { + this.context = context; + } + public void forwarder(Forwarder forwarder) { + this.forwarder = forwarder; + } + public void process(String key, String value) { + forwarder.send(Integer.parseInt(value), 0, 0L); + } + public void punctuate(long timestamp) {} + public void close() {} + }; + + stream = new KStreamSource<>(null, topology); + + stream.transform(transformer).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(null, Integer.toString(expectedKeys[i]), 0L); + } + + assertEquals(expectedKeys.length, processor.processed.size()); + for (int i = 0; i < expectedKeys.length; i++) { + assertEquals(expectedKeys[i]+ ":" + 0, processor.processed.get(i)); + } + } + + @Test + public void testTransformEmitOnPuncutation() { + final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; + + KStreamTopology topology = new MockKStreamTopology(); + KStreamSource stream; + TestProcessor processor; + Transformer transformer; + + processor = new TestProcessor<>(); + transformer = new Transformer() { + KStreamContext context; + Forwarder forwarder; + Integer currentKey; + public void init(KStreamContext context) { + this.context = context; + } + public void forwarder(Forwarder forwarder) { + this.forwarder = forwarder; + } + public void process(Integer key, String value) { + currentKey = Integer.parseInt(value); + } + public void punctuate(long timestamp) { + forwarder.send(currentKey, 0, 0L); + } + public void close() {} + }; + + stream = new KStreamSource<>(null, topology); + + stream.transform(transformer).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(null, Integer.toString(expectedKeys[i]), 0L); + if (i % 3 == 2) transformer.punctuate(0L); + } + + final int[] expected = new int[] { 3, 6 }; + assertEquals(2, processor.processed.size()); + for (int i = 0; i < 2; i++) { + assertEquals(expected[i] + ":" + 0, processor.processed.get(i)); + } + } + +} From 49ed1d7a7bfd7a5a6ee32a57338e06ed6451516a Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 09:49:43 -0700 Subject: [PATCH 117/275] new api model --- kafka-work.ipr | 209 ++++ kafka-work.iws | 1026 +++++++++++++++++ .../apache/kafka/clients/ClientRequest.class | Bin 0 -> 2117 bytes .../apache/kafka/clients/ClientResponse.class | Bin 0 -> 1880 bytes .../apache/kafka/clients/ClientUtils.class | Bin 0 -> 3513 bytes ...ConnectionStates$NodeConnectionState.class | Bin 0 -> 1090 bytes .../clients/ClusterConnectionStates.class | Bin 0 -> 3360 bytes .../kafka/clients/CommonClientConfigs.class | Bin 0 -> 3528 bytes .../kafka/clients/ConnectionState.class | Bin 0 -> 1123 bytes .../kafka/clients/InFlightRequests.class | Bin 0 -> 3688 bytes .../apache/kafka/clients/KafkaClient.class | Bin 0 -> 1068 bytes .../org/apache/kafka/clients/Metadata.class | Bin 0 -> 4076 bytes .../apache/kafka/clients/NetworkClient.class | Bin 0 -> 15727 bytes .../clients/RequestCompletionHandler.class | Bin 0 -> 222 bytes .../kafka/clients/consumer/CommitType.class | Bin 0 -> 1077 bytes .../kafka/clients/consumer/Consumer.class | Bin 0 -> 2045 bytes .../consumer/ConsumerCommitCallback.class | Bin 0 -> 342 bytes .../clients/consumer/ConsumerConfig.class | Bin 0 -> 12655 bytes .../consumer/ConsumerRebalanceCallback.class | Bin 0 -> 444 bytes .../clients/consumer/ConsumerRecord.class | Bin 0 -> 2181 bytes ...nsumerRecords$ConcatenatedIterable$1.class | Bin 0 -> 2079 bytes ...ConsumerRecords$ConcatenatedIterable.class | Bin 0 -> 1709 bytes .../clients/consumer/ConsumerRecords.class | Bin 0 -> 4221 bytes .../consumer/ConsumerWakeupException.class | Bin 0 -> 448 bytes .../clients/consumer/KafkaConsumer$1.class | Bin 0 -> 1403 bytes .../clients/consumer/KafkaConsumer$2.class | Bin 0 -> 1802 bytes .../clients/consumer/KafkaConsumer.class | Bin 0 -> 25520 bytes .../kafka/clients/consumer/MockConsumer.class | Bin 0 -> 10225 bytes .../NoOffsetForPartitionException.class | Bin 0 -> 530 bytes .../consumer/OffsetResetStrategy.class | Bin 0 -> 1192 bytes ...lient$RequestFutureCompletionHandler.class | Bin 0 -> 1083 bytes .../internals/ConsumerNetworkClient.class | Bin 0 -> 9748 bytes .../consumer/internals/Coordinator$1.class | Bin 0 -> 1724 bytes .../consumer/internals/Coordinator$2.class | Bin 0 -> 1910 bytes .../Coordinator$CoordinatorMetrics$1.class | Bin 0 -> 1719 bytes .../Coordinator$CoordinatorMetrics$2.class | Bin 0 -> 1824 bytes .../Coordinator$CoordinatorMetrics.class | Bin 0 -> 4037 bytes ...ordinator$CoordinatorResponseHandler.class | Bin 0 -> 4181 bytes ...ordinator$HeartbeatCompletionHandler.class | Bin 0 -> 5438 bytes .../Coordinator$HeartbeatTask$1.class | Bin 0 -> 2514 bytes .../internals/Coordinator$HeartbeatTask.class | Bin 0 -> 3425 bytes ...Coordinator$JoinGroupResponseHandler.class | Bin 0 -> 6376 bytes ...rdinator$OffsetCommitResponseHandler.class | Bin 0 -> 6253 bytes ...ordinator$OffsetFetchResponseHandler.class | Bin 0 -> 6119 bytes .../Coordinator$RebalanceCallback.class | Bin 0 -> 464 bytes .../consumer/internals/Coordinator.class | Bin 0 -> 19663 bytes .../consumer/internals/DelayedTask.class | Bin 0 -> 168 bytes .../internals/DelayedTaskQueue$Entry.class | Bin 0 -> 1181 bytes .../consumer/internals/DelayedTaskQueue.class | Bin 0 -> 2236 bytes .../consumer/internals/Fetcher$1.class | Bin 0 -> 1945 bytes .../consumer/internals/Fetcher$2.class | Bin 0 -> 2133 bytes .../Fetcher$FetchManagerMetrics.class | Bin 0 -> 4008 bytes .../internals/Fetcher$PartitionRecords.class | Bin 0 -> 1191 bytes .../clients/consumer/internals/Fetcher.class | Bin 0 -> 23886 bytes .../consumer/internals/Heartbeat.class | Bin 0 -> 1599 bytes .../NoAvailableBrokersException.class | Bin 0 -> 497 bytes .../NoOpConsumerRebalanceCallback.class | Bin 0 -> 1179 bytes .../consumer/internals/RequestFuture$1.class | Bin 0 -> 1936 bytes .../consumer/internals/RequestFuture.class | Bin 0 -> 5732 bytes .../internals/RequestFutureAdapter.class | Bin 0 -> 1373 bytes .../internals/RequestFutureListener.class | Bin 0 -> 345 bytes .../internals/SendFailedException.class | Bin 0 -> 561 bytes .../internals/StaleMetadataException.class | Bin 0 -> 482 bytes ...ubscriptionState$TopicPartitionState.class | Bin 0 -> 4139 bytes .../internals/SubscriptionState.class | Bin 0 -> 10200 bytes .../producer/BufferExhaustedException.class | Bin 0 -> 515 bytes .../kafka/clients/producer/Callback.class | Bin 0 -> 231 bytes .../KafkaProducer$FutureFailure.class | Bin 0 -> 1804 bytes .../clients/producer/KafkaProducer.class | Bin 0 -> 21339 bytes .../producer/MockProducer$Completion.class | Bin 0 -> 1819 bytes .../kafka/clients/producer/MockProducer.class | Bin 0 -> 9644 bytes .../kafka/clients/producer/Partitioner.class | Bin 0 -> 320 bytes .../kafka/clients/producer/Producer.class | Bin 0 -> 1213 bytes .../clients/producer/ProducerConfig.class | Bin 0 -> 16285 bytes .../clients/producer/ProducerRecord.class | Bin 0 -> 3300 bytes .../clients/producer/RecordMetadata.class | Bin 0 -> 1112 bytes .../producer/internals/BufferPool.class | Bin 0 -> 6445 bytes .../internals/DefaultPartitioner.class | Bin 0 -> 2282 bytes .../internals/ErrorLoggingCallback.class | Bin 0 -> 1986 bytes .../internals/FutureRecordMetadata.class | Bin 0 -> 3048 bytes .../internals/ProduceRequestResult.class | Bin 0 -> 1684 bytes .../internals/RecordAccumulator$1.class | Bin 0 -> 1275 bytes .../internals/RecordAccumulator$2.class | Bin 0 -> 1280 bytes .../internals/RecordAccumulator$3.class | Bin 0 -> 1284 bytes ...dAccumulator$IncompleteRecordBatches.class | Bin 0 -> 1849 bytes .../RecordAccumulator$ReadyCheckResult.class | Bin 0 -> 915 bytes ...RecordAccumulator$RecordAppendResult.class | Bin 0 -> 824 bytes .../internals/RecordAccumulator.class | Bin 0 -> 16641 bytes .../internals/RecordBatch$Thunk.class | Bin 0 -> 788 bytes .../producer/internals/RecordBatch.class | Bin 0 -> 4607 bytes .../clients/producer/internals/Sender$1.class | Bin 0 -> 1365 bytes .../internals/Sender$SenderMetrics$1.class | Bin 0 -> 1494 bytes .../internals/Sender$SenderMetrics$2.class | Bin 0 -> 1501 bytes .../internals/Sender$SenderMetrics.class | Bin 0 -> 9145 bytes .../clients/producer/internals/Sender.class | Bin 0 -> 14836 bytes .../org/apache/kafka/common/Cluster.class | Bin 0 -> 7232 bytes .../apache/kafka/common/Configurable.class | Bin 0 -> 232 bytes .../apache/kafka/common/KafkaException.class | Bin 0 -> 821 bytes .../org/apache/kafka/common/Metric.class | Bin 0 -> 201 bytes .../org/apache/kafka/common/MetricName.class | Bin 0 -> 3956 bytes .../org/apache/kafka/common/Node.class | Bin 0 -> 1881 bytes .../apache/kafka/common/PartitionInfo.class | Bin 0 -> 2326 bytes .../apache/kafka/common/TopicPartition.class | Bin 0 -> 1531 bytes .../kafka/common/config/AbstractConfig.class | Bin 0 -> 8093 bytes .../kafka/common/config/ConfigDef$1.class | Bin 0 -> 1667 bytes .../kafka/common/config/ConfigDef$2.class | Bin 0 -> 1082 bytes .../common/config/ConfigDef$ConfigKey.class | Bin 0 -> 1741 bytes .../common/config/ConfigDef$Importance.class | Bin 0 -> 1255 bytes .../kafka/common/config/ConfigDef$Range.class | Bin 0 -> 2019 bytes .../kafka/common/config/ConfigDef$Type.class | Bin 0 -> 1460 bytes .../common/config/ConfigDef$ValidString.class | Bin 0 -> 1935 bytes .../common/config/ConfigDef$Validator.class | Bin 0 -> 293 bytes .../kafka/common/config/ConfigDef.class | Bin 0 -> 10278 bytes .../kafka/common/config/ConfigException.class | Bin 0 -> 1261 bytes .../kafka/common/errors/ApiException.class | Bin 0 -> 944 bytes ...umerCoordinatorNotAvailableException.class | Bin 0 -> 936 bytes .../errors/CorruptRecordException.class | Bin 0 -> 946 bytes .../common/errors/DisconnectException.class | Bin 0 -> 873 bytes .../errors/IllegalGenerationException.class | Bin 0 -> 894 bytes .../common/errors/InterruptException.class | Bin 0 -> 1055 bytes .../errors/InvalidMetadataException.class | Bin 0 -> 888 bytes .../errors/InvalidRequiredAcksException.class | Bin 0 -> 526 bytes .../common/errors/InvalidTopicException.class | Bin 0 -> 873 bytes .../errors/LeaderNotAvailableException.class | Bin 0 -> 535 bytes .../common/errors/NetworkException.class | Bin 0 -> 870 bytes .../NotCoordinatorForConsumerException.class | Bin 0 -> 918 bytes ...otEnoughReplicasAfterAppendException.class | Bin 0 -> 559 bytes .../errors/NotEnoughReplicasException.class | Bin 0 -> 894 bytes .../NotLeaderForPartitionException.class | Bin 0 -> 912 bytes .../OffsetLoadInProgressException.class | Bin 0 -> 903 bytes .../errors/OffsetMetadataTooLarge.class | Bin 0 -> 876 bytes .../errors/OffsetOutOfRangeException.class | Bin 0 -> 891 bytes .../errors/RecordBatchTooLargeException.class | Bin 0 -> 894 bytes .../errors/RecordTooLargeException.class | Bin 0 -> 879 bytes .../common/errors/RetriableException.class | Bin 0 -> 864 bytes .../errors/SerializationException.class | Bin 0 -> 974 bytes .../common/errors/TimeoutException.class | Bin 0 -> 864 bytes .../errors/UnknownConsumerIdException.class | Bin 0 -> 894 bytes .../errors/UnknownServerException.class | Bin 0 -> 876 bytes .../UnknownTopicOrPartitionException.class | Bin 0 -> 922 bytes .../CompoundStat$NamedMeasurable.class | Bin 0 -> 932 bytes .../kafka/common/metrics/CompoundStat.class | Bin 0 -> 438 bytes .../metrics/JmxReporter$KafkaMbean.class | Bin 0 -> 5120 bytes .../kafka/common/metrics/JmxReporter.class | Bin 0 -> 5932 bytes .../kafka/common/metrics/KafkaMetric.class | Bin 0 -> 1940 bytes .../kafka/common/metrics/Measurable.class | Bin 0 -> 204 bytes .../kafka/common/metrics/MeasurableStat.class | Bin 0 -> 235 bytes .../kafka/common/metrics/MetricConfig.class | Bin 0 -> 2153 bytes .../apache/kafka/common/metrics/Metrics.class | Bin 0 -> 6409 bytes .../common/metrics/MetricsReporter.class | Bin 0 -> 409 bytes .../apache/kafka/common/metrics/Quota.class | Bin 0 -> 959 bytes .../metrics/QuotaViolationException.class | Bin 0 -> 502 bytes .../apache/kafka/common/metrics/Sensor.class | Bin 0 -> 6638 bytes .../apache/kafka/common/metrics/Stat.class | Bin 0 -> 192 bytes .../kafka/common/metrics/stats/Avg.class | Bin 0 -> 1615 bytes .../kafka/common/metrics/stats/Count.class | Bin 0 -> 1532 bytes .../metrics/stats/Histogram$BinScheme.class | Bin 0 -> 312 bytes .../stats/Histogram$ConstantBinScheme.class | Bin 0 -> 1333 bytes .../stats/Histogram$LinearBinScheme.class | Bin 0 -> 1359 bytes .../common/metrics/stats/Histogram.class | Bin 0 -> 2442 bytes .../kafka/common/metrics/stats/Max.class | Bin 0 -> 1601 bytes .../kafka/common/metrics/stats/Min.class | Bin 0 -> 1616 bytes .../common/metrics/stats/Percentile.class | Bin 0 -> 684 bytes .../common/metrics/stats/Percentiles$1.class | Bin 0 -> 1033 bytes .../stats/Percentiles$BucketSizing.class | Bin 0 -> 1292 bytes .../stats/Percentiles$HistogramSample.class | Bin 0 -> 1619 bytes .../common/metrics/stats/Percentiles.class | Bin 0 -> 6038 bytes .../kafka/common/metrics/stats/Rate$1.class | Bin 0 -> 978 bytes .../metrics/stats/Rate$SampledTotal.class | Bin 0 -> 1635 bytes .../kafka/common/metrics/stats/Rate.class | Bin 0 -> 2865 bytes .../metrics/stats/SampledStat$Sample.class | Bin 0 -> 1134 bytes .../common/metrics/stats/SampledStat.class | Bin 0 -> 3349 bytes .../kafka/common/metrics/stats/Total.class | Bin 0 -> 888 bytes .../common/network/ByteBufferReceive.class | Bin 0 -> 1322 bytes .../kafka/common/network/ByteBufferSend.class | Bin 0 -> 1585 bytes .../network/InvalidReceiveException.class | Bin 0 -> 618 bytes .../kafka/common/network/MultiSend.class | Bin 0 -> 3217 bytes .../kafka/common/network/NetworkReceive.class | Bin 0 -> 3055 bytes .../kafka/common/network/NetworkSend.class | Bin 0 -> 1061 bytes .../apache/kafka/common/network/Receive.class | Bin 0 -> 306 bytes .../kafka/common/network/Selectable.class | Bin 0 -> 832 bytes .../network/Selector$SelectorMetrics$1.class | Bin 0 -> 1346 bytes .../network/Selector$SelectorMetrics.class | Bin 0 -> 8067 bytes .../network/Selector$Transmissions.class | Bin 0 -> 1034 bytes .../kafka/common/network/Selector.class | Bin 0 -> 15681 bytes .../apache/kafka/common/network/Send.class | Bin 0 -> 325 bytes .../kafka/common/protocol/ApiKeys.class | Bin 0 -> 2633 bytes .../apache/kafka/common/protocol/Errors.class | Bin 0 -> 8391 bytes .../kafka/common/protocol/ProtoUtils.class | Bin 0 -> 2456 bytes .../kafka/common/protocol/Protocol.class | Bin 0 -> 12722 bytes .../common/protocol/SecurityProtocol.class | Bin 0 -> 2453 bytes .../kafka/common/protocol/types/ArrayOf.class | Bin 0 -> 2464 bytes .../kafka/common/protocol/types/Field.class | Bin 0 -> 2201 bytes .../kafka/common/protocol/types/Schema.class | Bin 0 -> 5250 bytes .../protocol/types/SchemaException.class | Bin 0 -> 498 bytes .../kafka/common/protocol/types/Struct.class | Bin 0 -> 8367 bytes .../kafka/common/protocol/types/Type$1.class | Bin 0 -> 1704 bytes .../kafka/common/protocol/types/Type$2.class | Bin 0 -> 1720 bytes .../kafka/common/protocol/types/Type$3.class | Bin 0 -> 1723 bytes .../kafka/common/protocol/types/Type$4.class | Bin 0 -> 1714 bytes .../kafka/common/protocol/types/Type$5.class | Bin 0 -> 2113 bytes .../kafka/common/protocol/types/Type$6.class | Bin 0 -> 2015 bytes .../kafka/common/protocol/types/Type.class | Bin 0 -> 1229 bytes .../common/record/ByteBufferInputStream.class | Bin 0 -> 951 bytes .../record/ByteBufferOutputStream.class | Bin 0 -> 1474 bytes .../kafka/common/record/CompressionType.class | Bin 0 -> 2310 bytes .../kafka/common/record/Compressor$1.class | Bin 0 -> 884 bytes .../kafka/common/record/Compressor.class | Bin 0 -> 8487 bytes .../record/InvalidRecordException.class | Bin 0 -> 485 bytes .../record/KafkaLZ4BlockInputStream.class | Bin 0 -> 5162 bytes .../record/KafkaLZ4BlockOutputStream$BD.class | Bin 0 -> 1605 bytes .../KafkaLZ4BlockOutputStream$FLG.class | Bin 0 -> 2914 bytes .../record/KafkaLZ4BlockOutputStream.class | Bin 0 -> 4572 bytes .../apache/kafka/common/record/LogEntry.class | Bin 0 -> 1218 bytes .../MemoryRecords$RecordsIterator.class | Bin 0 -> 3595 bytes .../kafka/common/record/MemoryRecords.class | Bin 0 -> 5566 bytes .../apache/kafka/common/record/Record.class | Bin 0 -> 7678 bytes .../apache/kafka/common/record/Records.class | Bin 0 -> 524 bytes .../common/requests/AbstractRequest$1.class | Bin 0 -> 1164 bytes .../common/requests/AbstractRequest.class | Bin 0 -> 2628 bytes .../requests/AbstractRequestResponse.class | Bin 0 -> 1424 bytes .../requests/ConsumerMetadataRequest.class | Bin 0 -> 3355 bytes .../requests/ConsumerMetadataResponse.class | Bin 0 -> 3068 bytes .../requests/FetchRequest$PartitionData.class | Bin 0 -> 567 bytes .../kafka/common/requests/FetchRequest.class | Bin 0 -> 8557 bytes .../FetchResponse$PartitionData.class | Bin 0 -> 674 bytes .../kafka/common/requests/FetchResponse.class | Bin 0 -> 6236 bytes .../common/requests/HeartbeatRequest.class | Bin 0 -> 3738 bytes .../common/requests/HeartbeatResponse.class | Bin 0 -> 1934 bytes .../common/requests/JoinGroupRequest.class | Bin 0 -> 4986 bytes .../common/requests/JoinGroupResponse.class | Bin 0 -> 5149 bytes .../ListOffsetRequest$PartitionData.class | Bin 0 -> 595 bytes .../common/requests/ListOffsetRequest.class | Bin 0 -> 8121 bytes .../ListOffsetResponse$PartitionData.class | Bin 0 -> 772 bytes .../common/requests/ListOffsetResponse.class | Bin 0 -> 6013 bytes .../common/requests/MetadataRequest.class | Bin 0 -> 4481 bytes .../common/requests/MetadataResponse.class | Bin 0 -> 7925 bytes .../OffsetCommitRequest$PartitionData.class | Bin 0 -> 916 bytes .../common/requests/OffsetCommitRequest.class | Bin 0 -> 10053 bytes .../requests/OffsetCommitResponse.class | Bin 0 -> 5093 bytes .../common/requests/OffsetFetchRequest.class | Bin 0 -> 6668 bytes .../OffsetFetchResponse$PartitionData.class | Bin 0 -> 932 bytes .../common/requests/OffsetFetchResponse.class | Bin 0 -> 6111 bytes .../common/requests/ProduceRequest.class | Bin 0 -> 7363 bytes .../ProduceResponse$PartitionResponse.class | Bin 0 -> 1077 bytes .../common/requests/ProduceResponse.class | Bin 0 -> 6010 bytes .../kafka/common/requests/RequestHeader.class | Bin 0 -> 3097 bytes .../kafka/common/requests/RequestSend.class | Bin 0 -> 1900 bytes .../common/requests/ResponseHeader.class | Bin 0 -> 1878 bytes .../kafka/common/requests/ResponseSend.class | Bin 0 -> 1762 bytes .../serialization/ByteArrayDeserializer.class | Bin 0 -> 1144 bytes .../serialization/ByteArraySerializer.class | Bin 0 -> 1138 bytes .../common/serialization/Deserializer.class | Bin 0 -> 466 bytes .../serialization/IntegerDeserializer.class | Bin 0 -> 1619 bytes .../serialization/IntegerSerializer.class | Bin 0 -> 1355 bytes .../common/serialization/Serializer.class | Bin 0 -> 460 bytes .../serialization/StringDeserializer.class | Bin 0 -> 2147 bytes .../serialization/StringSerializer.class | Bin 0 -> 2131 bytes .../common/utils/AbstractIterator$1.class | Bin 0 -> 897 bytes .../common/utils/AbstractIterator$State.class | Bin 0 -> 1323 bytes .../kafka/common/utils/AbstractIterator.class | Bin 0 -> 2342 bytes .../kafka/common/utils/CollectionUtils.class | Bin 0 -> 2901 bytes .../kafka/common/utils/CopyOnWriteMap.class | Bin 0 -> 4034 bytes .../org/apache/kafka/common/utils/Crc32.class | Bin 0 -> 28501 bytes .../kafka/common/utils/KafkaThread$1.class | Bin 0 -> 1388 bytes .../kafka/common/utils/KafkaThread.class | Bin 0 -> 1256 bytes .../kafka/common/utils/SystemTime.class | Bin 0 -> 804 bytes .../org/apache/kafka/common/utils/Time.class | Bin 0 -> 193 bytes .../apache/kafka/common/utils/Utils$1.class | Bin 0 -> 1182 bytes .../org/apache/kafka/common/utils/Utils.class | Bin 0 -> 13100 bytes .../kafka/streaming/KafkaStreaming.class | Bin 0 -> 2626 bytes .../kafka/streaming/StreamingConfig.class | Bin 0 -> 6417 bytes .../streaming/examples/KStreamJob$1.class | Bin 0 -> 1009 bytes .../streaming/examples/KStreamJob$2.class | Bin 0 -> 1423 bytes .../streaming/examples/KStreamJob$3.class | Bin 0 -> 1076 bytes .../streaming/examples/KStreamJob$4.class | Bin 0 -> 1009 bytes .../kafka/streaming/examples/KStreamJob.class | Bin 0 -> 3323 bytes .../streaming/examples/ProcessorJob$1.class | Bin 0 -> 256 bytes .../ProcessorJob$MyProcessorDef$1.class | Bin 0 -> 3821 bytes .../ProcessorJob$MyProcessorDef.class | Bin 0 -> 1180 bytes .../streaming/examples/ProcessorJob.class | Bin 0 -> 2995 bytes .../WallclockTimestampExtractor.class | Bin 0 -> 723 bytes .../kafka/streaming/kstream/KStream.class | Bin 0 -> 3250 bytes .../streaming/kstream/KStreamBuilder.class | Bin 0 -> 2434 bytes .../streaming/kstream/KStreamWindowed.class | Bin 0 -> 727 bytes .../kafka/streaming/kstream/KeyValue.class | Bin 0 -> 1045 bytes .../streaming/kstream/KeyValueMapper.class | Bin 0 -> 344 bytes .../kafka/streaming/kstream/Predicate.class | Bin 0 -> 295 bytes .../SlidingWindowDef$SlidingWindow$1.class | Bin 0 -> 2184 bytes ...dowDef$SlidingWindow$RestoreFuncImpl.class | Bin 0 -> 2396 bytes .../SlidingWindowDef$SlidingWindow.class | Bin 0 -> 9359 bytes .../streaming/kstream/SlidingWindowDef.class | Bin 0 -> 3388 bytes .../kafka/streaming/kstream/ValueJoiner.class | Bin 0 -> 342 bytes .../kafka/streaming/kstream/ValueMapper.class | Bin 0 -> 301 bytes .../kafka/streaming/kstream/Window.class | Bin 0 -> 627 bytes .../kafka/streaming/kstream/WindowDef.class | Bin 0 -> 386 bytes .../kstream/internals/FilteredIterator.class | Bin 0 -> 1639 bytes .../kstream/internals/KStreamBranch$1.class | Bin 0 -> 277 bytes ...KStreamBranch$KStreamBranchProcessor.class | Bin 0 -> 2147 bytes .../kstream/internals/KStreamBranch.class | Bin 0 -> 1576 bytes .../kstream/internals/KStreamFilter$1.class | Bin 0 -> 277 bytes ...KStreamFilter$KStreamFilterProcessor.class | Bin 0 -> 2137 bytes .../kstream/internals/KStreamFilter.class | Bin 0 -> 1844 bytes .../kstream/internals/KStreamFlatMap$1.class | Bin 0 -> 280 bytes ...treamFlatMap$KStreamFlatMapProcessor.class | Bin 0 -> 2516 bytes .../kstream/internals/KStreamFlatMap.class | Bin 0 -> 1894 bytes .../internals/KStreamFlatMapValues$1.class | Bin 0 -> 298 bytes ...Values$KStreamFlatMapValuesProcessor.class | Bin 0 -> 2498 bytes .../internals/KStreamFlatMapValues.class | Bin 0 -> 1812 bytes .../kstream/internals/KStreamImpl.class | Bin 0 -> 11028 bytes .../kstream/internals/KStreamJoin$1.class | Bin 0 -> 1187 bytes .../internals/KStreamJoin$Finder.class | Bin 0 -> 1067 bytes .../KStreamJoin$KStreamJoinProcessor$1.class | Bin 0 -> 1689 bytes .../KStreamJoin$KStreamJoinProcessor.class | Bin 0 -> 3227 bytes .../kstream/internals/KStreamJoin.class | Bin 0 -> 2375 bytes .../kstream/internals/KStreamMap$1.class | Bin 0 -> 268 bytes .../KStreamMap$KStreamMapProcessor.class | Bin 0 -> 2222 bytes .../kstream/internals/KStreamMap.class | Bin 0 -> 1806 bytes .../internals/KStreamMapValues$1.class | Bin 0 -> 286 bytes ...KStreamMapValues$KStreamMapProcessor.class | Bin 0 -> 2089 bytes .../kstream/internals/KStreamMapValues.class | Bin 0 -> 1710 bytes ...sThrough$KStreamPassThroughProcessor.class | Bin 0 -> 1842 bytes .../internals/KStreamPassThrough.class | Bin 0 -> 1069 bytes .../kstream/internals/KStreamProcessor.class | Bin 0 -> 1205 bytes .../kstream/internals/KStreamWindow$1.class | Bin 0 -> 277 bytes ...KStreamWindow$KStreamWindowProcessor.class | Bin 0 -> 2752 bytes .../kstream/internals/KStreamWindow.class | Bin 0 -> 1855 bytes .../internals/KStreamWindowedImpl.class | Bin 0 -> 4116 bytes .../internals/WindowSupport$Value.class | Bin 0 -> 1484 bytes .../internals/WindowSupport$ValueList.class | Bin 0 -> 3042 bytes .../WindowSupport$ValueListIterator.class | Bin 0 -> 2126 bytes .../kstream/internals/WindowSupport.class | Bin 0 -> 2341 bytes .../kafka/streaming/processor/Processor.class | Bin 0 -> 424 bytes .../processor/ProcessorContext.class | Bin 0 -> 1281 bytes .../streaming/processor/ProcessorDef.class | Bin 0 -> 214 bytes .../streaming/processor/RestoreFunc.class | Bin 0 -> 166 bytes .../streaming/processor/StateStore.class | Bin 0 -> 241 bytes .../processor/TimestampExtractor.class | Bin 0 -> 232 bytes .../processor/TopologyBuilder$1.class | Bin 0 -> 267 bytes .../TopologyBuilder$NodeFactory.class | Bin 0 -> 346 bytes ...TopologyBuilder$ProcessorNodeFactory.class | Bin 0 -> 1633 bytes .../TopologyBuilder$SinkNodeFactory.class | Bin 0 -> 2230 bytes .../TopologyBuilder$SourceNodeFactory.class | Bin 0 -> 2134 bytes .../streaming/processor/TopologyBuilder.class | Bin 0 -> 8232 bytes .../internals/MinTimestampTracker.class | Bin 0 -> 2266 bytes .../internals/PartitionGroup$1.class | Bin 0 -> 1527 bytes .../processor/internals/PartitionGroup.class | Bin 0 -> 4427 bytes .../internals/ProcessorContextImpl.class | Bin 0 -> 10324 bytes .../processor/internals/ProcessorNode.class | Bin 0 -> 2779 bytes .../internals/ProcessorStateManager.class | Bin 0 -> 7712 bytes .../internals/ProcessorTopology.class | Bin 0 -> 3668 bytes .../internals/PunctuationQueue.class | Bin 0 -> 2576 bytes .../internals/PunctuationSchedule.class | Bin 0 -> 1275 bytes .../internals/RecordCollector$1.class | Bin 0 -> 1801 bytes .../processor/internals/RecordCollector.class | Bin 0 -> 3402 bytes .../processor/internals/RecordQueue.class | Bin 0 -> 4189 bytes .../processor/internals/SinkNode.class | Bin 0 -> 3306 bytes .../processor/internals/SourceNode.class | Bin 0 -> 2590 bytes .../processor/internals/Stamped.class | Bin 0 -> 978 bytes .../processor/internals/StampedRecord.class | Bin 0 -> 1854 bytes .../processor/internals/StreamTask.class | Bin 0 -> 10527 bytes .../processor/internals/StreamThread$1.class | Bin 0 -> 1903 bytes .../StreamThread$KafkaStreamingMetrics.class | Bin 0 -> 2462 bytes .../processor/internals/StreamThread.class | Bin 0 -> 12434 bytes .../internals/TimestampTracker.class | Bin 0 -> 511 bytes .../apache/kafka/streaming/state/Entry.class | Bin 0 -> 1283 bytes ...tore$MemoryStore$MemoryStoreIterator.class | Bin 0 -> 2150 bytes .../InMemoryKeyValueStore$MemoryStore.class | Bin 0 -> 4032 bytes .../state/InMemoryKeyValueStore.class | Bin 0 -> 1518 bytes .../streaming/state/KeyValueIterator.class | Bin 0 -> 391 bytes .../kafka/streaming/state/KeyValueStore.class | Bin 0 -> 982 bytes .../state/MeteredKeyValueStore$1.class | Bin 0 -> 1824 bytes ...eyValueStore$MeteredKeyValueIterator.class | Bin 0 -> 2680 bytes .../state/MeteredKeyValueStore.class | Bin 0 -> 11596 bytes .../streaming/state/OffsetCheckpoint.class | Bin 0 -> 5907 bytes .../state/RocksDBKeyValueStore$1.class | Bin 0 -> 274 bytes ...RocksDBStore$LexicographicComparator.class | Bin 0 -> 1441 bytes ...re$RocksDBStore$RocksDBRangeIterator.class | Bin 0 -> 1614 bytes ...ueStore$RocksDBStore$RocksDbIterator.class | Bin 0 -> 2160 bytes .../RocksDBKeyValueStore$RocksDBStore.class | Bin 0 -> 7803 bytes .../state/RocksDBKeyValueStore.class | Bin 0 -> 1413 bytes out/test/clients/log4j.properties | 21 + .../kafka/clients/ClientUtilsTest.class | Bin 0 -> 1224 bytes .../apache/kafka/clients/MetadataTest$1.class | Bin 0 -> 1619 bytes .../apache/kafka/clients/MetadataTest.class | Bin 0 -> 4420 bytes .../clients/MockClient$FutureResponse.class | Bin 0 -> 749 bytes .../org/apache/kafka/clients/MockClient.class | Bin 0 -> 6787 bytes .../kafka/clients/NetworkClientTest$1.class | Bin 0 -> 249 bytes ...etworkClientTest$TestCallbackHandler.class | Bin 0 -> 1024 bytes .../kafka/clients/NetworkClientTest.class | Bin 0 -> 7304 bytes .../clients/consumer/KafkaConsumerTest.class | Bin 0 -> 2103 bytes .../clients/consumer/MockConsumerTest.class | Bin 0 -> 2864 bytes .../ConsumerNetworkClientTest$1.class | Bin 0 -> 311 bytes ...merNetworkClientTest$TestDelayedTask.class | Bin 0 -> 1076 bytes .../internals/ConsumerNetworkClientTest.class | Bin 0 -> 5664 bytes .../internals/CoordinatorTest$1.class | Bin 0 -> 1521 bytes .../CoordinatorTest$MockCommitCallback.class | Bin 0 -> 1394 bytes ...oordinatorTest$MockRebalanceCallback.class | Bin 0 -> 1662 bytes .../consumer/internals/CoordinatorTest.class | Bin 0 -> 17373 bytes .../internals/DelayedTaskQueueTest$1.class | Bin 0 -> 296 bytes .../DelayedTaskQueueTest$TestTask.class | Bin 0 -> 1418 bytes .../internals/DelayedTaskQueueTest.class | Bin 0 -> 2833 bytes .../consumer/internals/FetcherTest.class | Bin 0 -> 9942 bytes .../consumer/internals/HeartbeatTest.class | Bin 0 -> 1929 bytes .../internals/RequestFutureTest$1.class | Bin 0 -> 1784 bytes .../internals/RequestFutureTest$2.class | Bin 0 -> 1784 bytes .../internals/RequestFutureTest.class | Bin 0 -> 2160 bytes .../internals/SubscriptionStateTest.class | Bin 0 -> 4644 bytes .../clients/producer/KafkaProducerTest.class | Bin 0 -> 2598 bytes .../clients/producer/MockProducerTest.class | Bin 0 -> 5963 bytes .../clients/producer/ProducerRecordTest.class | Bin 0 -> 1691 bytes .../clients/producer/RecordSendTest$1.class | Bin 0 -> 1654 bytes .../clients/producer/RecordSendTest.class | Bin 0 -> 3393 bytes .../producer/internals/BufferPoolTest$1.class | Bin 0 -> 1569 bytes .../producer/internals/BufferPoolTest$2.class | Bin 0 -> 1589 bytes .../BufferPoolTest$StressTestThread.class | Bin 0 -> 1690 bytes .../producer/internals/BufferPoolTest.class | Bin 0 -> 6434 bytes .../internals/DefaultPartitionerTest.class | Bin 0 -> 2835 bytes .../internals/RecordAccumulatorTest$1.class | Bin 0 -> 1883 bytes .../RecordAccumulatorTest$1TestCallback.class | Bin 0 -> 1589 bytes .../internals/RecordAccumulatorTest.class | Bin 0 -> 13392 bytes .../producer/internals/SenderTest.class | Bin 0 -> 7552 bytes .../AbstractConfigTest$TestConfig.class | Bin 0 -> 1651 bytes .../common/config/AbstractConfigTest.class | Bin 0 -> 2464 bytes .../kafka/common/config/ConfigDefTest.class | Bin 0 -> 6953 bytes .../common/metrics/FakeMetricsReporter.class | Bin 0 -> 1214 bytes .../common/metrics/JmxReporterTest.class | Bin 0 -> 1563 bytes .../MetricsTest$ConstantMeasurable.class | Bin 0 -> 762 bytes .../kafka/common/metrics/MetricsTest.class | Bin 0 -> 10894 bytes .../common/metrics/stats/HistogramTest.class | Bin 0 -> 4469 bytes .../network/SelectorTest$EchoServer$1.class | Bin 0 -> 1857 bytes .../network/SelectorTest$EchoServer.class | Bin 0 -> 2168 bytes .../kafka/common/network/SelectorTest.class | Bin 0 -> 9432 bytes .../types/ProtocolSerializationTest.class | Bin 0 -> 5359 bytes .../common/record/MemoryRecordsTest.class | Bin 0 -> 3851 bytes .../kafka/common/record/RecordTest.class | Bin 0 -> 4539 bytes .../common/requests/RequestResponseTest.class | Bin 0 -> 11233 bytes .../SerializationTest$SerDeser.class | Bin 0 -> 1255 bytes .../serialization/SerializationTest.class | Bin 0 -> 4681 bytes .../AbstractIteratorTest$ListIterator.class | Bin 0 -> 1380 bytes .../common/utils/AbstractIteratorTest.class | Bin 0 -> 2011 bytes .../apache/kafka/common/utils/CrcTest.class | Bin 0 -> 1544 bytes .../apache/kafka/common/utils/MockTime.class | Bin 0 -> 900 bytes .../apache/kafka/common/utils/UtilsTest.class | Bin 0 -> 3389 bytes .../org/apache/kafka/test/MetricsBench.class | Bin 0 -> 3354 bytes .../apache/kafka/test/Microbenchmarks$1.class | Bin 0 -> 1745 bytes .../apache/kafka/test/Microbenchmarks$2.class | Bin 0 -> 1550 bytes .../apache/kafka/test/Microbenchmarks$3.class | Bin 0 -> 1772 bytes .../apache/kafka/test/Microbenchmarks$4.class | Bin 0 -> 1592 bytes .../apache/kafka/test/Microbenchmarks$5.class | Bin 0 -> 1451 bytes .../apache/kafka/test/Microbenchmarks.class | Bin 0 -> 5881 bytes .../kafka/test/MockMetricsReporter.class | Bin 0 -> 1503 bytes .../org/apache/kafka/test/MockSelector.class | Bin 0 -> 3363 bytes .../apache/kafka/test/MockSerializer.class | Bin 0 -> 1400 bytes .../org/apache/kafka/test/TestUtils.class | Bin 0 -> 2885 bytes .../internals/FilteredIteratorTest$1.class | Bin 0 -> 1455 bytes .../internals/FilteredIteratorTest$2.class | Bin 0 -> 1457 bytes .../internals/FilteredIteratorTest$3.class | Bin 0 -> 1454 bytes .../internals/FilteredIteratorTest.class | Bin 0 -> 2662 bytes .../internals/KStreamBranchTest$1.class | Bin 0 -> 1298 bytes .../internals/KStreamBranchTest$2.class | Bin 0 -> 1298 bytes .../internals/KStreamBranchTest$3.class | Bin 0 -> 1298 bytes .../kstream/internals/KStreamBranchTest.class | Bin 0 -> 4070 bytes .../internals/KStreamFilterTest$1.class | Bin 0 -> 1273 bytes .../kstream/internals/KStreamFilterTest.class | Bin 0 -> 3799 bytes .../internals/KStreamFlatMapTest$1.class | Bin 0 -> 2145 bytes .../internals/KStreamFlatMapTest.class | Bin 0 -> 3791 bytes .../KStreamFlatMapValuesTest$1.class | Bin 0 -> 1585 bytes .../internals/KStreamFlatMapValuesTest.class | Bin 0 -> 3764 bytes .../kstream/internals/KStreamJoinTest$1.class | Bin 0 -> 1381 bytes .../kstream/internals/KStreamJoinTest$2.class | Bin 0 -> 1299 bytes .../kstream/internals/KStreamJoinTest$3.class | Bin 0 -> 1336 bytes .../kstream/internals/KStreamJoinTest$4.class | Bin 0 -> 1639 bytes .../kstream/internals/KStreamJoinTest$5.class | Bin 0 -> 1795 bytes .../kstream/internals/KStreamJoinTest.class | Bin 0 -> 6152 bytes .../kstream/internals/KStreamMapTest$1.class | Bin 0 -> 1648 bytes .../kstream/internals/KStreamMapTest.class | Bin 0 -> 3706 bytes .../internals/KStreamMapValuesTest$1.class | Bin 0 -> 1285 bytes .../internals/KStreamMapValuesTest.class | Bin 0 -> 3500 bytes .../internals/KStreamWindowedTest.class | Bin 0 -> 4011 bytes .../internals/MinTimestampTrackerTest.class | Bin 0 -> 2134 bytes .../internals/StreamTaskTest$1.class | Bin 0 -> 1081 bytes .../internals/StreamTaskTest$2.class | Bin 0 -> 1275 bytes .../processor/internals/StreamTaskTest.class | Bin 0 -> 7533 bytes .../apache/kafka/test/KStreamTestDriver.class | Bin 0 -> 2796 bytes .../kafka/test/MockProcessorContext.class | Bin 0 -> 5842 bytes .../test/MockProcessorDef$MockProcessor.class | Bin 0 -> 1948 bytes .../apache/kafka/test/MockProcessorDef.class | Bin 0 -> 1161 bytes .../apache/kafka/test/MockSourceNode.class | Bin 0 -> 2292 bytes .../kafka/test/MockTimestampExtractor.class | Bin 0 -> 677 bytes ...UnlimitedWindowDef$UnlimitedWindow$1.class | Bin 0 -> 2395 bytes .../UnlimitedWindowDef$UnlimitedWindow.class | Bin 0 -> 3531 bytes .../kafka/test/UnlimitedWindowDef.class | Bin 0 -> 1322 bytes projectFilesBackup/kafka-work.ipr | 121 ++ .../confluent/streaming/KStreamContext.java | 9 + .../streaming/KStreamInitializer.java | 62 + .../io/confluent/streaming/KStreamJob.java | 35 + .../confluent/streaming/KafkaStreaming.java | 11 + .../io/confluent/streaming/Processor.java | 26 + .../streaming/ProcessorKStreamJob.java | 7 + .../streaming/examples/MapKStreamJob.java | 9 + .../examples/StatefulKStreamJob.java | 5 + .../streaming/internal/KStreamBranch.java | 28 + .../internal/KStreamContextImpl.java | 73 +- .../streaming/internal/KStreamFilter.java | 9 + .../streaming/internal/KStreamFlatMap.java | 9 + .../internal/KStreamFlatMapValues.java | 9 + .../streaming/internal/KStreamImpl.java | 65 ++ .../internal/KStreamInitializerImpl.java | 78 ++ .../streaming/internal/KStreamJoin.java | 13 + .../streaming/internal/KStreamMap.java | 9 + .../streaming/internal/KStreamMapValues.java | 9 + .../streaming/internal/KStreamSource.java | 16 + .../streaming/internal/KStreamThread.java | 26 + .../internal/KStreamWindowedImpl.java | 12 + .../internal/ProcessorContextImpl.java | 68 ++ .../streaming/internal/Receiver.java | 4 + .../streaming/internal/StreamGroup.java | 3 + .../streaming/internal/TopologyAnalyzer.java | 36 + .../streaming/internal/KStreamBranchTest.java | 4 + .../streaming/internal/KStreamFilterTest.java | 8 + .../internal/KStreamFlatMapTest.java | 8 + .../internal/KStreamFlatMapValuesTest.java | 4 + .../streaming/internal/KStreamJoinTest.java | 44 + .../streaming/internal/KStreamMapTest.java | 4 + .../internal/KStreamMapValuesTest.java | 8 + .../streaming/internal/KStreamSourceTest.java | 4 + .../internal/KStreamWindowedTest.java | 5 + .../PunctuationSchedulerImplTest.java | 125 ++ .../streaming/internal/StreamGroupTest.java | 4 + .../testutil/MockKStreamContext.java | 21 + temp | 34 + temp2 | 34 + 533 files changed, 2313 insertions(+), 2 deletions(-) create mode 100644 kafka-work.ipr create mode 100644 kafka-work.iws create mode 100644 out/production/clients/org/apache/kafka/clients/ClientRequest.class create mode 100644 out/production/clients/org/apache/kafka/clients/ClientResponse.class create mode 100644 out/production/clients/org/apache/kafka/clients/ClientUtils.class create mode 100644 out/production/clients/org/apache/kafka/clients/ClusterConnectionStates$NodeConnectionState.class create mode 100644 out/production/clients/org/apache/kafka/clients/ClusterConnectionStates.class create mode 100644 out/production/clients/org/apache/kafka/clients/CommonClientConfigs.class create mode 100644 out/production/clients/org/apache/kafka/clients/ConnectionState.class create mode 100644 out/production/clients/org/apache/kafka/clients/InFlightRequests.class create mode 100644 out/production/clients/org/apache/kafka/clients/KafkaClient.class create mode 100644 out/production/clients/org/apache/kafka/clients/Metadata.class create mode 100644 out/production/clients/org/apache/kafka/clients/NetworkClient.class create mode 100644 out/production/clients/org/apache/kafka/clients/RequestCompletionHandler.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/CommitType.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/Consumer.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerCommitCallback.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerConfig.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecord.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable$1.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerWakeupException.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$1.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$2.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/MockConsumer.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/OffsetResetStrategy.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient$RequestFutureCompletionHandler.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$1.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$2.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics$1.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics$2.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorResponseHandler.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$HeartbeatCompletionHandler.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$HeartbeatTask$1.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$HeartbeatTask.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$JoinGroupResponseHandler.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetCommitResponseHandler.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetFetchResponseHandler.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$RebalanceCallback.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTask.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue$Entry.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$1.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$2.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$FetchManagerMetrics.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$PartitionRecords.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Heartbeat.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFuture$1.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFuture.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFutureListener.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/SendFailedException.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/StaleMetadataException.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState$TopicPartitionState.class create mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/BufferExhaustedException.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/Callback.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/KafkaProducer$FutureFailure.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/KafkaProducer.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/MockProducer$Completion.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/MockProducer.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/Partitioner.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/Producer.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/ProducerConfig.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/ProducerRecord.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/RecordMetadata.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/BufferPool.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/DefaultPartitioner.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/ProduceRequestResult.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$1.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$2.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$3.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$IncompleteRecordBatches.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$ReadyCheckResult.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$RecordAppendResult.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch$Thunk.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/Sender$1.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics$1.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics$2.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics.class create mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/Sender.class create mode 100644 out/production/clients/org/apache/kafka/common/Cluster.class create mode 100644 out/production/clients/org/apache/kafka/common/Configurable.class create mode 100644 out/production/clients/org/apache/kafka/common/KafkaException.class create mode 100644 out/production/clients/org/apache/kafka/common/Metric.class create mode 100644 out/production/clients/org/apache/kafka/common/MetricName.class create mode 100644 out/production/clients/org/apache/kafka/common/Node.class create mode 100644 out/production/clients/org/apache/kafka/common/PartitionInfo.class create mode 100644 out/production/clients/org/apache/kafka/common/TopicPartition.class create mode 100644 out/production/clients/org/apache/kafka/common/config/AbstractConfig.class create mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$1.class create mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$2.class create mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$ConfigKey.class create mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$Importance.class create mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$Range.class create mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$Type.class create mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$ValidString.class create mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$Validator.class create mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef.class create mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/ApiException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/CorruptRecordException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/DisconnectException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/IllegalGenerationException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/InterruptException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/InvalidMetadataException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/InvalidRequiredAcksException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/InvalidTopicException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/LeaderNotAvailableException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/NetworkException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/NotEnoughReplicasException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/NotLeaderForPartitionException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/OffsetLoadInProgressException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/OffsetMetadataTooLarge.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/OffsetOutOfRangeException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/RecordBatchTooLargeException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/RecordTooLargeException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/RetriableException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/SerializationException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/TimeoutException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/UnknownConsumerIdException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/UnknownServerException.class create mode 100644 out/production/clients/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/CompoundStat$NamedMeasurable.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/CompoundStat.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/JmxReporter$KafkaMbean.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/JmxReporter.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/KafkaMetric.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/Measurable.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/MeasurableStat.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/MetricConfig.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/Metrics.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/MetricsReporter.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/Quota.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/QuotaViolationException.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/Sensor.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/Stat.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Avg.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Count.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Histogram$BinScheme.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Histogram$ConstantBinScheme.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Histogram$LinearBinScheme.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Histogram.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Max.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Min.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Percentile.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles$1.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles$BucketSizing.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles$HistogramSample.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Rate$1.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Rate$SampledTotal.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Rate.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/SampledStat$Sample.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/SampledStat.class create mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Total.class create mode 100644 out/production/clients/org/apache/kafka/common/network/ByteBufferReceive.class create mode 100644 out/production/clients/org/apache/kafka/common/network/ByteBufferSend.class create mode 100644 out/production/clients/org/apache/kafka/common/network/InvalidReceiveException.class create mode 100644 out/production/clients/org/apache/kafka/common/network/MultiSend.class create mode 100644 out/production/clients/org/apache/kafka/common/network/NetworkReceive.class create mode 100644 out/production/clients/org/apache/kafka/common/network/NetworkSend.class create mode 100644 out/production/clients/org/apache/kafka/common/network/Receive.class create mode 100644 out/production/clients/org/apache/kafka/common/network/Selectable.class create mode 100644 out/production/clients/org/apache/kafka/common/network/Selector$SelectorMetrics$1.class create mode 100644 out/production/clients/org/apache/kafka/common/network/Selector$SelectorMetrics.class create mode 100644 out/production/clients/org/apache/kafka/common/network/Selector$Transmissions.class create mode 100644 out/production/clients/org/apache/kafka/common/network/Selector.class create mode 100644 out/production/clients/org/apache/kafka/common/network/Send.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/ApiKeys.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/Errors.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/ProtoUtils.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/Protocol.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/SecurityProtocol.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/ArrayOf.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Field.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Schema.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/SchemaException.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Struct.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$1.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$2.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$3.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$4.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$5.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$6.class create mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type.class create mode 100644 out/production/clients/org/apache/kafka/common/record/ByteBufferInputStream.class create mode 100644 out/production/clients/org/apache/kafka/common/record/ByteBufferOutputStream.class create mode 100644 out/production/clients/org/apache/kafka/common/record/CompressionType.class create mode 100644 out/production/clients/org/apache/kafka/common/record/Compressor$1.class create mode 100644 out/production/clients/org/apache/kafka/common/record/Compressor.class create mode 100644 out/production/clients/org/apache/kafka/common/record/InvalidRecordException.class create mode 100644 out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.class create mode 100644 out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream$BD.class create mode 100644 out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream$FLG.class create mode 100644 out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.class create mode 100644 out/production/clients/org/apache/kafka/common/record/LogEntry.class create mode 100644 out/production/clients/org/apache/kafka/common/record/MemoryRecords$RecordsIterator.class create mode 100644 out/production/clients/org/apache/kafka/common/record/MemoryRecords.class create mode 100644 out/production/clients/org/apache/kafka/common/record/Record.class create mode 100644 out/production/clients/org/apache/kafka/common/record/Records.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/AbstractRequest$1.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/AbstractRequest.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/AbstractRequestResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataRequest.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/FetchRequest$PartitionData.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/FetchRequest.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/FetchResponse$PartitionData.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/FetchResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/HeartbeatRequest.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/HeartbeatResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/JoinGroupRequest.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/JoinGroupResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ListOffsetRequest$PartitionData.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ListOffsetRequest.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse$PartitionData.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/MetadataRequest.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/MetadataResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest$PartitionData.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetCommitResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetFetchRequest.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse$PartitionData.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ProduceRequest.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ProduceResponse$PartitionResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ProduceResponse.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/RequestHeader.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/RequestSend.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ResponseHeader.class create mode 100644 out/production/clients/org/apache/kafka/common/requests/ResponseSend.class create mode 100644 out/production/clients/org/apache/kafka/common/serialization/ByteArrayDeserializer.class create mode 100644 out/production/clients/org/apache/kafka/common/serialization/ByteArraySerializer.class create mode 100644 out/production/clients/org/apache/kafka/common/serialization/Deserializer.class create mode 100644 out/production/clients/org/apache/kafka/common/serialization/IntegerDeserializer.class create mode 100644 out/production/clients/org/apache/kafka/common/serialization/IntegerSerializer.class create mode 100644 out/production/clients/org/apache/kafka/common/serialization/Serializer.class create mode 100644 out/production/clients/org/apache/kafka/common/serialization/StringDeserializer.class create mode 100644 out/production/clients/org/apache/kafka/common/serialization/StringSerializer.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/AbstractIterator$1.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/AbstractIterator$State.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/AbstractIterator.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/CollectionUtils.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/CopyOnWriteMap.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/Crc32.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/KafkaThread$1.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/KafkaThread.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/SystemTime.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/Time.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/Utils$1.class create mode 100644 out/production/clients/org/apache/kafka/common/utils/Utils.class create mode 100644 out/production/stream/org/apache/kafka/streaming/KafkaStreaming.class create mode 100644 out/production/stream/org/apache/kafka/streaming/StreamingConfig.class create mode 100644 out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$2.class create mode 100644 out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$3.class create mode 100644 out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$4.class create mode 100644 out/production/stream/org/apache/kafka/streaming/examples/KStreamJob.class create mode 100644 out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef.class create mode 100644 out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob.class create mode 100644 out/production/stream/org/apache/kafka/streaming/examples/WallclockTimestampExtractor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/KStream.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/KStreamBuilder.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/KStreamWindowed.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/KeyValue.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/KeyValueMapper.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/Predicate.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$RestoreFuncImpl.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/ValueJoiner.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/ValueMapper.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/Window.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/WindowDef.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/FilteredIterator.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranch$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranch$KStreamBranchProcessor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranch.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter$KStreamFilterProcessor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap$KStreamFlatMapProcessor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues$KStreamFlatMapValuesProcessor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamImpl.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$Finder.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$KStreamJoinProcessor$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$KStreamJoinProcessor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMap$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMap$KStreamMapProcessor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMap.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$KStreamMapProcessor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough$KStreamPassThroughProcessor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$KStreamWindowProcessor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$Value.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueList.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueListIterator.class create mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/Processor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/ProcessorContext.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/ProcessorDef.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/RestoreFunc.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/StateStore.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TimestampExtractor.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$NodeFactory.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$ProcessorNodeFactory.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$SinkNodeFactory.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$SourceNodeFactory.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorNode.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorTopology.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/PunctuationQueue.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/RecordCollector$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/RecordCollector.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/RecordQueue.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/SinkNode.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/SourceNode.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/Stamped.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/StampedRecord.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/StreamTask.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$KafkaStreamingMetrics.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread.class create mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/TimestampTracker.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/Entry.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore$MemoryStoreIterator.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/KeyValueIterator.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/KeyValueStore.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$MeteredKeyValueIterator.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/OffsetCheckpoint.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$1.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$LexicographicComparator.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$RocksDBRangeIterator.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$RocksDbIterator.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore.class create mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore.class create mode 100644 out/test/clients/log4j.properties create mode 100644 out/test/clients/org/apache/kafka/clients/ClientUtilsTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/MetadataTest$1.class create mode 100644 out/test/clients/org/apache/kafka/clients/MetadataTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/MockClient$FutureResponse.class create mode 100644 out/test/clients/org/apache/kafka/clients/MockClient.class create mode 100644 out/test/clients/org/apache/kafka/clients/NetworkClientTest$1.class create mode 100644 out/test/clients/org/apache/kafka/clients/NetworkClientTest$TestCallbackHandler.class create mode 100644 out/test/clients/org/apache/kafka/clients/NetworkClientTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/KafkaConsumerTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/MockConsumerTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$1.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$TestDelayedTask.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$1.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockCommitCallback.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockRebalanceCallback.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest$1.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest$TestTask.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/FetcherTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/HeartbeatTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/RequestFutureTest$1.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/RequestFutureTest$2.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/RequestFutureTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/KafkaProducerTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/MockProducerTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/ProducerRecordTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/RecordSendTest$1.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/RecordSendTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest$1.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest$2.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest$StressTestThread.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1TestCallback.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.class create mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/SenderTest.class create mode 100644 out/test/clients/org/apache/kafka/common/config/AbstractConfigTest$TestConfig.class create mode 100644 out/test/clients/org/apache/kafka/common/config/AbstractConfigTest.class create mode 100644 out/test/clients/org/apache/kafka/common/config/ConfigDefTest.class create mode 100644 out/test/clients/org/apache/kafka/common/metrics/FakeMetricsReporter.class create mode 100644 out/test/clients/org/apache/kafka/common/metrics/JmxReporterTest.class create mode 100644 out/test/clients/org/apache/kafka/common/metrics/MetricsTest$ConstantMeasurable.class create mode 100644 out/test/clients/org/apache/kafka/common/metrics/MetricsTest.class create mode 100644 out/test/clients/org/apache/kafka/common/metrics/stats/HistogramTest.class create mode 100644 out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer$1.class create mode 100644 out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer.class create mode 100644 out/test/clients/org/apache/kafka/common/network/SelectorTest.class create mode 100644 out/test/clients/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.class create mode 100644 out/test/clients/org/apache/kafka/common/record/MemoryRecordsTest.class create mode 100644 out/test/clients/org/apache/kafka/common/record/RecordTest.class create mode 100644 out/test/clients/org/apache/kafka/common/requests/RequestResponseTest.class create mode 100644 out/test/clients/org/apache/kafka/common/serialization/SerializationTest$SerDeser.class create mode 100644 out/test/clients/org/apache/kafka/common/serialization/SerializationTest.class create mode 100644 out/test/clients/org/apache/kafka/common/utils/AbstractIteratorTest$ListIterator.class create mode 100644 out/test/clients/org/apache/kafka/common/utils/AbstractIteratorTest.class create mode 100644 out/test/clients/org/apache/kafka/common/utils/CrcTest.class create mode 100644 out/test/clients/org/apache/kafka/common/utils/MockTime.class create mode 100644 out/test/clients/org/apache/kafka/common/utils/UtilsTest.class create mode 100644 out/test/clients/org/apache/kafka/test/MetricsBench.class create mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks$1.class create mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks$2.class create mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks$3.class create mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks$4.class create mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks$5.class create mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks.class create mode 100644 out/test/clients/org/apache/kafka/test/MockMetricsReporter.class create mode 100644 out/test/clients/org/apache/kafka/test/MockSelector.class create mode 100644 out/test/clients/org/apache/kafka/test/MockSerializer.class create mode 100644 out/test/clients/org/apache/kafka/test/TestUtils.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$1.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$2.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$3.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$1.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$2.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$3.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest$1.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest$1.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest$1.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$1.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$2.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$3.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$4.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$5.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest$1.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest$1.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.class create mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.class create mode 100644 out/test/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.class create mode 100644 out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$1.class create mode 100644 out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$2.class create mode 100644 out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest.class create mode 100644 out/test/stream/org/apache/kafka/test/KStreamTestDriver.class create mode 100644 out/test/stream/org/apache/kafka/test/MockProcessorContext.class create mode 100644 out/test/stream/org/apache/kafka/test/MockProcessorDef$MockProcessor.class create mode 100644 out/test/stream/org/apache/kafka/test/MockProcessorDef.class create mode 100644 out/test/stream/org/apache/kafka/test/MockSourceNode.class create mode 100644 out/test/stream/org/apache/kafka/test/MockTimestampExtractor.class create mode 100644 out/test/stream/org/apache/kafka/test/UnlimitedWindowDef$UnlimitedWindow$1.class create mode 100644 out/test/stream/org/apache/kafka/test/UnlimitedWindowDef$UnlimitedWindow.class create mode 100644 out/test/stream/org/apache/kafka/test/UnlimitedWindowDef.class create mode 100644 projectFilesBackup/kafka-work.ipr create mode 100644 src/main/java/io/confluent/streaming/KStreamInitializer.java create mode 100644 src/main/java/io/confluent/streaming/KStreamJob.java create mode 100644 src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java create mode 100644 src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java create mode 100644 src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java create mode 100644 temp create mode 100644 temp2 diff --git a/kafka-work.ipr b/kafka-work.ipr new file mode 100644 index 0000000000000..e253c951c1028 --- /dev/null +++ b/kafka-work.ipr @@ -0,0 +1,209 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka-work.iws b/kafka-work.iws new file mode 100644 index 0000000000000..e5fce9c4d023d --- /dev/null +++ b/kafka-work.iws @@ -0,0 +1,1026 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + org.apache.kafka.streaming.examples.* + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + localhost + 5050 + + + + + + + + + + + + + 1438819547964 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/out/production/clients/org/apache/kafka/clients/ClientRequest.class b/out/production/clients/org/apache/kafka/clients/ClientRequest.class new file mode 100644 index 0000000000000000000000000000000000000000..7d0ae68b2ef3e094f9503f2947ecb3a16e8de7c8 GIT binary patch literal 2117 zcmcgsU31%15IxtHEX7uuBAT`?`EsGb7HO1_51?s6vD?rFC!I+;nIUhoeVeGUB_k^o zegn@u@C*6^Gcbdb8JOXb2YwW=_eyc>L`{bmUUpY|@9v(nyH|hx{o5Y^7V$KTYgo!+ z6w5dHvf_4w9NtHy(8 zPOV8Ev+5JoT_*%M6k(BBCI*V|{fRxDLtIA1$< z)A9E0$V*s;cS0K6rqh;eXussrJYh%|OSBbp6!jmsA$Q<(D!qkRN=)#kDKO;XoqguE zsPP1b_BOlY9|d*3?#fvVbahrxGcbv1AvO$L#|;Bx7&mYW?-;m=ynzYi8E%|!LixyT zHh91=K9I3E_()a_e1WPEn*?rV>U8C$e~YlqOvSHbm0@gV>avXny||Kr_fce+zS!Nk zIRj`IuAbq3T6;mycffTzWoi_SJ>#_7^fVdn7B9~H;u^#qplui{?p<~~kS;>8^7{Lj zr(f16{o7QSVjg6fVh~c(RHUX6NlmjMJw&aU#1zQ_ebt~LL)x$=$MZ^FO`m}MLf`Vr#l**6|57atjR9Y zR$lEwvGPh6sx{t4#v09Zp`C%4kO6a)a-P6{h!HFh>CJWnxoeh~E)uLFv)&(7!yvkeNM3&iWOZV;JUeWGlzGVva<% zN;Vy9KR`FHzJhALMSjyHag4WrrbKDkOHl;4C*c}Vff#Oy{MV6sslob(4w!}ykfIL_ zGxT{AcS#oLjx7#Js{+iAWJE)X5|{fDt8rrb>ti+{SQV)HgVz&-?Ox?_TzQx(D}O>$ zUO`C<1pU-D`jQOyQB#r1JrKypFqZ&EIS*#5FV7j1d7Aub8>0D$Hl%+#$7DM(c%sLo RzlYlS9_}UvFUS&${{pcixDfyV literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/ClientResponse.class b/out/production/clients/org/apache/kafka/clients/ClientResponse.class new file mode 100644 index 0000000000000000000000000000000000000000..e7802d3d34b21d1129c59d081a34aca60ff3aa5f GIT binary patch literal 1880 zcmb7ET~pIg5Iwg|+fY)$C@O+l6|oIy{6J9wseq`pzzjIf$oLY{D+EeXlN5(Xe~m9X zqcc)xbjC*?{85g3lYmen=*#Zr-rYUBXZPmkudm+$3}G&dF5GCtaoiMQNQhfP+|H61 z&f*U4W^gYH88>8%2t6vqn2h^EkIN_t{XoWqj7ecmW$-YAX$INlCbu?tm4THRl&a;K zj%{<(4~`ZYGA@7J;GWM=s5tJjZqyBPh3jj^t2IM6YZkYCPoE6*eCRnsd0g(*9oyp* zPIZ$(kBM~F*Bx80yN>UePEGeW>)g{9e79lxlpsB7*_MBw;pIQ0mKXmMrTCmdnsln1 zp`&8ie73Q^#NB7cQjJd3iennJ=Z0&E@90qSS1gaA7z5NoMA&`EL`d*6Yn{?23q_*2 z?O~eEiEFL!52=G`Q0D8g&Ii-lzG8Uu(P$WQ3%+5l%^3Blz0PRV6~pJYxj93v z%DzK0vFv4rNx z15JY^ErZe6&ymd2*IOi!Cf%cb0n_qZNN8Qs7LwY)79_1ZwT09tx=s)PXg!4MMFQzV z8?8tO`U&$}s7=Ak0>})m@h(Uru#N~#BuoCb}5RAcSG(<53#RHwpb~ZyMGwaN31n>LC z`v4RVJkW0*Rl!Jke8MXIWcerj36|yc^vq^27AdQ&qz&*we5_s74!{uaPye44~6 z+@fJm5_Pzx3Af_5CftrY)Ua2>ok?i8D+vQ*YA`iqG{_`$#WS8jR!t|=U?tIneHtc{ zurZay0yt`L6UZs4cWdwx@YOU(Am4;(+@oQC5(jW^BUa$P1nzIb19(uwLrFZ0M>ISt zP;a|gfz~0{%cgyMeDl8akekg)uU8Xoy&<3(5re>z=+KctdXU42Yfj3b zAdZ>@OmWZTexS;@eWDs$j-e8nU4_S#94HL-R;r zn3FqMHpEK!GuqC#z zM5D@2j<|WxlmnKk^OmxT-Jps>$6Bl-jHTl2^E~5#s!%4DjwkUHvrEU*ct*#wc#a~f zYe~oRIH=(T9WUY~9WUb*=CtZXl`X@ansS{qqdsnB)BWM6SgHbxs-<3)x9p5$F0Vgb z)Li!oMMfS)0#&0Kbf#7NNwIcQmXjKD-M|k#Be%hq-n8`mo|KMP@tTg;@rLsLO@YOs z@|wBMqSA2)2NmUE9Y^qtz~=44BPljE*Jev_oz%ENY%{5G*ZWtcZ{cl$3rl|1Y+3>< z<{VJ{>5Lv6#X&+d?3Vmfy8TqvnwCyYE)DPKc()Ynup1WhRKt5Z-p4T=tFeYb9*BGj z+*DZSDGeX!_z)intUPl(O1FWAk98c!Cl%>(!`MD)vYzLK7!=zn`*UeyFpwVG0pos7 z38_L~W?EatWsEo+XA-BFt`{9(DDdU)sEs(L}VICqXd^gA+z)Nxxn**j2t2vD_G5IBU+rQo4lVMlL6v4BOUOSfb0M zP%(5W5xCKoK&9J-TZb*RmPFMzEGOXYooB0Pt3fiTdW&xPqfVIq(~`#3^D3+oSU@*K z-?#3iuqc@2U0R?a%q)S34eSkJj05ughjxN0QJMWiCL_?5ym?)`ES zb$d>reyC#>F@YWHXVD;V7@u$w7x*69aMvlk*)@a2J9w$U}RI3{}<+GD4XXm9ud zOHZO{&q?SM)7(0L1}!9RnZY?H(E8m;EZEZ?Yj2prxhJr&C*B^PLEGU1WBYr?PekPZK^;$ybaJG?q zaZ;Vn{mq^cQup_S-UN0EB3vd1cskv}2BjlY?IJ;)CydIy^uo#kP z#?k_~OT#?6G;}&ihnnf+JUX|KE^frO5U>uO&Lim8V<-fo8#^#e{j>l#5FC}p-MA4u zX`g~5%3HN_6L+rUj)uA&8b($W+}TA8M~juPEEK>5Z(Z>fR(^?;ss?rsj!`vSz}0T9 Hm{tD(iNU^D literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/ClusterConnectionStates$NodeConnectionState.class b/out/production/clients/org/apache/kafka/clients/ClusterConnectionStates$NodeConnectionState.class new file mode 100644 index 0000000000000000000000000000000000000000..e8dc3026f4c3b3846d63ed463c7e471d6bf17987 GIT binary patch literal 1090 zcmb7D+iuf95IyUpabnyHZlPQPxTS#OwsC>-RD=XUm54%-NL8tin`BAdx^`r5_!ce_ z-vAO#C6M?4J_<4GSVC<=1P`7W&z_kxm))PgzWo5OiAM%zuvEe{mX)}zW5qxoOGVsK z`mPf9ig1*!6yYkps$*S8m0>E8TnYxa5yf4P_jvn2ct`xr5%=1GFG87kwI~clTl!Jh zRM;(sg@7kAb3c_*9QWkQgn`u=3XlEJmrod~&cA1@SN0h4wWvdd%r^W`?DUUYBHrb# zfILLROMP3T zeKQng)ReIwc4;n7rE$pLaxdUv*Bg2%%xFgaxGlDQ)lIoJp0cU}P0YhHF^f497hy3h zo=^9z?*|JpcQ0$5%6M01R_~s(IBq zq&vZobJXq%yw?+9hhg0r9l+>JvLf$jh!Ua zxBi9m*{UFIeIpPaZkbA)e~s0eS+MV6nL4Wl^(w%`5q;FprjwMNPe3FRq89a zPFh)thNV`Sq7)6SDWiE!Xm#d%M}GelQ&wRB-Tew}fTC3z+PMJ?>jK#Wn4bx&LGc&T bxKAmkIn0r9YPgl6Y+NI4(AhNBa23XH84?qG literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/ClusterConnectionStates.class b/out/production/clients/org/apache/kafka/clients/ClusterConnectionStates.class new file mode 100644 index 0000000000000000000000000000000000000000..19069c4afa541564af5b29d0f48b1424e6739a26 GIT binary patch literal 3360 zcmb_d-BS}+96g&4mMlxq2%&xwsUIPrQHy9nQ3;?Rgtj0STWy!Hgq3ACO*XCer89l$ zOh4ZGW}oeg&a{k5Ei-MO>P-J8rtP_#H3=ngpkrpTA9wHXp7T4ubAR`*zkmG=Krf04 z4q!Bd{kSUQngW06WvmI~xGrNN1R0Ydq%kF)-cj%_-jgvM{NkU|PkfqD`00j-m=&)# z6|VNaj5!&%6trQq5g&;AhvITuK`ZXa_(;ab685C+1wCb3mY#78+nRPXN0*?EaUh>E zwL(EJNQmV1OliuHmRYtJ7biI=jY|kwjP_z$ChlsVYRRHwn8`^kH^8scM%L1tVxB7+ zo4;Nty_%YpO-?&`!^#d!*!gTy%W0Vt`GMqsPf%tgWEN>~Yl+ZF^SbDm+GOy=nw0V;wkqJAanX_8n5bxe#z*#bw-kBZq zA+nOuERUvyXnbpu<6XBT_>2XC?aEcHfCRr~ef`c=e?PhILcBUynU&t~kbuep53XomnU^JrFL%6WIU=*sRRb}CsaTUiyRm~s>=DlFJ4a^m$cQW82UGsjF*&uXSyn2~!KJtz8&g!ameQX?2D z8s>tYmvAU;$GHFV_hXB8(OTd&AM=XM$e4;xkXKQFBcLxyIJMKXCG4tzPtD(D3*;7z ziNZnfDi1^KNn@dUx{=VZAP0U#!2gJmepEOdM+@Iti5?}U;E-^+^_4mF;Utp8CL-mCkzor{ zJ&iM*_ZCN+;NL)0M(@GPg6P@p5%t$2`a>O}y@WkWqJ1QK{&f)T!#PjURc0bHlm1$Y zh8}w|o(oo_Z4sJAgL>!}G(B!b^J7MrRNaRLKBq`_m_$cdzfmfAm9($Bqcm;XBCxmC2p*0Ae2cKdO& zVmrmP*9{Vzb;-nBueO)Tw3o@Wdt@XG;DQI^Fj{!KbCXPyJ%3_%kL$U}!!5NdA>Kw9 zcd8IO%ZQyGVwjZ`mh!o&GHOE{6up;)qb_crK}$Ubi)4`f-wXzE$fGiqOqcI*j31;336!W$s#sXN#$@k=dhP^0~*;&M9vZ}zXBmt1K}PK?i1njZ6jR4 zke9a&utie&UPI?FkK>0(|K%TIxliGVPE@+b51>Z>g{7J7J0Be=P9=ht|0Hxd&hTG870f$$+AxlSNWt7(gse@ VC!{~@CNY9Tyep(9bMqX8@-K5t;N<`S literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/CommonClientConfigs.class b/out/production/clients/org/apache/kafka/clients/CommonClientConfigs.class new file mode 100644 index 0000000000000000000000000000000000000000..5094e059ab5081e74f0457f9e174c243418700a4 GIT binary patch literal 3528 zcmb7H+mhQv5FO=0HX$K5E&)O^;kpYOnS^jlLcsBA6DMAKvCRgaTw`f$O;*wfX}re# z4qw0%RaC(T@We;($}doy9?4$qB@e0Ea_MM#db&@a?$Q4B-|v4i#%{6hDRz|IKEdvs zV0RsM>3*lv@AbR&%|XxWZh76_pwVfsHy=3cLhGs65w%Dp<66&TDjDB#*qMe-aw8J6 zC8CL3VqZDzWw_YuVXf0xV)q>O+J@J!uhskY!AAZ2LH&VO`d(sRW7~m6yX_ks-1FLN zgZtmDuY29aj=ICnR^MCW`wn}h>ovUQmbc%&0YCee?3Oi$oolq3Ub{bNt}VQKl-c5~ zHC%Vtn{d!+x4lMxaKGMo+*w~AY|QytVxZ{Kp$2yQL!{sJ`rRK6+uB6>2NvzNM-ICR zV!F*nZ_ul6Y_`0?lV*FZ^Q3z6V~1USrjd2ia@ZSli}rULrK{c8H;UreW2)=e?651v z&Oz7P>~#B7gI2xQ^LBT)LCv#l?C~8qDVb|_+C5wT<{EOT(6x*H8Hy2vk7txl;6yfsaF%zoXVQx!jy=l%Y`$bC-FjCsDwML_lG$vzXuYA(ngO^~P=D@5U-U-?Yy+ zS6$b=TeB7}@2SL>v=O|{GpF31f~_JTN~;>{UV;zCS`y7;Ng_poQodYp7mFG9LyT0( zgIOYC1>Huol~uT(r;|ufkZ40?G-968K??ZtOixg{8J}Vv^H?Kf1J_(9lKkn(I9w%4 ze1XJ7nHtA2fu1HSuEQ?I(uhD9;l|=8SB#|_qhbCab)+&=I-r^x6|F9-o_S=+ARF!$m)D9lfWHLMk7dlI5QGCd6ZMGezq#6O7RUqdl(DpGLp1@ zRbynFnv$J`8%j>_G3u1q`B@GNv7;5Vo5@rPBLiq^-p4dQX$wsVidR7O z!XSqvm7QM2IQMm&PK>0fVBHk6$39|f=@or?t}t*>BaCdi3)#QBccC9a{{{DS2MU~0 z_@kN@_AYNyfHA6JRe{)wi(tcAc1yc*_9IgqG7>brNZ2_~qluLzaj-Zp6sEW*A!`ig z;=V(!G=+)?xQ{lb4k%jhQT)^VBx*YmO~niin3Np>S!8mI3my`AwDCO}9D8#^{NlT8 zE=q{V&Se#Va2m|tHX`b0!VBfXs2s-B$Gc_NGp%u&j>{a$U zJI0PPd~yLDVJCo>D)=PusS17(_;dxI0e-21&jO#T;Pb#2D)=Js%N6_z@T(R48t|nG zejWIY3Vsv#as|Hye5Hb~0`m&K2K;sfzXSYk1-}RUeg%I3yj;PwzEZ&-0)JG&tH9SQ z*afat@W)`*p)W@KegcMT7BhDI!{3HGsXk1hR_ADwdDbZpD@hkDg> zUCZ=s&ozARTO|f%da>R-V2IDK$Q$jw#+#bKkbgT5Vf2hUBWuX8oypF1QL1|HErvwH zcCF^9-?avZyz9`395A+y7%G{@2|wey!`+^4_ygPRm9q1Q(;A-1oi-oXf_lm?xx7zl zw12r7qGj8){R+d%e^S@8Z3f9dwyD~oOh*>3AGq))B*#K(L^BM}>&B z_YG1g-~ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/InFlightRequests.class b/out/production/clients/org/apache/kafka/clients/InFlightRequests.class new file mode 100644 index 0000000000000000000000000000000000000000..1d3736a23d2a67116d3f94a9f07ceac6703afb97 GIT binary patch literal 3688 zcmbtXS#uOs7(G2Rq%)Hyfia0d2n+~hg@LFDNrXVyq7#;cT}3*RPNqpu_t4W55?n!X z7ZvpZpDbS}t9ZaF79&;alT`UT{1p~1=XUpO$wWd~sod`V?zfz8y?1{9`=_4)JcF+z zScAhdjzkc^;V@zt63cJ|^*9>AF&wYK37nL1O2+93A~-C*U#dYEFN@_B8E2~TY6NvS z8=;^oh$AwR;yEf7O=uYtZ{tEVRRbO8YA^vqEa?bLSYjh15&i~*2hiN$=RBO zGv&Lz-lw6OY=^$Kz^7O{TQ?ggQ2%#*>DwnNBv|?>>E4Jv;a( zp{iFmb!Uf!Q0qY3aX$L3QQECb=%#ismmbmVqw0u3N;F|5RpYp7>*CudhMbg6uRG?q zIXf6M8B|9{DbqT?P&rTmOZEdMVZmbbg+fHPbase{p|RYX=}g|E{}WFMi-#RGIl(l# z@nP*rP<-S^a$1gIT;Do3g>4l8hDZccHswg@t!Q9@ zb;as7)T}eil9R9`uQ*X4)g)Duo1=r4DDZ|#rcO&(#8MXVs@KcAso%;8y%ojSSi#md zu8T%RC-CW5j%vuDKy}j4RC||Uu!1U?EQ#}gquHXyStdgZWPPk)M~1DOoz(W}qOBTc zH^?S24ivN^$I2~^h68Fg#kx|k4(k=H#ZwBlU{b~f1yh)oaZ$l*c%6+~6miD^BUq&D zvTb$RB`bIXZz^~TZ!5Tj%Mw<*T<%~QFbr*6HCW1yws$J2WxTnyb4bf(EHkUc#%wDc z%WC$dX2-@XJ9Z(ZCsQ#erP*3c<-cjgbaUgFI|O5X`DZr?pWach9o-7v#T5w~7FJLS z-oq6I?~COFTvqTQ_AB@ZAItbe!Kb*&{3`fN?0k+ZGQLo74PPqwN`xa?;+CIQ32pNl zo}CO9a8bdbk#n5kd7bXSc1y*vY`5?@rZ%^>RhVj4I&GQpd?4aO-s~4Wz!K}63ub

=u%lLdp#Ne;aScV~n2Mf8(*#g>dM$;yIYhl5i zf6tE|9yXz@ev!>qHK`grk|iw5t8!!1Pa*GgWc%qRd=IvLfc+pb%iKgzH#*A9bgqOYULmFP*n2 z;=%O7wFR{_-bBOeX}Sonr2wx5J?JHvS}dOh&!gnwg%Nm&Cc+E*X?&hSJFt_hz`2`v zMG|Yh_VFoKC z3=&C`DDLHaSt)WAKG?L;jo#Vb8~^;#RK(~Nx=u~!IA$i;^F{&dI{pqiz&MFab@`|R z4mbrLb>)XWmTzu;cZb-)oKOX6`-!f1LDO^rm4HGoyHE(U*mxf z;G+-|C$tqCpnZsr#+fr`Jahcx=eO?w@B%giSPNi1fDHzLv_~l7O9ty<&twS~sfbRI z55&np@W`B>8_nOE7~9>}M+W0rncR$w))+Zy^Z_byD|(l~>@}?;q{5iN+J9sV)iH~j zQn%|;l|=0<-}iWg7dcwjZhat9YLsHI7h08N!@Tw-| zWz8`Dkg}=K7Tdd2ZJGv^Ds;G4NN$Cto)S9YeV0j{(Da6Tcz>I`yEU>9)(PznpWdx( zQ=zb~XGDx~VTNLDB=k?e(CDS^D4CqPYAyRF*N(wx*m}cYspj&<`JBeDzX`9&Y?v|l zRJZujnrUX7iHzcVsXW9>+x~l3S!oW-t9Hujw7td`F~B_aosn^-`XG2og#_4T7MpPIhNE3$r`R%q)Zg zYFllqR@+(>5!$NQ(pqgr$%o~6^!NTB{k1>$>Cwj~9|iqmo2F&Lp^UWN>BhWT?t;Hi>hXkkxq^zOUj!5{m-) zi*2}s=aP6HFR1uI96wB=8$U|oMZ6?Bf2`ssDt?;8T{siR%QC+r!>e)pOy<{;SRvwR zye=w#F4}$}7k??1-iYH@NnFO8ZFmb;RJ@(URlK9(nu1o#E-2_aWIKh7Yvs3|${ex_ z1=AT+(CL_Y$8^W{7}?WyK0o3rP!4jtWO_!<@Qi&Em7-(HwtCid+@f8kz_5aL%W%EN zgS&FNOrA%p*{tch`Kr})qGqkC>zR&%Uf%(uVr0k7%xNQk+Q?+>Qpql7_68#KCs{Vl z+!Ga=HfiXjf|zGlidkA%bm)|E*2q-7qLmpnJ<29Vi-oe`RUL}1n|-<~c(jaiAv5YZ z#d2YAh&N)pishm=L_|`3#}q{N+Bx1`bf{Q14_8ZLrgPL7v$#X3vW9icaEkIAZbrQE zBK=smKonNdEPHNdBm|9?qN#%i#Y8l3df9Q_O7$&pZ*KIhMlPpdX==XrzAhgM@y##o;zD%HEEHKmiyX6L zFcsxO?IgKxB5PJW)|ac|I&(`Tj)s}(w?=; z9-VW2^WTsSDa>kErR!7wH?uxKt;_?nkT;4}qr@dgjHG*Z`_%G5yqUm`**&TtzAI~m zg`6C+_4veEOlqu1C)4={9fcigU8IlXLq`k1Nj(l_g+qGc6z*JL(wu0%1c;sXu8!)Xn_ zM^QmW*lZy(F&)Qt+{{rLwX4m(ENeI%aKfHy(aH%k>HGbph>w9s4G)bxxl7k@6MvB5 zk7NzKRVn?o=vCN)f9nr+{R>B!hAGwUf>-Pjr~(|M;_@yuLPPS(uVKZ}Ui zr)H_*oo7M$s%n%wrxyZygMI9Uj)ut*!z10bvgxTtNKoD;jpNbtd>52hW!b9Zuv8>V zBa&*eLqzI>jb&6SW|=&4-$Fb%j|svgl5_87Fa3|>xMv4xXS>z7U^a)Y2o3rH@3mRf zSz)9>>CkwM7RXI3(_;QSnX}d{z8V3WFI9O{8dlXjl2_1^8gBL|l%T7fb6>>-D`$>X z`I_FjK-5i9g{aTp5hs_;RlDEFdbYz%CJLq(yz~{2uo?;3OXhJ0gO}5+Zv)arn%@>c zf}b9&X|B42NBo&o!ns8_#GlD3{!C)=XVQs3lTkQ#34uVtE`RRL~r?myE!Il1XP-&JA1g&0#WS6!+a7o`$9EW&`L$Xw)8Y&={1v3)4h|3 zrxTM%$dF98PSVBm_&q%V}yAeZGf2 z_hC1AbdT?lPO~%`IwUK>NaEQhgv--j?8hUalNru(7ERwq`zKtqP!$mKHxD#L7!4Kn zl5>Z{%MVjp(9n2!@y5^5)q>*<#{;B}pbd}udU^ucx?>s+;ycvTi$fR*HNL_yRcg#o zW5-pfiJR!+GRgnAvBpfyV96e?6Z}{CJP_^eiE{5_UTO6m@9?945-V{6DV+2Tr>T@G z>xMHp>?5+0AG=3RC|6NDM#(5z+73tP7s)jAYX^N7n3Cy;7a zYhz7#CF^({#S@f#h{X}7j3hI!CoW${`yF(j=vq3Bo-b#xY#^c!#6`V>}ot>Iz{`ZRjK)O+IQK`M^MTO0DhQ;Bquv|P1k8xCC;;bLd5N~ax#4*AX z7vJkJ=5d_OCaC1`Bu`tY_c6l0nQ$*B#3#9m`G9sJ+W8--kx2Kyu_^K|>UoN{BE0)F z-otnCy%6Qss3XRc{*5F3UktTwk1UO>dLJwMmqxbSF9o`IMKma~7bD8eum7H&!dk!H zf+%~4E^v`ay~M~aVgsJTM*g&sE|sR}X0J-ciBx|*ihj1$Gqf*>Z%4~lh^u&3QU1x^ z>gACUq>i5>trTLZON)vt(^xmRU0!NRzZMz@7-o#RvdD#DK1iL?Hf^GZZF5fx8r2kZ z-s`@qpkvzVzP8mn#w6weS-n9VrM!0p45ACUa7d<4u$Cgv3d3v^ewo7hPqCrvUMaB| zq&T&Ij=l&Q{kqoA@0-NuEmr9j{-ArCKku$$HQvEmTqA*AuVYDjb+2~Fx0(;m4ix-4 z=*2jzE%0xMzzHGw447zG3%ce8t$olGYpkv{=Iau6*i!GpX0mE(zyPujo6v8N)hV0~ z9mx=L$)FTcQV64&>257>Sz)IvbXlEyd1isjakg0PebVdO<$J4(WWVtz>% literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/NetworkClient.class b/out/production/clients/org/apache/kafka/clients/NetworkClient.class new file mode 100644 index 0000000000000000000000000000000000000000..3a959fc5e2946b7b31f830498b1f5ee169d7c942 GIT binary patch literal 15727 zcmb_j33yc1^*`s%@+OlPmOz372*Vfk{QJB`}2L=_uhT?o_p>& zzq8!)Cf|7Pi%%2L8R|AaJ;aSA^aQVzrb(Jr2Cw#$htHFTH3m2P$;WHuVVyMROS4{@ z3k+`Y(-3Ync!O(KUtga*Y?L>fq`6Rkq;|3=T?()+q-00&=<(l+yO0H=?P3CSHzO9tE^A4f2)6bXj zBn$e^(y9ClB8j4t`+p zOTzRIef%SN^JC%UC#C#TzSZEL`S@k&{J9@%{zB&bQs(^1$G?_2zY$))>Eqvuj(_Ll zSA6_?gI^Uz{lQOn^B)cVlkoUwdH9R$>94}~-=y#FKK_T0{-;oVO?v+2<9`bV|0CV6 z`}n^;e#7AZN!Odw9FzXHWZkz7e#cLb@w>vzdp_G*&@pP(sr85W2BwNwIF)WnG=avVZQ*E4WV=${tzh!bkH(|v z1xy`P1-4gIGX=&D)WzU^G^}W7n0h{wyB?fiQ;pHM)zsa&#Y(OfmNa-;!?E+j$*4R# zo$ho;6wDT4&snm`MoztcgtUU4;T;1tvN~VOSSe{aKv^7a-I{1?6M1-&RyeYgX=0(Z zY*B~|>sCyKP~D;urf~%%kv9}(5l?JqDgzi>x2_C#Ihz`qgXt1z@q`wIw(`)u42raX zmkIB~tng|?Q)}$4c{PwCa6U}_XiCs&i@u_2u)bt!9;4cxm=+WoSwISoat7EI)Z_H^qt)sXd%`=XUt3{*zYkW`m58S>b?R~U?67YS1vw)f?X(izu-lX5Zf&B; z+L6w9t#w^+a3DB3_X_U8c1erUX{)mf^G0qDN7Ks^$;>2e=ul6_@YiM*D9LaLKo1+d z>B`1LzKQMZ%9vT*fU)yWczX-#kTn>}Z72@{0@474e5R7-XnQ=I?oMK7XBSYw2cfUs zU1wL!N?VIWp{Epp;9w0K%FDF+M23DaUPe?6UQ5hkQ11sLJ90q8sZVsrp?Fu-RD_L! z=868{aRA8y{BoGMgqcl+49oUOUCBf`(VB==FY1b(Ywb+c6#}t6)UmfzbwBFBvTLga zcNHoCk0NUz$ZQX9wYt0XW?;*n)>uL;XlTqrU}{W&?T9#5ab^&rBoKuX1nql4BO2Mk zWVUuElMV4U9iw5(QIHki2Gska*=|o8ga}_V>jc1^=d}gVkHF?S!tqGVs!!O^i-@-b z{gDo&a=v7Rc&I5w8P8=l@dyNX_Q3fc1Sfi_Q3DurvI+3!O(nXMt&pS(`om*~gNf6M zGMA@k+g;f(0-xTn1_^}Q3|G1(A)ygDT@meMp_PVDNlEI`-6~!dj$5ZXqJ&~|GXin~ zxw9Q8;VI<8hBABvU4HJPB_Shd17;rzTf�TNbU|NMFrNUl(}}KV zD|SD00FA<@g#jKhr!);J=xktc+^}1C&!baa$JZaUvaumBwGe0-+Y{K?1P5CS-YrZNMDhM z_siX9%F)N97rD%&9y(;wVd^#MS$YoH)?nY^>}X)7kNL`P(wFGVa5+V16*>9I;!92X zA{{a5X*z1sGtzvOjzSpvT&A7=8pMS}C+sPARBMQ*EgW`DRjN#rz9O@}MqfASd3wR5 zU(idYDpNyDHB=2V`5azu(ofZJlYUFTGgP^$MyQdds!)}N8fB`JRKTQP({D^QTADFv z=v6h=R41z;Oyl#`jK!?>a7^dr*(4Q*!Q2S@0J*q38q=AsL7@}MNKi(sivw@ zQ%zn10jd!0=#RV<2VTuh{{-8lr|2M4%{pW+T{x^;k)V}KCX&HOw;Xtb4b`iH(RdKv zIMUr}C4)|U2&OxP5kpNg)pRu@LtCq4Q8F3cDT%$w0UmAg$@9|jO!BdpJA#qWnzt*W2m{Nnx|?r>-Rq-AUjB;_*I3V>FftClcL zE;cX|!Nt9qs!^>pRg;Ko74E>yY5+!6n(90uQgl-pB92dR(g2gkiRG*jn$2o0(g9Pg zQ|Fr;RO?Nis4g&6i>WrKjfUD}steUcOf!nHSunnvJeTL0>S8W6)n+iw(;3)0tr(50 zN6)y+986wRVUuU6dQ)vtt%icToVQft8paf}f^AZq5qx(#^{U|ZXu2bq3BW)_Fd7M7 zer2#0jj1g5nyO8;o2o-aO?8Rd$~3p3J)Y1uBFDm=IZBxjT7)EQH>NsG6&K@7h!$*j zmKRQ?e*#V#qH|#)l1(P82&Q+*^h;IJPzak-1`DT9e1VDAhvTgl{8S`s146pn$~qi8 zZm~ot`RfU{*m7ERn`)bgVY{$AKAVX{W|&PxR-zm*ysB#XL&w$B}-D-PbAH!}hS69gF zE2X*0hW%7JJ3rds4 z3KD@1GX7tHw@@VBrEsJ=t3gNmSoAWhu*kJ=C6pARi&LUKKP_(ysw@~z18cRHUpccM zB3Es>1YyBcSBvFeW3zLusjgGkLvuQyz*kf^plQ}s)vFgq z9kms%?MNoJ+r_x*6M#?tVGX8HY=)EPplnubMAj!dH37-~beYzW+5++#y8~gu7me5n zg`9z%f?A#fLBi^94ac*IVP(P63u=mmU!c5MfX9qm$=bV9g*2BmG}o_cYFb*qb}3v# zp>jmwso>I!Q$;uiH`cGJVq>mOik=e&%gP~^v3a01SFOtw@FZ)-E=r>*M+Q=#Erm$L z@Kz?I5;hBR8%(^oZveNYC~ClfoJv0q|h||9kj^ug*2MMp1wN1UUUi4_2F1-3qA%wC8)$Yw`U%YiZkDden=CR>$;SDSu2pG zI7=6s1S3trM(vUoB2^&;4&dG?V~Y-r&Tgj$>+O!qy_AU}5nYuHt;?8?N74Mn|8*6sJJ*NKjYXI6~l>WXHj&o&IBE4(v?x~=HZ z5m+2Yyu~}y&L>Sk)6*4BN|krHBai_D3dv-UAIuMOAUXRukZ=;B0`=;usm=J>R16DQ zIjol2xPyrbsV?W80Gsi(x)a}^XA3^wmy#nDon07Oh=`^eCEgk}fN&^V*3cwz*wshm?5BQL8g_GLNRTUteuRI-b>|Z(9si6*m znO=~@xkV&6ahBXVy?cTB&)Be%IYgh!}^MPE3Qfkwfc(EtG=T8it8Ax^CEo%?Kkmsj676=YfI=5 zacJ5>QlaSw$rYM$kldjo%L_xZcX3#}+I$ccj zXfrLKFg4K@YNl3$~q`)AIYi z7O7y9q3=XsKFt0Bz2u1DGK`k-6GPL2Jyf}yJnjRbr^r98hen~}q}^14H-{+TI)G6w zlKoU_x(V#1i%Ka8#ig;AZJOVp=3`=(--+}?`jO^$BK;T~Df$U0xJi|cMDI`OXAU3d zI((GnEk61&3{`qbuO@MQ-V$ZmCCai(lsQWUV1DmGT+gOZncRc>|ag(M}R6)w)4>YuQ16@6cF-eg#JHdB9<5Bc5cN zW1VdVcWZ_Wa5!N%8See$&eNbR&d2gLJT|-jSgijXsEVFEq((19AxyqdI18QSEL7qw zH0JREVmy>bwIoZm#G%Uc3jN+8^&oa6s9gpoC-%^!m7(d7*dCg6geJEfp;KU2Q_4^6 zp{nWlUp^I055Zce9mUeJvFW%fncGV<6dXwgpQ9mT!uKK{_Av|}VP@eQ%v`vFrT7dZ zHf3XUS(dkD^eX)UJo#XOB4-<;TF$dX&MZZF^uleZDUiu<(0CbZyCy)}b3C}Kkf$D+ z8J=L39H#0y2A)U4bVkxF2)<-aNu{UKz>7-#wzRTTFyaeMI7G8CQjM=HJxr(f(itw= zPs3}=o}`-cIf1fXn(Kn+n+G}6)|Q0=Wf)&)U*_AFvmlNIGVpAs+981NH=QMS zNCKas#kE5NLwl&chnDQ7NzU!kz|dY=ru16FWUb|aVS26oeMbXjk7K9HX+8YZnQ+Ko zf!xl4p2Y4iqETpHB0qFF4pu!A7?}?Xu7|lThve1+jT<1ZO~7Rb^n58ay#un{1?k=h z8Qw~Np@-?Op!#>veGQcV0gv!cQ2RG1y{>g59%U1P!Jjd<3~%M`uh8PdG>!g-c|%3^ zSjkWFDMjf&ShI}&i`gc0 zZ1=o@9zp&BDtVWtx!nuh@6s5z`#gjGSN1N240_X`V|_H6{06<{*8l82gWe`K=$*Hy z>@7&}UC>bU9_aRgJ&hguA8;vtc%A6HmrzQRn`ynYLS@zL4gt{T?xo6rI|TH=pEUN; zN=57U_x&m0t`)FhY76_&vCX z=CGIQ*r&xVPG~GdEFzI`ur4d&b&lampY6ai0<*`zf$<`+(d5!@>CMr7P?8JMf$6v* z60EF+fz?GAE=L%5Kp3(Tj9mk~Dbz!&_EN3LbNW*0fA}ikiWa zJJj+Nm2GfuMjmpA)(&vQT%AXJOje&`*w3XP%JN;%AwE~0LELZSX&q~4YR5*uEX#9% zp1FCu^=EgB4~vDVAP3E*KsJ@LL1kezZY?{N^Zdw{qL5yLA=D%|%`X+=qmJK(GJT<_)meRsgrz>tl=T4Zm;=Ta#*0)Z>3 zoSP`ftLSuI4c~hnEOQNQ;k9J(I=X__)75+dU5_!h@w74bU4Z8RcMS987PVI5QNWPzH zqbb}DVRX;}j?zlL1Y~2>%AHz#XMoch3h99Dr72WPukaZ3f!D04BHH8y&yfKhOM3o8 zZvbgx{6oZJ4TC#>n^t9AHf&nSuLf*M@P0_56k{bPhTlc~VxT1M%1WZW))VmbP=|KQ zh6o{Ai~aY|C5LHituFu%f-EQ2L!AQ0!xY!=dMNP>b=CSqnfC$TA-c4pf-ST;RY>+0D?RLGFb&9oXC?fE==d} zWIhEpI|a0+@TpoG2eR5Ykk!V4LE4aa8Eu?JZd)6Dbe#^aKq}CB%-|{n5`(8290Fda z5%uBlgg1yUdH_*><>9lv<4Ocr0^?47C_6&kEyypo*;?8Tm_ju>dI<;Dy%dl%=rXwI z%Ma5P&WkJEBuczMJDj~V3=nVyyvX^k+D~I;{4QsFFJ0}DP%io4H6nfcm=qE}#y63N zZ-%S5g(~?rn#3Q%pXYppmhqj`%pao&-$gOLTVs2HHu=u1Q-)QG;Rmu^PJ2}^hUL9^Uxe{f5!Av10bXw$z9GR)!rFntJi z-~+TD#39;+lk$G5%%Iqv?-m~3-?uf7Pk8*jn4u%W?dx*@x4rQ_0Qf$t=6jLO+y`KP z7QnurcJTvr6F*2F=7;D$ewZHMM*#8z)XR_2v;4WN$ag_0XCfr}0Odv6u^52zE}iy> zV|gyCkms^cer^zytt@v?7`pEvimviXThjf05Ch|epeh-}q z@y_MLdfSq4OoSE5K~GRM&)K3Ns8mv-q2T!~sseBLO-zTbjcw%jcEGARLU-fjIIl&{-k&%`pRDy9q1`PrLV#Qk?GZeG>JWXp)_a6* zX&LVhcp2k9BhlS9S%d_b?DL4~FMzEVvlx*YPNo7=qjfK4S)&Cg`l{gte726nlj$&0 z91mnS1fgjW%rcWM+oKj^l${ROV+Z(=oGo2WVUGB!M?=U&_BPFUo<@j<@9Cv|ibCV1 zo-`vb^I z4smVv!C6zm)AapRjeAfB3n=`YEHfX>&J**WF&>p5#6+7Ruxm%|6-Ysde?VTw;gEkw zBl*V^;GfWF{uz>^UjTQ%1Z;l=ulH-DKEHw2`z(CzDGsmG1uFV%u*Wv}-H z`1PWIcoB~0HfZfA{|2Q0CY<0g#Fw{d3BOI}hre#jc(Vy$ eRsNNfKXI6z`~ua8{c~PDuNtS273foZ?*9QO$Hj60 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/RequestCompletionHandler.class b/out/production/clients/org/apache/kafka/clients/RequestCompletionHandler.class new file mode 100644 index 0000000000000000000000000000000000000000..ce0fd8af36d252cf2c77ca3d364fd0f22a196103 GIT binary patch literal 222 zcmZ{fu?oU47=$miw(8^yxGRW3aC0n(gCJCJkMXy*jY(}%AI`xC@S((1T)W)e@_lgc z&+`Rfg<*_1#(-gDbYgCmB1a7K>CV&{7nP7#;-$Ehf=iW?c9tifgc5PcgvapJmh8z9h9C?5slw8TUzr$kjsR3uQ;9$X?toHoWy-TI?)>`MGA zNF|Cupq}|rh*>8hso;{ewCmk@GjHb2{QCXvCxC6dC_u)BiYIugU|GR46&fmHGxAte z$!G{fH4j~!n+moVWUIH|W~g<-XkhX)ZXY}5DL*>prtP^-5D!f|42FsCL}okmeK$V* zc;+-1vb$qPWuv>>dADaVnD6I}kvr#J;tUz;6=QB^xgCCR81#-CIQxm;ccMey_vl4Q zqaGYFysC6g_&GN{9t=z?j@)3-Fy>9N)Cq0wb$R59@DmO>;6726{*BL&Yr26O?=Y1A zL;uF;lIHQTOH~&vcM$M6i71j(5r%w|L?1i1WM18Yj%oKCbKXL4s z(t9~8Put_eVPgjNw_VqxHd=KWU5(~lYOM%zfKK8EmR3bcWRX!o2{%B@bbCanSn3Gu z3o=)bKGBWg7VWvzrW3x5+Y|gYGU=;YlB-{ly@D+8nQQnv^rDSXSix#)Cp8Eq80(Pe z1C;9K+9h(IQ}Zm{)Lp?nkvi_Dn04gw0PB-P-cliA)xJaSl@#d``3n@%iz=s=h3pS9 jNh6x1o;<>Wbpa;W#>J_W4yi)v>GU^dct~=HVjlkiG$!`S literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/Consumer.class b/out/production/clients/org/apache/kafka/clients/consumer/Consumer.class new file mode 100644 index 0000000000000000000000000000000000000000..b292fcd4307e32b88754b2eff60b558cac9cc2af GIT binary patch literal 2045 zcmc&#+iuf95S?w)q)iBExrF<@fYX#3B*X)bB1A+b(u5)sMMy|U8+RMFcD$A^Ab86^ z@x%ikz(*lw*KTlbk_$*YM4OqNIWuQ>=lI9ZZ{GpnF+9w{gBp5LoN*fkT7PV+4%p{aQbS!J~ zzO$dG#X^_Ber@k*VpRA0N6ZY2-W*?jHZ1aRp*JKp?b{41=Q|?wOt#BytX4c0;2lYi zz}BqpFOqBm>%WgA>I=VK%9X5`B|u;af6MskD`yB~U oVBSr*HMVb~eJ8QMnHj;|81G(;cONu`SCK1nix_S}Rax7=0A`7DWB>pF literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerCommitCallback.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerCommitCallback.class new file mode 100644 index 0000000000000000000000000000000000000000..3f32f950207f5c61f0fb8d469d68392924d4fe13 GIT binary patch literal 342 zcmZ{gy-ou$5QN7eMR_&~9)O2Jm`j8B z>1K*=W_I5{Uf%#*zii`^qKvQR#)T zx$wMZzwMoldAT#GCTmxuy4lxuft%m$)@Yx$dBp*ffAa!o!}Y_t537wVUkrbsFP0&2WSHT literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerConfig.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerConfig.class new file mode 100644 index 0000000000000000000000000000000000000000..375d64ce9fa9c8e98524ca4213684dccae0183ff GIT binary patch literal 12655 zcmc&*37i|%dH>!X*6dowwiesifIY+p*2ijna@dPO*2=c*JiWAEZYaeoUhe z#pus_sfW(+Uk~-tZ8BvrN5?+#ONso&S>;EJ@hm^ z(?fqte;1=?HTruV|ENYEyCuU)1PJ8hx2(zS2uyrT@_AYZ|?z(SK_6 zUmCrv(SLW+G9r!sN29NIQU^c&uSVbKBzg-TU(x8B8ojE~|7r9sjlSIt?eKrwdj$!h zi4IM4hGQj7bZMen6N@wv(}bpp9!>OOYq2;c;+j~ZiHkI`R1?cIv0M`gP4sEvVoj{j z#7a%9>Ll%kJ2Y`gOsp2PC_O&5KYKvXrV+=j3>tN#IAso=G|DH9!J<>EIW~UmvQ-&O z%b#JhyhqT|0~6y%k4$EVC&T55kvol3#$eU3D}y=Twd@MemSBzN8@7MUs5VSN@#eDO z@wA}TxlAsX9Uq&_XGb&RNAr`TxhB1p^UTRbm+TuK&*$}>OQjD(by1O4O=R*D zC+3@32cagi>D**4HF^XNb38jXJbpa7c1?&ec62nrYAJWEnuLvxj8DK}CetIZeMI~H z;3{`?6f+Zw`7N*+lY^?GD@Xk(=%Pm zs9J9_-Mpi6ilBS?Rr+Z7R&{XHsP8#%G7484zJ=5jrFKDUHlDMupsuu2LaJIiV%g?c zqgDW$Mxlz4#E4Tgs>cj)4>%Z%spRPK0+|4t5@6({ITK}2(9R1HOVILDoIcx0_1fmN z(jN9++0NQbJ!EfJJ5xgnj3(*G$avvSv*_>Hi^P5H1u#0aiB~Sk`9|?1Oehr;6Ke$Z z}Bx3it0HJu;MV~Avb%0h5# zpJ&&qW!k>SosHM1nJ)J>@l+PCtF^JSN$QNDVDp9yGiuuo$Oz(*uaR zxVV(wBWUZ~uI{yk7qkp{D_M%<>6o}IE-n{q<6@oAp=_Cs+AsuN++rl+L*k0Kc!O9U zr!rN*nAi{#SH{Iv;_A5Q7wcnUV_a+!n+5gH<#PVs6A~93$M<~Ks3%biPMNM36I+f>=d+RF1bMHv?QCBkTSL>I!Y+5yHay{L5^oD2Z zWyjS`bg~8DO`*5d{V7vtTcMKaMaQ<0I8l!noo5bn`>YqM4bL~-q@KqRObeIOr>$yL zuNfyzNfsL#fCD*n%XLk!&ZJJ6%*^zZRh(+h8m_7HtW~zml0IbuWVV{>sDrPoY9)BV z9(MddB)dAJTQzj`K7@%d&4PN&M2VR$SzghBE+tC2A$gMe2v}noLXzH;)2Np80xLk_ zJ8n%MDxz%eWvbhDv;MYi;pgr0dHa@RGC72314dHMS$5Inh0tp#Im>n@w51qgv7)@T zD|jxifui-A$>kabs_XqqcQs>1FHYf1V(O(C+o)Nn;MJLpTR=V($2Oq?+ws|qKB8BN zG*<1On49BA~K1D1V|)*ahqdQPJsAT=!nRg+19vh*>2f_#K)5+pFsqJT zVQ&o##n}tR2d4LQ*SxD?xe(GSV;MX#qy*!8x@lodsasL58MaX|YwWD$hArJ1-5N+0 zW?&umePwE|QSgecRWSQ~r*0KD2KLnh+sP0ZC_on_xN_Mup|Tcbj$hxb%t)ym& z+$e5}ixg*{_c5zw7x7*~@WJb3Fth5LyXmdn*yvIh%Iu7$qC({g&ORzY;Twg9A0-06 z3f$emIlY4LY1-D3=k8*tlR2h^UvTWRQx2oH)J)$f8NQLM8FwdPeyRTq>wkY-92D8O zIK+}Y%96pcEQgIM&A}dDLsGLm6PuSj-6$jH1*vOVrYJ$J&7Cl9 z?%k6{1CheF8{gC#9x6+2}B4NCJ2>@F!Ijth%bnNI}To&NT03 z=vAk}7SN4)y^04}fMv9)T~KDKjgqp8;&s47QVnloxH63BrKwbo!94zbw1I=Yh4%v~Y$aPH|d2FSMkbv*_U>J4EaXFiUIj(ga z9dH7~Z|GmjFzcJdJ$(~&d>$r4+o>Au;bg7XxR$b}eI4P0X9N8JdQXhN2rypAPz`w( z@vh5N5vxiy%K$Q~FrkHd?0Sm^`$r=|LDEYaG~UJ1#hYiwq4GA zB@2ym8J(E2DP{8PzN{zdnx>vlA5r`(JiR|RKA9UE#>Lb|-jZ{$EGA5{X1v5K-al`ah{<6L1?RfvLKn`cfNa0F$1*saz;cPdvwM@F&Os5PpnWbiwn zz|(AR8l{4*)}o`rp=6egMwPR@?=u3-unqkM*49icQ*P15ZjkgONkk32*mKzC7>6b8 z3Y=1|4@0v}j-VxJBPfd(9K3;`gH+4~3)U3Jb=k0bF5plmF!pi>)k~fLDlipSI_cuu zROA*7S*}}Vz>22Mq{AZEWe(wr(%P)0JJq(b@1tpV(UOM-^`}r!DpPgvYv=`-0R~{b z7SUxhEoMx|{hDlMkYm^qs45%*fe#^lFfX$7JRrSjPFCaoN>UATJD*kxRR;k$uTk4I ziqZp)V0p`vrTiS%f-bCD%`92q3`N^R!uHBDayxc1w5YP_tU&KJH=6<5q3c5ujyHAI zN^*%sO@=TrF~+rfT+p*DR4YX*VFCk$?4wNA>eAX^r%8^3+|bp|d~9Jal^*0+=xAKC z&Zg#OF4$f=H7{XgF5Cv(sWOFR>p1mbaloOfPTa%;AKDD!iAyjl3B*)J44Qi&F|7I* zNW2UI?9N1_74(kpUP_PvhukG|!Qtu_vJn+rw(*hNIj;`Xa>zem*rfp`$+|dK18_Ro z^{B2XiN??L@v4b^(!+yh8ZM5*{McpV_$0*B$GH$d`V#EL4F$9y9SJ8hP=`~)rGp*} zo2ga@;i_CtFE(&;mW5d!G)fNi%||-(Za&s?S16Bbd_Zr-X1|%NB;nU#(1HOO{ZmLP z1#Ux2W)t;+h2mr=_BqWCd|3B#Tw_j&X5>553v*-p_>G56Vv2Hmcs> zm@{5a2WQrakZw94D`-DX&q!AToGsDbmC!h-9oe1CyUik>nWRrO+X9r0@T`YJw7ecb zto7o>C-dnR(VKrdh5ou_U^yk72QOfDY==Q4zzM`fh6uPW1)+~-Mjl6@e#ahw%MDB; zyYXoRAPy@hvA}g3ZDwnBP7aCzyg8($c1B|7ipM8O+|o>;N!S9A+cLh8K%)n$T(ad6 zgfAdCznHQoH}Cd2vvRYF0*8Yzi~vd!ll)!dGgP1tH9PpBI; z$jWC&#erVr^3L}HNQ0(+sCIX_6F0&}b=sK0fhIg*Kq0PKL9RhRaA@=v^sHRmkRABY znTx)-Q9Q`t!!0YE2%MnOD*;t&uEdKjnj;*j9*rVIsLH$9{|DVw@a5n()04LWRtfIG z$JLwDRX)uz+^mBL%izGziM8c4Z8rj~OG{bqpg-(^FA5;Vz}!U8cJG@i@a;&^NBm;} z-Y&InnQ_&Pk^(18RjX|mH0m`L-?W*2nv)KEvp;x+Aa8KwAny1(u~^V`!Ck(b3SNOl zF7Jbp==xv?FZf!wLcvBc>^X)DAM`41Tju1&9`+pSpXY5vo5)I3{>-BN)q{Mc5xs|+ zfxFcp1zK1^D-^Qj3?4zt?PEXFsA0dwzH`v8RfD&ceUjmJBtfw{FR9v~64gb(Grx#< zJu@|V-jK(;M^tt76VJkxT&WryrKxKqo68FtT(AZtH3+feb-bjtJ#q!o2In~B^C-}D zFni#jpj`{!yR{k|EW(KesZY>@^Jr+{G+8a@e+gDR=R2DX5f&mI8QNvR+cN+7ASFA7 zuY8{T9`N7t`Y>EBz&?R@N3K-yns1?WAW9gT^baStaqGkgakz4HCF zT9cFSi|=-gjI-w^7i_Z$b!Xol{KgD_gzvxv4M)|ezU}xfoNj}p$F4UrejILw@6bjv zsj)UE+~+vBZ-ZZ~?7ubYdcimrU)1v05!~5xZ8{Xi$@OxK0t)?lm@TTcbJa^Eg7(W?-*J8dJfpyH+BJg_5+YxvLU?&1^2Yx*Q@520D z5%?O+yAk+W%zF`d1Lpk*yb1G-2z&(dry_73^V1PHjrqGH@G$0QB5(%tZ;HVCG5>=R zcnjbkiom;p|HBb@2=i}_z&8N?kqCSf;I~BJ>j3{~1U>-x$0G0u;2)2`w*mf%2z)!> zpNzn71pHGG_$c`M=?Hua=I@EXCoumr5%^Zj|7-;A2K;jo_zu87AAvQ%Z;ikP;9rQq z1?t8ZxW7oh1Yi1Pc+^J;pTgpI$0H<8oS}}fflUulX9sP5l9ml5x;LMuMGsM|la8wa zZA)`t-~(9JMZbc-*HR4cLiQ7M15j2>0%3Q5CgPLBR9r%uRGyNuJJL$JD+QNSuqf7kzoA~!P@!tmg-^P3ob-YS?jNZPCeg|L& z{jQYr_X0T|YsuLIIeQ`JX<7`SmIqRvrub=E!eU}XiW!W|k#i^H+(k>oHMCq@OY6jS zv|d~fId`|o`TO(^X8SyHz7w-b&Xr(rKIxh=t&7ngEDvOQ7f|0#?+Ikefdnt^*)&TR zZ9YRw{}e!{v`8;ch`r$SM#+`hrH9^2?*k5N>kpx6AvGPBTIG>H!jn?o`+@Pt^e2GX zN%nvmTWZNHEyGr)Y58eN@May6&Gu1Tq~&Ic;1c=(F^)#F^nfDN@d_YbmUx8vV1}MQMJN88rNLR+n%Kq@+h=J<*IC-hn7f$LHNafncTYR& zwTyaQz{S?Y^^CY1TqO2H@m3{<81Dw2*c+YLn7ENAZsLhl`@}w;NV7m)_%y!7-m1hf z&t!N;yd0g`n%K`X2f*jWbnq-4>cO8tq{BQjB4& literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.class new file mode 100644 index 0000000000000000000000000000000000000000..18b4025af95f31fd61833327944c86b3fa459eb3 GIT binary patch literal 444 zcmb7>F;BxV5QXnbX=q_Uf|(gQ;Ds$!B~-yw3DhF?i#dWD$F6LrKaPPPz>h*)B0!bs z0NcGg>(hJB?;o#k0B{9I8H_SGL^$>Co)Jc@ z;MYd~q{*k9gOlxm3*Pvu%{N;5CTPAjR*ao}2U&1ImYn7pSzC~LfcwOkIm%kUH3)`@5U literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecord.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecord.class new file mode 100644 index 0000000000000000000000000000000000000000..5afdf7d2595c8645cb81913f7783c91126134957 GIT binary patch literal 2181 zcmb7E?Q+{x5Ixr)vg|r-QPYOdw5Ul*Y{!Y4QfNusv}tI-iPJ)C4B=Z*6jzNZc@()r z-++gK8JMY^nRW(#GQ+#@3c#+UIR1i3W<2+v-MjbP-90P+`S+{e0Nlb(3YT#|ffX4K zQixzRiMQ|}FAsTHK2D&NLJId~tnA`(RwvN7I~Diqia@8b4|8GXUuzI9thklJ8r$8H8lN+QP|h^ z_O*g;nTG8h6m-WvXg)LC!fFV%4Bc^eDd9`KgfALLq>HC!D^fqIwWVcZ9jSPlzQ3t8 z{0x0XzVt=8GPEthMJ{|alkMtAN3LilOCt3ib~K4+vVjt`vj!AQBFdfOQ+S{jp>N@E&R0%OO_K}`>urnPIhbkdtn z#-DlhuA0U-_?8*p7dV?&JJaqTUBzua|4!gcUJVA{_p4Z9|M%3qyh`6d$fHpU@~S`P z6PeQB zbcL<-kJtb2xaYNShWLx|ZwF!CCaZ6$h$*zdX77UH>y7Np#@781GfcnflFTxuctE8U1; zmey$`X`LeLI^^)RG0aeI8OkP0D2F+U%v01dBwFXmo~NA=S0H@EHAH;GRYZNnb;Nzd z6-X!$R>QM7+#riNCh66qs2Pe{AnzE&KUBDsz|92S6|n?v(T@&-ro2aw(hSqeEm)zQ zlZfYDBK`}3s9%}nI~V-S(?LL(Ip=Zg;XDLsaGTbCI_alAx;rPNo5Y<^pziGC{_O6Z zh%PAl1IllDC~dNzcI_0ji$qIbc&GRXp`e$EpZkj6JP|GZv38{Sk+fjC@@u5F4{=6G zw=t4y<1B5;xd5MU;|)a#@P#%;m5YpDLT=-dG8W(@<2MPnG2UP7R##T#0fpJa3P-}x Lxl2>b*1dlLq!y{v literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable$1.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable$1.class new file mode 100644 index 0000000000000000000000000000000000000000..d2b830a226396cfb120fad0109dd97947626f304 GIT binary patch literal 2079 zcmb_dTT|0O6#kZ80&S2ASQWfe5sH$cC@K|15Jd_)!tfxD<2GGNAWfJgb@Zp`s}H{F zjCgx+c=Ev?<#;wp0j1+ig);2r?4I*|XZJgspTE9+2QZ9#EojBX1P62xZ;JPkcdMt7|hGz#?vAgtFyBktqaASWkp{H<_yFv93r6*MBmS0K= zH_a*ABJB({d@MLaAAm#j`H0XzsRs;)Y1D*vYDq}j^UhRCiY?sPg>G6oL9-I=ZKA#_ zJFp3s{fwq;_9NIAM7|=*u!-0Q-{&N?34iA@eU_f$6wT9tCr%R!`Rno#7|zmIrTEek ztTCxN+AR^^QZLsEgcJXPHraiJ!te$fjbr}!HM%7anlv+Pp(P{}e}`oD6B1iUz7PET z<>Y&c7lq`QMvU6$lshNvB1R~~ajf`!15SmI(+8x96PQTyq C!ZG&% literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable.class new file mode 100644 index 0000000000000000000000000000000000000000..8f63aaeb171bc237d554c1a618bf80150c0e6001 GIT binary patch literal 1709 zcmcIk>yFYu7(G)i1!Q*-k@fDXs9cO?6MqTe4{_rL;$;^x#>8YOBU`rHZrWn>9efx~ zG$xw(06vuQo3>n5>klGA+L`(0n{(!zeltIQe*F&M9`2-(z_kpLxUS+x1~HUW+){B{ zMVX<{wk@4|!lGxm>j{SsOu>-dwJhPZP42qFWk?(04D}1$F@M7~lUpNgx}ZTR`^Jbs zCJwD0MWi%qQNgb5j5L1C^#h?D^4Eu4(@jHIo~!A$4EuI2lnlWql+Cwo>h5-7fy(KSDw!-GB}mRIx1Am%O0|5gG$>Hud1TPu(KLShTQxz z2~1_k&qGtY7-D5va-G$EY(m>!S_HP}E=__#r6(^frZ!90tb#SVSQ+`VLH!&>2GlCl zW~yJHRBNBXKGGc+&z_ho&gRx;%$w9Zhw~xt z1AiYt`Ws>|h&%p)mISnNOwAsi?~F&Qj$ikL`YOWfk{}! kR};WKU4TvCvacc*!d$^sKYNSRg7Oq{@g`EFQ2Ic>U$GtCH~;_u literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords.class new file mode 100644 index 0000000000000000000000000000000000000000..174b0e512758573fe8319fa56150cf88a3fd32f5 GIT binary patch literal 4221 zcmcgwYjYE26n@?`?KbIBXep%#)QW(;LFA^=#3B}|wP`7|RtgGi(AsxT2;ADh8Peu7~TEQC%-V|u) z$(kw6)=hpU`fT0O#xuHr+Gm=&)sxY3IXx#(xpQD>WK3XFGHaz{+LV?$qQ{PDhmUEo zRL0OvI~Pl3&0Ky`w_-Ht-0aa)S!*H}7l;fSY1+?QG(2|i781#$+6gU|(adyg*Z5IA zWyj}Ay6H!;T$ch1U4^`DWMTu_6xYs_OA-NLWG5%HW^5!oWu%5Q%QkEyYkHm>LrEiN z2kpB%0Evy9b*qMk{dkjsSyF{~sWeuynTuWD@AR&Bu%a zn zI<8D*^QJ9O(c0E0Aox*?+|J1H)Ei4rlXLO7At~A9}D|lPQ zJ9t;1ww%-o-c#{DJ`h+{8n=C!jGopq+pRQ5yJ_z{nbN1^SQJ?5%*mF?e9mqe*IUf2 zxz5aIGV7(M4+R#NjBmFr?Ubxb6=(2~f{#_ah&~p9inG|I;uD-J(=Rp+73byB+PUNi z>H;dB#{~tSs`v~SReX+fD!#x)f!?|M?_NN;)j{t>GO;gJe1)$Se52x9BvlN`_}5^q zz=nHvqT)NWOQY*JU`mL)G-jJDi3%(!2+Qi1v!etxp<{KWb=KWle}D06!{5O@k-(DD z_MR+{J0;JTLoec#r#v_LuHtVU-UwXc_fVJ-ERJetAPmbO>dN*}Bu8DzpQEE3(Q>hoLLu7X5f$%P^O+PzzwN1y$4LsFumIY$~*g z39{OsQvrA9&8+H#)T6DYLU_(Glptej4W-mda^{sYPSZogd51nYuziR>1)BL`r6Gi6 zl0dwFg1-|&XhprO(&BRms(DY6aBV#`#(53#8f~A3Xz%zHp`Z9vff&DQc?BNlOaE&A zj#z^y`PArYV*`E2e^Jl+#X<=A2!A0w(0Luo zZ>YS|d4+3XCyIJr!FX#KZ#^ZA^erQ4arYM^X~I)@nm%e7(=*sg`=ywgs3l`^*SfeC z=M$0m2aO669#+pf5q7%~3WfG7sg>@yjs>%*8mm{PQT^vEB4g8t4#>D?P$Mt{RbX)4 z3>FAn!Y1C-3g)t~E4(Z`gGKVb?i%Vlu3_;d)OAW>gWR_;~aV4DL#yQ8s+Pum@c(nToJY+XG}rV+b+P)NYGk|Nu% zqo^-IakX}Ope#bT_LhT+g5E7RDRqhT0`0NQy5L}lwgmS!$nae#ngp1}H7t{vxCE+l zbapt=s;<5lqdc0DL`LCyd5K#6Nb*~f^9Op9hrh8RQS>o`M+62sJ-CQ zp^iN1y~*+;2e8YFyO&pqPSlNhg@BC5kIAGDljs~wq(pj&5a-<<;wv1B%<3vuN`!?| z94^9^hu$7CXBZ<~ljyB=A|~l)(L8n)k4~fIXRMUv?cdu>Ef@QjG0HWvk~o2-79um6 PS^iP}x4=F;R~7jWv;9(t literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerWakeupException.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerWakeupException.class new file mode 100644 index 0000000000000000000000000000000000000000..97941d8f3a5bf1422ddeac0a6850bab50b006a8b GIT binary patch literal 448 zcmbVJyH3O~5S+~=7ZGwCcR+(^Xb@1~h6Wv?IY0s;8X(lZU^tV+j`BeK78Mc&AHYW; zHW8vhG`JY=jJ!Le^~dMiJAi$x1?XYyW5vfRVYyPJmRYRIN?UVtc1)mi!eDAmRZCOH zGHaB_jqM>q|4xwXm zZ22H$A(Ochsk~36NHVQVU5SKkn_QKm^>2$?nX0BZc}`SObHE89xUx-|s8ikAZ2a+V z?@>O<5DN$h2mf{>Z2j)Fd2WrkXk-7x!X_NvDsKquaN+UYXXe1?irN^Ai>!!_1K#c{ b=%srSGWJ;pMi?-LUF~5BLuUL#j1ER$D-dyd literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$1.class b/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$1.class new file mode 100644 index 0000000000000000000000000000000000000000..d8bd6f8c7651502b701514de11c57da8d042d8f9 GIT binary patch literal 1403 zcmb7ET~8B16g|@|UAiu`prQy0YSmJZZAC;=H6c|K5%B>e@p0IW?b7Wm*^d%_mPXL{ z;1BRe8SgCHw0&y3$WZ z0TI@Zd56;m-b&l`0!KDJim>U`b*!0qjCm7Nm^N_*rimQZbyz0KsF-+y4U^7qQ^zwC zTX;_Tk12nl!jM<&mdj zkt?>N(5rdvwi8}LX+?FoYr_GPAX1-UVtfC~UX5XWg!AmhsVcN%YLal2D(nkYNgCX? zML;*8On0C&3c!E})s+#l?Frt!J9{EL_54;XzF@S4?cq`l0b!Wzo(99KQTEXT>??yR zl|7c}J;{Q!K`NhD#S2hRl}v9=ivAq(G&@7qXJ|+(PigfWth9WNROu(QZ{*T2OKpZ0 zLYu->vgeZEYhy$a6Wza1-+h#W7)MG`Cp#0rnl~3rr~M7c!+KGE)2k zdXH~ft<`ccv^*BHnLggd1e=g>E9UF*cIPxK5L1d)+@?NFPv{b^lVwmYlho7qAJ{r~ AX#fBK literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$2.class b/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$2.class new file mode 100644 index 0000000000000000000000000000000000000000..aa1f4153808f395d229942cab4068c4071df4c6b GIT binary patch literal 1802 zcmbtU>rN9v6#k~IU0N0?7ZFsvRxJftZpI%)qG>Q8)=L2SXX11_l%d<1y1T{r3cigA z#OMznz=txPX$n}o&=Q+;=IqRz^L^hrXXe-MA3p)C;Bg8`+%rQq4GZ@N@c<9akh3sj zA#Y)h!9L(#*5#g8=5CGQX-WH4hc~#pC!8AJt#QZoq)=htxLO6xy6~N1+t?Ik?s4Uc zV%xpIkPP=^kX>L{>?6=i6aP4xabA^5hHDHTa~JmR!{HdIoA=BPL!ziFf?>EMm3ZB( zmxaI0%O0(amNeyOhx^iew-*yeCuMI#Dd87A9t46um-}GRAv(K6vL`wLJ~xg*#za%V}GmN6bHIy#HP1&(|5%S*(yax^5%^?s@S@6 zJsn6@eI>#@U9m84!@+`$LD)7@NZS~~qJ?cl-{f3|Bt+yb%%UOF!4@oLdsn zAOF4$eJZ5}iswwx3%$jiNr++nB7zE$P2ajTPkOV66o%XMxLRNsKrE9ntp?CceMb*) zoc=?&OuNHmy+f?EGMt}10?W@GA(sD!_}A9K42?-Th{Su0kUc?%L2jcMr`cF0Bo&ry zGoS9!CC61dZ6I6%X!+$2Dfl7S>~|#MIE6@??ZgLMPVh5eCNb59>83D=YqWQrfMU2o s*7Ui+m%4!k0q_1da2z*V6mC)SNe%AcF2=|jC0a?E&2Ip+c$7^20k*K}M*si- literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer.class b/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer.class new file mode 100644 index 0000000000000000000000000000000000000000..2535a7a1d77573dd9788290b0b75332d98662ccd GIT binary patch literal 25520 zcmd6P34B!5+4pnKo#aj?7l8o+VI39`va*Tc%KAXe>a)$Uqtt*u?GwfpW;7vBGK&YhV%8w9?-Z$E!>mvhfK&w8Hi zoXOMgJ@Ob4EikqfktLq+;qnzzeAP#W_!?in&Mn{I%QyM;Ngur_zQtXiGR4!Tc&6a6 z^Pi83#kcvxcTDkJAB_~>F7>T-Vj0bhRT70(un=fsb^;(1fNz_b3?6hHCN zBt7wqUhz|Ie2F{%%qxD*FE9J(Y4Hng`K2j-<)f3tx4h!l{N)v&_>Fj#TYl>muN8^k ziQn_fAH3p^{P|B_@n;`hE&gJPzxrqa5BnS6|ITy$gYo<)zy6CauY1KCrg)Qwy=97j z`)Ij%yI8y<-ZjO0K3XLXV~ixP6nvMaG`!N|qtj#ozZZI?iILLllSR_!lf}~FZ^OLu z1XB+8(H4M~BZ}on={MylQ;zn^F+SQROT2QdSB~?_QXe(T@ur;Mqci11ubc$5Z4XUjp0l;oaG~1&gM^LURiF+3Vy6)RH}-pM^^LG9Iu?q z&GSq--zyh*z`RDN7$%H>|U!bca&mHfVnk*M{_ z)p8BLuJy{(ymFmS*2(n{sNBGxH!{)nUfIAen@qXcM~})aUb&Uu8@;lL@6D#%=9Sxh zbR8HhPnRt`-5JcA9X$MfjMkaVhWDFtr;l!j76Qlz_!7jcY-N($c&2uKviYflFP%Km zE*>5-M%-Q3*cm0_QZh}W5~y^QeLJUGhZV%#C_m3>e%c@DSi_Q^eRZ?QaA zo>wf-mlyEmLQ`Jkl@}L5B$sfX4|?UL{Bjw;e8?+5%=eEle=g^zkMi_a@a1Dn+LfmK zxGAsl%B!Jk@)JIKM!XHJlAq)O*RaBTiiy9L5x$N|xZW#o;CXKJ%A5FpGk>|oD{p1o zZ!_iXK6;6j>JHYpPkZH^UU`>S-tCq5;Dh{(SAN!%`+W4Oyq9O&Wy;Ta<$XT-9fP&x z=Y8@E@_xR25ldHoi7yYBa=%wT=%c@JpONw*zC6rd4)|oh95CgeDGv&=>cU|=TH76r z#q5|MqdU?m$iFcX?W~Sc3Agn{qjor69gIhMLT%M6)vYcZw>8Y-wn!w}9tz`gRM29DRb(QvRkR$ZGK3hak>*?U*pF*_Oxc8AWjG24P1vqgG(BH`*p zuV6fcH}@z^RMZ&i1U`LHkeUmjqfN22X~U9b%+7O|UGB+`cfM_EKC{D)gD)U#VGhFb zvuUj!28>QUo^ec0lg7Xk5A}d4GjdF$nZiPkRX3p*pcQJ-398M}KR>#xY%J&oJH9&- z?W$EhF`e0C$D^S(sPOC@^J(}_m+E@m6{Cj5`dVXc(NHf-8e^B|7@HrLM%UoRcrean zc>yHY9*hSCP0v3WFi@Q^;Dji!*}E#())ndKs0R+dU|&2^3yec?EU+^KjZQW2%7{X7 zxE~l8iP`NS!dXO3yP|dw@3V%m#5P94os4bN?r4kjg#l?q)IPhTC#k-oTsurw47 z#h1YzxTmZS_AZ&dO^~NH(hhMC-xvzp4ShYWcC;zj+Km=ikG5d&F8aZi%3wHf?bnq!h&f{AT)c&uDjRn{!Z-hg@4b+|M}+PducO3ijP;Fx@)BV-PD zhao^}>w?o(@c^m~s7#<_fE*V^0OXdYe8iN8R66}Aa#lgF96@SHre|;^2MhjJ7uA2j zc8r`g+a-cVoArE2W?O#QHbr_vZJUGe9vs0c2^BdpZ7V{tl36=k%IAoK9Z*4>2SNwc z9qVOOj*h0-DX0QdWG^4BSsBoIVj1tvz2I_=K(yPNKpsfU9x^CXfV$#7tO*srGp$9* znGl=yM!LHNojhxOu52K`PP8H0j6nhgNNA&2x@Oc?O-PLGFf14hR8J5=aTu%Co*uv!+cVtwPG3zoV`DMpNZgLXXYG&IeM3NlxgLSFcLr2?YTX$a! z=2_6xygE^aeH64y)v0wH3ef`P!!+EfXJH}xa=HObkgn4WeG8q*On%W^@arpGC{3W*= zKEh0b>Rc>6b{$o@kIEU@mYiMS!H*I!rHVK zuq(lmb7x<*FpWs`*~TV;s-#0EWi~3X zPNgZoMo5Pxw-Q+}3iPQ(p$;f@GEn(}c$^+SL%^eJI;yWy8rlml&ee>S(2 zgLIR0+~~7ih1&JSVB8Cl?}$sKw24;Lv@yd zyC$i_cef6sCBMF9gwL!3OWi)=*ajf*2&Y&*Lz#G4h~xT+6xB95@O z+YYj-`GRe2$f#$`nKMVwIYW^_TG8~F`L5-cnRUxOfbLz2N`bC7_b%N~qZLuDEvLk~ zEl1n3>~1-%Q$9U%g}%PVNME$gUK>&s>@*^)c$HgXxENuHfT$ESCp#a?2SKOnmi&tR zDi5B@my`LjfZM*tPxn}2f+!UI-$ZbIyinK=-V)RqzM`PFeo;`_L6f4`4C;?gWxAHYX zCmn5Qz23p&VZXD)NZ|+KY9FnN_dgKWvHZkXmfFIv?t3U=CR zctHN%5@SRO99C;25|5!A*jt6Y;d4-*wB#SeEvEdVCI2M>s2yc5Oa4Rt6BZyZs%ZxE2rB?3JKhzoZiIbrQ+wuP zflOG(tuaMis2xjzxf788f~VsGhw9+s90-n~+)>rq*MZ2Os&#Lij|Om8J9Lhn*38qq zE@)+5%oDsw+nCpCm_`&&^acQ{#i%25=wQr_Tk=i$iYXtpAZ^ z#23Vu7?F1@A%y|W#nFVS_O!DJ^RjW%OD_M>1WR~CfhFG+H(K&NdDt?DEwNvnGDw@< z5?)atXnr<`OedBKn4Cibh<4N|gQ}iTIH49Jofa&d-N8^?F`$%}QHlw=hoeS5RASn? zs@kG$tb+!;i%%3=hGajuTVNBQB;Bh7+OVswGMG^*%a)8bEpfN_Y|fRV`K9J^#lzFq z2I1k`1M!G*Tk&XLTRfobXwr=_R)!_|fso-5Ev8Xm8HI*vi9vCXpz)4QwdzS!`E*un zc1_^C3sC&52$+V~GKvfzJfTr+i5>Du3zNN1(3JFnN$$j)(dcUogEFLu)LD`x&Sbkh zxfzSbsdcsoI--#t(0~$da&#w)dwO=S9Dq6*(6khwI%Kw;!S0pOPGlWnclt2VISml# zLESG9j>H44b|8$-aTo&AIKeW8vwMt+m}QJGMp}m77-bowjWL!8u`K5vUCC;b-dDZu^6Ssc*~f;a+t_-PvWP^{4_=liF-UVPSKk3gBAfF;;Q}aie9dGHNYjwYY_Y z5UkoYjObdyo^Lbqa+YwKK*gQBjK*NAun*=i^W$B{FiU)fEye^+#d1f?;Ld5hY|9lB zRG0vNdE_J=CL-Ze15N;D8S9KX1RKeACkzPah!MvPl3(Cz0_m=1Lh_q(6&YVAH)VzS z0cNT`dj$NKv(8R-Syj$EGc4+Htk8M9LQ&e9l2Q-HN+AJ>+b(I(Y;x{4`qh) zj0HPvrCU9H-SJSR&b6U#I={Az%?PuNEqvK3_LxSaWi%PhY35#Mxq`}$z5sxu%8(Mv z#|bcFaqP3ER7u5|3y(gVUQ8KV-lJ3VqzasfCsNII*a*_+GN)w2(B9YC3xYk#?vX#( zaK&2L)fvv$+S}d7CYx|rPVb{%?k=8ig)>JfKqHnvj5@9{**j&|yHRl+-1%={p`y?=rV}SB6sJ*PYOhmgetD;GQI7tus*E~Xj5<+j zAv@(96rV8Tsz?N-sBj*zIQ?j%llBZpC?Tgx-^i@GQ6<@?&myC63L{d8DNxlwUYP?` zH)c-h19iBODRiAig#CzzQ}e>$nod>^`v$J&6|G~X)5-$?F)6i_+LAU$osMlr6$~e{ z>+3ddtZQ6@3Oiap8!Bw)s;g>4lv6icf>YZfc{Phs za1K=SqAr-VUhQnDC4-F%#E5zAu09*p%o89!KwuNI!DOg+h%4XDi62h;r#UQ1k?Xd0 z&bCcavOt2wGRJ^yG}0Ud>xnM8Bb?69ti;g*MWoToM=ugbZrycam%tK2$Zir8#R|!X zSGrhY`JIAG0xlSi_PgZB-iSrCD?fGVwHtLtEC+rWwJY~YWCv>9`Z{Ie&&~zP8LmSZ z^^%LWEFk&##b!lFMd5IWtY6)1p@F5yK$2_;M|V z{OBGrMpK$z6p4ngp^jhT=*lxrI;!8yjB=X?#!l^J7NSZP!7i1!bSSbtnt1B!>dwfq zcK%mlzZ`HJw$&$o;q_xKXJxzoW#Jy{fcawK6wrkLEcm5<*b9 zIPLtS;|*NMJ0>$bngF0&!J=zu!nH8B;Qdu~1~eVR$qkI0c4T5WxI-Y=6Y9n}1TNIK z1A-Zft?B8F<5)7HTSONHs=NxuAlAt@UBYwX%p`BIu2sMJ!dwWck2ZwW!9alnE_&Ab z9qT!sE{Jv8Huf+IY>mgloRBk(M{Rnc4;(JhyVWjRvLPJV4M-))LQ9Mf?5$Kqbk@aE zJtC=G`iqVVN24yWvb$T+=GwF$(-Xq*aA#)w35soY`_{w>PsD!NyvQUXE%#rde4aOP zwh)^WLzyDA)ZM2B)l?NnM?lvp^&vob+l`Uk32(dj$m=v)`CS@3OxMS>yM&(f7@wK7*mqrtrxRv3(NK*&MdjvPxxH>nX$+~^G@iwGr&P+fgb6c?Z+ zCgj>+@4>8|u)3uO^$rzC&(^HmiXWUdHo>rjZM)qSI|ypcB^hlu((9_Z+-{~4xMqd{ zQJXr-_!Vwk*`(n6La~(29)m@|YEEn(g6%oOoSZnj?HA|D;93#ue77BsU^z{9Yequq zE@5cHIt4$%z>eE4rxKoEV63jA0o>%(H%2X)%yC241c8P3?}`^)n27nM~dVzgASF`Ez=N8nX|s5S;s6C zOxHdXJiPi{Y5Ka;tq^Sic*BQ^*oJ_koJJ~NuCh^R^zcw3bH|s{SPh1^+fFW@Xfa~k zK9EST$C1+&kSArF`4iTRj@ zxnwDjMIeCDU@zj37Sxk&XZaOXz~G?QdaH{^wZP>G!0MF`)Ni$op^v|9-@=bVOF_+c0ZN% zQ+YpCl>007=Le~(pQ>vL%KdZtX>LuSe_oAQTG&tXZ=*?vXaUA8^q<^Miyov?`e|`L z)%4Snn!ODluTT0CVT5*t8O4<%|R<)EClzPh^rYQqddw^C0nKcPy z*5dnVE%>&!+`mpY4p7|zT3_zp&`%o&slKKNU)Prw4N^lt?l zzp=FNAT=4Z{SY;`9HMP4<^Jt@9t>N}Kudk4#RGJ@nx$|=d%3?we=ql+q3JL8@6Zne z^u8Jk2%XtdYH@cSjelzg==}#sl>2vrCH?e)K?>FkL$}tJQV6K+0JY=AW+Kq11AWT< zor#%u;T0d+@ozOFf;&%Hf`4nvDh4PtKxZAGuF5BAI@r?f?@?0hIY43b2tzax&q3-n zXdj(`&;4|EStY)edUXG2s(%dqW8B|x`a>zA+`rUwfMP%}?jYEgK<%8w!|uex9w-Zg z<*D$G>CZ$Qyx4ny&IO?J@O9cvR9uGp%mF%|HSz+bk^88mrl1UF=b}S&af| za-FBg*`06X^LfdVn1<*SdOngF-Kf4R={f%i_63+u~MwUUs?J*99FGZ zO~d3f_;X8Z#9B7I90r8A1(rq;vo09g|YQ*8G;UgvZ;g;Ao#aSCw|&7Nr@Z0)72;Qe&j zAbuE#*AJIJP|vrD$I-_Fp^QO4rK+&b7$bL*M+RxQY*S*>j%T7~smm2{3z(q@FC@lw zyMxIYbcxs@-skwgKLUmc2%8xbUOgWv_g_9pA3aD{7*t+g@z^rsWX~ASq`PQb#Td^# zEGb}HTF_4)yUZi@9exe3>#+{6WR&-jRa3|hrG;2%5;HHSwU~P?ty6bRMH#{mlDk30 zUK%aWrCIVknv44)c|I+d7gDXfh}Po1NnWfdWN$VLqCOL9#=*trg!D3}(b98bQ8d z#*cC{VLfdDm@No3`0i%Hm0-eEV8Yd4!Y9ClPts~Uub07#Pd+qU2yC2j(1Zod}TDV}`7s)|Tdl1wPg4##YsNLnr)=h1uY;#eoWIJ9{ zD{zqRENx^E-stKR56}&dPgym!=-OM9@tJava4K$5LD|#+ zx~ay5b~`QJQu83)Ea+w`N8`Dy_Wg8AKi!HafiUQ{Lv*`}vhJwyvFexl9++CIDt%Q6@ei!WhKKbMiz~bkqME-~-%IE1M`2tnRAJby_B5cXev5a1( z7WoTmmw%=%`D=>FS81>OEu7tJbd&r&-75b`cgjC2QLvjiiK3zlKB0iBs7rLirJ1yt zJ|KF~=A{z4MuhRi)_Z?aE%%EE+ENLV9@(qx3-?MW=u|2k{SFBjrX~-3hzDy@9)|Oh z@Hie~f8iMEvmrwqlXTq0lZqik^ zi%MLys!R#?(-%@{neu|hF!C9rXp%A7h1KLFR+GhEmIwK0v^ZCsmxdMh0#<;2zPP|) za0f78iF(V*D?~rtpMs`RWdk~3%yvQYCLwvng`6t$SaFfK81to)X70t1_}hiz52EU>ECAtHk5D*Z>rk z@Nc%d>*X<-70)=8JjQY=F;-B(SVc38S~>~$IXJV(IloSv=Ozi6nY<*W%FkAw}PZah-;bH-SEuZ-D7@)bpWo(0IQl$k)L4N)?)W zXe5n8Urok%Xa*VEK}HLWGA(;}MNYgQJI?6kIcT4zOu;zRkWn4fLjf)`Vi)lLU z6~-kl_EaVjtrYBGm_3!^3M>ydd$^a2J#9{)@G;@IPEiW>u=eg`{L3G^nTA0zIc(iW zX89lu)>j;)gMu0=57Hw7GM|lyLxP^5M{7Jx?w3nFDx~C5rD%DqpB~?L__+kdjtg*f zx`CPjs+qQ{yQa8`V|?Qyp!lOS+PDIW@G+WiTuIA~kJAR@Drzu3L0gSaQZw%FH?C0} zU>%W#!Ig1TII7kZSHDXpuBQ0JTU3>Z zFh2PoW788EZ2HRavFS#z>1MF$7O?47u<16i>2|Q`4zTG?u<0(a=}xfe?*BtJ32}}1 zl(QNRJKEI5+Qce>6ymGUDHWf5jpGxi@$2=dEKMqdhJ8bOH17FzRFGD&=1jdGW+d^L z2qSd#P(vG_LMv#Ex^JL*b=Rt-G7#g7;L!s#+}Ka0#)I&857A8HVX86?P>t~jtw5z~ zjgbn_rqOJ1g}7F!MN?9Fn)H%#t$qHMP#LV5(syX4DXu$A@FOa#yk2>Eu;GRTGej+P zlb6 zPzI)b2Q{(*`tIX7ImWr#S=0+8dMT>znp5*QfHb}hPJIuY`aZPj2QW21gqeAks*UGq ziSYuhHD09k#!tyMenw{*FDs4WdhR%sQf^QjTY{nhdr1RXV42efa+6%#>*69WLEZln z^w+%-{2Wi7>UXFF{G6PFp*WQ9c+}Hi?-fO#T)SeVVyvml0e}eI^lV!X?e&bDe^0#QR@ozfOc$?-L@8U0aA68Vb zm9wZ?g%uua+G$fQd)-{ zw}TybBzcX9Pfaq(VFzakCH?eVj#9=51kVZorGhw5F?l@0XoP3DOB$21q`@r-{2;w; zsVMo=Na*iO*j{=Gh;Tx$B$HOUn4uGR_Cr4^ElBwxWbeY1bE2Xs0F1`b7^U9q{N3`%%IO^rQ2{AfkwZ9DZ`64YfpVl=eb||Ka=syBwGrqfJwiJ{sqhilW8jkx2 z&vF`r=Tgrqnuz;!PpwM}XJlz1wh z+314dBoQvVlSmjO|4h-D^>y;71=hjZ`{F_RsiZ^n5_TSbR$qpqI%j~Z{Xef5q?Z}2 z{};|v*#ArCDdztbKk@Ki>%E$OdgYrmZUO=XRT}>dT*50)(1p4_p5U2^w>+E3=h;kt z&lVc%*-BGAja2SwqJ^GjTJG6Kb)M}=WKSpCb4F4N2o>_gK4OR1hrkg(1ED5yulSr3 zaq0k?dx4+MxBKvoGYK!D%x;_F^LWSaUL7861`k*Ze23`Oma_5z`fY|i_vnN`&+JRW z_o=F~dxws{F;vOF2gG$6o_9GqHUbRn(ms~A7G6uqPg|=vHxbPewn@MDOfpw9Hc+>)1O!HPSA*c`pdO9 z(FA_Ph99Z~e^t@g-&*{CAE1A5G&ZWA{<%GclFH3IS71qf3`^=tI??lSs`gw3!*I2W zJ*PPADMrdMkgSks)cIQiQJD32FdVqzSF;YTtqxt1Pp_vC(Y(11dU`#L z_T1p&&6q6Sa7zLtcr$v0R~!_NIPj{S<;xXr!TKcy=UXaXK2V3Ob}OM;JfSSqVHJxTMGtNwa zfV(Ivf&z7EQ4lxWP_`z7DBuPPxZ*Cjpd#)Iu881&?tSy-O)@Dn*3YkB-`scax##?6 zzxTBdKY#Ci08UaXLpT=`7Cs$989q~mb8w%9`$MR}guFZ;kI!28TnJTW=JPW11)=_8 zh@%H(^pHHhB#$r4#tj3Xe%)+;X=G*e}9Sh&B#zK5g z9^aS84=nsJgnB$KG*4LgQ3%K5$1?L1dHghlIy`CNXCXA;=koZ4Jf0HjU&`ZY5&bKf zn~>43h38WieiOnf{8k8lSB>A}4}$!U^7DOf(TsD;&|eH+)IBbvP4^wQde4+Z3vH zLI)ivd3v14DTKcw?+~} zLy36n)~wKu`0hm8dj3(Eksj_#M^e$G3+D6!yJJx&o=KBSJUu+*q*|l#jFXCoW9il% zzRf!_;fzC$R?3MaQvFoy4L)R4YW$wcVTAx_h*SqGM$; zJe+p=^DVu@$#Cg3A4JX+4NG0xO>CL-ZjPdF8ocXu+7*%PIiR}>*|b0RXB zwGq{xotO$h;glj66+*5sFCX4WG7It4B3Sydd7e|KX;_x;9O7=9JXey#s@z!bqe162 zTx7WWGf7-fk&$2&i|efPFVo9*Udoe{InTW*{RyS;1RsiauGc8T01eowN$t<6@qz-YV zf^{P`!ru+%*Y1eNom6`)oK8F3G1s?)Mq8|@xXsE$hn&Q4MnP>P`$!^59w+`+*gAb8 zrYnwz*3gScN%^aN>^#Ip$DnWXVPg?AZKa#U)5cO|3hVzXmwP-IuHpWEQ*sqfE)=ha zwymytm8M@PtSVU@+Mq0xWW9_Gr&5fyGt+4{PO0?%&^j}f3XixExu#?pK5b9;InZNT zwD;Gy@jS!x2oEv}>qsT3QExqGMjb5?@aJqB`MaC z&+)K$-95=whrz5UBY6fcin^dlCDhKC1iQGH@q`PD$wB&m(_DlP7ME%s;KQ@l$`Yjs z!a&+_2IZ(wKy<-|zfpEon&e{Zy_w9f}uNcmNO8I$9bzD zTQjpEPPW#N!@Gub`TSFv6dZ*M@=j%(o8G^hHJ`5Vkki&Qt+s~9Tu;37Q2LxiN@1}Z zjQ(Sp|CpFPc*r)Q^&ju2Yq`e4-tEJ^8!2oqLe%t%PURPh@ld2sZUpwysV19p-1L>6 z(y&Y6nl+s4=e@#%L768MP-v12UDdl@oUVAiB~mN&{fiNsa^wQZ34L=?C~we#d}(;l z8J0Hgun47`u0*EYy;&vKZd&6j%Uk;PI+09UPl37hQ^^X%*L24bueY+3?MMu#BF;Hc zxpvj~?V>oCdq5|@$u#uEeQdL;0(pDD2mI~UcN`-jfvsJaSZ8by9z%5PR()B z#?5NJt?F=-!pc%Nr)gicz=oravJoQ)zAnsD**m?-($Tg$MjdObdU-5T$Juxhrq4O3f`$eWXg1(Ah{mBPWmH%}%FpqD-H>CH)?ZdVuET zRpl&DR#S`i>~b#SZk8;5?Fq#7%bYB|_ODx=Smy?u%qE9-)?hTf)&2BT0(|lhl0#28 zHp~)aMbjIGk{NOfq@$N}f(SbJBn7WI4IOUx%SS*OY42`0uz}aQR`otIib?a~yI&LK z(}=H@y0aZZU*~7i&W{Ex#*AUzG(=V=9F^~xc^-6)B1S9cWsyuGd-&}#*O<(mZHWG1 zhWZ6aA zj$M2n1B=fVvRjT>Xr-7HSb~-OvkI$mA}-|WJR=Gh;bOi~C|-h>63ELqw)rbF7onHG zi>OqX)B^EruE<)j@oof0G2?cQjm%1SkMK78gD&qJN15s6x-nNDZb1b7+$WrGq=ZWT zHcF^VIu45(XQcv`d#Xl>`w5x`o$t;2k* zC*E~fiqo~0^+Y$H|Ni1&x29_F^liWZ_E2^XoviheQ>2XN$?fDYNfIOfZH-6$3uz!`OXKT6&guBoW27{}2H2hf;LS*KTyVT&m&^v3DoUs z8pR3xyJQSYH&38Jl$XjghGktX&7){MfExL2I$suAM$s(4&D=qQwTPvf$FW>&xNuFN zF7OcQgvl6MwR?ii;*S;OIKM8?G>(;JI6qfdR#RE9a0FN?=CD{CPbHeLm!=xTY0T3N z%*$>hXyPPI`EriZG+f3rDwXgIID#wj1{!bqq|Y=IUc;WNTex}06| z3Kou6P{&t!c3H{>B_Sqm-sxFrC$7^yQ|7M64TLSDee8-R*rb%3o$vH^_L4@ZuEV=% zA+d%nAy&f>dlsv!T|>NE8)Cm}2=Qk*DT9qo&1w{@b8S=py&4Qfi@_*uOeB)dLm2cB z23>@b1g4Q@Tbs}uG$yzYXxTRsoiw?3Y)Kz?9nOQ)#9rX(psli%K(bGXux3l293$?_(=A zro508qS7o+<|s~MaNfo_b0RVBd^e@;_vtpvlQ_$jnA1(xjGXr$QqGzJa&DrWn@f~) z3qIgA!5&v1TlASg8%xyM`zPu0LCW}$PloO3BHu{z4+*j+y4->fmL%h2l=1NrWhkQh zF!4)>pGW_g6C(Av2kUYtMB|U!$^Q<`zqnK4tqkOk;G-`0ef%w_l!dO>Ef#H4+&4Er z1nVBGzr8uv3C+oHloH37DEBZ`$9b5W(9%tEFZ3+9P@fTb_GkC}7>l@xKa&qaCAKtc zALp8snjqA3Sdgo?pP=5id3NY=8UOcDEC1h9wgk_? zT%D=XE>?KRS4@d`S?eW@_(q@KgBJ9$cnJaDRT|(kvw+W<7Vu*P{4E0hHUU3Iz~B8} z0$zl>agY%t8P!D#OGbsXs{?sAJyThKNJ~D>Th0@{a0uB-ja80Nas zk*&QSGbYf!tM=?MY`8xcLB57%Fp@_jVG^m;{pC3WyJD#Jv`Q>j zcEqDpl&CO!jG=rYm9$-k=AQq^pSTxs6?#K!qzXIxBZgv6KM01JFO?Xi@mQEUK8}cK z_&T6LoW>FUA&Yh@6NWQio6zHP9!!Nd}6 RHA0k+I@a=D4MGL1e*sl(iERJ? literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/OffsetResetStrategy.class b/out/production/clients/org/apache/kafka/clients/consumer/OffsetResetStrategy.class new file mode 100644 index 0000000000000000000000000000000000000000..9ea44aafc01f39475ce74fcc11bac828612ee40f GIT binary patch literal 1192 zcmbVL|4-9U5dXZbTf0&iOhkwH4NKkNc^vK zBw{oge)f+t-g_f5B0sH7?s9jZclWuw_kRER{tLhcHd0WqtRsiKidhu}9aC5lVpT(0 zr{JnUJk>BG^qPi@(58xK3`whWP__>kURvF-ZSs%YIdsiq-a6){<9lup^-QN5^!gn) zG;6I^&yDt75_S~w$ZdZrGHB(}zEzP)lxnS7W>A;vCF^b3W_a~(?6k=f?)Tju!|Pmr z{4C0D_k+ujw!FZt_B%~CJm5{AJgQ7zYcagZSx5YYn?4WPCiUe7?P7jx+8L|saKFw& zPt>_6rUbl0H8cMgq77_$ffsEv%w5BylCLu;(V<6kNZDRH;88!MO!1ruXj`T=NUY>nB#HZ4@QyZz8{_dGGD+t)08P0Y){4RPCM4Xooi!{%7m$3PY@4BW$g18Gbf zm=Pj_X@;rGrHF%Plz2XS$-(Ig~E$*z-Q<$Q6 zr0iq`1AsJJPz+D-q(GW^hTt}RKKrO5+oFVx|hDXa?9#`=|P{%`w zS%ij1lCVhgo*EHf=?9cXMvb3AJ4H$wx+0AUX&4D7#*LKI*vbVa>x`5BCVIJ#BQ> zC)B&!$Am_hoG{_~KpQ^DCquSx)G(IF(jbXce5|Z?ei>|Jr#dC9AM*R0(`-M_a?4>d znZ}&y#Jp7IBxXzKyfTJu7^^g8$sZ09I||fPMW-B$)w{6@qFA%Z(jZDqnormU+GK2v ziqjx0LkFCFOzb!;G9UiAWRKt{ro4&V{zO7LmL6gIuYrX2kscZ4 zG$5Q;Xh7DSzvRljbPNZN&T9HFj;TzPo4YgX*9KuY>zo_pm{D z`@g$Jc>XUXE{R5XT8XWaLK+>-KC9@k(jB-r2eC4v;Pvo~qpbJ~FDhy0^l`41cpx*^ z0HG;!SQ9I%%DNz4AeO{7cRqtIisQDdEf=H>__FWJ5^x3A#FNp)qO1p1&az4d$9@A#!coUfLD#U1mBOy#s#w1`xg&MkKLASuKZyVU literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.class new file mode 100644 index 0000000000000000000000000000000000000000..24976a2274a3958cf5d18ad16676f846e7f3ce9c GIT binary patch literal 9748 zcmdT~d3+S-d4As2YG<_?BoIPIT;h`fcUNF? z)Jcz|ZPGZn?m_CKP3x+0+%|TC6oob+OZQ(N!l;e`V{F#M6k09*6 zJfkarp_jkZ%d>jbD zZz|BYBKS7`Q#bxgFaNEAd?$kc!FTobd3_~DA70SQi%R=@iuQeF^@B?M5HD5YW&B7l zujt1AD%@q=y;6x+@sfqfN^}Yo^Ce0yQXv5;qlywFl9UrvLJDG85+((y&`U%wm6q5M z?2$PNU8O74lw0OnGB1Mt#zazM$$U#{EmNu=VvBSXjInS;*pBv)#B()~{IkdsO1ySEt3 z4JIgcmk)VQP9M*AdBM;d&z(tUP8FgsoMoI2j3~1$SJB*uVo5BuyISNJl7d^WORiEX zf`G#eiZCJzLUsJ#g))|9Y!3*={iu}2>&%>WcZW0ST)IDEL&rzk(8=F%FW!MHP^)pwl_w`qRxi0v{M#9e#MD1D3> zB7E)tXV6?jvCR&%8pKVlEgK$T%5`7As-ADB)^ANYKT^KH%ybRzyS)B0cUse+2kNGl zzq7;n64*8k_8N3H-Q1`=r!U9a&lYFuDMubwUN+Zkt|!gl8qfwNSl;B9w^LGXzH`P& zg8B9=9&5vfVSG}va2(zM#_(;Btm0p@m+JyWplb&yrj4CP>x>0{+hyUo4U z_;){xK_Ye1GY9+b3O`mQfuUN?AD)8Ml^)U2@HH_RDwc=IDc2p$JY0qqcf%R4%LvMI z>70|aq>fX9hL@X0MKS01Go~}sSFOSP{2EShDCCUEVp?00P2Juq-EDi!cBD8ik)d~A zPy)>)YBb9P?<`r*Q@O6ZOqUQS5cZ=`V(b!>>mD?DoX}{>oYi{IB&GKmUU{vvT&2(C zD3206D6v@en^fCxQn|c8na(oqB0KKuj}Pm?AZy7DeD6z-WcuSf6FN{W^6heKb=tQv zh;xE9*F55wK+|iO4qT;x=LMQCzTPyX>%J z=MX~`w zfl3+8DaJ8t17qZ5oR(T$s0G1f+YN!i*P_+ZEU?cWXrdW08l@&AN zz*9Cqk5eiod$a$ANe zAA6nLAYq3RJO-GdlH0*-;>{#bn@H_UCQc6K6U>|PrkKPFdY$`1r=29fMhX_@)53f$ z?BXoK0>$ZrCaEntvbfGO)D(<-sUkc_!E7%TpiHXA^pq=jILtXnXCy+*n?j3ABvTSU z1}4n-Ii%7u^Z4y5d7Ni0!Dipgd#(87abZ?J7dk@GTbc$t9gbcDCB-+~-P?C?Yj4*M zb_buR@2O5k08Zc0ApCTss64LlzH;^=YCfKt zF|^qa$P;$vnR_nfPq~rc8sHiZ8w_CIa_-CID#d8_8Wu^XyqZIUwOoTj-ggNtOSO6k zn>O9IV+DT@I0(UAu=u+MM^*l8Qlmq$xpS~Jch0edImYJz9PV?PHR~K`&7CtX@3n*( zM1t3Q`1XegBE)+~bQ}`Bh`^Duag<#^kk92u#u2&zi;rPGR$M?N+I#_((UuFaqpcS( zCt4T0fU2j6AHXSISEGX0cG6srx!8ck*vPdGG-C_ap%a^N8+UI=~BlgeIZo>clE_tO~mWbfg7kb9BnM`1NDA47Ff68&)o=%zfo4Y%cnN~FLo zf_srM+#<-5a)8e{jCho{@m=v<%43*&csk_+q}*pH*SU~0C<{*G4ENN^pC`0hrkj$^ znZP{3&B2SPp;70zg}7GPGJ)Cv+N_IMaHM$*b@f*37;YHH!nSa<`6B9%)CGg};rh@x z7L8+Zedxm^=9+wwao=(NIMh$q4q!PAwvoq|om5KAe1JcksUh-a?#6vMYZULsoADMS zd^g^Tx6wfS2?;-K%v?)X-H*4E_HwMjJMc3Myp5=%NO`6+uq1#>SlZkA3~soHWz@8x*X*=TV0jr1w^HL30T{amP3*XeoI(}FuA#V#7#FMQ z>$UWhDiU?c7ZiyV6p2y%cTt!s=E*PM7b(au5#FX??#jEl@*b{)A(ye*!Y^~@y?DSA zd<%Cq>gKB-_`mC)39Kwb(aIqMk~4xRVS;;t2;PTt9;uTgrD3y}Wb84l>WJ3j#xdOF zzBVm>Z1M$-qcWp!jnU^Vlq{XEm{ zWT^txkV`N)hBdrbP9Q2M#%vXNg02eFPV;CZ<>RWMme%|e9>ji?lr&oJ@zkhcBsRaF zS?+2vI{BhfbiL(Ld>_N&94S6nB6*&oR;c^&Yowg_hP(3XTv1(us6424f=M?VAf~;<2J9=qzK49c_D^X`5vN}DvDFNPZYC|Q1-%X5pXvJ@GEmWeF!soRLBX9-t zg29_S%^o(IJ;JM6_6};Mo~VvC6a=R5MMt@1>q!RgAC#E1x*%G$5r{t+>ABe>`T&V) zDcu}hF^*OxY^@l>+K(Z2d$fT|>k6#4d3);(q+T`@S+6uUe~P3o&~sy02<8CK;Zb}7 zF?@>k`qM@vw~}ryh_u-gi2@iKX*NAZ>$(~r^q*$bsJmJU)++l9r!Vk}sB@y)DncDjP)wtIZ~Us$0t4 zN(xdpfz1IND~5~GE z$!}<3VQ#nawJ5t=NWMavUqvJTFkv0OUT{<++s^Nql2~lD>@yfOB_*GjE>ylu+9rg> z!tcMrKZpqE#~DxVyS?zOi8hR*{XB<&3O=m!3PY(Yze!hqi^=xw5?7WM22G~YG7!*j zJuVm7M4p=dS=TllYMU*BUfRLf<{%%8Z93b6^}z|;X6!V9+XZdq_2nK^vpyNH`nFJg zXbf8~p{C8Mx1PqfF?8wfcERZ67e*&P=rW>{iuG*+>m%?Ttm2<>Y{GY037+R9@dA4A zA}6u$@nhom*>HcrM*Bm~QZI3qc-g4h$;z;se_Wy+DMZL>e28t-B8?4rl1W`oid~$# zK5POYTL`YK$sM;=Y6o!PH5*;{rwOkibpwKzQBB`mL8FC_@Zm>T{|XTT`svHF;74e` z2!SWcw1hIg@-}o0W~}(An$9z$u^PXZUBVr-6)jV0Qku*&>7YJ&CJk#O)`uDYM8N*%=X>{_T(aAS0 zEe7{))*|3C)w_Zw{xQrZOoEQWJ`sKrX1|g$j7Sh~m2$j8LU@32drrbQFBNz~B6v#p zO;F}wTy(B*$D#AAZR(W@&d-O8r^2NBc9Y&A?BIl{p0ddAJ;qa7Tqg>in(#aoz&NWU zhh@_V)MqZ{Qw7YY={@yXBSKgB*EjUltJq+i_Ax$RM!SWHR}gy{0oQ^2=IJ_+vE!C> zZEKYcD<;rej>CLrd*4UtP36|tGPe7R*w5+e&QBJrsp)r816$^!PHM4O7GRasp;;Cd z3Iotm(5R(Q!CSlvu2Qp(H&YybU!-c5oDcKL%};KluOv literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$1.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$1.class new file mode 100644 index 0000000000000000000000000000000000000000..331e5cc88ea38d9c017d887a9833430b68764f38 GIT binary patch literal 1724 zcmbW1TTc@~6vzKV3#DbH8L}6QjB)>Y|iE8e?4;$3zNdhDIh+ zm`dSU3eyb6A-D4ux9x3i9WX3Z9k*uk2DkQvdBArMxM|r^DBm+JM|nYAxMn44RGfNU z`ju#HnW0OwoZVed_#Q*Hy3Y@}8TisR*LZ{GJN!NAo2#0+<;b0~_OYEJ6Oc=} z^OND?RW9worRm}L;*CK0vMyFWSfb%eM};6pRzoq!1tW%mIAnc$Us!&)G`%Tn6vS|& zS_~1HTuXrs@h%XazZz&(9Y#zY%3dqH-s>mVdD;7@EQV?n+tW*Zg{ZZ-NjfbP1zpZv?qhQX zMb}aE3N>uuQ*=izICrLLJd5#w-oP|yH4?d;4lzJAMLDb~?(~c%>EDB1vg)JOE3}0p z{l(HTSaIwaZN>3pB#PgW{7Um}=%=!Stie1CPfDV&FEVoRaK8&E# z)*+j&Dev`4~U6LFQx>`Ip2a>b2o9@?rf1PpM8~1O?orR)(xQsixmQp@GZ! literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$2.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$2.class new file mode 100644 index 0000000000000000000000000000000000000000..a3a708cd76dfcbeb6b9e52cc79ae897206241cb2 GIT binary patch literal 1910 zcmcgtYflqF6g{)8EnQZrsQADKYLz~aMM6SME0Hvkm=raXR6m&vyF*!6cI!S&_~3u> zFPQj>AB>;Vm+e@l3qe9NZDWXmtJT1k((>X8qfsWKu$FN=Ba=Pmx*m2D~sv2sT(=djD1{Du9%tL3G9oPkuDl|;u z0(Fgn%Tp?(PgZG>;mIh^CaNp8jA3~ch(3CTXD;8y0&?rM+ z72}{TOQRJibWx;Q`wUk73gsis6r82s9Ib(}iE}ibI$VVboTr&YkZ6o9su{W`vEC4N zgkJpwR-5~VjKs=BSgL$pd4+=VI>MWYb^OJkf(5&R$rm9b{x4%k!uuDFkM_F x&g*nXaxNWn)_x*8_XTIt^&cS}@?~SJ8@L%^-XfoXEWJ$cphzQ?&~x-ppFa;9Id1>} literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics$1.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics$1.class new file mode 100644 index 0000000000000000000000000000000000000000..ea8613c180d87f328c8e24f4638d86fb0d4eacea GIT binary patch literal 1719 zcmb_dZBG+H5Pr5T94!Yewc@*0t=bmk08zw(U#cdMg2o!YZ?8*PxbBj>D<=E^_1_p1 z@a=;?z#nCtyP^ws0+} zeBmnYcve+ww<#6(wOe5r3_)fL5Ki{k(yH)XX?qOjvQomWI^6RFiM4)h6?(w~^NUhR z|1rb6;{OEh7pqcQXGm0aQ!u1!Qi-)ryCK|Hyy1{0Thlgo*10PK`B0e%2xM66*JK1Q zg&AVTXbbLjTrxWwMU8H^wX)jXELq)bRkhlXt;$O22@y+~CxKxq!m+;`W{4~Ont`oQ zk=1p_wZ$_T1fMyUzo{+0%ZZ_<%67CTRg3Iz>ZXY-W=xD=%)~IvK%6yk52k@R6Zi4J zz`TivSTOL&L8;H_n0w0RRuiNWV4z7D-?~4Dx82^ls z02)8|1N>3Oxhp9Iielp>x4W~m`^@Y!GrK>3eg6Sq3iA;}amR1&M$wCV5u}lcAdYMl zV;GO(J|_G*BTCJrh6fsQ8Xhu4cetJMHY_(a$uL`TRL$Vq+}sex7GK}uhG|k1M}o!UyB z2KUX+S<>>JF?>w_PvB0m@|iV;aKWhxhFHmxV!2VT2(`*9Hq9hTj>+vct}I_ZoDBN} zGAws$vJEe(95JKS1$P^Y%;wrr-F9Jy}Q^2N+N5lfLLfnlPJV}IDp z5R%S&23DjbD?1Hkir1EZ`1rB-O>FWVP7Ezb({@}-*2w;bQ`IqmX&o_K(s2=ezW7MP zjE={c)$s&<8uB{k@KnPy9nbMX$4g)H3I$564s*CIf8?(3qG5AcGgc~_!t@xjr!AZ$ z!e_i0>*DFf=PIc-?l6347r}GE<=fD5(&?a3^QEH$muW`U4rdykWgAO;J4o$N+E4JI zU}>PBEHN8O(OqGzT6OW3ijh*UyiizKo-Z>DH&M@8+PXjJ5}N6dFiJCSn!LJvYO2nAj7xL-;D^;B2%Xmrs(fXk#Eq)~z75cRR_*gjZxd>>ueuLyk! z7G9yYhZZ6f!&Mp&2AgBJh9pU}DS)I}$93E|1bl~Xgh`j&MR+8;2eyaqZ|FIICU$Y* z0Fh6hN!x`Z?EV0IByc0y!cJlcH+|b=YnTq?yLl_vc^jkDhv|_^Awi=A>EiU-^-~`u GnZDn302gim literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics.class new file mode 100644 index 0000000000000000000000000000000000000000..3d6c77ae2bac505f4796ee95e356333360b9e142 GIT binary patch literal 4037 zcmb_fX>$}s7=C&-$xN0d7Z3=jtjLjUcG++Xi6}%l14I_Vh*xLR*$nK?EHkrFRJ=vJ z55Tkh5muR`%2@sYe}P*1L96_tR{2Fg_(99Jd-h6pw%DS~R;GLUy`Fxr?)m25FTVz` z9iO&h2Tq5v6KCY?K^v~xMbj29nXak$MXhWFz})T@sf_0!`LF1UNLZ4#}yr~F?0>vwliT`+_Rly!gI{D z%MdwiS;864a@Q5~s^?0Cz7gBWBzc~vr$q8RpFGc#X*-v*tz?lUS=2Geu+(2V=;V*{ zoFGdh=lC>FX1SF~rpS<$kxWwD^hUTRtn@{O-WwW73Cpz|67850-0{W*uSj?2X~pzR z+d3%-yP2_a!t!VbE#8#rt{-679iTsJN+#NfdF;nN1$!`N6)^3{AuK1bMc7#I(fmLT+3*r}%hQ`PxqN>}l?p z^1Zm&B%Q!85`dv0@awDGQ3-QI*lF&+S7u_L%! za3p-0CX|Zap;EjVxCjBI{S4j&~w>7w<)I7bwHm)hly9?!|&$s3O6qGYtKw zrbLua3x{V!)H8GR;OB)EowS`OkNOf(M_ecf*OU8spP{dwu_ElG>v5{jebX5oS0lKF z>(opZ+0JoFyiL+J&2D73VUx$C)+I$#4ta`UgS0_;OtluZCrcZmV#~VGJgvEc`p5)B ztg%gp^r5knzJFTitA%bLH0zW% z2{qE4L5*|(sgc?_(9t39EcDLkH;j9s;XXh+3Y~sn*JJ%NV6i!9vG@!^v3WF&&7nCl ziD+(g=ev329ekoh|FW@SXUd(TD#hNn&!|kk4_Tmn#Hm*dATC5D3ZI0 zyt10SN|AetaM5Dw7uIF?bczD29{9on?-u|fL*N$p1**M34y`vq6EU-6mt8@|&1 zz|Y#B_+9&pEz$mFG3_5V8q(O=kinc#8@m)*&aQ=eaFBHFrR1T&;}8zho1Do+($wq- zrBX=Ej$#B&>?~1^5sGGZ39E2|W-S;cp$H|Dlst<2=`;Hzt!PNm|0y+l0C&>3nl$ae PtJp>BTj*W>#r@$wCtKge literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorResponseHandler.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorResponseHandler.class new file mode 100644 index 0000000000000000000000000000000000000000..3753170380440ce9e888f8685f841618b8cce605 GIT binary patch literal 4181 zcmcgvTXz#x6#mX6&9u`2(gKyM4N@UZOGCMc&`^Owp&BkGv06atG;^9xJDmxWgccF+ z_ebCG*=Lt1go_K*r7m6mCD(Gf&t#fTNn1%-n1{@{?D_V$FK6#FfBkd+A%K1OQo$y? z(nNnr1)FhNjLs<7fV1N7oPt-8QjkVQ!D|>+@H*aT#E6XZ3NGNH*m<)VmoO@J-jbmy zXu)X(W5^0%UHtJzj0>5Zj0rI}WK7DKVrX=DA#Ym+&d`~(om^bYYuO1NpVG#sw0PDu zxK%8~2i#}M6YXON6(@{B$8Ls$OTUyQ*wQq2;)Ye^j-{D|W802ySX$9`1W3ZLjAB1S zBD!Lf&S8e&fUOh%HA%zbXG+s!+{tKTCatt4?W|@FYmOo2-eOQtXV~Q9GH2K+&C*To zFsMV8#hn3DD-<}%W>!kCV`(L1f7?QOSMzfeFFpl*iA<_5lj-Xxy$yNI5dlBEjN2cH zcxQ4_yQ0NS&C12kj!p7xk#L&QMvh`xatN!Z4lH6q!JlxQ&2TPSi(QxfDSo-c3&p{b zVA3ZNVlYZ7R*sQy{Um^zY%5*LW_f{ddz@k zWw~i`J>pafTIAaG$c#~(i1^Au#L&gEUgD9WO&isADptq1VdWyaQJ`R3RHC^~JRhyw zGOnw57w@TfA0Np0P{l|1Sj8vU#;~XM0$@*1+g6-Aj_nlU$E(oQvm@hE6`$dA6?8Nw8FesMZjqNJcI1z07ESL822)_0a88Ru_U3nly)TF`K5Sf>=#zlm97gi zo^(5!A?}Rre%&>VPN+5V%RR0JhVAh-4srgAN+2Vhjb(BTCQC1uHQi$V8PQnO(Fa zp@-&Fu&VQ38d1HW9n#6R6LOH`OrbN_RYt=N499LESVpJ}xs1j#n)>fTX>ATYfI1TJ z_R=H4t#`31J<{M8UY)*&H386)bQj_7AJH;$2d!nSy%{6ub!TF`y31HEtO~kT9l~m& z_$?Zw@1ROIu~qsW-O>-(E8W6j={8Q%JV{V#=^idg_c1Cx0GA$OO8Uw5#6C2TKabJZ z<$m%LS>gFaj9#D};U^OZ@FH2u$=h3SkmQ8NB2{_vm?w3pD%D5330iFc`x~R4BM%c? zKVGUh4quUoP}$z~3tDbqy>L-m*F8KlZ$6p4cHY)DLWA@xWa&4wO21>H^aq}m{&elA zuxYQdZ^scFb=kLj>;>mGQcVg;s=`_93C@CP0Bobj^H}&O1y}@}0!&F&;YmsLGd7U> zH?AnD!op?8Jvz-U8UIOyI5A>PLxGK!q0tO-F>yjuI-0<0k^l*g&Fm0H+ci_<*NzffawVAt zcInp9b)zl4x0H6Jm#wTV+j^l6l(l2`tsnMbf5XmM|A?KPJ+CA?4py8?&zV2S()&Ko z@3}wkBYpbck3Ip=gHMA9|_i`f8+FWa1^z^RXOf(Q8COao5v<&!~t2jAr@ z-)q4419-3jKj7;>4B(*veiTFtPBq|R{{Lf+@JJ9p!B6?)=^$>!qa5%tZpY*NawdQ$ z0(eqDy*;T}EnNb3M-4L*R`QBE$->i0dRhsqS&iwo6;=)1D&&|M)^wYhx{|fR{f1$t zG+nU`vzI{nG+ndz3g~NHF-qH*fVzGo#RS|C)pQms!HU8`^ z)^XJZY&ysk)1F|8-EZXbS!Qd7eo)aNL^NMMa zn&UOh?{}P;U{;>I$$Hy-G%<2HLl2wm{sObe?PZZ=CIJnm2Oz)?!3)@RvM4~4jza{P zei|++*GJcdQzPN(zrSD?OtO>0dgEj%LM2NC^!Q*dss|a8c3=hc`tUDRCkhxdO@q=y zNDA06KetZgOZhdjauI#eDdo5l&MJB)JZ5OA-o2D1Ym>I3PLC*gFMk`8T81i!yd>b% z|2Zx`3i3!DZmm!}J~72q`|~)OL+kVF5d<50S;h2Jp0mXYedXcrJkFCA@$a12`w)CA=(QFTO6}XLu!m^AcXg zYZ6|^8v*=W!Y}ZqgbR2>!Y}cbfb9vUGVM4^$yPyCnPsI5S-ISK)pSR|-a~vc8JtmBp4%^A<6NUn z+9vsVUqQ>Jn8~y0V*#y4bT*TxHb!A_dcI6fsZNSJWeLB>A0+${e+uBw68?g}O86VU z+{PnS#NQ>{g@}Yc>=BTt)^6dmr=n(ROc9?0v_e2tZ`r+4O{wK=UyyhHd_hxIskrtT zsS_0VR=?TA?E`t!unpD7hHW}%tuV!`KxYg!4!<5>i-n&pcPyx3DRcctRsm}ir$w}M z@9d%u-(2Cs5k_gG(yaK1pGHdxv+r=XB+iyTywje2-F`=g4@bd}=eXQ=D? zoa>BmlIfN+={I@PsmD7)ep=PqHe91%=R?p0XEK%E9cb8Z9pEu+bjs2l{0WK%GM~J= zE)fR#^hToVV)59Z|0L|oLB6b0omY%<{JVMlZ}Jk>%BFj!!cilWVf?kGiTk~BjRwaQ zn?UcyA8l&f3}^k0p4qTGnp{Hhx&a^; ztu_^nzxjj~&Y^O}draJp=AmeG@IWLweqb;*n23xH#ba|C^>B?&`T;JfhL%}<$-;!C znwoQ*(130;vTDzA3a$!(5=Un&m06-`kco=t;Fs`ERSVs}7XHowy(NtnQ>|ktgWqBAJjmBI;5%`cnd!&)@4p!3#B;d35 zOAs$3a0v}`*H}dGbP;O^UfR3)eXK3w3$J4RW!%u*bOkqx2nj_rU%(~WTSt56v6k*` za;Ua&s`Xc}L4=FBF|_$IzF5SUF5nbVZE~o-!l`bqpt_}q%^_!(=kbG&^YHj>j$ zW4-t&Hi?g+MSL8c;u-7`pTL0lB!i>gX!n&KccW=WXS`ZO!ickPYymaUx)>X6~Z11>+x=;~2Y9&L)!L{uO#%NW%Mspusw zU-%{b31&*5d|>#@k7C%9T}N)h4DMvGN7C(S@7ZVfcK7tx-@pF>;2OTkVginZNi1Y> z0c8t|SUOKHuItea3(I&<-)?I0eG3(==+P|`s}`;Nz{H0p)=YfFFc$60uuwKpWf;FB zco6Lg9yNJ*z+i7GC4#EQ!%&0_w;O)YcK8GC?hEIDe|Nwg*ONj;q2u~0j5{K5q>4nK zxEDHA-w#?+@yHJf1Gp73x+0Z~RvGHWGs2wRWiYCKOE63}q!L?kXHNu8zUR@*g@*5P zZZm|>)KDP`h>62o*F zzV zdb@I|)VZpINPD`jT+a`sYTp&nzTdJ@!@7+rOxws~!^UmgF|lc*j*m?=Y~00`jeGdS z#HTj4weAj@Ha^3yiO+3(fiG=*g|8VV26xHEYj~Yf{Y)~wP$p-N@w*{pSQ`fX)EF|X z4}+14=U^#P)&;Sm$|=@$c8(PFz1tBM7;X-=*(oEmq07(7k5Yjr6;voM5QT4_yFXNB zPhszOI=*t^NK$`lQq>UAD<_FQt3{2lBRx-sg4$e*YMbIjW%`upq9z)gEzy5(tNp$} z+^?VH{vQ@nP6omi@=)}#s9otM()72|ou``@DNmQQr$7E*o>qHj%9e;8`oTdpagDql zs(elbVK^7k8Prpo~FSK9h{Znz|u7{TgMEY&6A-00n6od zbECuFpev6TY4#-=ts+Bzz$(rC0#^DJnV(6@;AOgxkq(%JS7@B?YT;E}B8l`9G&){% zd+TBSgi%@p)Jl)R9%JM;7*8-t@$JiFGee*XZ~LBq`e literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$HeartbeatTask.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$HeartbeatTask.class new file mode 100644 index 0000000000000000000000000000000000000000..176aad0080cacadaad8be4c19c7fa82dbf573227 GIT binary patch literal 3425 zcmcgvZF3V<6n<`+X4~yXTA-Cyg^E!^Dx{^nSXz+UQfMtfX{mxQxXs?SOOoBNFSMZY z1^xv50Y5XN#g`d>fIr0#zI{+<939WuO_~jq88^e24>#wW+18)RW!sKnTDoUD4IJjP*d=}z-O zfVM4TNJ9~Z0;ypIs~Xl|YPgP)g0hAREDbgVqHbC-%VpD*)FKV) zMbtT{U(_8v!j3x6*3qZ8sG!+Cuu}#K_ZCfQ2k#D&)=M10wN6euWxd}l}>u6{?6%U)JreL@Ob`~3Ni_2DtrZ5fPUV5xVYVzzx)pU3>4`tF<2ksaHx5Hht z{j#xjAE>N7D&;6> z@9ydHfdG?F+mcuGvkKEQ^C=Ss;g@&;!u} zg?s?+h3dODq20&87L$H}G&7rc?mm(aLpgi4<374Vt~(;_d4Rp4v?I5#v3aLXK7WDz z4aqJhH?g~L=sWaYA?W%x@jOSB4A47H@c_T02Y(Rme^La05q5uLlt4Wx=&dD^m=!4) zq64<*#JW&%SM&!_4-%s5g!$_@6kvO=7C*&Ui~lg`=A4AP^0aH1BnsLy=*{l!2Br8& z2oI(SLQYUBH}nXi0edvPY>*QkrkANva+=ehqQ6SVNJ9e0iA&%FUZn2<+P{bX;JiSR LUgG&P5`O&)CGGUb literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$JoinGroupResponseHandler.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$JoinGroupResponseHandler.class new file mode 100644 index 0000000000000000000000000000000000000000..d55482de84e7d53effebb2f9aa28ba47f6d4e48d GIT binary patch literal 6376 zcmcgxd3YRU8UKB`HrY;TOrfPfWv!vj)ov3)u}ykxlQi8n*)6-9ln69Tb|;%jHZ#lY zENwwh5vAe@;sG9@s8u|mST-ruiuFM2fwzM9jrXnm;nTijDXrJes@BF>* z_uk*;%-P3J0cgf^UbJAaf|{fktr+q^^c3S@WR{6@ebzdojmg{ZtmsgzH+>qr{Cknd+|O# ze7^?|c<_ENn(%>gJjmYaur6R5 zricoxm~4uN(*<+vUObuBu3`B`*`JcNq`y~J6D`|F3QMD=93SbH(|L2Rh^k3)I1;SD z?Juw@T^MBL?yGSy9v&E0;^y-aHbZW8yP3Ob*G?zy!N&y-&H^z+<|ZQvT;!N2_`yU_yKm7<&WzgXf;p#)ImACUYK8;M{Fby_H_yv9`;a7aKU*k6te#;X39XHQn2PMaO($ohsdP*@BB05n5_*89B zH%4Vs)iq{dg5>Y<2MK><8h_&E&n%O=`4O@e@)!KogTG1mJ8}|2_=mvSnMAsmYnF~f zaV5=c2&_C8Q=_IqBG{Q#Qwhaj!#Pa}sYg=|r72>OIIR#*mv9QZkZia3B>WTqlJIY4 zVLSfA;`^_JE_6%SjUYwA=^V|RUK}nlV^u-QSS+Liep=wzyjW0*XX|D>JGtJVpCvXnq5D!2vkDaFF1Bb+#jYicPwyYM;Xpj~B9tr-f3I_ML( zN=|RVg@Orz^NHuF#zSmfIim~;hyAVvvVd2P$CXT`dh^Dn*$c(lfm)_><;tj(RH*E= zypgX%k|8TylINjuBQr5unw^7*Afpv-)Vz^0-c4@$nwJZ>mclcijAjQiaYMBnoT?*J zA+2m(h*qWn(9=Rwlte@sB(oiABU@2TdMS{|$fPDjMl!Rlf@x(^gU!SKEbBuhLYcZu z+->V}&n#(UG6X#v3?)v9jUq!$JL{v$W<^QtD{x1cRfPzVHPw{AOU-cDS{0|4R#KVR z7nrz%!1BEQ&+VV2>&sc!OLr?oVnC72n4B3AxXta}GdqVvybE+I$I0qRJkX^&7OUY& zK;#JIUy4aTGb{ve(sPnrEv}EIx%Lpa!0As8?^)gx+8YY*5B0Z&L(!h@j!1v7-Qmbq zxaxjBa&^fIC0zaR4P}*o2xxGFcTGapn_qL){&r zSbsDY3B)=&`>22mhM06P8sqsnHm?kZdIMd-_Wo!GVG(33*xeECiSfhwT&?DxrAn}h zvy4BGRws%(vdjAEc4W)gWjhHdshHiAf#oDAXtUFHW{ky-TQ)ET|lNM z<9x|U{Mu=Ps?aI6o2Spw;8G`KB&(U~sB-SgwY~7s4|lEfW2UIAxs_Bb)NSQsm8T6qjcqc;+v2ri9jjW{amMk)P=*h@&BZL1$ec@7m1V5sgmifGC^ zjwSo^eQ6v^t^PR9%VF6#Dyv>ZTdGcAc`R(i!E|ByjJC7VhsBdd+VALcxf>jtWDKN@epmljJDs8rPNz%9b9Yg zT*s#@ka6TstsX~h4su`p?Pl*F~M)c!ZaXme{WIQM0I4csuBNS0Z-<4uetQJY(6RKzu!=jVEdqm1IjAl~# zSzJobwJ>o?$}gA*_4eT^G7z4NAWBA3Dt6HmzK4vi*D}b$+J0KQ+M4nFkC@-Ee9h{0 zwdk`9@#U2PGO=s&2HQ+$i)r3lSN{Y`NbH+uGkLGg3nA>rwRwb%M2p>~td4}fWva0| z9i}XQ8su~b@~7MBEYqjcV-W7NefExm9LR0^$$i-c%Wyqp3&Fly90{5&f`VQ2ts_ac g(4PQ0Xy!T^^S@Ra50n(q9~%suq4^{9eHumo2Ozenp8x;= literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetCommitResponseHandler.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetCommitResponseHandler.class new file mode 100644 index 0000000000000000000000000000000000000000..780eb55a82450054f7ae9113cf78e9b947636fa4 GIT binary patch literal 6253 zcmb_g3wRV)75;Cs$t0Tr!Xs&cs-cZ-NK%$y03|?K@_>cSE@ne$sJc!zlg+?pcbS=` z&|0fiTdmLfMy;*&fo}`Kl2Cc5Re?S&Efuk)w$;|w+SYe#rJgghNp`~|4XMfZ&EC29 zp65OPIp@x?Gy4w$sK=RNEW~Cv79qx?J_&I*5=gq?!WK7l7{xG=;ZX@JZhDsS@0TPxH)Y-1sa$$B)nR)-Ukmi@dOdiN55YjRs-DeC&t}CFN%Zw4;-2wHZPkRiRr}(wF8c6kZsd~H86{DGH{zOEHbtt;V@0m## zTOu$!XgRTwJfN9vs*xn+)zwP8JErOavM(N2^~RWD7%FXEKT&ln$B;q}DbH1KdvZ`o ze^suXujB04Lj_4iH^?DfWB9$%I>kXxG|Y7{*^U-n-I7ubI&UZ2$fzPc<4pl4CYfLv zA&!Eh*K)o<%1O!bh<4C;^vG~XX;5vHr zyvAWz)h+kz-JwKEkEksgC-~AVne%wFj0P-~u?)A%Sd6y{EF7)ojVW2eE*ZP=W9GC+ z!d@9^^h+3!u^)HHScA1Peu4uM4$3%$!!mw~pGo++j63lQ3BQ!_D;%M)lyMZtWZZ?j z1*YdzY8(GD?%}12N7FGk<;b`f_ff_eb_!Q#8TaFWj9)XODZ@(^_+&hQ;}RZ}@eoeP zco@fJ{01il=&U&Fm}+;0oy01xzNVrlp;u6yu$usyknvlblJGkjkKj=mr}3DC-^+L$ zPsn%@e~|G0C=EqBTS;l|xiVQzmWvqrzAau4zM^AuR zF*H`W7HFF&*DMJmkU>4la9sGPYvN9cnyhn?lvBwh{8!-0F%raUG-(q?f1OoBFBw(w z+lVzXK6qNzp|+}~u0^O%bSyjLiaf`DYh0Asc{ig7`DV=o&2-=wWmlM0%Ca%_d$RXjnu0c0d*U3 zBy%>Ba&hZr#^CncgveIyO_$lYpJDoR-|+bx80sukIH(ZS|vWO=X>k2?d-dRFZ!rvM!n zn9dP~p;VU<(X}LBo2e%?RdT?xb3xu&37JG?%>ccq_V<^q1tUT>j2gM_4c z<_s3{87y^@Kh85CeI)8e?AugrRcIWg!H#Zne}S%OeAOMp{-Ju3A!7Sy23uM}&F$gV z=5}wBx7{0V4+g`2Z`&#|dzLTI;r08P$kf(WUwb&SHRRh!i^b};h-w9ZfF}b9r*AAO zcR_~7o79>I2}s+TvxCJxU5D-DU`he4RDzT!8JL@DGH~IGCkvt+?Et1L&b|$>?bFu> z)&_zb0z*d`_BBy~^ZEVFtGxd3s^&m*o44H;3=n$bs@mPA(q|D0&^}75R{kR%MHG6; zm}-(@W!aK5pk*=V)YN2E*UCCn(C4p3^x-cL)3~h%F|*#5l$LVCPvZibTZZMd<1%^+ zBcJ|(sns=mA*yS{KIB!`?nVACn#{)vdQRpI;vh=IA(V>4@X}m4krNSCqM3eq#KQ8> zLSvZ(*fUFL#umSd=BVGKCktIc9GPCe1Fp`RGz#{i@OFC7v*Z;ry|^<=jyxpCx>3-& z;1-tl6Sus5kWfq%$?822{gC#ch(^U}xNl6OgxJdim?SWO$pV2(2QWoohgd;_Qh~#m zy2LfpHGpY6p1!1jDD!6)3}8kcy_cm?zPON=&*SCu)0nxasH|u=W~DLvR@C?7f@yCk zFYLz~ndD6_*cw#KEJ$O{4xAi%EiX*t!ksw6v=q+zJjWeE;-7G%h3)3`J=fO+}Yi7<2U40D)oaX2rdPa1W@ zvW7`5M~KThvFkC1{C^Z>;uzf|??RKf8ymzu=n?l~i?|QhiTiPrcmTJF zqId)^iAP1gI3-HNX)#+oCgzC8MU8ku)Qcy@ zLh+PXE}jPBAkWSv1;!%dN;uq7xe_wpz$Oj&pne6`B2=cn8hBlh!VS`xR75c-O1+ ziM&WcSi-v{C^R9VOG3m&VO&;PR#P_pRn)7Wl}%lkF+M+|NiQ@@)+RL3 lZ#B7ZA$^l?ByLmbQ~&zHV#+K!qOLsrh}QYfq<2af{{tB~c$feH literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetFetchResponseHandler.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetFetchResponseHandler.class new file mode 100644 index 0000000000000000000000000000000000000000..cb6b6939057264b2484a7fbd57509616f7d5faf6 GIT binary patch literal 6119 zcmb_g2XqtX9shlYtdr#eg)hBQr@ zv}u!cH*HGOCg~*IAq5$SmLzSu(l#Awx|8m`o3^j7@0Hj8cPCkrEeQjG=euwG_dY#& z{*iM4y6|r&Hesxq9tkJ9p@~R3aTvN2Hl#!t{GG1B5!~Q}h;hC;T7zS_u?DZgP5kj{ z5jWRh9&Qow8vedj#BEMA;I&S?4!1k-`fA#|!HGL?Cx6`K#2fLZ8oU{Asiy5)`Pti? zcst&~w|8@-ckRRYw3hyy1$z=NFCM>)X9xT+uLk57pBq=-*3*o{#&)wF@(8o#E;+;UQmj4JL!^2j0C z9f_+-!brIzS|XJmQ*^hQFcdu@$5U>Prs+{NAsd?BOGGxS3Dwxju(|n?skH24uz9qo z!Z6RTCX_&WY*^6;<>5G;)cduF9N#DFD&J=hZCrGQCBc!Alwu4hMr1Ukq>?18vO`Wp zu|AzQ;6XPQc!l5|U_RV_L$kD~Hoc zip*YuCGm3TSsfWl(a5@N8aHy$sA&=gqm+!Z;9Nr9^5GG(bQnk*Xo@oPCB(R}@J&TjxWg!}^`rFmTJHt^gYpsCoi^0C zd$*iyGLH#v(3bVBvgQ%IYO_4jVSVCg4usA1 z4C%}&u%(nyN|&sbnao9;WVoRMi3>EUi;BpC4%gIam9TMYlfroC#U^zzC2Cj&2f2PQ z)v%tqWFgB_EuTP@BeAF^mM1d3fRxNaXIM+?5oJK-?Yy=?*7Y1*LJxW+T#ak^iXggX z4kH&X^CBLW@EM%qG)^lWYr$#AjC1fx`Su5cTCOOn263*hYB0eYK93G`)mhc!J zm+*OffuSLfKh+Z?Ji)KlmJ_eo781V5p|_QTu4Mk=k}Ba#cwEGnC42>6W!T`=bxn7T z@R>(V#9UUJaE;KJ!8>UO4J1}8b=`PV5B*7a5?>SXbqU|VQxd+3Z;AM}gzw;K5#N>Y z48AAf`}l!`AL2(Mek|c9cvix5{N$(HGe6_2pYz8r@JkWDlJIN%rh*-I@d4619#N9K zpEE4V2SwP>N#t#5H6B%T?gXOKwl|@SC#k?FQI}$>rY0=ya;oK~o*tKk-{N-?evhX) z@*lXUf0Xbi{8_|bB>WYBlkj)^gJD%bb44gj1sem$06p7Py z#EIgXHuH1vg}s!f#F%32Qpi#cTI`eKX_C-E6wK^q=q_vc%8ibpMvg?3RH~`7a|87( z&YhD=#Yegh(r6TmDg3Lg+RQZTlZ%B*`gxk+O}S_o6{*lzc9I5^}921EV6K;K|6G&Deeo?sxncegi0_RYrFRXoHR_6BwZ zg8Ktg_d`A^qV+z%-@CofKeXK&@P_&ZeZfE;uZ!ayD<#sb3T??R=<=B1^l&Pot7bc* zA>L5P`&(wl(?t-V=t(F_G^EgP3pDnRQ86*Io-AjVlSw5JW#}joM1_NGQ7IfuFrF8aZ$n|`ZfwY7Xiq?H{4 z_5NPaFHW@EggLYaq}I065L(;V1m>Kiy?`wyETg4}tsH_d`18fsMsLPjeU`?xIT~kS zJJ6Ov)dcJh5lkVC8k5FCi&vHgr{=`~QDJ;~pgWRC7(*2t#K3*|{`Xvtr5dFUX6)(Mcx?I-zF zNA=zJVo`@BcgGp5pM-l3_MgB?(kDk}1E*uppp$SXu~EQ@^Ut++Wbh*HQJd*e9m~kZ zwWt&7&@I%%BQzi=%*TFV0Y-#{P=!T^)B3Qm7&i+`aEGuI_Xx{zpRfWC3M+9=aN!AI z6`m4S<5^)1o)^}#Dq#(C2+gcoXkiUP8(SxIFpuD7141X;C2TZ3zk`1N^-uu$u#+M% zNF(De?54ABCLw@5Hii|qei$UzANZUucxae(R?$2U|@X{%$ z19W>GL9K@Kf4EY_%S62VJYphVA>x%HhE^&K(6hh5F93!JSn<} zz#MuzTiYIktv!RTN-5#A#3l+JM6wh%5KW#|_Ez%XRnwE9Frn;OY*11vS?qqo<@^bI z0<1J|ZUUanRyy+42&FeqQ9EK7H7Q(AI{^oA$b84qNAFfL*e3d2{93wcprG)hk$!eR Sf}p(`QM%{(;6jy?FPa7jqFjR0LTN3I(MV@3ZN+rkhPllI_QN@B{oPadr=- zU{9XHn|Yb{=FRud=NACn!YGBq6pjU)JNtq@>ZqL!k0I#N(n7$+)pUt#lugt|=FS*W zEn|HBCLo(L31j0)K)#!IKXty8xI*B8wuu?7UDbLR z%w6LZJ!!*2que>a(5&nHO=~TM69JRIGDee)f#qyyYVuOsNZz6eyvAn2<%|k6XcgsX zOo6HraJr*5oLEafHy8p1Hqm9m|4R8>EIASZcmH&d2oTWYTLAR>0`{QKc)vxEFiy8I T*ABW)21oo3n91qDFv)%a*H(x- literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator.class new file mode 100644 index 0000000000000000000000000000000000000000..0ecbccc5f37c4aa1a9419d25c91e8db21b8d4855 GIT binary patch literal 19663 zcmd^Hd3;pW^*`rk!kbK9AOs8qP=ZDblVDgy2%sSbNFV`8P@}ClNhZm_FcW7cYzkNv ztXtKpRcy6tRZ3eoGzm!Ex@h-(vwLgzeQ$r(<@Y`Jy_tD4^P&N4e}DY=e0cB8d-tAu zmhV~a0-t;Rsb`32p1RIQ2e`EWmo~Yqm&g^4R^|qT?X&=(G0#q-n>gLS4#h@%|DQrKQ#D9KDtKu+RHzdVSnP~pBntKk8a?fNsFHw{0kr5#J}|NuYCM# z{*A%E_0g?-i8TJ5Tz+rxALPp)eRK!^NiKgb;J@%+eYpJ1;J+LEkG$RBR=)gGbn`E{ z{Eu8-5y1Uh8onx&Ui0$nqOvy(MLxw!F_qLuqp_|~B#?;4nuCd0s58!FHbx@BSbaDU zj|bz1%3~V4F4z$W2O^!p`an3`5$N1xC=b&lNAGpPcwaOU53USEy28O2KCTP~Vu_Am zAWQgp5F`)?lJ`hU2${ zD+KHAe(ewB7P^KHO6#obwOn@w8DC#g=~^M(?($20A4 zDtVZsnhhQc`)x=OBf)X9(y)3ZXBo8;U{KZwGeU{!^bx4QF*4MObgO49Op8X;*Q)um z_+WZ6O9jV)mJzX$6!|!nHy+h;mt)4HD_o`}m2PE1wh7bop6uXKY?t*N!7d$c0nTEF zAFmF@CB^{w_IPt$WVZ5y-6UR<%_RC)WRY}9ml8qN9FH(gnkmO9u^?yTzb?K#(uwIt z76OjbGJCVg{igxtifi$&u^Ga)YFQ@;Y%K9DPp|>tqJ`rFlF!QW=L8Y~6chjh(;07# z6&KEcgQ8Tdw8-srzz0EV;P_SbHWfBH^rqHMQVukNP?+5b3PD6l%{kXc zHd$#J1^9(blf{3fIp$0xY5ln2c>WJltUcJ(n+tgYI zWa@F76xh(Q7S56jcq56V!{s80U=us-Wx2HUNhaJ#zooENXOR{YAk1#w1X5XoV6B#E z`+q}l7q=EwjB()PoM1o}f&!hLSg)&?J!=-zhBsj?sl<%s&J^e{QOOMimL?suP)rB6 zVVLBYb7aie+dJm)!6)T5uw0UZxx&G&{}u;HXUJY2PI5C(xIrPH?!B%?OC34;Ym0Gw``J2zU{>v3G8YcFzQ|08MapP z$=4d~k97u@hh$Z`*kRPuMOu^nTxHV7>1mUmqvuU}kv?P6APt%HRr;h!Uy%==r_Y%@ zhh8@6)AWK#hv=|L&(O0bJw-=M`Wn5=G&?s^Wo6IoX&V!n^dclm1A5GSygBY^V}bjZ@>XIB2R9xE!AhPZ&)f8w?ov2PS)l`_7S#143@^m7fWTLPPpjA=qzCM%#X>koyyWP^Fa`gkyg;iFqZSdK1F zuJb^u*=mld=Bjyyns2Hz)B>hy<=p{zN>_O_l4h=4*5x|&7TF^7I#Vq$)k3w%q&Ecm z6<{wR0!D&chn3A#wZghPc}pF#A1dS*t0ksds?IXGfPJPq8+v6?JD(~)r>QzqEu)tW zRd1?uIN#*4Tw?NgTvP)C{tl$n#bSZ&vczI?p(D2W4UO41?<=ukM~PAOuc3Ev%K=Mor}rHi@I^R$in5tb}XsU|@>hBO`X8{$92c~)_Bt)l+nU>{_niUk^a(zH3 zZ7|g(3acLTj)j~M=rC2MLhaO=OJHPr&SW!F)u4~b!%RId7kpTYMA!x@k6QaO<=s(u zU&_%1G{{obKzdMhGtJFq%2i3uiN6SisMRY%t;_9yG(y>AGh`Y`I!k3AI|B<#AyaKU zE|yY6v6hy362?@URM=FT1$3t;T`^e?lqaX|Dd&VwcEqri5Yx&&Ggn1fDo)Nv((_{N zKaqMTV0E-%wwdGdvM`6e9jTPlWRJuqh2NHq~C~RRC3bH)tbZ0oly)kot!JN zg@p7v1kSK+4u!*^co2~4!sY~4%7FF!%BCi%rMLbk%6*KKds-~Dd{Qx6?`2`pKbQOA zC3om`_=PM)37N%>{6vhXV9C+2k+YD%G(O{WKs`JYML-}Jd7fypf7L_rR z(We0LoKbP=-`-FPCZ!oYlkh=zqc?ytx1qNuqHi+a&aO_-Zd6Y&u_}nu2EI_dPH)M| zHf)uf6)X&87T2}1zUl>y^8(?1M1z7b%If5EDkCygkC>pwU$NCSzRo&fAuQD$jXjbP zH)ELzs_2V`!%R!vm>C{p8Q!HqS`hEYlA(3RBY%q}d0`0L0%z7Krh;H{;~MM08Jb_$ zACaSwsSpkuoRTY=(_h!i$w>HSJRA)6L5NDNqOw$FVVO;;=p@LBg`jg`1--k19sM}# zd78a%;9Tv=#f3OPnRADs9*RTakpn97ZCo^x#;=~}h>1o8`uecaj-tqA5GVRF<|%@8 z07##v1?3E7>6d6QCXHOXoq;Yb0Xb5d32b{!3p5i^>lw1^1d3ALg)N8dZN0JRRxzIz zalBK+v}cSH2~qCZps0_+oS5FYAvsxk#2f>-(@30v%T8A~2gODznrXgZ#SI_Lh1~Fdg%smT01G3#O13{ujQv zs;$XUtwZ_t8v$37OW#JTsor?fd*_a>w#jqf2(ijs$t=5ITAYY>M#D9AeIW!uSi;w| zrm7`kuc_;Z<7{##j+N_5_$XJ@Gh-#xfk56DjRRu4-DJoH*)V%p zbzOlzq`c$gr6huoZD$k<%`p0&qhsqW>Wyr9l6lA)E_G4=<8Wg>_XiZ^WTOQ51C+=63N zt&%C@P_!o=+5tiycm-OORyH{hZ+ozO$m{DzJU!FtBEts@2MN1TR#%&_9X*^m&mKM{ zkBnWkX8H2ghPDm$Ynq!I+i+(6=&X=|jdVGew6I6kpY)be`kN!II%G+zBli{4sZPz< z4Xt+R*eJ4?wk*pV+Ui%j#~c@}NY8?m4R!0zf-?DhDu52A16)DYF9AdmTnG9q%7HxP!k?trM5>z;g z+U3YnQ8_4GfLhS{cS~9RVU>>-YtEjvG@B04#rWrq#WYDaeehHw+dKM>%^iKm-jKdy z$4B3>O{DMGw$XR&+~_;@hV&gfNcxVA4t>Y&k-lTk19v5RKKhQWC4I*}6Yf)qv6J+5 zeE$X!$H<^Dc)QC#M5Fyx161fQD;S_6zZ#%X{*vMWGW^v8r2ITUWBq0M1LX6cI6x)- zGS2{w^Bv9^4>jv^rfC{Jsymiue>VnpS;JlkEsE7R2OEnat z`LvN%(|qGje`I#r&gFY-vIGS48LwvQ%S&-hp=A7>1be~_y7zwx|9xLUOC(ORFbrPu-q zwo)l=gJ!qW*>oASdpYc|6Z+o;4PT+P>PMfmX&U{EevYU4R87C2UqOSVw4Qzq&4_-( zw$4TNQpjqx`c10UZ_(;^_*O!m*XU$}es9npmcByj73i8}ht5JLw(yj;O)VIoiNB}& ztA?ni8T=Pk57I3BoxM*#&UudJK8Bz3?jkR4^Pfc%?;$#4{~O{WmXLmJplk8#IvC+b znC3lT?q)iXZlP1@RtRz%-1L2PCf%V0EyvKK0k%KWU%&=YZ3;-W^jG>D7)^?2wfei= z>aX+kOg;<=kbG-Uvj4P`=rCPFb@b@ zBrXO}oHT@gfPlv-K*M51`*HLdt&h;s_8~f}?{lqvRZ12B`t9m*)}u zdxg+m`PQ@>M?kyDN&9g~F$G3Gla>HHw?L0eY@IJdz$3Z`n*Sgb(*0=l09Db0XnO#y zAEJejqLV%fr+x%IAERsNar7Smg@bfEJ%K0uC5Xi%uv4gbg z9>l}cuhskCcnKPA7V{KV0h;*RvX4q}6G-CbfaIfT$2&uZsKb!rDPZjx=;v8F4gC8N zlxnb5zW{+~DUJ#>Ak;U(9nQrOfi`SvOL!B(QJEWF?qrc&UmY1egp+hs{2H!b zN1Xcxm4Uf(uyPuruAg2?v1-!f6suKPtm2Q+NUaYc z!&J>2qPG21pdZ$!xj{rhv!8*Rp96Zogpz)xIgqG@?)JFyl#Epq*u(i+{0TgY4Gffy z*wQK)tDvIde9Mgyt(EK<+syq7qe%w2%2JBE`C~aA+)F9j2C&} z(_`eT!rziATnzjjH$?AfmUizP^)y}Fo_CNow6=SyTif&T5rDho5CvKfQwLnxh8XDG$+fw}k zjh%UfLhb%){N6Z7o6Yo=w(X|B0atlC+jxmB2e7R;4t!%f-{$*uv=YA8!iuR})7v7-o(YG>U*wu%$Wd5b z#fK@JM~~wjynABxi{!=Q{Aa7HY+^0jm# zUx&ngJ!c7x6yLGgPQryls-n&S+>OBHl>-hDb1|4HIq(H zXs@*(?bC=hq4!s*5)pyMByA}=!A8EN6K(~Fq$;`3fovdnvy>lzc?2+zw&Mj>iHGRY zW;~VKRf?1(x=NXk2dhe1vY~_v8Qh_%Kc6kI{5~7>axZygy2N zP|V)V>CCbmWu+7`V*dM53cb%(=ybY_C!nxVw1=kfL@onRM782MR?pwqJuTJh_B0hBMAxkCivO>X?Iuu-$p^`};PW*4Eq1y;O9v;1zGWu*(! z(V`8tkjAS;RIO??o0emCy$T@N`V zI@?ydL5n;@H)`ut9i;bIrn#vddG%&nOp6blJ5>{T)haSo3l*z#X_{I`GgT|q;Z%*J zd4btXKzKTWxG1&3=G03|5YoR4NrYz|+lmdIky3|Z95}StM^7sz?-h1$ft#YP1k>-` zM`Npx(5-;dZG&|Ce(DwY-GvAT=Xl?KYCl4Ew4>dfL$t32E_c^KdjBAO;9i=KN!{J; zB_G7Qdm($5b-&NL-9JPhlF8zels8BZ*rY~ROaDXk;2=Gej{J*I^pLuk3RMS{s7@+Z zU38l220DAGR)uM$+DsRy2yIYN+NAm@rY@x&Do%UU7Ofk}>b2Cw)jU(1XKxC_d--&( z5i8SP9V?v}#v3ue)>{#I%16ISO$N_;1y$IO(Lo_YDrCrHb#w_x3!!O(r59+5=y1P0 zC$0Wrq-2>Mi(Hmnbrqs6CqwOox9x&l*4)X$+B65&K)Srx%X4`i#t`HR(=wW5{US-{ zmP2U7(qac<^Lf7HM07rb7r1mbN7ngFUg*-faAchqaji>d-^e;I<|R3uhp0fb_|`Hn z<+DNG{uGoP zI7AA*~-3{%jL!blI}{>Z2TdKCA^JPK2d1DJ#=x)S$; qxbMLI3EaDIN6C(v!vRcC72O0n@+V~7P(sY5UxH2y|3B3vD*IoKQTrkQ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTask.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTask.class new file mode 100644 index 0000000000000000000000000000000000000000..9aed806dfd4fd6f9a786737b18eb5918b6f62571 GIT binary patch literal 168 zcmYk0u@b>R6h-fgNKp9<8m}?q2aH0aVD!&=BFWmA-NcMvQ}_TMWo$alozvVi=ktBN z0W2}jk>?l@2EKEIbhe!z36p)(d83=sDq*@-D^EQiB-D#a9zs|(zF@hFxhmIksfCR# v)wO|5<3e{+&!ubCJF$WP?orkY*orH7cdFF18G(?*7J$;2fwU(K<0i;{+FmQP literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue$Entry.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue$Entry.class new file mode 100644 index 0000000000000000000000000000000000000000..0d6c4aa723bdf3fc1e5821fecf425c51931e6d17 GIT binary patch literal 1181 zcmb_bO>fgc5Ph33W8*fVH07IyQowNlAK-!-Dj)^aC<4(&fgU()+$DA6tSf&Y^*?bz zqJTIc&ip9E>^34%xdjfM-I>{W^WJ#(=dW+y0o=zO3kI&5C}LTenuTjvF|cZpbHl(* z1Gg9oDNlwB4?0mi@c04u_k=g(?}ps-!$8P1@%%_8*+|4*AX5=b9wy#~2>GGd?NaF) zgOLU!5oIX@YcrIZfeg}hhKJ6~nC*Ixn6;u^!BFl5QoP7UeGzwgKcvV)C-QmN<8h$; zaj=l?1qnm(sZ8TT276OV5w}8~B!a>lvr>7LiA=0&P*Q68(ddB3q8l+hpIP2CW_^=5 z)0h?Ks?*u$@3|LpIq+Wg_l2LXsl2%z3?xspn6%5kk4!~cnx2|U^M43lC&|`!l*PVy z7O2Trr+&OvaaR$xF$c$n3CqT5*aqr0Tr>>aW_U8=wVIldjU}98sGOV)!{Q_~QO=Ne z)b*L4Bs-BD&@~uG?H2|`CiqB!5Y?-_r@(Z+< z-7jEn;|Mvoc7*gu8CyP(rOvabo~4HfR-48?0yNM)3-(yDf<4bwZ-C`Wr5&bZAZ zs#jEO_Y?9B`o5y@f&AkhC0+Mc<1QVa`-5xaJau4|$AorajCPOCXkl*S2a2CjItjI< dp==Fnj-m3ns2g6wWxZa(BCU#Y5!Z1R)^Gb@CuINt literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.class new file mode 100644 index 0000000000000000000000000000000000000000..1e0e3719edf023834ed907e33c4a76839bc3d202 GIT binary patch literal 2236 zcmb_dT~iZR7=BK&VNJ3kjfhgMAXN!aj1@4|04g80&=63l7Ob_0a11NSZo1j1T=f_9 zt~YeXYj5gBXDHM<)9H1mGrjGfY3Xw|;lqyJKr@q*J@39B&-*^lIr;scUw#G9k3tM5 zaLvT^7(%!f#c|v)F%m-rquMg2m+?4m;$yvh66K2t9Wtr6pX#?+6H}3IwrU@3$Z4nP z7-n!wZ*Rvji#ZdYnYbem8FzedRlu5X9OaD`Wu>AjH2bo$!r#{HBl%dSs=i%J&3Lx! z+5YOSs;a7tKy=nFIMT0rO5nlU{06hGS4hdS%rC3dihQ^tQ~9E;9KVvvyH2HAQeMh- zeC0W^SV@hkqFhx=b1WWIIUAtM46|CppzYZHus}H3Mcl@yyTrOjvbLjg)zYH!=Hz0L zmPFRg%ibuhMHP<#0jh&_>&+W|7pOECW-n;c8u^#q#tXl0^qv3w2sJkIyY!!aN(Xuz-6OzQBD8 zNpxBGQos8E?G_&3oD%`hDd{gWaY;Vq5lK!4Qb_M}5X|FM9h~8veu8&B z53y#SzwdDu!fD>pLd%8g8CpgV<`dA=y$R9%3ZeN;gnxqZW8mGvwUHjcyD*sN2s6FU zomGp$S)AieP!OHsW3^kw|JC7*W3rv?n3>R?}A96gGShA11em-5s0MY@(ewI9qW_qWB_R>3G z({->R`d(y)5Oei1wvSEqGrt~H%`!mCWm@70|4o=C(w(}qk8q{lS%InbtUGFbHLk&I zM$?PhF2SzWakNcEe@62QdWC}~YV2HRi5p~UGyvN|5X`azu>)5*MG+Jy+l_q8V6a~F zTNc%X4b+Q%kKWe{e)kL~YnEe#kn81G+g_q2jOSZ_YEEl7S^`e8)W;OHkmDex+5XH9 lznw&=ho}1+*g-b12bn?Cef$p5BZOfp4vgV}ppsh=`xmj68eaea literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$1.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$1.class new file mode 100644 index 0000000000000000000000000000000000000000..a7659082c238d388c579c253731d6296518547e4 GIT binary patch literal 1945 zcmbVNTTc@~6#k|ylrD=D1iYc3Rw*sAARub-f&>zistF)IZMfQ{ReB7MjCf`rP$g zZ9!Tt)@B8URfXDDa?3nAjr zQ63MkFeK*eRnE{^GzEX^S4-Sk)=L(xbQf(yw^nq=l=E;gAtPi^9}B^qIZJn4PSfcK z-ebefj8njU2T9&+)`?QKSn>^ny97USL^9?Ao<;6%QeJsMM%3nnGlt>jjvYEkS8t02 z-L!m%VjsvB*Yz!3vvg6`7JcEFRX+dP;G3Rli-3fYRsT^&484sY&r0ju@PebMC9^Da zFCf}(fCx>RyBr~%P1v>8?u^RtNfqw_iEJr8<+m|WNUTiT5FT*^lFyN z!DE(|No!nVXj$bYzs!(pER7}4A%PVdDq*?e*stWPLm9y!J<;tTVIh`IOUVJ$lm6(H zPtd;$-L!j}M$ZvLochjOeg`Z!vV&OeE8?GMDUKd$Tgec3i(YE`$e2v2KTEwFNl1os zG*$z<`W};#3@Kg^l04Uu`wW(+Cbo;jK7kT!ZWF+}29l0Fk`DCa;sKKS9>XOJgd{5= ziMorH?$-P_B;z>D@t$(>p@Bo)<4`GMmk)4+d+AvUR}#2NATeZcE$FXfi24M5WruNwMhej>)YI<|KKcPH literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$2.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$2.class new file mode 100644 index 0000000000000000000000000000000000000000..7d227e0138edd748f17951c5857e2de84c412378 GIT binary patch literal 2133 zcmcguT~8B16g^WarR!2EDky%TR_T{40>;E5kq}8VMF=3z4%=y2*v{77EeQ|)C=*cQ zgYm&9f0XggvQUa!u#(uMJF~NQ&N=tay?6KLudm+#Ok+BU1Rko(n1()NHDr)WAg|F% zA%S87j~E7bxSjExhGi~u*SCDjkqp_gUmQJJMKdvn(<6`kR)Q znP7MlvB$QA^gY@uy=Gmwh9!OBN^X0`g7693RRA-VwES6ycTucoBSg?fO3ke^#OIxg zU>GP{QoL!_H-x*!H*8uNEITH**STw{`*1O?Tw~B*Nh#cUn|q$1+outPXR6FhlBYxM zB)g46qNF2No2Dr|g0Gzth53MIMR*Mgs3>(MWJSDh3eR6?`b}4qR51G75Qrhy8R`Tl zomVx0U^ovy!(wO3h;(UFKU3b~JKV6jtQloT)=IM!+~lfNlUzB+u=_t^7@*?9a9mmQK429YYwVmeSEYB1J-7nMsBh z-Rx^ituS5;Q{6y>)C^ZWAuDCe^OvesPxv7;Lp3TPBfHe=Q%W(y={E6H1wC1nFd7e< zNoI0_`tpw`?lT)!RL({U{`<`8CZg?r<<>Du@2oz0tHsi3l~e$Ylwq{eL;rrHK)Fb> zw}=HR1Np)}SiZQASpGA5J_d$qy7rPG*fU(B`4Aa{whiJ6jcTSK=@@CQ#?~`x-WVop zMPVZzP^9vE#IT20q4)*y7K5fx*>m)>*??vA!1{S&iT( zJ-bEEVz`~2Ba1tW2j-2DG=lJZie}sDUL92fE&IAYv!s*N0<=gi=KmGIl4*;&hH+9&H2P3!| zT<*qoq#_7`%TL;Xkn4nmWDHD3VBv&3OzFrP$QiIBXh2>^A%Yf3bP|pM*MO&^7{Nwx z1Jedh={OxhyFATE@`nsOY~T?Cj~aN)z~l1n2?I|WIAh=`15X=x#=x^Wo{JzR*|y_( z120Il7Y)2*;H-g{4ZI@tc~uI1&A{tA-qi6HgPs$fW2Ib%uED&MN%8_uO^W0cpP1sw zR6du>+sObWIpm}EF*NvBhn>PO&k2Us!4rI%C$rqnBu9v!ogpN1e8v;*A>k3jG(+c_ z{6~cC<{hGJbVMreq?c&T3CU%Ud%{l5Ff=WP5AqB{*qgN69lIHNt4PUO!uDLU#&(N2 z;Uq2F6OPTZZqipyiVXK#w&m?-I8#l}wK-G~Titn;p7L*so|R1w@j_qM7(?h_K22_I z8nkRNT+EFNXOxd;m80`1o*m9dGM6#}FCG7oC(iWXaSrl|bPnWGS1t z5eG~h#5*Rg!L_8>$^t)7w6ba8n0OcOQGDYQvO?oIKl|c-X1FHa#|I|%fxO$G^z-pV zTE~YbKElT)`f(k@w$*x%pT~ZeY4l~d)W|bqmv@ar-efqAGKATMDr6-B`B(p zMLJp#PSg=6i^BD!L7z~kTTNJ?b<*`X^@|&)Gvp-`pW-uWNy}vCIHkIhO*<=B%&kZE zy2TDj?i+k(1uZKbcc}HPVQ&dn8SGL6wW;aG)@SPNihZfAmSSXLzl=?gly00n84o<^ z`#)aY|3UIaEjub`nv}fHaB%>Mmvjy6+VlT>O4wzVHTVI^`ZK}A^-|TAvP)md?&Zuq zD<`5JpAvR-BJV`GpCu&?6BQg(sJkQw+C^7V+cOpP5nls8UY(L~DK8bMB+Ff94ax0Z zU!V@=i?&MR&l&1V-pulhi5sN-ixH$Nszpkq;XrqlrgSTVLenqj%0gLEtrRll6f z9nbQtyshwQyhKk~b~=AbPFmS+NaMytSwWllLRRya_==%9QBqBrEUW!%hQpO6UeM$r z{!7j1w(%1p<&|Ce4-kf(mGa6Pxc}Z@sOgk_G|USHA!{aHrPX#-bl;%=QDZZ_d`aX* zot21$v1=_&{4Ph6GwiB25JP`eCD!7z_<}l8g}lOqzSVBZ-Xb+TO-{LU^?A6|qWh{0 z^KzFD3GNmh`F5d!%2(@^U6w3t(j|x z(u?_Odgtr(I;WvWyB0aR)JjuIt<(|KO6^9iRCYjfvmBRn*1%}Q+(hfmbZ90{wE)Y; zx@W;+^Uz{Fv#5zJAar~lwedNGTXp)(!I(u|tfw_Hhx%EVvF~6mpy7CHBkk&1o7zJ2 zXkI`I5w*@?!y@=11->`{ZzS-h<>1W<+!lb_34DnJ_q1+NVzv@^s}ggmLcFX}j1P_~ z@a2_!wkbZ_3A|nL*`W|SD*5=}oeI1w054P52X`uP*P@sO#7O1tdGz>Nle@?{3BpP! zX-S2@dj-31DGD_>MBhUcJ+NO<$9_X2`yCzZ4mEvpZkZcb__?t!M3&aHh30VO zzS_3hA0g91o$KX!%X4$Ms)Dw+k~Z8HE~BOMR!J*e7E&%tpq}*WK)bdR9ojC$v`+MD zUFg?(Fs8-9wFD-#Bu;9(aa!A>Tyw-%zT_IrV2m8Xq(Qh7cS%Rk?l|pg=*RQ{+)dCB zI>-UCq-w#jSw1rmR*o|RO5bt!5N!*z^XS!aua5ivMjw5N<^2TEh}kW)Mu=oD-oOB~ F{{Zhxe}n)4 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$PartitionRecords.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$PartitionRecords.class new file mode 100644 index 0000000000000000000000000000000000000000..a7e366ca1148f4bbbf04d09e662d97cf18994bb1 GIT binary patch literal 1191 zcmb_b-EI;=6#h=xLV>ldwW9T}RjVS2?uA~-h8qlt?WQ%Ba6fDZn0DDEvxCt`@Nv8# zCSLdezJ-6UVLSu0QOb?gY%=rBIcLxL&fn*sA3p(jiU$Q`aVw8SY}#YX9=CJ2o5MDt z*pVje?RPqnf>+}hesj0l zY*lOg5wgCDm8lW@^3EJjbJ@-?s6K2FoO*J|>{bF5%e`U0Ep=10BbKZMNhqS0(8|82 zLdW!!P4%<_6J{)OU&OK0^+=?thHu!{c{D<%vf+&2yk|uQYkZch5q*I} zUy-M8Sf=k-rysb)x(XcY0PDEIw?&o!jgh6086!)=!Bxg34n48o;M<}i({iQ+ud)4v V>UG>;?E-G{A7DSk9PY4n;TM~}NPz$V literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher.class new file mode 100644 index 0000000000000000000000000000000000000000..6073f21311a0b9df5bc69e01fb3156d81e610364 GIT binary patch literal 23886 zcmd6P34B!5_5V5dC3%y{V+$lWEE+b2gb+3t!lGf5UEJ@w~>CgY~Up^n+a^GFf zJ?DFtd+wcl^WBFZC8D`jv!7~tUm-3%^7OEu?&Ds0eMBx_m&>Dac}y;k%jF5Vd_yiz z%H=7!JZlY<{8~EwjXeEUF2D2f z?~C|#{)3;^^B<+;Pd2~drQ_ zDYsRHpDs|-{i;&Ukoi^lX^NUDud{q=woGx3U(Hn~`_w7&a;mLP^V2129+;=9Z8hId zm#GEvWg(cR7TM}_KV8MWQgVi^YW(yeVd96>Vp}aKr0dkuLbXgS_v3P=ysofStxv7= z(?^9ZXGyaOb1t+opL)%j|h6t+vx+iex` z(*r{O144dOy6OOa)dfBkE2KSYsq_~wM9&HN)+uEd`qd7#(^k9u^eBi@J4AXHiF_{> zu3zG(CyZn+^+SFiD5PR_nY6pyudYy6LT%JlGW8Go)YU@7hvelNpSspo*ZJvr?)9k; zMaqzPxc+)5fOSRd}mewT6jTG7VW0jl>hdNMc>Ey)(o#x~`_l zySCLH-NIB{7maPHinq7TJ-@0hx&_3{XDVoE4~HTNrX~7oup`*AHB_}N*tRWLWj@8L zS};!M_E4-U97%*?kzjkgN`{uA`cPs=G`4Mt?gZWCCBoZ5%4CnuqT9DeBUPP=Fq$`^ z7Ha#pha-!3C1n0w%T%~MxO06loLGhT!Zu-AWB4NUHZ%sN?OGgc*%obU(;q=&%eEyk z;UjO5enN?0YcLUHn&dIECN5qD*6T_b*p7!H@n{UqYjfB4n6{Q=C^p$cqUg1-F}x)b zOmxOT^V$DNzYChqn%}f;{-PP){SS$EZjQIa!W{w=(0=-$#%RoH+?WU^L;!Xy1-9XJ z=LatgYVlOnh2sf)+|;jV!Jvjqv8q3=Q*)fZ2ty3p7TUEm6c5G1!FGr`#x&Ps5QCgl zt)R@lD!Uarn)e?x6H0`L5f_U7WOw+V_6O0Szk&tfNI0>G`KqZNRJmaE0GGE;)z;Q- z@aX2KJb91W^w%AaZgbCQAmluno0&j^#V(4H%6-1UUVAXIrK%Anku7eC*pI4J!H)Sk zd&;sfE2pi4Nm>$Zg+dOi3r9lro!d8uVokx#?I?jgYzeln3j%z2cMJ0pTf?wkiG5BgK3&a58hJ+ZKl`hu1r!d;(oW$AQ)%D% z?CMC(X`u%xN1aSsPlXv7+kl}K45V#f2^4`J`u~W3qud@7Kbqn!5CiOmuK!j|I7V^g zWJP~{DH7{t5Dl13_L8CD;ga^wIO2i%qGb7P;;xwxSRlse8%3DfvCsvbp?D%*rMqr0 zZ$PbIw_Fy1hetQ=y`(SVChKBRuNE^V(}bJ@rO0Ax_3FjTix~>*VJEx_9jY(oAjHTQBI1uFBz|UU z^BCh83H1`I+tkGJX*LVOvm-&5l$=&aJ7|d@y@oKV)MutJwgSC3u*OH40;vRBeHKXr zrj1kkW9Pjclc@**3?){#wc*NCam+d>Kra;WK^EJgyoG6v1gPVQ7&LzijM=HjARaR~ zm>o(W%+U?O?*goIP-3gQpziJ9Ws1h>$4ukOX;>i^w7HO8bqu6r7I%Fhoz3*RDB>^y z#87RtL#^qx>WHZW_1x#ixUE>-8Z7H@Mt1@|5vDOM9~5QUpkn=%MvzMYNU~dFK}2{M ztS(D&asjcwnxtZH(_cFFk-65tClsc;WV$+wJ{?Zwa+&=sbXK$f^|YXJ>Rcqibat$a z>6-rX%dNftPpM)m!P2}{=9f|p*R;0{0{VlJSRb?tkX(0gq_wD9C9apseFzoh(DpnH zQ9wNmi_UbC2U%VQ5gsYsc&ScmT%0BPK_Mm=vU1a(Q7su-OtbnN)6V}B=6jl&%9kvk@Gj3 z9fg9;oo#KBQg)LGk#Mw1HX9b}Vl_;lg zVPw!x%MsKkqKRO;yY*pSmP9*c8)oJpyT&~7u#CYfq}rq9ky)1&lU~|aJ&tB{`p1CB z3RO>b$nx(1wyLo+h>dG(3EJvzTipY>YJMC3BW(79u|emtlcVLD2A&s0kIXW#+lM_1 zS8^5}`{rs_Z$3ZNB54@E-c)ewpDbLVWu1Y8)lRR0)McjB@~ZOo;h(89bno-*YBm;AUuwP!mqvFv_joJyi)XrIQnGF`ZOA$HP3c%+=kn|5Q_s!VfkCF)IR_|QfRIpcGm$k_?C^pGXH^>|)~jq6i0eGc77eTh zW;RAUV=bZOVc8jRTxrgbu)^W(9C7GR^oGM%@dq6)!pR4m2RSIpPxMwFwL2)V&V9 zLkAstlU{dt7hmM4&#Nw5eZf&*R9`}N)=^(pUvboZ>Z=GRGc+M?GFHz~Uz1Oj14%jp zJ{@(xG#Zzykwj^>dcaW+s)uZ~$5DG#63c^u)sH^ySu)9PWXT-0PxUaJIIt5hc-rb= zNA;>l9QAd%JczJ~(XsirI*c?nKyZF+wM6@ii zHB{Du5yFW?s5Mpb;!ADy4M#nxo^sUF>KRAvSI=T+HN##vZD_0K9Q949@GbRic)MJ* zr{eJRroF2Mw)&2vzN@|`Jb2zwFQ|tc^`d&oQQuczVVX6F?u;vP_#TnlNdp^ts;;Ac zpnfQ0zwGb?^&?xo;;0|1E+n}0OtZG@9Q70RQ%C(wz3Ql+3*LUAerc;;IqEg_Ye)S? z{T9hRNBxfLko?X$xW}5)1;3+yFHKKBdei!-35=FUW9h8IvYjmK9+?sJGPHjyeDiLb&fRoiUJk{Y+nW(n1HkeIs1y=b)_)IXq7N2wYU}Iy_6{e^{`y zJk4jZDBxLVWLqSXj*81aA*1`X z?d_p0!FFt;CPHcAah|OSM={FcTce%rVv!PMTZ0`Pp~wtmw2T$Z@;(E^?pVdvNXHr_ zgO*ssm^RnIUv2M5hzrvoa}~g7aUUnw2Dm`WTRTHiIl4I>Z4V_vW$hYZ@vY&GGD+yh zw(B+Tbl?$rBtUkCW0gYDty|H2hFj@tEZS24-`j# zn~;gf!Or&3Kuc*Iueu!T1gp%kCcxNO6Rk-|IAneiMDl;+^5>sG{-8pDr{@IV^yj*nHrY_VK*WBR#zad(-*6-Bb?ZZC?lO&C=0hr zu`3eUVwKm;A+Z##Y2?e=Jj)ZC7}&PpecrOF9BZbSpeHS?A7&iu)C{B%8V#;NKXYi! z7M{J_BE!(j*+_JT=L%~T=oG%Qu0%s?46U;q)2_*HbW1Ye`V8QHI zr&_1=iD_$MvEVMT)t(VfN|=4t#14Ok5h5S9<~r6qxKqfn+Og(a3mj`9y>45J9KO~% z-LcNFY8-2^DAp{Ef{Z&6qOq@yg0dKnZ-qob5K3f)l@4zaJy{~HXRmdQzcynA2wJEq zuISB_%543FWyshLb2w~qXSf~4A7`M}MnXF~U<3gw(_@QZ#j%!J%jB{g>k#3s_yTd} zsi7R}OwqCxGS6DMtdz@H;!v-(>TGM3W7Ug)yvthcSZl1aZL7ht8ZAiS9+bety(yep zYg_9aYrXY8Bn2`WRf|J1!GvrF%o~WJ-1_V4Rq9m2F{~9#pr<3V)Mhf%l!0R!!Kb~j zoHt0prf-rAfkdv>CIsr6B4fJpJzinM=MU~wpsa*_Ik;}NaJ}LqxZ1JKzHygC!^3GZHBQx94=|fvZ z!B(BW2;mGrBHF=?+lOp1&(e$TcHuzQ<|co@=~`rQ!tol{OAP8j3m{z-j_X;Pjj)m7 zc$PJ(4~1IAw-%f%fcKpxr#!*qEp72MWf_HFJ6r8L;i)_V#cb zN9rT;`;B1;7~_e$D3&FyqBu-tZj#dLe56(_Y*u9a#*!#vqZaq9f1$*#t}ic6nVt0N z5!fG@eQ-}er2*wzwXCUTX-yNhck{58Z^zz+*Ti#!tPfUomxb|8q=^k69JqFUgl}=Q z6-I8>5$2b+SqU6Wj?1C$PMs(}!rrZS6A|mABMg+<@x>A&m(biQI3T5?3&?70dX1QN zi*aO3G?Mip*o>Js5eF<)MpByH@C_YuobR-3xaJ1uK)h-lY!HuUbg(d#Nz)vk61i4_ z25c4K_*r_b1o2PGHCF@BweqtYs8qZ?6zTvLaJCL?#K}iQW|>$a=!K0OIz4c_{vgWI z@6-?=)R+2I6S!}FumsR&vn<75(3ckp$~nirZr1${ZY831(H-~=RZxzQjtoksAhq6E ztIyx!H&p4s(b&c9hz(NK%%@cf>m5YL9s~>&kg&{xLr2avyS^*qY%`Y*1q$FNa2^(f zxGl8GW;A=gjwa&zXHKwBwZe;9&z@|(hqSs)G(B}NQiH)YsrfB1H}FMyXb(E`r>@lZ z2|4AIVikU?^uH3j!ADwcB$|YnEleB$!wJdEB9v*Nr{4AlPHqQZVAkLWugyb!G*OdE z?g(RQvy>8o)%MD%t4;ret`TQhy(XKRf}_R={hb|nZ^KP?LFv`6q;xcb#i8|#+UEM3;Harx?ojcaNe znrfSBSJyL*SzCWr{p$7g8=F?Isa>+M@5j6y!B_+$LjJ}aubMu!(JSc8MduOXhJI|& z;H&Zkyy@Ic+#&AIHslop6*gye1bi?@tFUK_pO!+hx<}|in=Ea`iKMj>LMb#V1Fy!k z!V8yXnErV7K%c4T1}v%5^C^ov5-GY?EUR50y2lzOBbLwBWD7(`O|A9POOiaD15r)AD1Nl{w+vPY<#jOFFm)i86p$H`N8PRLPC&ET3$?WEw(%`eMGdkC#o2LmJB zACj+LE59k<*ich{Cd{<$+DiacC)fd*N{=qGqY?uJH|vJUjOz=h{-1q`s40R^*(|5C zj90}#_-S{Bg`)b7HUf3vOQZ!78t!VKTOdFFo_0hf(h@h20bMzhUuIbBVoH6W9#X;2 zwy{tQ<83rX(7}B_q=8k5sLrWo;< zcbxUWIEV18=w0-OlPFRy=SouLY~dXTQPS@Pc*mKP^t%rfGtRa6NpTSoUq&`ch=S$a zlvf@YvX}D9EA~=BdBC@q?D9b2Uh%4bm(3xc>sf0?BKeHhT*!2@@O;psReJX zR6-#dOKns}TWAVxr3wnuEZRn=QadfA2(6$F+CUeeRg8zA^)0%OAP;5f3*>9&3O|B! zHU|unCt<9mpM0G>>H#GiV}@!#MEbS=!EC*xZYse?4!<|zkgF{+=)C%O1L850%3 zTwl?5hL@&hIHexb@>M43#NCH~<366Wn+m!PzwEwE-3>@qcGI*jTIv>+@28ROV}<)^ z+JmHur&p#b^iX9U>6cVVl4c~Ss_XFO84gv`P>}WssPHFgCVh(L&`m(^&9of&JDYB$ zX1Wb>zXKTk3`KEI(4BM*-9;a!&uI<{x5m(Rp32iSN3Kuv?RqZfiY&f?0co_zx9RvM zd@H4bgJgqm2dTp5%HlBx2rN`CYG$BT70+~8`7bb1F!^jR&4e<}62|5C&}^JF5onml z(!9_KH{MS7qM#s2bE*ph1<%k8y)?IZAD!Gyr&QbWB~b7nof@zKpK70e_a*7H050=( zQ$sIRH_P+<$7w;57IxF3>cT)_l1@+38CrF>Lhv6NUf8+{)= z&hvQzq&<@MGPV&h1EF1zVES03?TfS&+hFRSPL`JT94a_OBW*q-?=Wm$q0KdU@?!I1 zo0r(U)aGS?0>MU2J3#z44LJZJjR9E>5a5g(03lYr6i^U@wh`>~RV3-mRns4&6`v>n zC|bF=i^k%uPQR_f+mK#b-CVq;o6derbflqTdXgHG)MQkoVnR2qJvHC$=)~PH`O;@; zK9nV+DKoTP`~k>$8SdmqG@4$anlLwD9pST zT(W3l`UzyG`MOcgCBbwx?!{6Q|$qLxZ!W0>+ahaJ>1EmJmU1mzv1(Kgo) zIY@ocmqFQHRnIJ@o!8hx2609G`2<@g6!aNhjS(qXaPQqWh=~k=`T6-ug{hXhqZT?`A+PmmDp=Ud)oQMCL_fbTHxa)AWnG!aC zD>7!K0q`TZem|V^Q1$^%g)|3|Q4MrxHC$u^f{TqjoGw5Ba5)##)i4j&^C-FvyzheZ z{x0113p|c~%g58}d;_3f+wLC@srsW3oVd|fguK)gY{gjNSkZ^Zo!V!mj;PEX5b4iZ3?B)djtDNwbnLwZ##5tEI>57bO zwCjZ%^m8TRn;A5Xt7ryX-8`N}OL;b}fPr4abLkwI=Z(;oHa-oZ;XH&c)oJ6>Mkl$Z zI)*@TS^W$2C!yqPFsxVZrV*9BbQK)*2a|Ml7d_ugA8N)2_|a=b z1WCFU%+vaD-5&bzDfuP&57PBX`p9Q#egCgFi2Lp#(S}dr_lYKG*ljM)FX|N7N17tn=Gm<$=^peC+U_h znk>b;E8K!xlXP1T-Hz^h=?={2Gnh}t^cm$ki)>i!60S!axtdOf@jZjjrbcd{^SBX& zH_=XBOV{#x`UDi?9&V;D@dkL#_tO)69zDk!=|$XM=1ue~oT8ODy8)Htw&bh5cGu*spPdJGDSx(}vd4swW=W^n0N;jnYy) zG{lBl4X%eSp@NckslnzQSXFI0KoupWZ^OA8AG{Ot#wmOwb+rUUiPR6dR&|vau_WE8 zLnH~?btHc|9&~&(r8=)dZgl^)abZdgukCgyWLk8 z;FF+1%%=<`o!z6wA1Daq_0qk~r4ri}b<^iv8+-Utw{1PFduoj#7<6e{Do9>zg! z3+M{W!{A{SZPCh~4^@t8=b1+@(uuA?c!P@cy2eHfIGZo#GL4o)X|x<-y{-Y){>7J| zXGM2|!b|aF&>|+@jQRnLYWlb=-G|`Dw7fjs@(LG0)5$(WCB|YO(m_4KyMqMNeePQ{ z8P@s`jWS;kAemr*btNE$-=mnoCBG2B8i^dh?XbfzCkt`=LXy6iq%S4u%Zo%szf$mZ zy06)i4~@-vdIO&-xKb`#COq2;yilkc`)yV;un+jlCGSj z?-&07FG>2LxRtI$5-(y}FOR>*oy;EkQAu9r53A9#`GBFMTPe&8eU12@nF z{wSRI4ba$+LH9pSOZXGCoIeSl@F}{8Z^WN_{WN`&Z-UO>Ob_xcu8$Znd=3b@v5jC72Rg?HJRmG31h5Ur7<0n-IKds{Y zjJk~XtB>$=>NftSwmV5GM%HQ?UoAN%1Ri7fLwpUAZJo4`ujT6y?!N@u@FNHW6|d3( z9RfNCHeUiS{y{?!UVfWD0zM2yQ28`M#gF2zACIF)_+#*E!@-3==&XScym%GX>*H`G z7xA0gQrWab)q-o1+gh%Kcaqy`RP(TE{gl`sb$iN=;ps+DgfJ6Nt;pk;Wdp)Ym(EXP z)oVuGfKkl|H^IJ`K2l-k%{2TSsyPfFi&R97%{SY83&R2K)~nHXX_d{lB0FL8ZHH(M zma7}b9)?pySlX{{x|-BM``c~418&q^)6RZ}&N)m|34a_ZS39Dy{^{f1qDiomZ_#m> z4vz2|f6HgWx>b^Tl3v;{UHEz}xw{>9tJUIuTS5Kab`wK(sH>Nd7*o z;|~D&AJSre87}UXbQl9#Q-Z>iS?Efc07=2b2em7+XdJzdKMSbj(>$am?&8nExNQ@;r6pJQJr+?a$j7Z^M&C) zXx5k9#qu&>O){x=dByZ@vP8X=P9kDmiRaVxoJ?}ih9a7ztPOS4-~%`A6Y!zSukzR2 zF30JXNb$gI3I8~|Kh(URA8>1~0lf22!-v6<=R>*I=(2V~o%HBFj4r7IDZ3AJNCfYL z`k#u4_+Sk7xV=ws=_o9&-ABKEh?I1nP0SKBFn)M1C-F^jH~kj(m4I6@vKqfZ*XQE? qJ81f4D3^Z|%?*scn@=bxMCSSu4pRV`sPX!~2Y(k@-minEH zF5s3nwl*JWvr?k@xP;rdQ^2alWc@P=_(Y(-)!yj#9=11n&cRph?t#;GobA2cgYDg3 zySu;95fIw~mo|MbjHDOs%h5y``nEtJa>vS_L`>NcxIB_!w54Pa^_7eqV5_F&{HFAD7e8<>n$V7p$V0#A-YJ0-g= zV0iwwY%^qT8YYRQm`gR7GS8e;!l6GI9cCgr_c0-yNDhu3$>W5Xv9wHa=Co8je^;HJ zh38>SA3Cl#P@SxV0;OSpTeCfpBY}l*Dp+qSdi(U)4HPxL5H}R3%mrG^MA#&cD2_h~ z2I>n}t2dJ}zV%d|$TAkNRz?+<%b3?@R+~9(-ouRo?v`;6p9-x0FI_olUBn(q?}@$F zf2szYGzH;!XI>pVlF^Xiu{=$tzk?ON*Nc2)7OtX zx~z!3x9A!4-^M#hnfu)9{Jbt{_uv2uZTnst4fpYb~i>LH!nJc{Ert5LI_ z{Q#@@yk<3&MxXMn#p7J%oYq+FE_J)dZr9^#t$3Erlz%hfKf}ysJhLmj_dY#~QwHmc zztJ#FQd&wSdrV?lvPDhI;Ad%KniuJuuiobw^%m3Wi?6G9^?_DzIaTjA$!hhAFIzsV i;ATd_Vya;4{CARh(bG5gC41b;@H+3!eZcZLeE2U;SOAj% literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.class new file mode 100644 index 0000000000000000000000000000000000000000..41f6f93880d481a63734758ebd287ad4e833a217 GIT binary patch literal 497 zcmb_ZJx>Bb5Pbt2L@2&a6GC}scSxvo-B=S^ePi_?Pekc}k94HzI?JoPl~(rj<{WIjmvS)5Ef z_yhb=#_HLRVBCNkIZW47Ro8pf)vtg4`u+pJd%Ucpif1)EuVKB07lgT%aVFm;R5#w9 z5N6u`R1=mu#_9c{AM5m3#+Dp9-znkKl6vho+n+ON literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFuture$1.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFuture$1.class new file mode 100644 index 0000000000000000000000000000000000000000..da45aadb9a31bf32b177f374499adcc30a6b495c GIT binary patch literal 1936 zcmbtVTTc@~6#k|yER?NO6uhB$soIvyf_DT7NFXt(mjJ<}Z`18q2DYr&C!{T zbyK}=wCM(2wHeyB@QGwJU)tsS+;X_8Sx+mw!VaPW`e_WD4NmM@s0$J1TmC5+^ziLpG!&lfuJ*9 zq->X{g!HBxdbU`XQER&9d`8zz9b8kk^qj5W5^ypcZQYNfbF^&=3`?p7`9t)xIXplO%Io|eZvaz+en>D z=Jm8K$F8Q81;ew>A?$!u1SA3&NWFw!)W^uLg# z-!QTib)2j+5=fCgS)4ooD^49CQJg+NviKFLPf_#;$$p9k-r@wwG1@`#^c^FuEse`? zifl8AYx@|D`x%PXJe+v-zD)5m5|bp7U(olLARRg15y1NnlFT2H49?%Aw{M($ z=LtT63uH~uK{U&I{tw#z!?YiW_TztOFX3{Fwpm-7Hr1lLf?Py<71v0oa2ytcftUDt6#(ljMaL6+Q#K<{?a$hXvuY=iB(bxIpH(mLK))~c(O zNuZ?{dV#i-TU+R-cHl!lzzi)z9D--K3^VNve}jL)Fu-&6U9DtE$;1-&Xy13=%Q?@v zAL$?e`O}{P+=pMr@I}}?7{Wyz8!>dKm%OZO%A=seiE%8-fbK zaaqTAdNF|S%Hw(2dqKzdz}} z4aRJtV7ZfdvsAK58oKP#lvA)YXlFG+&){k zG@fzX3n{Z`X4kFMhPk$3rm}h4DtM(-)+v|658?8cl~ z3?%GYGwr-=^xCCG%Tv;8xMLT^CMNC9I-AA3vmVXQ^RBv zB75|_F&#e)ILuJBe!%paxm?9FHOw>5;}jJlVbh zLYQqBVmy#jmvrOD#OjTcLF>wuoWJH50}bBza&w(jD_mn9YMpf@ntHb!I~V4c6U?P8 z7c@#U*ZS@88i!aRCg0Z<+QHM5;%!;SadS5N1IL|ryz~~0tg7l$*D*jF8mTCGFmIVT ztB&7b!I^NK4R#teAoi6!GjGjV9#es5YKS)nRimr0gvOSfvYWMLSR>gO?P@jd6(2Hi zFH#2X!Z8iE`cCCHHfh&2FH6QZki@8gyYVFhBS>gC-slOcw)tgDx|yH`hy9a*$Am=& zDIISLmA4E`U{c5127ZKhSZ)mb7!Mlw2_DjrZlr&cxw2BA4Ez+Q4Ezj}27ZorH9Xuh zNp?;x-#vC*bo-uxIm{b)R3N>N(*{;?*1#`tTEpqR(4KcfhX@=bR4^;UUMR7Fsyb5* zk%4MXQSL*Bsd~gLE186chA%fVYueptNHq=AUG9=D+93p{ioPmCF(-Q)JLEfDY3Y0q zt>BRgxo(!`txG)JS@4G*rw}GD&cimQgKf-p+C4U*J8e{&HX7_oX>L79`fcRV$%Xl) zC+4OX&rL2YEKbeNr_We1tLAZVo?`e`h`1}fYSNSbb3L=RebgW)Q=M_Xh zfu646GSmQCRs{4IKT$*mPl@>BH)QnLbr{X>(s&&BNO}_~Ca%~mnFYy=!eQb&jv6YGA zT`K1N5Ul(}r*fstM4if6l()*B@>bbf;GW7I9fyoq;J9Ds;y17u0DqM%#n1o64#m(y=~bX#Bx-0y=;tER%jWfAf2S@FKn+S5GNu za%v?!^(0TJka~(mfjv@y>=}-JOb5s^$pjreSr6!UWcCU6;nNxo^+SRP+|ltgVN}CO ztvtXL#qyhE`3#;7SV~w)ed!Nao=Iri_(Gzdfgc5PjP?PGbTs1p+OFeo-j}gfCpcRzyMgfFeMRl>-;FaklBU&N|p@1V4)l zNF4Y9{3yh%6KJ3nhgkTqGdnZ;-uQXe-+z4l2H+uf9ay;K;5IgD*sNiT!S-DycVn)G?yLT>h_p99muKXwcjZuV^I{A^i)iP^^NWrP&_o4X z#s%n`*`5z_rR7LG{}73>mWe9CR&>v(>-8B;&fwFy92TFAp$r$pld`%ny!}r~oa*lT z!Q&YV{p1jo<<^&=K8FjH}wz9_%%?4UBh+*JGev2l)Xv}%YRQb!=t%T z8LBCl8NI^BKLy|GSzD;k@5MY7&@tGw)@Zj+mXWRA{Q~x>NH5ZA(!e#;X>Cjj;NU{B zn~huqEQU*1qMBvOF5ohGE4V_Qfgg~qkhSmM`;3KCe23z|8Z~mW;Mkm0O*stzRbn?i h*l6N9RX50Q7N=~gBHn542E``KtG~T9LN&dt{{(ebX+TA2YB|oah^Du72UsUT1Vq}a920&)pq#pHWHXNE_qG0v3cC-oL&=!Sf_sc z%!Eix{-R$~;LiC@s@v32^{~uxDXH+LSfDI2tG|E`@kV9jJ(^?tpgi5zRxM=m;r-W9|x?3S1 zq<^QxSl*1K@B^((6bj#(FrFwUw2726G6=;$nNdgTK#fkHeDx4%YYqvuP&qAwp>iP$ zm)#Qroe}2R42YzOhBAm%ksT#Lgi=!*9UT$!+dD%-p>0Qsu+Y;+_2bF4au@PCV5Q!( zK4T@rx&9Ul(VY&tg(lm~DBi%v&Q~3_MOgd8;@-V{lvT`Oo^brncB?3%OxXX8v6G24 zLOEw$D9%-sx&E0GHhKKX5OP>g7`)p+1v4OcsXEIiJT%O6USLJ=O1u_#r=aKbN{w?V t6&qOOeKos)dWx%H>F4nl!;+YNL+)yd{0j;Rg8zzhhFIllnX4RDz5wuQf?5Co literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/StaleMetadataException.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/StaleMetadataException.class new file mode 100644 index 0000000000000000000000000000000000000000..de35577b19f153c6231f18c1dde8d56d1261fdec GIT binary patch literal 482 zcmbtRu}(rk5S#@bLRhk)XV^FEY7PhbET}=`OyB9qV~%|Nn(A4UNzV!Ofh&BfZ_FZZ z|L?Yw7oKwiYp4^h{+~)XTQzZc?yS<@J0Dcf&N(yVm3eW1lI&e}h&@!0$XyjyLQN`K n3@ErRidu3YIerJdEn6CbRgr-mGzA;WI>827!sHJHOW67W;0At{ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState$TopicPartitionState.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState$TopicPartitionState.class new file mode 100644 index 0000000000000000000000000000000000000000..32377e1f4d9c842cf01c11549019012e87c56fe8 GIT binary patch literal 4139 zcmcImZBr9h6n<{riOX9+gjgk2ln`HniXxzBrJzt0G#0CUU6xH)3A>r>M(KZOe?a{M z{n8KZj1HZ4rv2U@)#*7mn`9ey%2Fr8%(;7R?mf>r=Q;Nr{`&j3KLAXjlEe_6uqbqq zc*^2a78|_xjK4l(@toBcEIw!P1@C>yUth8Kn$>TTH~}?@O=xj!C7>sfLNS3-97Y^3 z1rnNN+O=JMOQ3J{rTR+EmsPWrU$xBAoIt#&JK8q?*ri#!yN2V)gWIaTp_YxUb;~v! z!!iZL3xQZgt=as0xB6P8eY5mLw{?f~Dz62nsyce3St6Ivd6_rfqchusJn%ByP zZaQ|}^`@F%D;DXFG|;$6{={w>rs2#BgtHSH0+9u4OBd)}HB9|cZFf_zK2{Y zorh}0&A>pmonpQcmAuhaqKPU&;Fw`AG4J(UQmTpEq^fDUZI4Y(P73HjTt8}hCfa=y z2<;JA1NjOgKkEdtOk8O1N(<{MV--7r)gB}h(1`FFqc)T&uX3H0jZXOE<@Saba@woh z^eQu~eBTLY-s3?WbL*3;guH8NvS_~Rgs&6-(^-#mvdWsSO7=U$G1XWTre0SCt5((Y zC4-yg(1G@KnFoP_>o}{>I3jTxrxaxHp@LEVx`2xcKH#rmjIhWnxPfT}m-*`?@14Xo z1!-2(te(NyICd12u`4jznlsDgvR+clvS}{vX}WBo0*ec(Xxi9?CT}YtYnv~1 zjb>&~Yl~+wjbe(nrmq!gfKjJ!ZzBT7y}g}ACXL@YZZ5aq%U>dWi2j2B;s`O>^c7-8 ze5!OY*LmC(iD8jK-Q##3+gpnJsE; zuB|D$!!;RWCc^Eu?vbqpX{%mU4O=1Zj|VJpMYmg6A`AC@El|^REiBW@J`56L;Wy}E z;(kCp@*^VQpJc|Fs~DdVD;`^eZe+|#6rt2z(U-tY%(!NmA2u5$d%Zu?M@VvBx6|#h z6U8jQ6}mi!Tkhpqw~Au9(S7u^qVnFZzkyi&w)F5eS0mY($DO8~CRh9D4sM5dB0G0| zgHeJ7^O1ir$AUXi59S^g++bQ6-A9s{IXt5u`#aHfR3{dFGm!|Ei6x&1sK$ata32r+ zxQqplfcheFx$NsiG?)`B_~ZZwZxIO={;Dssd|HPS_z;f{AnzmOAAP+`wBFO2?`09- WGL_c4EU?FzpfyQ83}G8tB>w@c2|)P( literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState.class new file mode 100644 index 0000000000000000000000000000000000000000..5a13a3814f8ef5cd4e90749e631bde6dd65e4eb4 GIT binary patch literal 10200 zcmcIq33yyrb^hkOB!y+CrVSq=go@G*Bp5vT-4Sum%VTYYAN-gf?y2_Xa2CKli@(=FMmn5BT%< z#qYg)-@WIaD7R&c+cogL5+Tp(xJB5j8XaA~PSB z%SRHp9}h(F(HK4!M=Ktb@yF%zi6}l9M;pdt_>|l~9m8kj{*b)10moG)}-tK?ML#Xne@$ep_D0Q^0~t$yW}WXdvZCauq$g9i%wBtMzMUXm@Z_FIRkn* zr>6JcZlAEb%cV@V`><2$RcJY!8OqtEa)J97SHInAhFLo|)P1;A$mE84`?zCPMXav? zMQ^U%oG%P@+oN{+xYIps4-VVi>HNq@KG$8Xx)#|+jj4t+5EK~Hb-z8zlkW^1!I`c7 z4IWWvT{h$7N=2%aE0#x`LU$%tatb*+TkJk;oHpwFbxF0t+Sgj^c{#@!DEcbz&^C-v zu7bKvVZMHSgHuW$-$lzarIIsnaB#5bl<2Kl1J0mb&Xx{29Mids-_V%C#s*Y1GO9Mx zH_cl!xlE}~Vatl&QpU<#6(YOx0|dhK{!Gp}P#!tv6pq-(vYct_&!_F|t#%^g;qu-JxolvnkCANs|$XGP-#@5 zo?Geyl96*xW{M@^-GsCnmNNn%?B_D7Kd0I`W22sfw$!X`7_oi5QPmt(Q6{mV=#XO) z#DFWjLj2L$utBoAWz^#eg=y)mV;6kfGTeHif?u+X!ps#b1KJz1SXw(b1w}*$1@+Fr z6lkpLcK0+L*Zw@YMqyRMChip*oAWxmn0%r-c{%-3oWtThnktT>-C4j>ee$er6SS zluLP!G8AHNaiJ>{b~d}qlp2nzGs~2=J=NAVZdZj2b<&7ZEeiXpg7zEL%1Hvr4+5t^ z*-<#wa5gpI_<&@u681_yx0`&fs&t&q6ayPP6#oWnNVK;yF@j~+ZrGeg$RVJfZx7S& zxLw?l&FX&O=6q52ZA^vPBN_G$xuM$ms5`IF;RCF~(4N^H>ErCqD!ElzK}WfTR5yFN zQi)R&h8Lxh@bqo9YZ?Y#pAJzxQw#t;!)B;l6z)i5irOI4JT#5UGzpq+XSYRP*aHLJ zKA#uZ@gB~X3u)(ujC34xCN*NK1)3I);iQG%gUv+JSk?C3WEYRKMp-z5=UR9^o;&F+ zlfc4lc!7oA!HX>XE?&g+wb)?arLeF9`aRjKGh}CV)!2P9?dU#GVQWyEWGT=8*o_pE zc5Wc)$A`m$lN>3R%FOq%q;oQzEf+H<9MSo9OC^+Lsa9nv%&nYf8ta|qOm@I2&_%l0 zOj>H1n$FNCUt$-NIp>5^Nc#QE0MA}+sTpdfrP@SVd&Lbq3I%&iVlAe=g6FDsHWHSa zrDn66u+$vfWvRJpp2D)mNZi7(ni*9cmYPrP@iL5A_y9=VaX)Ta*{eu9qR!QpdXB{B zh@Ac!yw*|+)Iv)o)ist{q^?%DW%9X~*^4VIIgn4L%Y_0lovgq+IhZe0Vv~%j#g_fE)xqPmxD)+(Dl=tGDmg-Z%WkGk;S??d(r2Bj3tes9f z#p04RYt|?@^=jT_reakz+$P>Tbgdlvyeo02ZvMkVf;NLu{~LmFIbmnZ&cQ(`hkJuq zsnPCQ(QfccXcq{eK2wUY=aY4EsyFD7LG!RPrt>C7f+)C;H0@R~qq26rtiAP0s4HpL z%mep1Lh>$PHU#zD7#wVpL3p%G-*2h|$AIzA22k&u%_8ltD`D1Uw4n*t2?Q{;^G6!b zcU}oyfBPP@S)-j)GOvoK0(?d$tPpf%o_2X`tOM^`sq?}gX}n$;*y+~P(9>FG02`N8lMWG$;-XPlYc!uO1NvcMKTkb?#W;x7bmlr7 z3TV>fYtn;ZWU0XjcSR7Ib2XoHo=$2S=bHHDrNPq|i&E#&M!&ST%^Jt-PX23~!^=44 z22`ErX?GqSM^odNKY^?EuQ~^|fd{(I;W-MYu$T`E6du6BEs@TS$T*VYxaJg{-4U6< zqSF^26U!pnvJ`(32SeXVD0C4Bn+Sw$I7$!S#*n@Mqj(`s@IFQWh|g2pH^A`c`6ePZ z_%^@PwRsQL;>9R1HbuU-Xt|jwaYpR&M72)@V>zeC162&Yk*21%v5leDR(8UEL7&vjU zNiWM6ZMilpX_H-v!`v%@SSS+DVYxzzm|WqyJsLTUh$MK$mTA-qL{tl7zJOvXv0UgL zB%04c=(Z?cevyxgcZ7J6yyh1W4-}kqb0h~hNtEBm2V+CCo_twAY?CY2Fx@Np{#Ez` zUPZ_@?h=2rr0DRKXOXHE^vZxET6{;e_>OoL{?HZGPrD)%71yGYXVF>f*jLl$KhpA& z+6Qr8Ufh+(-FP+Eh*JE_b0yzFx#F{KH)yLucmQ4dSA7Z#f|Rk^g_O${tHcyTUTs>j zJ1`A@%*1^iA@ury8r{Ac-L4wbi2OUa)}(6;>H@l=+^4wDGxwltZmVZ*_gSo|wfK!p zr8fnRYW0t5^(}67Ef(QZg=KARSnj?=Sl-~{vx-gvCC1RnTbc21)3R#mx<3fC z#C>#X5NP)VHhVx*cw?R8nFW)Me>;!AHz1+HW^eKxJqcIW)_VCJ#ML{sluP4E$uAhR zK)~Na10{pvCUw^-)Vldy0o^>bnlySdcfQ4!^e`m~_V>BPzTZgfQ8{*lG!K5WK&*b2OtMdhD- zqf@vVcjHfWn(T8y7Gv}9XS%16G9+APR=t(0%)6sBLda%1!cSvv6mMfaS}pacV&k9t zFu2k6R-5om6}<^2Orja``9Q#HZLTtMOcO#gwu{S8OVwZWZWaL@X73PE_y&ScUM z$y23JZ*6SPYoUhX>oMCs_zM>&-=THVtsHTiwyC1KzD{(vL^>j8vGo+Y853a{_MXGG zu!Q;3fiN%NA2fh}2M{ADQuru)xsQ>aA7r2ParRiB(ArAxu^#g^IYqc?sUP$se$e~b z$VuZ`3A(H@L6>)b$#=n^{}qG2u@dyR^EH2S;#E?lpDqyUHKe}kEc$BW^BKnHp()~H z3bDlOZ`ia*oxInt#5*ZhDi0+SP@!>LS1bARl>BfYNWsV($qMga;@7Wa+iRKX(SWF6 zyggBWi@%#f)Q(!{eB~0N-sLBalrZt%8bb&>CBt_q^t1KQgwVkUO$aP!gvmW<3g377 z;uDcdN|~PQ>-;2m3@h;sbn?EMf6XCngMfDty<}Fr$*rHfWR)cnD<@~h)V%=RC%=Ke$*BrGwXV304AFyA)0Phj$S z-{whr$UISa6od9mLzzh^gmB2 z?m(@$W{QhGtGJN-Ult0Lu;o*XSe_ zIvt28lSLBdhcT4eN~^TP&^Yp+#IxX$&>`=qR_U;HJYXnv%}6p-eWm4n77wL;5W|R= zx^DtnBxoJsAF_~6RKn2qjSab&iC`l6RE(#B2a%FGP58{3Q5HzcFSGGj+TMI3vLuzG z-aL@AlsvZ>+*l@w2p3D1Ofh(Uli5ICtK7!!UvtiKWf`mR7%u*Ahv9HpY~t8xewSbD z*EbH}&`;2-Vh05jDVGR2aH(Rs4~lEVbck~(mrhP!!Cn?UWs2T{-6Q2bDukQgU06qz P5cQ*mjm4}wAqSgZho*y8 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/Callback.class b/out/production/clients/org/apache/kafka/clients/producer/Callback.class new file mode 100644 index 0000000000000000000000000000000000000000..1d037d57c50b225074b19af94387def1dd690ec5 GIT binary patch literal 231 zcmaivzYYOG7{urE?+_A?fxwHwk3B&)KZ-dbLA=dFxbZ8I= SmCqG`YJ@6kr9=Ir6*S(?VMCAr literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/KafkaProducer$FutureFailure.class b/out/production/clients/org/apache/kafka/clients/producer/KafkaProducer$FutureFailure.class new file mode 100644 index 0000000000000000000000000000000000000000..fcaf3b862ea6ea0635aee9df40b401f19ec5807f GIT binary patch literal 1804 zcmbtUTTc@~6#izrbfJ{1peS-tLE3Uz(TbNM#vmkMy+kUBybas2EG)atc2`K$2VZ>g z#TWmD51J?>n&`Vf%6MkGltLQ?eb|{fXU_TNJKvnypTE9-2av{e42>AqFcE_Sy$KpF z#b}t6!(|O=4OcZxF*J#7LliyJu^E!tb^e^|zGqsx;n;@ny2AGKxou&P&#~wBLNg4} zY11~n8HS!jIm_a9fkH#DOmdk)opthpp)G6L;(@=pD%>1jwaC+%bqsDTbJvu1=v2Km zv&7Ic?|Z&0=DBIn#1LPwZQ;&Zyi^jjeLL&81)UeUu_p8l{%nKmhGi1Xl3sM3yl)6s zzbD;~LTl&%9x?B(%aCjk38LA0houPS@?)Z}KB30oN5W~P_OS3`cmdI0; z30Zs8EZuc%Qa^nJszK8P*Z**JT^L?Q!!-@p8M;WTnoSaR$t>7hUNVDG8&IEc`4*>- zul|d%~TKT^S(B-{SK)z9F{Xl#mI8B)WG--8a~S7tK{9P zO4q7%X`~`9QshVgmI%T2CEh{wPf*noM=4oCMUV^*Phm8e$8dt?7%8IRifp;xU&57s literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/KafkaProducer.class b/out/production/clients/org/apache/kafka/clients/producer/KafkaProducer.class new file mode 100644 index 0000000000000000000000000000000000000000..e5b26cbb48f8489983f4f5cc0c541db5e1b53c5b GIT binary patch literal 21339 zcmch934B!5_5V5dC3!EC7rC6O(qOJQ{ z>+a9BT3czWR$K@~RO}+QYF%mC3D6z>9Nr>h`U3nYZYQ^Y5qZgpW^K{KOLZb zW%Li_`Xhn=Oo0By&&uO-^6_JbpLh5Ln_rY+FWLN)08QqX%lN1KGn;=Npep`_412}l zS8aYxI{i|ff93G69sZ5n{?_K-IsE$o9nOD{=RZ39C%OLF;n(H*7l;2E;J@+T<>3vR z-we>Pg40`N{5Jo?=64)^*WvdZeqW$}5TH5yPwDzEhyN`t{|WGi{84~E=1iQ6+22f$_mgjRph8*N0m6rc9avK6I7|K0*)#J&6Vq@az_mb&?+?)fYmTZ?d7PT zqlVjRM1VXsQd&m^)ZS_zhl5zNs*sk^jv8aDedTsPAfm>WQKuRwcOgfOm*xrbFws%_ zgMI1%TOBCVO|sQwM@=s?Jfqn1na3Zc?UX*tnWCpoG~ z8c!}$tJEn0b*eheR;LH(eszYe!nSG-&_>l_t5!#OGJBhRw1ZQs!%-0#d8V|imXD~d zIvo{*%&EAex&pM7U$)g39F=fX(ov}ZJ;^USs@qX(1hcjBvCdIFGR0ZKlCuM9y*kHM z=N5l)hxsSV`=YJB9 zC7G;fyqzhyFrH|yO-9=eJF|9SyuIB^OlKOiblH+QE9&PiJH38k!`#KoPj8rWdgI&^ zR?J;oKbNU$;hEtz;o9z0BwE`NkF|6s5?(A-8&1VLBQ3SF^raz|@(ir9u5cn1Nk!tA zn+bDZx^P#xrNgUT9d274u5F1%Fg#h?m58@?w|I%#rP%?%Wk_du-7@bB-Ci=)7&!~e z;0C5qon9*38cv1hd8w9;<&jP=-kn;6Me%&5p{aN(99`sf#uGheh?DNlG_f!Oi+q@v zv@G1x(%sn|#hQThF@?uukd9!%iC8$AtX<}{#1pNvvH$?DBo&aW(3yw`=hUFyVB40%*aExjHf+Iw>lC{ED^e$iR z^)z~kNH_}qgNBC`9G3KZg+<(Ivtt0Hv@y~i3yVH5P1|DtGnOxyJ`+%et_erGy)59* z2WzEC+1ge;CU4wMPR}wP@7?sS3#3HGg;PB)ewOkbLnm%PE2wfj<#e#<} zjKsXf-JQ)|VtKeZiWaDSOE|hR465SUZ!Ai6M8L>{^N?pC9t%Wum-@&64I734i@@li z1JHo7_MZ!a=FfN3o=zleSZ=6yc>if;lyF0c6_EfkMnR z1BOn|gT7J4f^xX$BV^)x228x0IPdXfM#?|OFky)xT9Jz=q@&A&>GzWQT>#?lKZ{6Y zT}VzprgI8F*khzV2XkZvm;Aq)|Nl(OoAcSzfoho6K;Invx2WVAi=H-IH3jEQTYlq{ zGDgcWKEGjrWAYUj#`?_8bxRXoTVx&X6JA$50p|+aGAu`I7DkdO7@i>sxVxU&;g;3$ zwl;|X0%p$TVKGLf;nr4ha1!s7m`U?HDxa{6z0}%xVs*WK1avp;K^KR`VR4FSWR6{d zCylQa11g5tSU{%w0Z7i+P3k0<`3u!{YLD!jPdM>VHG^i?nilg?wGH@ajJK@zQnL(I z^hD*2DG(11LmOoTlE@s9fFKF9)dpLA(^lV-yoaAgronXge?uz&&$?5Jm^10kYFQ1o zkMuQKhdw@`rXYl5@N;|)FO7Sg7f*DCQ*+m~cwG`hfS>_TCuznZgMv$yP>$(@Jt2e% z38zoV;SHoE9XOjXT|~$PS$h@<_^_Gk^2k{sdtnCUxYIB1`EoL&ce<>6(mxO@gAHti z4(YX2!Vm05qTt6LJ_&YXM143KZ3aVk$qK8b1@bXAM?Q4kR3A?E6OQt|2wZs8m;en`B~%G6XIi!!gxE>Cvj1S1n6)|&P=la}|7&M;MRL_9lta1($&C`-T`8|Mx6AH`#_aC4 zHZL)ET}QY(Ibe7`2{9Py;52t82eGc^JJZxW>li4H87Yxk2shj6UR&LVk_JeQ`h>1R zw4&a^)Rzy3fzZ-E+K@G)HG6``{|^ltvNjw^Es3SI4l!9CLpfh!B6|&-FN`7XylA*9 z>B0OJH+$_7v7Nd;ZAq$8 z20`90chb%@ZWjZEl)D0cvT$HnaxHWyzBa>2c+it!&DtV~9Fr(^A{*J~__%(>p=BO) z+KaVF*fcy6TN7XHwa!b#JJWi{G-pqsnKI+r74~rl|u)S>B6#=nb{Xs zTiwspyyuJ0*r(klf-?hwwW&dL%~(*EvkpJ)UYVzD=HPC5h*ZdwW{?C68spuG7H?ie zS2g;TC2B+xQPUbs{6cU|g5vU69w#4%xavFV0r}V{kC(gjF@1vSC?aO$wXyb4s>2KE zDMLtZb@9?x54!3h^{~tP@_umZgXcDkHwCW79*9z>XtB#9cqF`2q5hqwsOJ~=44x1g zzqtmYSCb5Pc169Us~%CCzMERXN%99cQtH&Ckn|X zmjpZ2!$@vi^<(us)1+MuU1Z2F!(MRN<#MLIGP>_2YFebI4&yf%&LC0C7PnpXqI$`t z9k8S{OsBCm5}P8A1!qu90cCVRT+=2+?i$~`3)k4!Zuj)? zvFax-m#{75M{~lI*K5S{y3DL#%$jxJRnw`f=q?w_gu`;`#Wg9^@oe?7tA46}=JGJ! zD=iy(UjPM181>`B$qsnQj5t+C5I23xRXsgR>TSNtRsT@$xawW?FjPZt4CFv|d9-wU zk1t1L-QP!qlHxx+Sl0Jt=+%J_kI&(AUG<^*$W(Ko5l}r6lfk*nx0Lz_(%pXov+T?iC@=JfF-U|g%PMlYm2cmPB#5Rb zT5C-rm9(v5*DA4WmoMf^b~37eQ30Z1IZV?d5KUmf|MXX z_)sJn@_-9^i7^jBblSa?j={pAn71~heSu6i)U}2|<*mJ};7&q@Dj1Dc2#}|eZ?Ox72Lzg4YPm2H~5s6hu zyU)~7!o4bZ-LbkOb8ui-4(OHT1 z?oONv$zj4cVS*n;z$C?D@l>eUgYvh7I*4Du-+Giw1aTqX0GDgXY9we;BpL;ajusmR z!EmhutOLbZO>(Wt)@WD_jm}Pww`3|9t~JG~WO{N5Tu36Kt*0*9WH8lph}k>{a|!ui ztwk{6prf}2OAK`<5edSv=%`iDZ|xQ<2(gN`Ri7D;#A-szu`drUhGWSr_h2mJFi|b3 zXb-^Z8VYP1F`U>4)CYMhsV!8XedwV}kKX_Mxlw(QlCbASj{qK#99q16^1Yc1?` zOvqRhk<1Q+7$HGOkc{|8OI+(2)i=v@XqNV zKn{@XlX;C z;urqNlLn)oRt{wUh%*c|T8MThm=@ zhIKT;=v|ApupFif+A9%67Y#5heK6y{jZMzf5Eo&6?#V;hJV=CP<1O!|EvnEZi25L)h}8;L_6L+u9f;3L zvkiy61Qcy;zfgu)3Of!3Dd5|Zeib*=t8HK)u4|`z5E=#Pg#o2eT(bzwM_Q0o0puR( zU0?}sOwHb1)Ua@2L*v}~C5z`EFVm^>iMnFPv~-u%`aNDzDOQwL*8_J!R?`owDv5fr z_7wJKO2T^Y5w)+~8tPXI0bwYnofT_sh_zq~34RbMm9QhLTV}5804Ehsqd6kI=aN&0 z1jtbUX%r9pOMr|#8yQ6s8DbvXa%R#=Wxp+9fB(pWRu2$HzotFqWF{|Pb=@d)Ug0q_ zW^7?$#g;Dq=||am8JTxkIC{7TtI~w>afg~sp09WI3MTZ%FCn2Ha6Z(APd7(l=H!{Y zu;}H72bNh~vSF1zl$VQT4u8@#AKV1(0C8r=<52`h132Z&s&MN1$qn-hDtZCj=FdO5 z;E?|8P3{&abGR>lKp3lS33hd0OA2~Ha};TXNCZm42qJK5 zR=#rGfG^PF2G3&z*58PNxyqD9nGVeZau)!=(g}sSWW^c>t2%WME@jGAAYWqgH-t-t zbt=7ArX@E4J6LF*&(ED)bCQQ&9fgtWU)9w|!t&tfhfa&qCtY#`+LzXb`3-s({Ox>G z`NMT=XXGNEi<$lxT3soM}cLfQ1cIT4qoJGe^`*|XfgOGGBab9F)# zStC?}!{u=~T3(7TRji01rGwunZuOeG5h(0pX_=z0wh2YZU^?*ygj9TCd@a7M5tai0 zC8!!`OdM?hbOdA}sX_U^pC`;1_|i;G0u`A&gvg<6+PKQeB}Y7s*~%K;53uo&aN1eI z`~Ad*8IJ(f&Av?=-<+m2jAEU1fjoL!OHMZwW=mSp*lem;eJi??jFBBuX&VWx8zIker8_DwvBx>TKCltH*anka;s5m4~H+7e2T}6uM+> zc(vD!y}yz+{WT<}*?Y7EQyn#I%@|BLj(AL>5|jq?iKO}Y2tJT51Ayewp46E?xS=n3(%YWN6UAHz7_rEJ_OJrjlAJP(I642jqN{-@mB1jh;7&PfRIl=-B$-nX33dnIo=dEB|KDiu_Rc$+7V&J`V54R1!m@D=0!iv>zO<+#$Ls{g*BG0bJ90XWrw`~3Y$*SS_M*{po(Er-9xX>z za1F{?47th~2z|vl3w_1$2Ytnv34O&00)53Mw!Y#-h`wUiSzmD+LSM1xtgkrSqp#RH z#+Qrt!M)-N%!57R%<~vL~ zzrK-%&`mr6&!bFtp2+*-2?O~6J`f-$^?Nrt^KME8g-Shfl|S*-RD!9AMwp4GR&Svq z%s#a`&+Olz;j}@|UTMZ;W*=c@7dWzWu5lZo7j9{cvi%K4(jOT4{VOv4$6&RFa8osw z)%XmdSJUp1=K+vsqegHrd1|;8z=S-mo_Hvb7i5GB2l>RfcrY=F3lZZn5aWpg#L#mA zc79@n$kOl3{H9kv5*!Sya6Zl8R@_WOu{u((S8k?~hsX{(eN?KbvT73rz`L@h&E#&P z@~t!kcSEa!!+L42UJCZo@T%a5UK-g;qaLEYdugA#qN-p;FO9A%t|;!MF}KrxTWMd6 z*)KS@m&QFrp5C3bMWR+bH9lW!^p#%Mmbf%BTUm~ znzo6KM3dMf3WJTE)S}gDTXbDp< zE!|2dG;O72fZS*Rr2zz&3y2kZP;pgorJ1)Xc%tc66+9{3(qwKn)5&!%*1f8!!j)0@ z!|eWgtAeNK<(+z6Rq(X*z|+l*%;+=zotlZAn<$3Yv8rB* z7xht>MH^@cUiZ=$D#7nqwdtP7c2A;vQo38&?g{B$y-ByHM0mUVXpI7>(RD?YrI1S? zv~DZ)GzHJ9!0p-F9)yk-;mF_fw3dpr0eqXLLuowSbU-NYOEia}&aufZDWAr-r&|ml_`a9oAZ}7M1O@4^p;>YQ2eu3WQSLi+d1HI30 z;I{&b=s)Tx`cN&PkJK{yM4e8bsTSIy5^ymWu%#~JB6Ss)sB0Ntz2Z{!5SOVf?5f9k z2+m;+RnPJ$oKPI5UgY8GRUU~mXnSkRaurrO8>0{9!?5o8>O4N2kDy}eQmc3x?n>w^ zbpjvBN0Cii)NrnY2Xk=b>PemsXAs~fPVfx8Eo02YNApZ_f#CG9 z0zLy113PFK0aSK!fwpv%?|M$aYH+g6cRiQ;u4h_hFRkyTbAspg(id;WZ-i`s-9^(D z`qF(1x6=8r62^*N0DZ#?(YeiZ;Wi;9f{!tD9u_|y_d~?|^B5Y#`{HL(_RF}debKU< zSLpuJcqLdwnpmY|4WGxy+I%7~Itj9AB8C$IMj{5gvWk1@qK&vU@Jj>&Ps+eW7fweB zHcqju^4*|uk@C*?WO;WA-WmUUs-EmLGnusoOh+)pcUMBOFNT62wwW&R&BDcKy0nij zvuN=Knq614m9EIp;!4m#UiZ?Md+91X1ivEGd4vLDL9X`Ou1Qnt+HJn=5SZ6#m~}-0 zv^W>&^#F9Mh?2^YaA&t{rCXb1 zWg}rzJ#LfvjZPIoj}RgFzWQyZI#tecQ%SQG7TH0;lXX`2DDcPABNFJ@9Ioz^)|4|g=HL1m(v-Pq&2w8&uPr3 zZ|ZA;9?;hd=plW*nRFpK7JtO0@%_EVJ|O&J-8UQ;u5apOJQ#=<2iftUO2;3VJ3XW_Ki1^hpJA>WMNx8Uhk+}(z|+xafOLkseKh~y{I(R@0e0exKO z-xXt>7KgbRx^M>Uw7E;7tp#mGw25x;@3zy0IAf<^J71z6d@EFeO0QEZw{bhvV>BJa z9lRR0a{*0)5=H?bNyBj0iMtD7Czrz?SbPhW;x4Wg_-^)8OeP3N zi?3d~m#CN6zX+0dn-iZ9e!(fZB6GKcQdFwVQp)CTyuyEmWm;qNTB3>_z~0Kd7L&E^ z6WSY}I}jT7f6=ql=Cc6!Y+sw#11X%hG};XI6^ogsyCFhHLPQ^d*!s7d`R~xQxF|!}&f)=lwK+ zAE0X92<|;dGvS>Z_+eVck5ChD0&jaMfgGxbq3GO4U*$*XYy6m&rsLr+m(e2rBILlK z1Ymp#p0kvyshiKkT@f9GvFGEiM97rpH0~~dSr&tSH>4FATTVXVb6WFvUn(9xL1$V3P&PoqujdHWbptnTX)@;#ZM(RV?bF$l_o z-@}c>jN^jePgCm$+s0gkAv3POqrjM_q&q_Ceq*wP9R+D1I9>d}7ahV+(+vJ0wefQ~ z{-ZrZfOe?00>areO5VYL=w)L8ha8Hxm+|HPqVDk*RV%Wx-6Sk9o4xcjF!lunPe0s% zUliF&8=HbZLhAdBj{f$caWnxmGTfg45)!`&&-@nc#c$I-{0@!d_aOoA!bZJEHT*#a z;fatD(OltEtwB8pVQEPtY?ErXxu*=+Ucpxy?Vy{0vm|H}Dyx8`IJyl~zDTf}D&`k$ zX$9G+rznNFO`~aO`KcskFr%MQIq#rRNaZI0o}_){+JJOwMo-94b%GJA93fbB)w&W$RRBuUup{eWHY_IO$73%)d``PLQG&5|(DIQbH`53i_RDd4$gd<7nu(s_ z?T7@mIR9G4ZKo-6(Xe~zH0uu+Gt{Ey%{6fDsr(u+)Rf^ zpFf&8fEL=JD1RDRatrNa`urIgSPn=uRtI0N0NQ`SZ7lfL^ldWuH+?Jc{;u;aNfqDN zPNPQ_-9jUEy7i`RdXmmu37HD?6kPx&KDQlX>rkSZ$wg8L{;<=f{vRZAT#K15_V7tXC1e| z5MeUCZlz*v;r64zuuo{R&9@!PpU{3>4jTG-jKBfjp9?tO_&Ez>i1(HYW;Y#0< zTFZ7ixuOUU7sIvIl~k1UA(5mbAR4IB+=_yPTu0h>QH=;`I+^$f%(lFFC-^SzMnZ-r z1mDA*PA*>FrU%^zwjV5j@ptJ|d|Zd1w2$7m=|0S#V?R{~7>L+Ioe86~n#QS4I#|W% z6qTT6mC6ux6fH#5brgKxII8Eb!r+6bv_4H#-037jw@Jh$UTeLMv&3w!=R5f>khcV} zVk>_QgcG8Q@iTDm);?9B33;WzNjoGi8b;P@jpE%eWa300|oHKA(o?}sTQmV|jcK25>&Eb!oLN*5J3@Hg>BA&U42*B~&zAC=+{ N;Z-6uhtH&n{{^A7RT2OI literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/MockProducer$Completion.class b/out/production/clients/org/apache/kafka/clients/producer/MockProducer$Completion.class new file mode 100644 index 0000000000000000000000000000000000000000..e2dfb9c6c71be32a0d082a1cabeb820125f7e84b GIT binary patch literal 1819 zcmb7ETTc@~6#k~Sg}U4-UWzCvigk;u;$0NAM5CmrDa7~bc7TQLZryH;f5hLwvkBFx ziI18X|B3g+ON@zlW?G>j+S;UZ&YU@y@0|H&U;TXg9KcCr;?R+9$2N?}<)~anA6-B@cJm3stS^ zTev~lEcsrcVhP{O3Cr^f)Ag8fO6H4l#i3-G7NTu85WdTuvU$6fFemO-L^;T*9F(oi z;*OK&))K==3;8DN-W1sxcqQAq&Hca*Y|mv#Z4vJki=J!FZt#>nT(Dg`m|%F_a?sQk z*_+I&g@`8eX(7YfM2_5Hh)j9~!O)epU2(Hg%nN^(=N+;nvK|fk4)<-juNx!5f?Z~a zPkP0YBWgp`Z@8}TCmmic3pyKb0l~CqE!A(=Ke(PtU2ZKi0T@VS7x_JII^3N%a}_tR zi{k2iOO#~UafzLgSO3497&>MGV#+ixsRk6g&uTcQ;XHLQ<5hf1T(f1nJ&l+~Wnmq= zFs`Eu-8wqZsiOx89ox~XV+VFp5*-(CQNx6eOSr7z3d6bPkE5|nL+cFP>&@QEFA9sE zOW(R_bN>vNo7eg`RTzfH>{F0c?ICzsbs=d+s__ z7^b#@u{nCvr>KD|8<{lP97yBEVURwbI(=fJNJwswmXJhIl4znNF-1ut3&{`|@o6`$ zN!mR?7#hj_#$zxe8L1*<3|0{~lF=$6Mlx1K)JRtmd#FPDXx*m*-XMavh~piVdrv(4 zfIav~a6XX)CJ>WJ(T@R=Qj^7*(j?4~(j?G`(j?TV(j?dzc@9!c4dOF8k^UKDg%GVE zmVSvi$?!5_z0lH+&`!JP3k=aguT<$DVowkcV;P;&z|v|bdk91J(5LWyCi#Wxe8nMr z!)biS75q^6ch)Md;m_azh6v9=icKT3O0O@Z;gE)5DGzH%ty08qQ@Cb|48+NJghE0v PXl$TyF}kl)*o*itaEtj$ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/MockProducer.class b/out/production/clients/org/apache/kafka/clients/producer/MockProducer.class new file mode 100644 index 0000000000000000000000000000000000000000..1c3ef7e66c4f648c9f1b05214a40f36e5747ac0d GIT binary patch literal 9644 zcmd5?33wFc8GirlG1<+8Baq-CN)W_kNeHzHnqaC#sX-0}3PmXGu$hFF&2G8}O7BB0 zwRfv+#oE?h_R!u*2x+agExqsizVG|KZ0-B~Gdnxk9Y|uIK2M)K%>48J-~Zj;_rEjC zC;olnJ^&Y}i{jXb6ER$YNx8gFF7KDieRBDLf&1ey9D)xTcp#3r^Y9@9AC9BZdH6^i zC-G4OAB$r?PQhV{(|1FNS z_`B@=M+5$e2W9lH2K*Zj8489{hN@GD_vMC1GIlYY%PN@t*{q%K%UFeiT_8;=Qz{hg zyuz~0x%^O+FggN!9!M8DmOfw%XalSBs~fZBUZkc7Dlq8D_}<^ZD)&xu919h zpp>%nT~}8&aDTYKRfTz*ud@zYU8Q0=)3rHWC~_sXBR!P0ilw}*u(@h^{Y>Pz0=L_# zTz+8v&Z~NM?&|5~AC;yVuK6^0zK{9k_7h7sDYjP6%|zmsTayE87>}7Gegz{$f`ERpMLaJ%T#tpM#?NNnk6r38?cT3(Nc%0CQI|FwOu2ozeKEnd&C^AE$ghv$yF{`Z>w*WQx@X+YtcpiHmv z;gln$r`^}IQ@L!al+W{U>)Ke7Lrl6?ULQrl9e5hv3f%!dKm8_O;l5z@=6JhLYgW+7 zRIQbB0&lU4)__&CbR&Q2?DYxC(rQV4Rwi@6N*$Uu6!q-1no^--27dMU5|(&?ho^YI z4r-y4A%xj513u|2fmPPZ6uMlTuwC#*p}5@^9+D!1I$37R%}KE;KP${BPC*9>L0)$n zUlop^CJs8Gk(-DYD}YI1CV}vuMSdux(72<>01@Tr15RVokfFlF*@W0c*TmEc6G4hr z2fGZlssP>$!r+DTf05eptbIdqn}&7@tAY#(^msen9G0yf7_Oi=g}UMhUnCSFgGAm3 z6+&%tyfk{bSlE~&vRl#S!+$%9`7{xr-$WN#M|4^U;LxC3%WP|-U^d8bN$ zb}*+a-59p1NJ?;an_m|P=kBhaJ2h+e5AxY+4ckEp=!kb{wA@H4lPeHaEpOWt+?IEy zhwaA@P3nW%DDo*+qVwxzB=ZVOr`bpdPjFVu<&V&pQw(t(y|rdC$h;ZmwcuYGWm^oi z>DHsYRVNIyf$tMBJD<I#NFQ|2UH;{#Xt8iy7_615@ zALQlW{4`X=P*Fn}hN@TCKYN3{Q!=3KI3oRqkF39{+h^Xk1J~K9V$ZICx!zi`012lG zhNl%BwIf%`r|gYsdBvFT@5`&D`b|6@FEz0VSDAPWcA40WEhetTUK5YQUhb&8mtScW z4zg-Z?8ajmrpm%)`Mh;R{BB|^wka&ETGB7oCbnY--EFFviYxTiep;wDrgB(bZVy^K zAH6KG^P=_>Obnx7sE~=Gw`W2_kr|T;{O?Of_FMnQDPrXsBjWwWvjgI>S^6 zwU}5!2h}^1Om(JOVyd&$*``{mT1~YKZ#3}+ywOnSm})t07oICjyk4^FO}N`s=c<(q zk2=p(ZK|C`XR4%XS2(A_w?C7yhpfzH`JvJQ@vHHLoS^Pi(Au$)@-iWn$ETs58ADHek`$D_pVvJ zdbLojW`lVgM_o)Qny|)Hk5DTOb-t+{sV*?ph4Qdg3|XhJdU|jBBgK=wVuw@v3d!kt zl!4BS!doVG!aO7XWjMqWuU45UO|0AxbQ-wLd5Tp|bjsP{ea_>_We<-Oc}_0pkvNSI zcKe0pgL5i2Y4UeQZ@6rh2^wn3g)ZJS&G>UA-uT4+R2 z-$eoJYPtEhyD|y8GW^iGOTKYxSWXykqzhL#+1^~yy)Vb^mJ;|y2ve`CL7S6sC2OZt zh_vTpBiytxofr2^EeTJ0^OQ<)OOj@V3;fut6(ZiMXGXXhMy-(%{ego?U1>&cw)^dy z(r3wV3VndXtLc}t^`DMieKF_ks0w^p69zs+$QRi1oYFhh$7)8TyhI1I1(@kGyljX- z%81SsT3Xwmsw>QgL^Nuw@no%ty&D4OYGqAnxIOLtSjJz3HbUXis7I3^HU0?JBsfjq ziMY6SHOFm{_Mr z;*^jciFsi?62l@GP+W53<{Ed_>oSKryG4ISt%%?3) z9i8J?P_ZVi7V)u{7AT-c~HBJ5MI(Nj;T z_GxDd)qWc5sFz``4kPQTx0@8g(@3sTV_5i34xQLXb=+*NREtC6DP~+Q@mhMSJTX=8fJ=S83V_72HF^+S(V`OKpn!xfp z?B+0ZJIq9k_N}-ZbND-xY58( z%D~MAo@3xBzuG+)bh{IfTc`?OV7OI!#(h*n%0|KwqrXbjj@@@Z^jqlyj*4@%&=-;Ii?NW~MI}c>v#Ur&sLY7JMQE7nETDjw;Z}~s zdE0oG(vf77TyCi@o?Csz6Z5Q$;WoUSj4EPeO~g16dIi^BiQ8Qvm-0!h3Li&jHUC%o zln>V^FEbUc$H?Ul&sUd_qi|J~?&#F}oj#VThNa91OJVgYUd0R7@Smhm4Xe8}E2o{8 zS;^KrnciA9UX9l<7!^O1*)PLuahGfVm81}AQ;sS++8oE~YCqiVQ`)RiT4pMKcpY`j;`w+YJhxs36PWnQ#*p*3scr)HY8IAmt&f_mXp%Oj3@;gJiN}YVk;niYX zkFxX)QXHcWkK8KVx;xx?5<5B)VU|XBBoUe5{|ds|Xy+I%yaT4(nxl82a}sO!Oky3o z{zXj}kD;3jJx%L5XpWBIlBQmc#_*_dT>1goau1D;B*Nr>*#tI(&>i7KR8RCd6H!it z9g0MRbbX0v#{@1nQ1QxYG*Hve&^Mpu=yRBh&tnn3fL45ot>?=;IKNI{^-Z1x-=ZJB zt-U2Zei8A`+gSGzQa+0OuGe_ynCHP`Uf(~akxv~dZ=;XyWxN{L+@$cseT``^jmX(MCg=8VR1 VoZRX#L8K-J0p9_6%3O)~e*mU0ZHNE> literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/Partitioner.class b/out/production/clients/org/apache/kafka/clients/producer/Partitioner.class new file mode 100644 index 0000000000000000000000000000000000000000..b159607326221170ba64afc766f78519adf8504e GIT binary patch literal 320 zcmZvYO-chn5Jtbs-cmNM2c8Fpi zE{cK&^?20h*ZT*6E1VY?6*w($Mi|zt$t4#9;bn5a<~KH;LuKYkU8u?f@~~KoP380T zzbtBHxMsT&v*G22jg8xF4CcnSO%g4qcZ9z6u@ORdGJPV9=CM^Px6WrYe*Enao$V?J zm&Z!)B&5dF8edy0Y7UOF;xB%mO1L;y{8YuTbXBXo@G>9}I(fSJJDnb(hdz+^07WK* IlO4zK8$JkJ(*OVf literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/Producer.class b/out/production/clients/org/apache/kafka/clients/producer/Producer.class new file mode 100644 index 0000000000000000000000000000000000000000..7590db31e4cd09e8ef2ee176b8890a18f32af01b GIT binary patch literal 1213 zcmb_cO;6iE5S;~rfi^AB4@$4)U|Okr;e;bDNRe6+piPK+ooqJAlJ#!1UUTCo@Ppvc zAHa`79or#FOF#+g!Lr|Y^xo6knZ3Qc`3(Sj@G*d`06qk;&0r~!ro-SzShKFjafqTX z`2f2EKyg+YU)!bBL=Gt)iam@ z23Pa*SDHsrvlSgYty7u7cj31#J$BF|F*3NE-%m7y(jjWyMlpEWGwEN?uRqS?zxjA( zcBQUPlRksRaJR*vJVfV}SJos|>lo~W#aL)GJ>Kw6nciP%)Kuarfcgw=)BM>uwuYbH zYko6bn;&QpOVSbvgN<-rz@N|TB5{fd6g zR+GW!*_WEZdcMy2xfQ-B%e*wHxXggT0zHd#l>|$4fk27SGNFYLUK!(6`o9J*0OFX5aI_Z+M7^b}BDNZ|fmN53iuN(>rqXj}AT3L+_+_@%Y^iy(dTi)I;6$ zF#mf;551S($KT%FO!Hul=N?ddyMeDCMXQM|C=?tCNv%)G|jc?<9H_zUCa|bkZt*+U=+woTgUvs9v=a zpwuOfy3|pt1bZOA!%?dpm3Pz{N3BgBRvfj?QR@Njt50{-21i}iNnJ1ctfMY>)D@1p z(ot7A>S{+_!=SHqlm7B2qi%H66LV^#qUA&5V+V#0D%v~}MzdSJ zhF6~Rx1R83PIz0(VXYR{@ll@%X15N>$9{iikD|-=jgL>3CMO1uOqGfg$BGlBsRT}b zTRlJHczunvnyq1fcP;vEtC=)IJb0 ze&B$O-j(}C#)l41jgKYg95^~Ml45q*lCzo<6UE7i;bKPUisXSzZ{X4KrB2*1G(LJ{ zqF5>o2`6J!?ilT??5{3=fs2N`s?EMv7B+4Ug>~zsn|ql}T2N9UV2RGQ6(Q zli=Br@rg+YXJ}-wRI(7ik@+?}Hg#ZR_~4<*gtw)sBfM5~4yY>HbYAPGG}8|kkF((n z4~{@5&6ycnmz^OBpXaGB`0gJPC5n!fD;v zQv!3HSE-czDDbMmgMKs_YA2!SIepp!{b*b58FPo84YBfUn75UBr$}i_TQk zH569hZY~`O>i$@(HVqWL=_;P&N5ZmKJ>~&(jLpNG_H;3C5GR&m4-a<2U&xSCwBy1F z0@#!6{%kNkb$?#-45zMdM{1#wAhxR=odG?xU99zFeENRB+}yJlZgl8G8eUJks_QYeu`db6LusD8-COb{P-eN z@q*FbW8-Wct*EDwBEl+?DQ4TMl~APY0kSGZFa2pz!%A_%1o5VG(zJXas0YoxutOV1 z_7j_q!D^dx0nA}36RQS(y%}!>ODe6hA8iH6@64S`{2Fg`KPiG&m?kR<>sO%pI*ww6Q$FaZyRJ1yi z;~_7e1B$(LiXMWk^{P$u8b#aAPRAafctw}MM->u(>PI=XxmVq!`g_$DRZz4_oR2lH zXl;fBkTlw!Q(Jr0HnqK%>J$PgwIio)?o~V0u3q&dwLPbv+^cR;w<_v88^fg&Mo>E) zhRwJcc?~}Qq#wmOwYyi{ruOF4Q+m~)+NY@hY;cC)Aptuiqr_QuR0iE@5I5a$#+?h} z=GI0SHQk06M6ufp-B#?oGhyWV$U3L7ZVq{<+nn=VmJ_0t3s+xmGI9Sz|1#99R!u!oi2kW{ZOxIqo6coW1}NHd`^BT-?bD?wZi!An^((-NG7 z9|kq1ApsfBg{^AEon{8;^>e{9{GIEd}+m z&l|zlgmR|sesD`8#AMmHwlnx_Tn(e$n$KYw0&2K@ns+sC!7b0Be(>GOLfxwch}6}E zP5l6$kW=@;f_m6wF*aejnxwN?;Px$qEr@Qxok#KE)nP4Nmz$a&3-u+>`k|zN^Su8m|aOaxIt z`j(k$Fgw?Tbf3|J&x%F!#ueovA2qZe_o^dmFDu-gN$8O5Z@Av{XZ@&GO{kJm50?yA zJy(3aI~)04svF49L+#4S9*`GC7zYPlg90Bc^Xdym!{Eq5F?Kwi@q#L_hnj%Fjj#!| zgAZjPMP5A?AL&O?2(LMYNkI0XxooVAJzN>f2JVV&NR*l_5S(JAG#JS7yva|FZ$TUQ zyC0kso4|}YW3XBc=lPl8jV8)Q%Xkll2E#&ufyJ=K#LlLD%>hPXo=Ml{ms%9q0at)c zRUZ)rDIjY=KX?PT3y?F2F>H3s6MkmsWCzch$V*~?#7x{Om*J^mb~V1X9W$fP6wl-} z8on24s0h2w(5Ie5?kDyhajez&eZr&REP+@o4|0nkisCm_I z`X6r)`3i9v}Jo?u_U)A z2un^=*)GbEqq$yGPH_U z{28xRHAw}~?h6XQ4l&Gv3Tr?!JE*ui)@r|{t&!0%qUkqD(d>gkW6`6y4$r@kK152O zcUK?WX&x5bA&Uy|1zug$c-m{COD|(4r{9fr2?N`PGL^;Y${N;1Q3J1vCAy>#dM!d+ zaT9KzB**D4txeNND4^WdYMm1wM~y2>3o-hBz3Ltma2=mwz9)<{$6|Y}A?7d3jfY@m zoAtr>2pR#soAF_KnA5orG7tD6cEvRaA!tDGTF*3(r+ucXEj#v*silG1sL`BXZNx5W zTEwuL$GIxFi#4zn#JV?3^`)urmllKhC!gkn$ z_VLK*S!O9ARrC`;APmWP8-^tyCK4_r6i!Fs2{gSOU_6Zd_v@Xb~mIOT{Y>nc-EWxU2|>%aZkQQHA0HT^LXyNqYmH zuFtcv=cqN~Z=ztnT280Z3kc6HshsAD{DYp% z6)XGWXiCj{AgHx!u2(}TB2_u>)<{SPL?fIfCaLSy*_vb}S+Ib}YCh*WI5qRq62nX? zw?vZ1nM6jhXaeugQYqtS+Ubxglh+x0gp)jXQpw30JN_DQW3v?m4WC@U>NU8)f{sh- zu1SXKAzf)jb-!XvHCYc22e%U=*GNlEQ?qjZDOy(v0gn$vX;*DY_5=icr18-Ss1GU= z?foqLA-Qg}=U{FuxUM%%g0$32%|dhGCQ^5l)juZ6c~QiT9kz0u;)6oN7G3+)0xA4Z zM6w@Cs|IbAC}y=WbM^-(Boz~<7Gb&Ef<=ei=Ru(gM^FvDie_Nu>g3>aRrlf@SQ*_z z$XnQ10>zk1)^ZS)TQ$CIEt~w_IJ>kB0!&lOwJQdLEoxRy?KEOF^nVZ_IIeq%yiBSN zMdZN1-8DG~gdv);>2TyXP9@WSE~F5ixcP?Ggtw5ky^tXUa9}uv1A@3jJHG;@0#UL# zAkM{_s97q+_C%-E;8cbcK&Pm3m}EDc;<<S2c3d&Hf*sG(LriCeW~7MlfeI6U3NX zaD%O=!A+h-$dEI(!XPA2Gn5FyM#GXD4XscnO`G-?xXB?c;Gm%!@azDRirxdxv)sIO zk?2a5!!0s2bP)#S6yw8D`oMJwu3N;>8zg!AERco+Ukx@?l7^zBTZ)QIfdp_t-)GC> zq=vgN+-Y+AO5;C>Ua65dhXjCD9bra$3y!lq zX=-pPz!fvP+6e$_6ZpcH791cT{H#%DS|b6fV#$mXk0p+y8q{>j+NSs7owdi*KCRtL zi+mvNG*TwAhJd`2A*-^H&Rh%q|5RNerltDIz3N_W#XprhqfhHq_n{0?FS(2L9qO87 zjYb{7P5*&8v_84)gX5|3)rDz&$IY<8;f%Z8&!w%6P(_T4Tf&ggLi;O-MkDkf|zruPA8owBx; z(e#xDD5Qo$9k$nmUK@tTxWGc@gmyR5bYp1x)~BlM9tA)`mx6tXlyr^)D4bK}A^ zPE1Y3Oafi3xN>>N1Jv^1LS60vjh2cYaSsiBRmd7fW7eDfxFg^mg=k?@y6)95OyY;Z zOuH-&%QynHQKE>fo=J{3)#hapUWY?ADDtxpC z%{~d1sh11NdOdP*4m_OG3)0|_mgcAco-lBeV2|VzbuM#ToG;4WYD-)v6PX^zO`9a- zL(%k=Bzz}pyT-sR^MwJtv(w?Iv|a2Jm`yI2VnT@F@w z5*p;Zf+S?_+bqfyUhhdVkcs@OcqyMsjW(n7K<6A$sX`lLV{I+gOfMw3xRKD0+E9T zArerO%v`d9MKxT22m0m$Y9ReijWP&Dp2T_w3zbJrFK)!@;E-_tt%F{Ad(0bIU6Y;&(!mrCYu#ly8ip4aVoBTzxF*6yu8_8q zNy{=cy#S_%ZVGo)pkH7S*E1~?I|&%gCxM8q1}d!iMrK-2FX&slB(}!|u&$6eo3O4_ zo)1l;bar^Fp4IR;ICU%iXvZ;>1t@PxZ`LXZc7UDS#!0gjm+D-4$lawt zPOd?5u{p(8TjQnltwHUfB=yG&EZRQMs?_dJClT8DWn0d2e31Q}*|Y^wUxbRyT?EVx z8N*{SlmJzcsMDQ8Xmo=)7^mR{;lNl23o;XToB3pwjYJV}7iZwM1{ADLb(z(OqlBWp zcq^u$-^wn?TiKY9cc2PM#D*Ti1EcPY3;T6GawIA8{7BKTH8OlTH#e-cOR*R-f81BZ z2dDWRpK^0ZD8hRa+51I!!3i&$!?*_d?$Wx{)UPYzB`Q?2Lz?C_edfinD!)W75AkMK zCpIg(#k|2KQ|47i>y<6@#NKZn;qA-pCe<92!->R`2%J3++(pcG5hnzD#hg7udzO!LRm*& zj5Irk$6!b6`-W|a4eK{|AheumEGs%@bII`DI{RX!0eiqtb@0;@MLREyWtv*JC1GGP z>fkqJ4=?$q?D@;of(RZwVpIPAuOB}>%UwQN+&_GD6jH@6q(+K^V{NZ%?F&QPg@b$R z`|h!o!8`{eI!DIuf?3LsQ1Axyg^7!8jjp3E{A`YB6J5bC@Z#I$)SzdeR~X@Y2d%?6 zw#LV?u4#?$#eB;eKNa&Qt?_-BpSQ+O#{2`;_*Tp>SmP@(|DZL#4)Z@_jW=WdXRYxL z%>SG<-i`U6x5j%g|4eH)AH zS>thxf7u$}f%U&)jmI$mJZpSC=6}^1Z^HbqS>vM^Ki?YPjP<{6jdx-G1=e^9%|EjY}B+mNlLRycb&IGUi`ojhA8kVr%SU{M**p!T2TCcn0I&vBt9)ztkGvkMn-l z8dqr;D4t-PG}8a&z>T ztLRl2cF*!LR?i7ji@C2BP-@Xb~`jl>ZE&T)5Ft<*F(@HqqD_rF#ufsRZyfaww zdU^xKtR#B?jU~166s^Khr)l+R%JX3z)?r=LtCq=O8sJKLBfSZ09C|Z7qycq&93$R# z>l?i8&*MH}*$pQbDFSDvD)*3#8y=-O`logtTJt~*Un$X|boZskbzEC>emET%AzAV43bbNWfzjTJS^U58(@@8JSlZU%b z)06T~2GrHJlYuMe-IBkR7w+bn*0%PUJv=i292tug!&~yV@$M8CM~DS7 zGtZ~-+-ZMa=Jm@?QH4d|v&bDYnBl=JA3ldI%*KEzxF6H`6FjV9_!_!|l}?G$y#Q86 z(C4e5(L15L_d{DFXz;V4ZqJ9py@K9?|8J#F(%a}u^e{a}Z&#PmBkC%8r`kyGP}}G| zY6!o}^60&)O7Bz8rAO8C=mYA-^nUd+`j9$`;buQdYt)|GFUK#^*Qnd^`8j;v ULEUN;pJU6r@e}qn>KJwZABX~H(*OVf literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/ProducerRecord.class b/out/production/clients/org/apache/kafka/clients/producer/ProducerRecord.class new file mode 100644 index 0000000000000000000000000000000000000000..97718a00f9bda920c32cc9a8fcd6cebe14bff294 GIT binary patch literal 3300 zcmb7GTT@e46#fpmoumOF+_ZR!v_eRNQER;*Qt?(%P%DU4OY0#ygg{JUII%jN>FuF^ zp;LXYFFv$06=v#8FE2XN=`ZO2=$q5_TYI0}kUBCG)?Rz_&~@NA=5%;LL^s(%$m3+7d{N&dI+tUGI2xJnoo`#ZoS9ATqgR-L+DAtB_4iJLOyF6X-6(k=duOMsg!BOGjo9w-(4$TQM+00{G+)-&KWZ> z8Ea7O+HR0kjWvi|8!tlS4 znOSc3FQ$ge@i3OKY#^MB*5-+l<(a8uloRAQbC`^(=|FbhW}jjh<2b3F$U+!r@xFof zx`#U7O`>XZbktF9^Keqdq%WJg#cfNG`>j&RE@TW0H?~t1&``dnyf9@=_-gz8FCP0j z>v!<`;6+60LM|euQ7fwh18xIA@Zym&*DV<1rK~&WAQZ@iRWm2ik9mU z&l+8IJGb#CYBZX_ukCY3Lk z($C#q?r2H*YbZ`UZc&{0>`|O}>qUxu)RhclA9r%IpYc>tgGBA8bSsR1Y1|jUfdJkK z;9vmn^2ZaMfkP_Z!*0Bv(Hk$%g7Jir99~0E0`jPU(BzNNJyqQw1&`nf5s8ON#&1DhgtK-todUq zFItorJ<6h>v=?$VNSU@hLfboWj8uw0&Q%KGi~j`Q4`g&A{7QIE84YWIYgDRImxuw5 z6^y!}G^z`%S9j^P)M+n|Q|-WpYR&a(XE&jCjB2BJk92+T{`G3-UQ4Y3PCHlKpG<<( zagMnRJck*1;}Jr_uOd9j)ohTNlubQF>-9CXMYgP>UGmQ*(xG|(Dk70i@(B@kktF%I zif(aJzrHqQY#v?ww=IT@a#uIVoLj3dOt3b)=w6JkVSt=&5M4h(?V~n%K^^dh&+_fO zK$uiGUgE&ikr}7m$R5TM-^1s9ocJ9*58(4WPW*fgc5Ph3CU#^>!G&SF?3$#s~21C_b5f>yRs+3lwio|i_Y*IIg9XU?_1vgIo z0WKhcM5)AqAHa_y%q~_++X&T`cV=hz&3ijL{`LFEPXODf>Co{Yha4Wt;ZbT5KRV6U zbgW}T4&@9g8Q2Wzpwk(Nh=J`gf_(;NTc=3a)l($DQbDNhn6$S=T*Y2G1CFJRhLs|4-+2V|^|^qCqzLt3&W z6`M{t&@Zrr4aH(Bc5}*NPFYOcR4kMex5(3Qn~-ZrOh|^7&ft!wAtCKV9x)Cil_gPg zxEo7zlvJvV{SArFl;iKY)3glKlnO)nsaOS*R~v)3$jW_L0w=#C^W_qL`yYIEil?ra Qjn96%gx{Hox74oHKQrIh*8l(j literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/BufferPool.class b/out/production/clients/org/apache/kafka/clients/producer/internals/BufferPool.class new file mode 100644 index 0000000000000000000000000000000000000000..a546ce56e6ebaf2ff0dd9b3d0c850e621dd079f9 GIT binary patch literal 6445 zcmcgwd3;pW75?rp$(y`9h>0N@17t*&Ff5}eh+(Phi!5pa6sp#jnU`c>G85m-fLL3# zN=2<}sS9psMQVjAAVCDhR;zWZOY2hYX05HQrD|r_7SUvDDYoGud2Bb(Au~JV@vzM7G%4~C6Lr{SV0QqI2Jlz_drXYd zdmk63AC=lE=&P z`ihAAxq)98cvYb^olM*Dc}`o>U8SJrC|K>uWZZ6wI}Nc598S$vC~t}grk~$x zXHw}{GQs(!3VxYZ2+dt)udu_JbSxf@BomR0>pF>aSQb;^Gab^dok-8+>v-;Ib{!sD zUa)7Hb6&>bQb|LsC1IyCF4yb2u20YtiC8i`WmVdll4)*s-0_pR%~)y2(vF)_IHiY8 z-D@^AnTW>381Ixanh^8s&VgaHIcYZ*;km=+Cf$~>-EK!(o$zwIdAUtH+uD+eaF!)J zPqT`B=~x@}AJpR(Q6?2`Bp0~@qPx&y@!^CtIEhr!<*wohu|zCANg-V~XZF&=?#n6H zBV&O|8o{`oXbCsa`b5k40!-)G?c;|nqK#9NQEFT^H1*`MFpx3wrf>;&bBcJAD<1NtB;dQyCDUT9#T#6IZJUaqhf^!u%xEUjiu7MWcgEGwGN+yswD1SKD~~@a z3=TH&)=G^CHf6}v>bOoYmI~UzR@;pRb}AWI=UxtqGDB0p-~$u)?_VMMOG$qbF76od(`uzutq#-bhO@?eI+ObyhPJ+ z-6R1X5oEX!HcDY=c>PFlWvkN(6Zds$LY)4Nk1hNI|1|I~3!h+(g?|eK`-J)w|1q%N z!e{v0!T~{UP(Cqwt~d&{dG}{ygcT<36u>fcv71b^hy}W{sHeKrEY+IK#G{F!X+kDJ zJ<3wR2bNOQN%@58Mc*jDDq=LMVhfjHwZhoLJE*(+QsIcao(r0mIT4m-9UESFH@3RT zmHN6KkY}7NH?Ny%XTdJ8hgFH?Yv`ehbld%~jHbL6g-M+)hK2Xu;k&5L24_|`+jVs< zJg}!3uieY_D*e0lt+ZSV*+a1bomsT!vKvIRH+uo3*?LAZ z`*mLYJuJ`pU%Lb8q%M%GjCF@H^8CKt=|kko^6v7kCtmZh)O1!>ew{EHc_EdwC*$!M zF*il$9elVBb~!wjn&rf!Q&vsYpCSx-$CgGUli>Z!FBU7|tel!lQeLDv9lJVJxKrUy zP9AO0VRLPfeUJrN(c=3aFT;G7&8SP*iDb%=Lcu1r$d)UPYPH4U@mS|{u}cP*dKcAA zn-}lRzo7v7UQQ_QQe$5(qNS8hA4|oNxS1L&(g&y`Lc?$wG;5P{(*>m(8L7T zqDb7aS!9pQ;ow~&`U|TELzm(Nen!sZ*SCBkvvD9*A-#z=vOV!ewjJKc76S@rlJ=p2 zeC#$l-xqR@J%`7~w!<6Qd3Ync3yvmv&&E0A`zoNup_ntk`Jo+9q5j+93-$GHN3YN> z_?PTJ(RLJ*G)M-vqa;t7q1x>zm4_9oCRguJb@6ue356a)?~1Yx9I+kcyHK&D;>c>F zqOv-$1AXgDxL96YvI|v9w&JKAIJyJXTPV91&gHWnefXTo@Az3Lp#^2M{}>`LnC;o| zL|_ar>3Sk?DqE}>w77u?L}}|vwpTWHp2c0X15Jo<6e`M7cr}m`Bzt+3VCnZLL8r*0 z1fOD$a?s4*He^Q&S`pw0m55;(PhQShD|=cC;#|}Cw{eaQfJW0_&J!)VWlYU#+FH8y z4up5%m?gVVLje0#^zXocnhw-f1W5=(2ac^6NV)^Z?ZBWtoGR|X;2Ms;^$2q)*@+?i zzQJ*t9;)y->gH*_VYk54RA|J5dVM&XH|}kn$fwnBwOzA%?X4Ics@Z`Np(UKES?!3! zJTak~hcR-#rVEZ5DM$5MhvB<0s#EcBfxgF+t~ep5^yodkCaHSAKIX*wA`)hG(N3J? zL;sq)aV6jT?8F#_lZ$0~tfydhCcwq=sz6n-lVv+up#$UkuR|yc!9a54AgI!`hM?YG zGif7+g=+doX&aw{xAibiSu@u3EbVK0ruOe^o;zvdft6e}ccNb48(&{sOO*;u>y+Ibv6d+k`JXr1T ze!jl+Q0~$_44#QN$5+cxI}i7H<8Jk_j6TCx>W#I&W4(EuGS6Z(c^xKjGsZGRCo&Yz zU@)G8tqk97JSByTk;c_bk{cM*w__FVVdr`u*SGO+H@=1^a3S8uCD_Ym_Cs9CrtUK3 z$K|R7Yg8q!P&K$x4Z&4v0`8O=>STtB)J0Ob879vxM?2Hi zr1Vv^U^a>=zXdL-0QRcUv`LcrLlvfF87A{SuCJhlMU;ODtz5}uzF6JK-KQ}vnT)xT zKExI9f;pU;hkEe*ZoU_U*?2zE<|zkAWT&Cn`xC z;OR8w8~pXoz(ogGh%N7phkO4x$?;pF+vV z!qnJ^vRqwYn(Wy^;RZ|jB8pqY(kr8m;?Ytv@eEg=!%=u1{qO=pc##GFWlX^<^7|_H zy~=|B8h5=)(_Z&X4H4HVjQwxX<5JusokmBz$dJ+>@yz#m9yd`GGOK3OEu|g^muEqk z!rkHvGc;Vx?!a_`*ymkJrnmAYFMziS;59oXnORASnyty6Cow5kjMQ%_=NzQu(#^T@ zyxKlhuZ3ziXDe`M5;H6SAh(Y|7qS zp%L2Da$fHTo4Sv3|HJ(MtU&IdgXJ#Oa_4l)4O6aoT5cZSCKB3lM1|c=QNck)*@G1O Yi=3;%4Gc5zijHMWs*em!^R}NBXd81}Cm0uK9Y`(- zgr>bUTc9KBxb~Auan<%0&D8=WU0KgE3rnW&$a75?3O1awz`d;J=MA%DS{t^pWv*|T zhE;Hwv}}}oZ>?h4zTrr4u30D>_w9AFQV8ZvKX3xaa}^9d@aDZT&)duu8ptFwgyt#$ z0)mGqOIdgF6<=Z*XejZqSxQUfY=AT%oQ`8!fu3gYJEg+{`kPJ1s>g!q=8at7J8nLm z8fTNDIVbO$fl7Y!Sn|_-B%)~Putb3uHLu3d4aw$KH*G6OuS}gPJK0^YSoBp~h*7Xf{8rRM$O+L$I5Odurig*;0oXQZV0@?|1L=i zer<%gC)v36^7KHRfxgr}MCvVCmfxavc^{!Yg!j?*3$&kT)q*~*^7{kmM+i4?k-Io! z&=c?BeWfLWSDJoNK}F=-5N&u2Lr=rQM;|I8RHt zWtxzJpJMBPFA%{D&a=J*+kOcdLVk#k@CXyg^1FBr&yiClz9Pg4bmBwyCPvFq3~?8w z{cYTWp`uy!23U;oG0S67)PJJdWAWF)FqhK#1zOu6SSnC$T5O1KOL7|)mfg%|Q|vhY z3O_g<+Zn&{*%v_V$VWTlSO1OUSf9IrLV@vRoMg_q_v}6Ap7WgNo?rj`X$!zGj5rb) zi6MzmCB~GvsKg~D-i+e1hAVMIQP40RM;s$cycNZS(vwQOtxjHzVoJlcINrf@8)h)8 z;d&hJ;yn#F1cH`R6G%)uZY|%i?hG&Gr=40&x}ySN Xg^>N9#Yve7%uH}oKYuYt_ z3@*!i0)cXY=k6L-Qwp^CdRp2wZ&5%@(Zb+Y8Z94M<^>{Srfqtc1bWj4qm~Qh!u-IT zKxo{lGT_l^)0VT%l?CbEG!`scm}kYX<_yQd!QpVOa}CWtnBQ7d z`Tu?)Atz?ZL+=%cjuD<#GlFTAM!e`W-HMzv71BM20W_qDpra2LboAmm9Vc;0$7v;= zM@q*FIK$@IP0JGK*iRbeR}?tD-(F~%R#mz>ZsHaRl3KXu$%fj^>3AQvb(}ztz>J?y zYIRZCsfM(xY+GtYHX24vraXs#QZ@D2=c!fGTTBtp5pJ?6pOI?WvP;!;wS6YN>kCV= z;%O-9n8$4mWgQ>jLxIccP5d7|Y4}LT$M{4b^UtAnNXYk;E*DfkB1XN=N(D})rv~;k zu|yx=gzO}1I7S1Cr@kgt zeNDFdnuPT=IZGX+Umsqi?C0(``h}@;nN5hy;5q`C>^g$U(0xQRxpjmxxyR4~YY6AI z))CGmL+glqdl# zx&eQs5)S$+m1+nXuFoU*gyF&(1~p_g*$&z wj(?z?vg;Pb?`%^Pui-)qFg8$m0kK5#2Xudj4wanXArMLRshje3>HyyO3zOB?m;e9( literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.class b/out/production/clients/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.class new file mode 100644 index 0000000000000000000000000000000000000000..d87dfce5176e4d94fca3383e3518fe858cadba45 GIT binary patch literal 3048 zcmbtWYjYE26n@@Zx@lNS4c8V3Q9#llYztVFh6=r5Yf7;xXz}`X^R`{O*$tcBf|p<7 z_!DqO9cgFmjDGWjzsYetXLm_c8Y$?P_kNyp&T}rifByaJZvd{~+br^!?8Zq<@o+1P z_c5KN)eNsb&`{AZn?(kb8QkW1t{ZpoVU~t@-c~bM;Q4MA8t$>oM;boP;vg0o<-Ueb z`0!I6KFi_=KIhdJeD@^}Qp1u4Q$tOl%Tvw3_65c&uD4>yhBQ}|u_l+-q+!~Ya{Q*z z@Z4HpD$lSSUwMwSo5r2!XkI-GRMVdiF-8P>JY`GYT32(+%S}b7Vunl>{iPMq)q(<{e+zrUiR6rmCr}oeyka5jWHzmye zsxQs8S=or$)z_sRs5x)a^IT6L9d3z;;sAwypVb@>39}W6(iu_IWi=;UhupjJ67GfW z+tOJv=9V5P)29#|B62$%cUP?yM{>{v%-s(9bxN3v zY^RPhSk_U%>pI@RX$>nnR{7*z4D!knIQy?rrZBaVLg47O-&NnUoRzV_vTKTH@8^`D z#|61eC0ckSEuXFn%dS_@@c?TYY#ntB>u})mNrR$as5dX_cnFV8Z0hh42%PI^%j9Ea zQWm;S%Nmwh*KrMl0)uFm1$wu+ zic&}5ct_VFae@4{z%=(cE^1Y4`5UG$^`2KUty7z%8j&0yfzx{z0N0CbG?YW-bD`Ll z)=~Va0c=~nBPA$5Pj+_gXQwNb>FVV8-0g{~fJO~wU3o8O7uEKO;>?S*XM8s@ z==q9`Ef+Z4$sb}kP26)U$?qui`M}}6)rto>c&r^@VbyaVan~Q=FGP>;HZ1dw^n5A= z*AXanY{_oD?mEW8j<;OdOS0Megc0q#v$3P#O!#r?Hr;x|R+Kv!qK|DJ)L@Z7jynK- z{&U;}Xx>NXXKRCtQyPjr;PIT~}}b)Lpk=q9Kh0xQya zfdokm11QnVD^?@I6)Q^6`aIr>)eO^|pu0r!36CLiSNQsDh;}5pZJ{M_0T=0Ba9PqF zn|dwW1A6HfBsP#LZ6N()3oRQWl<3Fge2A!r=z5GeMDM!j#U=VzVa&@h=J)hIO?PF+ zJau6MUBjuvsqawQg0{GY%;F}xbJ-2-`x!c4@83XA?f~sK(7TD(et%{|vW=@jqId|s zr0Xi_VXjIchvk-cW%?I89okpM+ZZM$S>h^roWeVyrQ$DK6?6vr+e(I3YqX}u9ox-e zW3KZN7r8%>+rq)cc(;i|Y}sLEC$g-EW{`#~;r9`fY*oZiKDLVW5@uhhG)3SC>42In zt(g1?B8nCEZ{bL+=4hnt7){Ad^lgVC3hZc`lKxnUpd1{D5e;&M5z{RH11;LKVysO# z-4ZV2dMwO&#lpMu=~kO$q9r-nlDrg4W+LnRpU=lin|vmc@4ZPG^IpvS5XFcKMk{eX s9Dm;HzBX|!R=*>RA=r%=%!rLiQ?yRJ2Ab#5Kf3F+@IeGYG?%C6zkfXga1l2=jN+_|a~?); zzKKJ)Q2(#`$?l?<55-)Ph>yg4EatM9D=y|-Tve!dbslXfoa{W(Kj^T?<8BzGNmOK+ zN%C+ZEt34x^l{RWusMZ8dD@Gk`#Q_xJWdmZb|=l&L*3KSh6y+I+NNgy)>fK?%P&MS z*IUtjb8l^}Z*m2-$l@l;(oEqB#=<1KUR$H?@g0`m@BjgvUPrci6oK2&fP z(p94{+KCf$x7b=S*|J{g($ngsk?ua!SuFd?>Es)6U*YDEHoGw?_QPJ5t`?EW!Z^uI zmgsIjykEMO%=d-q=S!wvbaOW1tfmR;+}=0m#a9s%E_MA_$5g``@a?2je(t`O;L7&3W;ZqzWSqUlpbbIzyohbwi*!NtU9C)v5aA4$c*Dm8-t=)D zQ$F4;%?Tg#Xe-RWOy>VhUt!F4W$);{l}9F`+|xU*J92ljqDK)1qvTlHcVx;GsCq^U zjdhdTSA}zYo1=Wo4tpmL{<|qG>`F>@B_Xa2C{c#?E$;3zLvfu9oDQeqr zf^pQh;r_w15xh;?VkB^qc7_#>5$io1p~o-t@eZcB_wWQ9*O%ggpG!(w;h6K=wUqoPNcUE)rn z8fCTDcB;uY3%CpyS9URIR27c#OpuOSv=bHDZTf`2Cum1cKY{-%ccn51Y@G#*HABvf z8g%PXvBUPh3f?SWq0)nuwT36#j-wvFmEFYQ1Y!RG`YX#`p)VY0)MnpvId1ABF*@U1o50x9Su4IL5?V z{NV%mP{y-c42htTu*uHsIeX4`Zr{w0pWnU%xQEI-vbf}64woHVz!e9}xSGSY9Ioe3 zV3-SgDk!WnJZc!*_4t5`p7i#4XPFXT&P)GGuC|Eg2RXO3SBFza{NU-ttMa*f4_oyWFZ+9x5|&84T{W z*3#B|9t4uajoI=jtWj0DzT`n6lf7niM|G?9(qn3hNCbw; zgk1jN%#haRH3O?NI6Ee?LT;(J{H3whSN8Y;r%pZ5!Z(4^UCN?o+Bw{CaTB*(9KpN` z2Spbplyg{daT|AN;xi4ru*Q&&BYQsAUGG_IPl}MCJP!J2m2o={NBMBvcnZo<$yRn% zK%6}6J59X#zcUJ|jZyMDG(FuNYNAm2UXu^1F$>ehcy+iCGN5C!N(bfbq!3f2T`f&u z!r%QjqfSTK=f?O%#{1JWY~)Ead=<$^wv&Rk=?ciwg+SZ}%jaWy&{dJ5*TGTJy7aY9 za-u1(d;%+fLFz+dbBxy56F9(eT9<}?3phbij1Z#{ZLx@xL##U4Wayt;{s{IJ>Cec# zdrxu-BeXXO+A`9y9e$&^IEB+mG?yZsA&)f9CZ6YTp7spgXcv*EFNdtMwA1qosFY_% literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$2.class b/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$2.class new file mode 100644 index 0000000000000000000000000000000000000000..e9d1845380a8bdeb5e531648e5e2eec37e46ce6b GIT binary patch literal 1280 zcmbtU+iuf95IviuHFXUQ1zz6VAh_M?~i3CIxOWyU2XU^Qt?vJ0}z5}>})j7=IqKjEva&aD)T`c2@gR2g% zIVdsA#sd|V))*eNjP3h;kBfoycX@A@`yxjZUAX%(+CY?i0Qs%$e98m2257FtTnr%Bk6_9gEGq*-hk!GmpXRVojag|rL? z?}^sZHUb_+lEjVa@+j4)DkqdYN-PDgjVsH9q0xRQW2;2ux3j%rbWio`&B{Y+ipT_p z)iJsJ!EQ_G zJh&)(sG#a##ltP!rio8A@KTMTm`3&kuKWJ8&W;o@Lv)lQlXh`{P1PkaoQ? zh6#W7-%NTvX`dSt5E<`I(y&8MhWXgUJZQ_%*!`@kCfx%wbSn_S!HUI{BXnQn=yfqq zT93ZgNzOFYl}}*RFUWn!Y>v{JdIE27jMk;0-vW-4lp>^P#9b`n#1N}VHU;|URX>7# zMgB7i@7|M~!y($+3~d?t)DFMVJes5PrP2D+_^@0plG86lgnWCWiP@7}^F%R4pKe_T6>e)>~p%P6FB=K!`ts zka!ax_yPPV#5pUf#0p|sq}aL3-E()(o&DqIx9BAHkv;-H=F>qhrfztOBdBvM2s zFw`gH@(*W*oHnl+Sd+oqFsT*tv2x|l@1efF#b0q^>X8V!%7yZ>6+>q-0En2?6??tTaxdlFOBC>1)^+cGkCC#$SUcR_(}1{!j(QpwE;-5FE# zdN@p4pT1T|&NQ{fPhhn#nEH^}9HG^D0&j4X*7=d&ERK=n5F8rKFXnK3gw-UQJpJ=) zAHlvN_Zj(j?@6A*80~F_wt$?o!*4VnCvY;0=2N6o)7Ei&(dG&eD!1F_R!GwVe23|B_;w1ww8@R+^y1o-MTb_`DVWfPU z-{E#!x}IGTa*1KM;?{jG+acY)arAb%9E5e7x46>~_AcMq<+kIw!k3ZV3WHkP5uxq+ zQiMMDBKwALg0OalOQ*54N{I)TUEh_f49R@qCPQjHsA&MMFRr(nRS|CSsz;V=IdHgl zlZUQ4ca168a3h9^jYI5x=>^@SH+)}&>mH9Hb-Z-~sB4bXZnixxgK+i{b*PObuhkfq z^Cu_}bD^3as47|*E}b|FL#86Rv%AS#nj)?4WfItc;yM(~;0}>##2Z4)-ci^+k0F;Y z#JYOiuiG1>mm->?-ulmr|EIcH3EH6}UU!wWxmdJ5q8HWprIE!L!Ks2ahJwR(=$lou!5kbo06$7s_ynXzip|9~r zO&G&yufaE~w}m4aE*`&>p0$5l&}?lr!rpMjotwzKCPp4O5hAq+BQbt$H;O8N!kIU z^Lt?P`$%l>A^90nceVF8eNU4I*w;w0Z!p5XMUHIgI0h4#q7$RBI@`_n0o6#6ZF>Gw zq_#Kjsb%4F3?y)?*!dVk*NdIf!Ci7PsQd&usuXHXLKVxZVD=ph_C2!f2b^U;>Vi3n zprqY`YB!B1spL7bK90mgx=0$Z3_N9E2F$?If9XbN4}pC|6$Z$=bQIWN0;|&>BC`); zJ3F+8;R6NJyqqd_CR1IYGnk{mITZ971D(Pk0sV=9{!BoBL5}?ji~WWo`&|RJsGMSc z4$mn5P&$NO>VrOl_X*w~7-xU>iKf_AqABLGU=JxMp_z+m>yAl*GS8Do0t=X=)g)wT JT%;OP{{WFP)LQ@m literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$ReadyCheckResult.class b/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$ReadyCheckResult.class new file mode 100644 index 0000000000000000000000000000000000000000..ebd6d293674be6bbe55017bcb8f4a0e83fa19b93 GIT binary patch literal 915 zcmbtS%We}f6g^I+2^ogAX_G*C7AR5~BrK%vQYxVU5>*H&sk-RSaZKW7;)y&n(5(0- z77-E)mOT1Vi0g)j(ju|w#=Z~xo^$TK$3H%O2C#*DJ_@*A!UAr%+cl417Mj-HdIYi?NUc#fRd}P;eQKMzP_p zT78FpSB*MK+YX>HP;&TGMPl-jaM&)xYG#HeerKALRyKKdq?42~2MmR6ZFIWBP}!VE ztnVJ&-)Ha|RNs-+Mzyk0SJ^$$jVQ9xjHQV7h1Jgg74*`9CdlQ#G} z5=r93zy61wNGp>jd~D-hCY9ydq{lc|w!%Z&F*Q zl>N7tGv$6K&a70=wNqO=L*b#DUVtUs4N$~|02dJixP-+bHW*rG$8{cg)Y6k>iG=C% zSKUJ;sipZ%H=xJoLc$~3&jM|39%Uy3w1wp5;FC{^$Sylkbkn|&+GWa>$$L*f7U-^q zAHl*0Q{=++De_^}o5K4*i9A;5thq<*3q1A}CH4)A>^my#7^{>E;kg2+;4&PFW*#nfB_gfs%#IJ}M_N;lyjg!bmtd zJ}i5c)&v?7I!66C@WrlG!l{fI-{dBD?+71u?oT-!?j8}sZjtc_oqlfAr|Nv9>^Cve zTv_QCQs^ULbD!T9LpRP#e!Efselk%e8>q6T@SG3WE^?+3AY{LmT!50j zDS~8kibk?EMfj8J0Xl5g85MMakS@`tD=gAAR_Pa(_$|5VXP!0A7^&u^I`Ygx9eM5s U#}7FR7;lw*#2V)C6f4;H2LzVfA^-pY literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator.class b/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator.class new file mode 100644 index 0000000000000000000000000000000000000000..b2ce89ddba98d74058a3564dc6fcbed1e6d546b8 GIT binary patch literal 16641 zcmd5@34B!5)j#KDX5K6>I|)2c91t;NA&3xkqC$u)K{f%!ptcOjOBk8V#F+_#OKq*T z)o$9)){53zt)+`?X$yf!Ef!nszG?S;x2x7JpS6{I|8wt~CCQ9Jm*4MmdH22h?mhSH z+naph_#>Ypq6Nm90Nuq)g4Dqsa<^12ohDxxBrh+MhvjlvA(xeM>5|JTxm;xO>L8Wt zeQW%@*5q|Ts^;}3UmT=bJ-dA?e5s!|2WbjlW^#9sPT?&=dFH@n!GJYXUJ2Jtm*Z$mGx?V4rHwchhLwp;*Q4V-hkl)O2G5M`>%I!k+9dgv$-e z-Q;%!X(M--e3$peQT>lVduNF6<~3sz~l#Ie6Pt51^Iov zSC+kBzj%H(eb`CI&LrpaB2p5(w_+{ri_oStN=cWE@!v)xHESzU>Qlj@8|(`k9E+TcWc zcXw`gdUkAZ(nIkKQ;puyF*xWXdb5*d@nk<3xht_gmF!PBX&ljb@Sqv7XirHFPC1j)q?yRhW!{5ZEU4Y>Zv&FolFt0US#vnC2-Iqk~b9+qxs# zwa2^_TsnDHv6ijq8P%EN*_!AeHUu`t`V-O2Pzq=4Ie{(bmBeFHaxm7jKAOtJGT>%=0jvw1 ztB0Io8%wjGK=ax!z`xLrSWYdOHW$I-<$!hc^b8FQ#i1#wStV2)$N0{RC1RNinC@)o zT2C(P%EWGn3Zk{+JiuTC=JHekI>Y^T%DW^Y{h9g;{)r@IP!wCyyB0t8mP${4*GcA$Q zUP+%XtH9EVYoc%piT+L7QwSN5b@K_}6ipAMM3u2WsI+hC?uxlY(MhC}DW;bvU$4Y5>6c&Yveor&g_<~)RVN`bpN>&X50^fEoNv8+ z>E%m{aB#dycShs!ZHPtf#cXo?;(YjVV|anPbY}{pI=JQQ8`5FpL{P~?U#@m(h~Q?l zH=2n8ng>}05{Pq8fUG<$4MpWA0AZ$-;GrUOi$EFUxYsxX$<%JWJ&mni6*ifS-m8&z z6o^uVEG0@TF94oE(|Tb~wAS>++?*VP`Zx1TW`@ z8&_g0ZYJa9mJ>45>hXyGJ24mTcVh<1gTSL4DiTB_$|n87cv^}}c9l4`vPAM4@TgZ(35?=Lz_?5? zs9MsU$Q|Ja+gJVj%0D*Cg_MJ0Rq$u)`udO+i*s{&B>|I+&ysv?Q*tvVzkS0SfHdiG>a*!7UMXJDcJlU$tqh50D`j8cqpb{&~EY%P5zO| zKSuG0dJm*+kESn4K#U->tYHqy7_Ta0m^xoxH^&~{-UVIFS2vRJsf1n?6M)+nO9?a2 zEGflY^4*o_ONxewNX8HmxSpW?@DU7ULZlPDOLg*`o>xrHxBL0cVKSj09^!Uk&u-M63k6rzlAmaZz zIdC${X)c#!sJ%2n;|#4(6SF~(;jwo9>kx`D=S0N}pEte)L0Blb}AjEt$&X3rLlHh?BI`i`=7EwKjlv zw5JCRhFSCG&12gAui#x!xSnJtr_&O*kznis0{-)Q(6LYjZg69ADAnUEk4g1duXCY- zqvnbyvgl#@utmeX(4zOy0~WoT?njB2cTz9Yf|0F1v* zax%LoFsoEU!J_?iz@m@PM=kmseV%FN#ACF7$k$5(1R%J?28IU0(ShVpA|r&1!xL+uo`|L4o@hd- z3^!b}-2u+2Fo;SdGvOHiONZUAUwpT$>76t8-=<+cEWkxOS6yaN-zKOlVqOCAsKQGy z+~y!LRCt9OoS4FW2wgqxxq5|;SVF`Qmd;=}j&SuK!2fKfDK7rGRI4Q`W&RES78%>b z_sW%S7V@Gyom8~n2@g7{mNcrUL~nt%X*gj?Rnx#6q()OtxHpwVJ=h!W9TM(jCF!27 zJZ0lVz0$hwnrtg_5HvtH01cCWXYud(4;KHCpJ6&pIT%%4cg5pQKRQoX>g37`xS-Bx zLbxqRg`<5)^c7FDUO_2|YAw$d?R?ha7M^SIpZGbZMHA6_IdxY~-QqvPR|AMrC5eV3$&viPq&RZ#w$#s4d;m_6}0%|tDgRt*pJ*7FwsonJ8d9~S?U zU$p35bRSbkhgLjM$!Iv?Tobs;c7bKo8k0m{=37P`0*XakDkFHUkiq*+P zzh&5h_EgK5W}Jeg-ZD-#rX#brjMKn>Bh1es4MaJUhm@;TEMtbKQAxp!O{6SirmU{a z15()(DhSIs-I!$=XBe|B<4k@IVlx^nJ{!mL?`3X|(ZIC!WO(K3)|kf4*xFSj(P$Y> zJk>OsEu+PN>m0`|(8+Yl#AEYmtYx%9VU2mlSz{Q~kxE5(OZtuM*)rxE@JVNs0yCy& z5Tq7Z#zFzwCX6^2>4vdLWO|Nqu7HBlOv*D*9i+Hu84+2~4n?3J8|PWZ`G5iSy8xR6 z+G4pZF*;0RsbzE;7n;U0%UBNRe2!!^4~eK(@In;Z_`syT!sH!?#L*r{3ijP|En}t8 z1;9KXjf{yjmDb(~5;2WcmT{4>nyH9)91kbk+R3t6IYjhOlh^2$f!5BkJ*byRr(BOQ zruPNX^M)biR4O$zh;BxKt1ZuSURhF2{<#b5<$DS|_C&5|g>gFz=qBa2DHYQOsk#-S zG-$CkbV+WWX;{_u-}s7Bx>u{(+AAA$5jk<$iOgi1E(+Are>=$s8_q1n&}{J32_x!` zK(x14KQL`9cC;3RUZNWEB^wBN*tzDrh{T5|t9t1uyYkxNet{|0UehOLp>0&VQOTEd znPXR?00T+*$oHQnMv@36p-Oy9OgRHcG&>T#DxUtbDK5ZqF9d+)3=C$n-yc*{McLK3l_l-eW%l@=BY-`nyoS4DjPD1ECCgD=dP`3m z$PUEfs8du~0Dl8h!7m-Q6jL&7_hS>F(jU5u;-s4WrjerMNmyV5PN4H8F7lD9hldI^aoBHlhh05Rd8d#@n!J zm8eYFKny>!ndHhUNAD`-YRp)Anb-tC<~i7zMD^FBJ}3#(`J&_o&$qG(4{h<+WtuTJHQ10Ce?rY(wA`(IQqQcY>73S#22SC`2UEK#K|gZ1>o4 zYZUsGpc)|$-Af(*fGS*>xYj~6l$z-;?>gDj9$AzUvoH*|%+`^E>Eu2DyQ%gD`I48$ zHzfJx7zA=sRbeq0@Z_pvi?IiK50&@yCKEt6+_0*E?HkZ~f?LZaxGEu-JU%QekM5~}cPT0rl?olwzP4pqWjd+`E zkL?|%>1x4g%{Afdf*G=4X2d58eAxx3H`mO{E;vIL%#N6{z{G;4nlp!~A>vn%<}}wd z>V>x7#&Jy$Me{JVGU=FI{6D&D=>}$hGy+IaKe-HlxD5>dhCYs@KcTZ$v;eUH)HhNP34EF zwYz5CK|1S5W79#JAMx7W?uhTG?H!@B8CWien6`P0R@!EBi|rYvg%Q8)KSt*qrnYYI z3d;rb-7+q`!ve*ah;4$hD5TXSu6dA!nG6F@&2-*HT8KHf)iTy$RA2>uSBf-P8 zpu5SI=lT&s5`K_YMM8GSJ@F7-^cbyb;Cd`15Msqa4m{G!tv?$La^5nGceUk5FMai`n=DH6QL_5xzn#g-ck0 z4^dacjIW26y$v6j?tynb3O)QXeCw0&tlxuI{RJ5FEd1(UX#}eO2z%+1TtScWIryy8 zNssXc`ZV{`QQk$L;n&b-`8GPn`|<7wJpqh9uPo`Ez+x%v;hXd=;546B(YNV4WCC8N znhWBDud2BK;0;Jmn1XM+>Qd|2`{CWUh2s~<-b$yGLxQrfz0F7%_jXBXOGe{ z3QfeyeVfCKEBsvHVs=u?>E)q&09r%m7pQiWwo&;@BlMR%TDST6HW#gbVG~k|6FUhg zx+n1{)fQv8%P_+D+$c21I}WgV0i7r42-Dy2|NqkSxFWxRZkDNF+Sb&)pVoP{9-wti z;RCe3v1vbDEY8?td%7duqqe7Mgf7kBQ0=C@H)TiXg;~zzAK=bT`kGyDKCjU5WS+9HCdp{VU;UalcjWqYs4MZFNT~j@s4;q0*7iR%urrql0!OxPKqj%eWp0SkdsE zIgF~pYT8*;T~~Fta{AU@q<#A-w5Ybew!Ydy3~(0J3D9gH1en4=*fzPoy1w?#d$Kf6 zshhlqrqxZ^L-lq3JyZ=?o;`qkkovN*v(2uv@20xCsy$St3FtRy^Io#*>*~FSX!|0s zU7=KADn@(ik-f*icbH<`wx`}(XC9=h_Cb8N@ETqZrmyEqWOP*{A%cMT0`yyCkH4d8 z`U712ACWCSL!00{x6*USCI1Z9`4>tet8u4OtT2pK zpY=4dSA9YaPS7xFU88u2>Wyfw{PQ(N289q}A(Q96Gg`QfJ?e;EHua_7ZpdyniL zJzDlKSaud&fJj5M7;#(1Wr*1_YELy2#1KyctLvzl>**|>Oc5|-G265T7_R4ON`-{^ z&B(dMAC*C2o0NDFiD`o?-p}H`9+c=+Pv=(EU{1W%bn^a_D#7^U)T($d0a&>7Jeis! z(-?-BDCd;T@SG_0iyQ=G|_s}L(ns^J3bnVtt-%GWU zgwjAYF{W)i_xw< zq6qVB&5{Pn>MWu4{?VDXPcZ_CyBzxJGPtngU$%YwAe`Ijr(ieHFQAAR`4pa^p0!%+ zMCc)&1KVnZ`ZvQOTJU#vbKzlIX(`X6i+Dc#{{l+zLio8hx{Vjn?MNi=<#Xvh96>U9 z9umm&@w(FxVp^c$R6ZT^qV}6}o@sN=Gi?T56;MOG{HJlgK6Q5P)H9_H2aH)<3utO5H(;lj!s{qh{vuVGJV!aF#aZVB-^6fGG7F~! z3v0xGaF?UNze@X&;!8|!$|bZ!;;qg=&Uuzt7gB-0)6`LzlW!~p;GxP?NY{|xmHOoDm$)Ys5C9YnqRdw|SqMJ4`phqFy^fH$$JstlEctT7T-n8{%$w^(|C` zHqIozf@brTIaeaw%l1ANxaoZbRXT`aJH~?nBF)- zZ^B<_KqpjGnv5Ieau=*=-cN5Hrnl@J{gtMmT~VO9Q3H)2l#^k;i5mH}G@ozI@j8t7 zC`qqy{pJ@GtEhSK79E!Zwfy8tMQ()t<^kv!uy_F?=b_lFBO?9&KT`8mSvE`(N$ z0>KN$vbYXyn96*ByVOY9e7p!I(kAuu`MKj~sI9P>n!66sJ3m2-1p_HsfRTC?fe2f0I9@&Et; literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch$Thunk.class b/out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch$Thunk.class new file mode 100644 index 0000000000000000000000000000000000000000..9bd35ae8e366b2250e3ea25a675dca706d4171d0 GIT binary patch literal 788 zcmbVKO-sW-5Ph4rR%5jF`x`uX&??4*;K47jAP7}KMesP?t~NGq>Sp8r@*si-e}F$q zoNYzDSgbpkGoP*GaYn)@q5l38_u8 zslparwzs4systfR-fQi)}D}iGTKbasx&<2A-7591i&>@#5=(=2250~q<&3IG5A literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch.class b/out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch.class new file mode 100644 index 0000000000000000000000000000000000000000..52bcf3ad70c7512d0437381ea5dfc23f8a530bb7 GIT binary patch literal 4607 zcmbVPiC+}w8GdFDW?2SA3{}*WHJW5u!Bt~T1I7fXR47CRYl@^H!|t$*?9QySvmhpE zdL%v5D^0KTh)oZBCDlZ((4=?Lrg!@f^zX>eU)$&VW`P0L3jK-i_sw^_-}}7xzV+`{ zUk7jq|BPZgoJt%-K@eAvq97B3ObT*AkSP@xqp0+bKC0reC^YZm@hCooCj|L$6d%Dy zqxcw}RPj_4TkvrepNOI!pRB^C@M*#4GZ8$k;Qi-v-T?z%=U?~+et$}x1;NcVBB^zb|F=%(d;SMxEq-8(Aw zRgW=e`{cWNUQdk~$#H#jTu-L#T+X(V-jQUVk+YpC@67^bs&VbSnL4REuIZY#r67Jo z4yRX~oph3>24v+TL^egq|$AW;WSt7Q|lVfSIv$@3B*>wvTM+aZdpqx|DLs2ecTDkJS`4{k zCbe--nTKwyTOT~a3b=XtrDac2gRV2xp3fUrT7erM>KIy&%{ujLc0^B&w>51@2+u7% zQThy5PwTEOlsx)>xyuc>1;>pkA{^>qKu+k{q6i`?QOc{Jc^v{Tp`$|&xP;lK=cVqM zZ)mKvZINdz-sAaOR<|*kZNjXPTnVFsgX?&9jb6gi@fF;+j?2qY z?l%Xu)6_sx%f+DA?({v<4&7E;A-^y7qwUm=?3# zE%7Gb4R4hejvPs4OMw)O3)Msi?4rXwVG1v=^@+Ssgt&$t*c#rAdo;8P(k93e+^gX} zLGDMph7NQpxL+jImC5bKF;FhQXm}5fDcG`{S=t>(pAzXsLl@qw;>Q|(f}d)55ihCu znT9L4s^Vo0ui)AS-!JE|0_wDadpCl^r!LRJg2{}ESq*cT*KinjD>zj`n~vp-LP5_M zg_vvekGZ`iznI)(k3JroG~Kb7uz$>!4zM_IOSjT7!*Oh;g|T8hF{}13ijU|(n^ol@P3@p@LT*&;CVyCSq!OoQ^W7^heZp? zx0k$ELH98sUd*^)q*xZLOsrTioEFxW2{Ub^V~cSgv#mun#7233tr}W2yoEn%co=_D zuyq-y=fWLDGn+OX1v`AYkFPA0^vf6VXF>kLBHA2tmP@2}a_pmMX zAH=4oSvKqAzMSU;Mw`&CPZhnk$d0P4M<+(<&G8<8q7>UqleI$i{uYsXa~(a!92xh& zQOc`BWu6t?L_v0rOtjQ@Fe6fWeZ1TMTuDYs=oneuH7AyGKQFPfY$>N(&%IW)d_B?@ ziM1}X+N6t?V0Ua=opvstHN5V!r`%hsZ+kws)H3*RFXaMhps>tV?2XE3%Xg`Sf9;C! z|KsHZDi7YrU`=rptf&hLnAIKZD(+xcUzij)P0Ges0=IkHA`Y!_vk<7QE*<@(Xm z$f7ORS-0iL(k(}pUXFF5&XU`4&I9O1nDZXIkF=L7FCoYk;AmnNN+LLofFMCZLWy0| z2n$k?Xr6|e*gK6#VgZ$dvxrWkic^i#>S@$mmAv~%_Yr}hA?)T)ygjJF?ffN}#5U~5 ztwebnao&l$NDoo!ci{n^PVh7(zzLk>C}yXKK#?ezuua-V;o|I+S}E>Pb~^o3_(N2; z0Ck9>wGupwu7o$d1aBA*;sX-iFa|jj@Pb#$uuAP+svEomtw1C{@P zOD|(vT|>od*ghDf=#hcJP~#kS3=D=_2IjFdh~VK+eMo?s$E^xpKw8Y}L-TlNfc-;# zYq;f2G%cV}s3OwX5Wa@k40g@pw$=*qvAdx{na7?WE@PAYR$5A*yei(lL)>c!&*RPz zo(w6M|MM&tgia%kEi;H;rMw3)iHij9sjUOGl=~F@U;sNf#;B451l~^g$0YC}j1l@c z5l=$r&0#!-NANVyJ%bdUM_Q`%IPnjmnmalwc>6F$xEto#TAnaC+YK9|j1u8bmzETM z2^ktq_|ql6W0uss6&2U9Te&Wp&~tC2szb&2=C{d25lw`rbXqhZFUA+pL|;$T?VUlh zfI5Sgx_#a}FoR^>e$HobK=?{P`bveoSCoQl9*>eh5m*${3%^}*hc>S}IETAl!8Q>| XijToAvdc@R=NT&TL8iStb(E z2Y-M+%6MjNgrFtJCOdP^%(;AX&NuVp=eO?w7O^-C50h?~@-c+_KJH?A1P@Sh(nAlA zJUsSLW*9%@amk2Co0fg?@eMbLOr%x}`Q|=9p<2Q?|L0Q4WSL>5e3<}e@Km-La&_Gj z4ysh*Rkyt(%og8?X=SXbBOY&aBc0w`%sKoFK|?8F>M^%gkoxQrSV}cIj;~c+w>z;& zDBk#o43+-*&p1-bP2Dw-Sds3GBfkqf=h{Ryc&VaTTd8)} zMY5+`9x4H%jLDkX5$n^n$jjq7pQFti@KRD zyT155ii9PS&lAnxozv64*#4PQx7%8!SILH3vMngz^xwxmYeiIT%Q%*nZl`JyRSDzT z3C5bk*trzO3sde?LERg$R@ufBJt{+B7)GX0aJ&L&CX&&^lc9eUdD<A$5NriwB#)=gd0fK{(#R+rqz;KMglhD31WUH89jZ@)^f5I0rfg**?;ua;#;5P2ioTJC_9uwT36#fQ~EwBjGdcRd&E6~aUMg4&>MrxB*w23wSx8JLs7%5hB=1hHn$DGD?OvkP-{AF$K+ew>I!p{uWxeGvZYYIXKuMpyKf2Cl*$*b z;~DoU8$shkKuLx z|KY|!s2A24VhyJ)7&1+%#FKunCERDcWz)`N)3La{#$6f6!_8PwB8G)AyzNcjs89j4 zp5R{JC1=%96>xezN144r3(P?)8jf0*o%(WNf$C?40z+v;IDau_h$`n516wBEtU7(y z5{oiOIW_1|X@hTbD#=4-*^Vbwhg^4^wvHT1I&@@pB#{b4Q%4yo4HX?#%xkFWxP{w9 zkE8Q%e|Z?PK^UgZRmWUuZ3xR}C?3S!pJ!0eK?Cm}R6gty>l-g7@8b2x|dflG?<0xpKpT?#EPQ&u48G_H_iPzDiN&)_)C8p-ti E0455ifB*mh literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics$2.class b/out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics$2.class new file mode 100644 index 0000000000000000000000000000000000000000..f4935876daeb5799561411aa3b9e46b2e97dae65 GIT binary patch literal 1501 zcmbtU?QRl56g>mT23!Ow)NiGV3be8m@hccZq?$-jf1pkL=dcX0?J`Stml|KeKR$r* z8H`D*#veX_4`sZ&P!n2eE$(J_=FZGL_sqR#=I5{PKLAv4FM?#xXVefO9!&s zUBr;0JHl;^d`8`o`Ep3Uq%GQK0~xx~q9CCmkVtqU0;nZL(Pb8(ZviQqO;fK(2rMc+A&(>OyGIr)$zaGb@t31AB$dVzGNuno42 z&^H|Wg0Og(f?|E5i2%Xu!hZB&Gd_d#AJ{a`ktEVOa literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics.class b/out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics.class new file mode 100644 index 0000000000000000000000000000000000000000..4678a4241066af31874547067ac1dd352be1471f GIT binary patch literal 9145 zcmcIq3w&H-@6v}p#m0EbitBRYgyC<6&F`!6?Ut;Dxj_p&|MW|q3r*B_uiR1$)peX zZGXSs$M>CczVrB>^PTTbo;>#DM~P@D57$zLz7nD=9g(L;HTuU8Rnrlj9t+V|>1%cL zb^0fr9@puK5PgHbDMa7W=t-TvtNYuy{6Oa8XXJK26|PaH$t=tvYBva);S>eDv)xuPWyDOfr%W{S<_k9xmM?p z&UNq%8+BaI4R8-P)^QV0spDooO)gV4o>oU!@)V6*YUyg89^x4s4pD|r5Ahj%rp7a6 z_$;`K&z7f1h-dL^xy+HVxia=Pxtt@Hc_D7)`8v1hyg=i2jiXHU{Z=|**;}o=os8$0 zjP7*Wve&1~T+YI)HsVdp@5$IhF*9q%hppHybH^?-7SD`~WYRI0B<9Lk#Z+g5YZQ`m z8Jnr~gq#kgFj}^qu|3Ad>k3xE8c2><-pCZ&!dNcvADeDl@r<3wZL{n?XV#-`95nOs z;r`@}MgD0}IcrSQ{ZU&C~5B^R7fK!rssJAD47Ox5c%35%(*Cz-Z-3nPP; zJzx%|Fw)$UiJPfQO*<*??qIc8#?)0|i4&a&KDIYe+ElTF55U;n==A+fA?;vj%bii2A!WJ9+KdR)MT&BI|( zUA(oo%;ImO)U9`z4s+)`;)+tX^s6SJ%``;~5YwuGVJl+pwrq3AilkM^ zj%0R322oUUksLOcNY=6=S<}uZ^VqKBE$VK69b#Z7)BMR6c9o9h@+S7Eb-RZQUd5}+ zmB5zAv_h1?)2S+O(ra&(RhoR;Zq!B$4Z2wzH+B5DGd_}tu!@fym_^*o??7x+ zrx48pEC$^mqNh1D(b9w?dy`+b&_|N#$VhIS$Ve_L0l-uq{&ANxUDC#u1P6k>NDy~c&eqBnab_jjA&Y{~%C*a4nlf$s=vq)xFj$aqY2W`h+S5nEf z4{J8KdP|#%%4SeOtU+@rh9b$Z0lVfb^NSS+IC}fi#gUQlKq9U{u?VV07Lu>`OICxf z{z zg5I@N`k=+9t^@Wy>`OS&&Rx6^C)Lew>ZCc}>0fwCsWKA@ zrpqe1cfujH!f9i?p=PdN%Xu8fuKX$q{b(i0rv|`P71lkG1hsZ8Y;%0i)L%)Dw_>4M zA!ugNP{D!G3K8*x#R-yID-l6GpYif$YU@NoP}B}tc_-jZ%PUBqEZ$|&^>S#T{WKFXdh@=Cbn{Vw-n3Io?OU z0r^x?q{K>-R`+sBra6-p498cOh+m<2I+O1$q*4&<5C8Eh5x!4KL*#z@2s&>txpjl2 zV#lPBbO`aGb0lMr_9+=T7}P$Qo|WivT$SFcJ1t+2pgz2fj~tD@J=Sv)pI>sJV%j!)v#&F+{Te?ev~5kdY#2*`4BbBIT%kD3*FT6M=6P2JH@5;Zl+$r|rLV!D;yiR-(Fs*-pQ5(T!mEqIu? z?H~o(+8?H>!&JTfAl39nn}Z7;A?*<9RdiW%ZD(~`dvj=v>K-QJFx78wZU|R5H%1Rq zQ)f-v0W!ihhiM8Jo5$!hH9r;e)5_;tFh3phkTgR{3MO`3cr7RFW1cNsB?bq)55QFA0=O73DHTxg3-$ij>R7QJ$+PS1QVL zLD^ZPTq%^E9cq3R=2w@`d)le_^Dw`rIA10FYXO*pbp9BvQxsjGSnp7@%k%~{y%Ez7 z0&-RKZd@P2Pd9{)t`-el1NylZ9-=usO!IgrE#h6Yj#G3okC4e}x{kAS6JJMnuuXSy zp6=xWJGdJ=rd^W#>V|*)G?suXCeixe1cXN_&<57Mu zp7!uZ`F(sh-_8&49ejj8z)$gBeuh8Dzu*t?%lzSh#vchx=Z^*E@y7!z`OZK$-xau= zKS{Si@1;~D2}19o_u@^SBq0OT5?D`r=zXwmUSKi3pKiyGW$OYn=nnb-IQ|l(29nI7 z+6yX{CfP~5z@wmOgg0(EOq+m{3z|2NQTGD;Z{C8-7+ts(=~UG|Mm@dJZ}4_E9XqSx z<{)Ocn3EXTy1-4(UX_HxRoAPc(iW}>S0AK{I)l-0(2vM*Yi@Ja?9^n@um=84T?x3j zC?KpKS3;jFp;k$#6$v3FVMa+pNFd>_U5emb8&OBeFz=pufAF6S@M5PuQh zBp;-`{1A>kU!wc?0FCh&eU}f?Q3T~ze28A*!}#g%EBO9?gxmR1eCK-%N2RagWb-vn z^Vb2`#}%wT0RY5!tAbE9R){JH)nJWj3PM5JOE1!g0V|F6(R1_>RB`;IN}s2XVyu>q z(x>QS7z=ST?WB)mtd3{WHS|{)!{Llt>8~+X&)3jQ`UJ)r1a2OP@w5-K`1J${c_*F% zu=`573v^XQ=sMc%g^&IQo~k2$op09YZ#DWmF#bJ#5=_`a0(S$;O%T2leuob(IYiim z*8!B5F&%>2NML)2a4y`(o8<}J$zyb7XHC`eV7TTGT?Gm)j6FtC%#bHVadoF2)(_Cz z$LN~Q+O{x2b*(>JWT2QXogo!;Y)w6nkog!5cGiXKggd@xj1m*sJj5G{%_y-*c6TgX&a+ZpckPkMX+e38 zU%NcG|Jd{3oi7q3mwrEe4pais zFjc=!r&nwA`C~LoqX!sIuku;1K$S0GIzV3p;XlxWxB|Gyr5fZG(H3~O2;45bEIM<~ zEJVN$8&#XV8YR6Bc0t6xGZ+p&L3QDv!%^Do&JxM|YbgI4n$Ew)2KOSZ=9jR^{f@f% z_sXejr1PRJp22u>a#F?~f@1>E`#fcO6)l2slH)bB8ksCP9#ke{i#bN!FxgGZFP+%B zWs_Lv)z6c*?hml;kFf4fu+nmVlE!Bt~B9^+7mU}@QLA6jpOuO5J?4woTpac=s=Lw{%=5m&zI9t}>*U_#V zL#2F!A_1b#0Mo`mfVKpxs6S9sw6zlv8iNf%+K31pgbjkNfnszA=n>4a8at%m+tWX zt=i4HcEu`oCAQi{3ki|p($d;$7qzWzZSCG_TeYjTRgnKX_q|z?nStQ{w+Xy=-+lL< zd-ikgxi62s{=kDoRIfF<>1LiK7oS{advKW}t+^gDd7iXRm&+M)Ia4mxa+xm|zs@xt z8ppNrus|*gU0heji+FJvFX6KUR=v(=yLhQ|E_3lYGU!|nH}LW@ZsZlxS}CnnE7nbn%fsE=BYj~PZ9wY^M`c)u+ATmg}{|0 ze^lo^!kyc7{+KLshcM_)o%ed^O8%%oyUW8L=TCU(8vdltpOT56*7-9Yx|a9pJRo!L zcJpWXbHdDfr2AfB)8}>mf`@M4FA6KaB<}(`-{;}``O7-*XByXTMnh)&LNgf;2NO(2 zb2Mtkn<9Zk!o*`qB-YI|abqmrU6+V-)o-uc80+pfN_Xc*inU|-`M6x}6g~`OtOpZWs2PoxsypLNi zw}gAmSPG=+ap8+F!jVKD)4T%f#Cm&U(YjPJ3^ZjX7SiLXXfzxJ>-Z8TPgg7+G@Bx^ z1R&@MGn$CS0dh?NSc+ANx&jQad4U)mUs zhLbCqUaBs>T&|dkkIFEz_>cm4=L@d6y}7vn2#bvnsVHI{7L)t(q62gIolp1tiy*P4 zSP1+s-x!XXt*PElGrlFz89@)UIT(ms6o`lA+3s{Cd%}=OeS!5(CKHjjjW*N#0ssnA z2IGR3;94!%G7ziGA!Oa|pxKuU$D)8fu007Iy)e+HaD|fxan&t~#;BrkX3<4k@m}z= z6`SbNwVl9{+jHD8^g;{)uMPHKRxT4BnPUV@WkBYu25CB%BrGU7j-q z48~}e%__XC$a%RjoJitrSzjy?$-t=^sY(5PX>F>{nVH6jqcW+)-81oT+WG+@(c zJt~84h9?xin=?XCDg^Xo%v>YHFJFmYriQV@8>t>Em+L$T5$imp^FahHsHu3Ga3GoX zbd^1UXeeTCwwyV{-xrID2W%XJ4q9V#sTJucop&nXF`Zk0pJOAPNlW$>HU&!{TVj3T z;O0O)33`+Ty87@S6fjN5^k*qrTL2XcQd*hnMsZm@c zlaUEECz_1+W9~ZZZu<~r17O)a)s1oGwrkEJz2uaU1{hdJT8xFTwE3S_6

4FvDF ziNFtbN~pRvN9{6L7SiQ{@;GJ03vgSMKRyzR|I7MgQ#>)66qCC>+#L-hQ*ju>U2oJF z{u|gTdxPwH`yyu2RDOf0=L8trN|v9vSw5$z;703pON(0C(-}yZo4UFZCUP-XFc57+ zRsdx;{|hM0N8(l)jGORho6{alWsV38bE-#b4jfZXhaD1wUF^74IwPSER7EFZe5K{dxYz=+~ zX^iCXDzEouR+3#sVoiU>9V0!AtF6Y!&c#JlH{Zjj7k41jhwvAke5##uNp3Gb7TIZ} z6B!@VSq0|W?6c=IS;h(aE>O`{5}pmCICcUd*nB8vu;L|KbD%+a3jr(arImU|!?C(G z{cs+st}di6;sNi&wiP>8orjtF3rV=p5@U6+r=T3Ay8=7R6hgWO`%*KJn73%rBBo1D z&9_+lvE_x9wYZYAPeIBPinI`irf&1n+2v_P3W2h{o*Q%GbI{v@c%4~dLn~lMUy`lEB$5PnVuMf+f zvdq$hg`)EY-A*4f=)?38gFZ{2Gw3Juj6t8IPZ@L%9Wdylw8x-3=uU$^O`kF7to zcNz2vx|8X=q9hlvRcqCeMFFhB4;VB+cN_FU`VeX(*ghhajduIgKHryy<7=+le8}Lh@L`=FH25KY*r5IN0Mn#wh{|k8nHFq~1_Pcg5m6@MP7j>f6VHgRdu$@TJoq zVbWs;KhEDa_&fYvgTDvq@e{J}ll+v<-#7RN{6iFxvOL=ui*{p!Wbo5`#NZ$CkC~?D zi`ViKvI!KxKQZ_jq2s6gGo7C`_~-l!gMZ1tGWgg08-xByuNeGWdf4FK@uS#}%d~`V zS2)?@lS&wvh5feBE!=DHz5ObKf6vb`&1zDqOb9(`F=4CglTD0AR06yA{={p z!6=k61?MP;k;hI`q#X(;f-zJhgUF!_I>s+!qiOIz`4yf2#k8y_l4 zGx#VU6GMI7;Nuz@8f#kG+D2~O46Q`erhrHa1zkgP$mlhvickIq$>= zi}Bf>DccoQWXzdkxl|?O)$w?sUs76Vg62bMAxoFNAuzPrV&#@gQFE-}uhCIMo1@K5 z&rz=~2qb!tpc&dc?Q~r`!_dyusts+v=1*(Z$Xv|OYP4FWx?(is>#p(^g6jgN+LKOe zvkUF;Lx#3ctJAebhPGH+VrXY+^@et~G5~EUYOeX9if4Ov#9J(AER)MQOiNB3fUccu zXbl?D+lg7Gx1nfnm55tU1>(s|lyJZK04*J}%cf~Olpxa;L)1+aa;g)OWh$U>HmhuI z=WPd=E_*}iP=O57LfEc!XA1OEhOQ9ma*u~lj|!QcDV%>)+FQ`v+)$>>DaH^X);dSQ zVN`E85(y`;AB%>t?M9^>#b|#4h*p}b*4WY$kL?mEE|+CiNxvHVhm&RRua@@nmx}om!~CB;{zQ%2p88i!PG( zLKxd+qK=F7uDbZFaPSx6g3sG<}ziY{L?w&tA76{^rrN=0|zI3c4_On ztlJH*2lcjn%+@~SF36%i3Q3jHj4aK!n^Jtwu?BP?8JDuZb(mNMCS+|TZ9tf#Lx8R3 zZj5$B)opTmq|z2EeqbC*I{8Ffd=Y%W6v!SY8@;ieK(yRWOt9@JrXsl)LcS8Hyv(WojypOIn~g#0qbkM-*P*DnQY+l;Y~<5ydO0k9E#F zaa3tYSxcp`s4a71qe#0DYUxO*(^6+L9teW^Gf!-RHc!R;bVSY+DtJ)&>gL>CV!Ggk zGX9u=Kt-*1IrX{=kQO}lt5Iz_(wrX7ry-k4Q|xnbP;G#!NHbd z+1f9M+iJMPHVCe?M;Qs?+9}MvK+VDIh52+8Ud@^TUEUQ)tj$Tag+sZb${r;vVL~0I zrr)c;)C8^~2ZJeK^~qTy%=Q;#u7wo*N$!CL@|o6kTefX#+wit^7dNfjyrp?lD>m1# z;q1AEkxtOp#6mE<#V24=26_@la|t;VPX)!1EGeWnqYxS~MReJnUKbpgFoFY?y}1x1 z;{D1ojeKo6uhMQaxxoa&E;KF@vj9kyCtBy!IzAGxTD8^fg*@;m{8TaY}Ew-Uz!6cjkm>ZGYYE^HmxWmu9Jmv>%_r2F;tjAk1RTyVk#8d zRI40aVs9a;o0Z_r^jSQz zUwoS&q3}e9q(y_UMYF(yk%&kwB>@ufzUXy@qiNME34~Ca9+91;xsO1H?*&w)<;2x% z8G<~pHx}=2v&Mto6Ko9f$HUoms=_eSl(b2W+@%NyyCPxazLQ1_u*%8M{yzCm=bT){ zvgycScbc;NE^&4vP8h_N6|0glOY?CSpRe;GuxAe_zo7Qe^g6;T= zrWvDWf$|AfJXXOW?XC+}Qb+j)d)*UMrALRJO!#mqy5S~Os+ipy*;E(Bz6ijg<1^Ie z`PucSX6@F7aP*e~BEqifmP?ARKN?>Zdkc)V-AtPmq4Ty^N>`?cBIIs=)(3+4^aQ@% z4c}~iFvoOyu8yh?>}>hu&RkBj>1HyC=~fy?r%3@9pN>wHVzRoUcC7BGFsnPt$m)*z zvAUy}tnMf!t2?UBxUZ2)v$~^hs`^nK#$A*8H0~3y-aT|LuAe7jH#w;cy&L>PRO+ue zK$?HS0n+`o2Z;TZjsxWKIcZ~X4=Fs4+sgjKp~-A`X8mud;S+^Qvv z@qi)w2~;b#>;zOqMm^fjWCTis0PwqW5$B(;nm?kj|(&T~TFKm<%RW-vjrGy5K zKPyjD)8oCj(>WNf57KG3Q#o4NFim6B|3^c4dL)!bG1R*U{8n?PTN7t6%KYVO;7n^d zS!pq}60W6(Om9&jQ)Q$UfbR*q@dkKjsE}<%1uK<}>VdT{q`%a902hE{tXaU-b z=z40P_hI(?G5=;jxDkT82?Dv9c0xa|f_7d9y}S|U9k<|A<5n>K4*c!|V?|tkFnNNp9jg; zD(qM=YmjC)IJ}PiG-nTVQN5bGO`his)9Lug$vK(MTvqCZjOH_tTrMqC#U7@b2A6Qs z>jKYe9iU@|yh3@asx;HT&_M^OZdvKT@ki8Z3NBhMEW1d}FZa3zX)%<2)(|aeaLa(R zs!C;&$3~)FBwQoHIos}8T0-);%}MsGZ-L%- zK*KMCjwc|bo%8_2{UCjS9)cK;VCL8H<{M}|O3%&mRE&o3PG$`YnA6<4S?U zO<3u1z`KL4q;J!AV3T*l`oD{}9{L{W@&Sf5^9jtf#y^=E{}jfHLY+zaF`BE>_jUTg zF{;q%hmMun>(r&w(>ndgagdKGM8Il8b^jLwIVi%Xf{FA*oF%x#r_kj~5cu!+j6 z{1_|$gbZ+|f_|!aVOSGwx*6P018uZ{dc<>j0c~{_+IY}H^eaI7HSqpT4kV8aNs|*q zK!9`B?!p-1ZcEJ|tr?`IL0UUV>y|q4xPA}4Np$M`n#{WkluFeO6N*m5gc3tTskhYD zsX5Y8I<=ue_j~pGY2#$Nu+n+`>>=7TaQvA8dS0My&H}4+LQNG!^u1{h-FArDI?!w% zqAdb=khTsJ0vVXYLk{)Xl&tu#r8U2DQF|xe3&k6@OVA@>CFe}Ee&P0 zqW^Ez(7mAnF~#tf$-B2@F#8;YQl7;Mt;+@}FhJ*ymCnu)bY^WPPSYWYKTtXS5w7u1 z(6Sex952!tXxGq7)JlJbBm66(*54?F1CiZmUqvs&3H}q#?-fL&e<2FJ0^R#JZ01#Z zf?k8%j?!~<3~ye?({ZlA7qydcaJH07@bP)6lJ<3wXsgnqpCKY0fh9akKZl&3plbRB zG{XZZGoc;7fpt9xSN&7^8T9TTod=8gJ+%H-IPd4^c`D_l87R<@jQzEMray{qqEGA! zQ4B4y$r38(<)H3PXrI_s72a8JUjSTd{EMoO=yY-&gR4>|Q>`7N6*|479xi^BrW_^1 zdI1(mUhxU=IYfJF@2Aes&|Ca9wL=u#Lt8|_Li%AcJ4&jcTwU!Qj+*_{ z-QMAZj14qBvK#mc?ED4r8AL=mjW*r0;UM5^G)G=nSYY`D7>Xs=-}ZRFD^z%yt&&!jld z!e43l@aGkCaL_&%$M^G;Kv$|1V{cjj@ckk(@hW?j7$uF^D<^8o^!Gvl4?_UbCqmJc z^U|s0H44n107l}N&0-|9Pg7H-32Q);@kyFs5t7(Lx?_MGCHDf3MMbF+xnE)Av!-P* znwG(6nvIb-OeaF&dH5_uPML9qzk&9;RF)JvM9B{17%7%O(cnNcW9iOJWL=qwtlQua z9Dbkp7-j9d25C1Cmgww}PG$D}Lv+P3ye(glB#gVopQ*%Sg|uv! zs~lc;Lz!2%{M-Kw($$0Xj)&=*VR~l?ZS%UM-)avWf9Mch+fnJLbb9q6x~{=}h_3IL zagg2xc53(2yNBsL8hPD=^xh$QUxVim-C*V3?{6qOM2#J@DjjL{@Or#uLv&-7)hZ!} zg;sDQ`FI7b;+3?XSJ7MfJm~mp>g6@~3#2CK_*%M+*U{~~p7!yEjN@95lwI6a8Lfho z6g~F{b25TCYzKeoI|fH1!R|0L{Vo+m55p~r^O3&C5a>(cz&=mMAxa(Y7oWqZG~h~F zjqeGtokE5-7tXT;YwV*b>_8e(mJzRl0hemSfTv2sco9l0;=POGWxg`3zJ>&ya?}(Z1#~mUlMkVvF5ARBr9&9Z?Y9P(5i%sDgpE=I7N%VT+~X& zhT2qk()`KV*e0LG(+W)HF-!*f(=&3J4IuL4o_O;?y5-AMAzWc47Kh%9a>7DTbeXXDP>E#&~%}JG;K*2iy%ysNjhXQGt5j1 z0e7Jmal-{=5v?diK?QA-rd32mKm@G|h@vR&0xIrOYvcFd_vX!;N#-;hdrr=syZrb6 zzkk0odE(f?hX5>AW&q2u$G|58@ZjzMK7~)qaE~7gu-Cx70T|dLi_etfv$(Gu_v3Ri zeBQtp0;s?P0X&E=mg7s0r-#%KQQpa0M5pqGCVDx&&cqjGCV7XKQ6;hWc{3hpDL8b6Wx)t z!sPa=!aKvEflM?OYLBKfEeih5Xm3268AwGG78V>YX-}kjL*Zn&Yeyv17w+i`hq@B| z{fT&}gZx^S@%qWha4HkcL=*9J>)@tDGTOBvyP+`2#cy3WN%AZ6cZv(hXVp>^h$F2n z-jmRXs>UEvxU5jDHbX2N?+tZkQqg#Ci;F-EQyf^KKy3sXH5vi@&z*Y0i|Z0lBNoq(+4dfB%m8jog{DTL}>>{cdXu}GKr z-VN8aJAaFUccoT&QhPKW=@{tW9!YHyLn&0ZC%VG1E#XvD*7l}XkSokA>Tex*3M#2E zuPBk@1sxQmjw|?*Y0LOVGOdG)(8M!Lj*Nt@02&h&3Kfp`BrFta#&I=SMBAv7!3-wE*35eD+C;X2A*ek);;!dHbRAK$1gI+HML8Hi8!dx&Qh&PBmKz?GiX-*{0Stbe|sX4 zNoP`FR-?1rpmlGZ2nQqF-JOc0nYuG9^zle0)W)GR(bX5ptgucQM1Gr4%84(|A}O_9 zwJDt8(%Q;SUB$-dBjLEQ5U3qW#coVQjcS@}%*$fYn9bvQP@zn8vpUyl^=-vv>+DkS zL}>f?y!6$5?9m5>5RD+F;%nk!tYUpM@mu_k71G4-@dpzx z;w6@fVsRu%Vd7@IY~YV3{)9gpc*VqD@K+Om!^cCNV3morGK}I5gJsQaJm=M5 zDkaKK*8E(~FqH={nP?NWPAvx4c958$gI`)Sk8um0sY~si}OpL*b-Cp{)ba z7)uV}*qrO7Zl&v%)|tvsWit5r{bxa4Ev&9Hb@vojnUh^+77+t{fXZVFzm$f?gTBD3TG#h&Lbe6}D<6@A^_u`82%i&)dq>u5AD z1=32jZbtS)Rv(miCE}TIG@iaV!ned!H(8cD6))+H5Y622PRGI_IbQ(m{Z_A2sH1(0 zMZ4V`XB9gam!w0RWuzVBiA={pEXJy*zs0ap=2=d!eZ;DjDdS{|7+#^A)3u3orX$?X zmBH19gnXYmslLsH-)$FPUu4i?7fvQ4ajMhg*4`#{Ti)8TvyC3jqtDuMjIHbm8qV{% zgd8qYG?$t~n7U0k)yiO~) z+uqT%g_Rz%us4dc#$a9O#!4=YHr^am7t1FfWXyZ9f#W@x&v6b{^BAA?sNy<^M$BfO zw4w>yun<=^+u~J3Z zq|;cVcH%;{3v1P_Xj2Dpk$MCdtHWqlFJPT|5gi^s)_bb3!808jJu`8MCxlMVB5d-s zVzZ|kTRb=7b=XLhL(H*O{>*v_I+;zoaSAqJGiT572ih&%Gbk~Dt;}jaaV+Mo3s6RM z+clj*In~r>Gd=JK`uK)qg3S6ddyolAfoU;HiUV}Fv41>~VS~JO9^64T9nJq^* zvs-2n&OE5_45Ej3OYoq28ok&-dmdCzBFb4Q?FS^8FBZB!2VI{dU-m5deUAK?Bfp=& zAX`%7j?{!BwbhZDbfjKQsVPdm6IG*hWT}C)fsFneFt8Iif(1u0jngAI`xs^zxJLhR z*>n^J_m5(+foqST^$0vi(a4=q#*3NH$i8G?(7<&D$lBsQkC#SK1K(J3;3%eRx#!rG z_j>s8M!bodu}(7Dd>nykhp=Fqq*BQ+nzEn6IO74HDIK{!u?4;3$D8pMtIFbe-Y(_t zW;+SnJxd#O5_UA%DcHn`&cIN!xA71vw#~S~Gb-7?P=+&yu_&91hXnyiEC{w}D`{%A zkY1G0y#YFT4qd#8AtgcDKqt(>47?R@1-?`ILjmLrjX(FnxO?+GD~L!tH#Y#NN~Ix@3C24O@<|886?XZ zb~Mdz%DM%Qp;SGvfVEFpmuRfj+EO>55;rnMcQM6xYcX{e14m2{@5TGbz7pklKP~Ig zRw{!!iYbg%p4@^D*y3AljVklR7pilMOPxwbPNgkJ;e+@P`G|$w{AOjN8!W_KbsGRbE?lB!g=KC8^Zb3&DOkT q1bhn0#u9D}VQHbT5tpzgTbKtQCtD9bfh+kLppbd|H`x_F|Nj801bayU literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/Configurable.class b/out/production/clients/org/apache/kafka/common/Configurable.class new file mode 100644 index 0000000000000000000000000000000000000000..369150543fdceae40ec6c5f279897c8670ec8036 GIT binary patch literal 232 zcmZ9H%?<%E6opS2hOxmjNc=2xV@GTxR>Wcw`#Vji%4lWM@o*L%z(a{D3$Z&V`R@Jh zeLQb>05f!Av|}U;ZDCEWRaHuc{%CWgEAh(df^X?EAMZVoYGud;VE6|tK72t&@zj;t zsQF|WrjpcFrI2e~_`SW=*Fjk!r*8$rz?O>Bg~UPf6Xhr3!k*98@L!gp8x9JBU3Qeh ZH5nLc{s(|X40S}lH+-w1=~En1>jgoyJ&ynY literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/KafkaException.class b/out/production/clients/org/apache/kafka/common/KafkaException.class new file mode 100644 index 0000000000000000000000000000000000000000..c5ac04e3f564c81f0c43b865dcd7b3f374c8cb4d GIT binary patch literal 821 zcma))%T60H6o!A3%akZ7A>7iCmT*y}>MBwf zeE=Q`an3NJkfstd(ixxQ-=F_u&&}=cKLGZzRX`qf4zD@9;jrPL;b2ps9I8OeQC|h2 zw&wfsM*;C!pwzV{jHHSBaull^arhAe`2%fqbSThj_AceMbVt$*+>d-RB+030+pWiXm?2-C{K&oXsAPhX3qu#SB|AORPL3$ zxRS1Cecu}Q3nxcEJvENVbVtDPRT#=a0{q97K+cnKNRulw4N`qZFKoN|xc;=GOp^tL z6B`Gf`lPv+rKh$&u#qB)C~;WhQ04GS;O+Ao6R7=d=38tc?W>2bv56dABwY$>aPWe@ z^OVh@fGmw^TZ%8Ka*PbUfYWO4P9T0Iq6LZt5(Dc5siRDz#=Q)Ss8B`=u!LoT@Ms0A zq-N||ian&9*XLX93FfA$8)UALPCtRz-gBkBc+jVNC zGgg!8$x)|Lx7>D%_8wWedw8w+wJS$xBUZH*stW3qLeF8R?zl%3PWG?WUH?h|aA2I~ zk2j`?E;HiP?X&IqDZ4dhP1W@M8)d6DZnYel`-?HR>hPw!M;fh4$!c2Vs$H71X6CF? zxiLTAsF%*zq^f+D3=;a*krgJcG&^SM{}=r0q)=`-@+lootYuZqCFbla<5sOrnl^P& zQ7|gDJEoIjcYha!0ncm@2!_{DNRPT!d5)Q*$!6*+oL=T#|m!q8ByRNgMxw+qGJ#0duud^K7SmD6xwRK*Gi6O=6RaCTpfF&DmwjAju zJuQ2oU31x!DBG9YR*hw}B@F)Asad=19vZly5N%A&u53X5Chaxcs@Wx9jdxuas@SDempmNWN~{(+hdGk)N{j3r)N1%sBRR z-l^xCY%U98^A*Nn+2u)c$SN>#1+OZw(?5-qCccO-(ewZAJ0?bOMq%4BMvtf`+D>g+ zsOwv%C_f>+e%Qoo_%b;i%8E@8SnS99c}R^|r&vfBo4`s-aA02<* zXVW*Z@o%?~Ie!DaBhld;_iArrd$Ew)SIEUC_uWKa6fb-wqORTfNAW#ciG|2JPYN%f z4-@>%y2$8WBGQZW{E|kTr?30LhN+iEwC~?Y8aSf< ziJZ^f<2nhB`d6v)Dlw|2$Xzx&qND`}8U-RQV zUEv4ws{!ZK4?Oh-o_c*x{LaIZzWowMNz-`Y&xlKKz0~W`i?3;eULMDNgY0Ns5BID- z{ASm~{dmd`VAykV_2I8}Iqb#Z1d;9SiojTHe{8F7rfapZ54(^%;>4D?YCliti#GTP)s2Ji1s|!j>Zc z@5rS1A-10JxG@LG%SYHk4GJ<@!nPmk6HKPtko|aPKQ5EdKDm$E58i*npq1LHrEc@pIgTU$A4p&7Axt3HcRY5$~{pyvu%m4JYtx zUhFq`7Qban`W;)+B4+S=%;OJe;E#A6@39-dAENqdP&8L@mgf|0Pvbdu{)lc%y=+Nw z^IzCZb~F6-QG;I#9(kChmXy-bTiC%~dRO-DCFIBqZ=K8D6);{|mUdP*VT^ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/Node.class b/out/production/clients/org/apache/kafka/common/Node.class new file mode 100644 index 0000000000000000000000000000000000000000..e5012ca4112102603f06530b7a91cae91f6608e1 GIT binary patch literal 1881 zcmah~-%}e^6#njRb~j0v5>ir>go3rL`9X=bQL8}F($c04&^8sNXs1h9lZB9tNpSpI zeDTR24?O6M1jo)eKG_-n7ykpt`rX}4NohtOa_>3ko^$SZzH{#8&wqaX4L||qC=w_{ zVBqa2-od+4x+0hN!FkEsP_R8MwrrmmAZPsatm)wd~U$a_{)B|IvQ*+t^XG(6Xnzx!(rDo@M ztj~9>e8t_}bsPB=W+M!};YvwhDxLX1Up1X!VWV^$P*1pKwQD{^t=V#RnNn}r?Y%my z8!C6K%FeRY^uUo}p4ngQS#=^zr2De1Zf@I^&TQtPfa-2;zerthD4@BWn%yGB9k)ku zwCwJ+D)y2i35m#xW+X-vDcm-38e<0Tn7D+JiC6HdiE+6kF(#1uFM`)v`ZB*{9$G^j$L^7}IwMzzEWD zVL)(5-qRm^X<+d9z+mlo@RfnV$$_x-&|`)%awO--%GrR4lQ0a; z{2QaBV%U^3$L~u}`Xl-X;we1A(E1@x#)rFzu`Z>Hk$BvfSzoLt>(YLThAu{rPgbRG PI0^wKfw!1l!R3DefP+8B literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/PartitionInfo.class b/out/production/clients/org/apache/kafka/common/PartitionInfo.class new file mode 100644 index 0000000000000000000000000000000000000000..1b86de35bddec66b0e2a96884ecd719cca72571f GIT binary patch literal 2326 zcmai0T~ixX7=BK&$u4oBG(eHBkX9*S6B=!+wbG&%#fAo22?ZkcV@VdeCG2Lx(i!jl z30`~QO>ex=8G()+xpl@r;H|&I8O7&pLik$D40FzV-aXGb&-=X3ZusNhU-tpr#A*y0 z-fzPtOvvGm9PY|tQVt(z_)x`1G2FwHhL2EKT|O;5b@lqT@r}rpPA3iY{hievjwkaJL?kyZB?`8*`Dn>0wO06typH+stF|W zZf!klR?X5=E4yK?ZJ61TySeE)**Ujt5vkRzYQ-*@TLQx?tu}YqPT?D;G~a|o@7Rv* zO$yXfXKCgB|F{;nwRk+XD4c@TD&A9@ zRgRv^>d~QwIcLqKG~KDOvqcGnQ)5&($%ieZy__0rQ3U1p+^jUy0S+w^3^a%EOp5;}&F5SVE~QoeDMcatMqnPk(aa$U|O>ppEf zJhWnCX>Zm1NTBn`@()*^StU=!qK+jj(*us{SSpG-K1WK&+qh2IbgbZsK<^Q(E^?zn z3a}#if1x9dFLiu{ujODe@25ruM$YSPvn*+n6U+|#*tK#@**aaCiLRkzFCH<n+wrP3-w*y|q07}PiYJNw00|%ltTM{IEt4mkwhTmnQRP`plncHGiht zIBVc5Nu0nHuD(MEQO-$Y55h?AAY=^gAZ(29Kr#B19Yl=2$PS`Eb596wa7;ku*bjw1 zZsQFdv~m}-eRT6MPkxh6C+lZo1XsD@={`5%N0MF0k0igaA4!IaA4!f8a_hxtgJG7l zL`IBX5c-KhokrA8UiGij>l+%^@r6Z-8Cl+OaQMel;PF840LB8q%ITYL27-rR1b`!_ z!4rYt;XrWoG7Im)Sci_PVCsUhK?^n`T9@+)){;}`bOIUAlFO7AP>ACSnT zhpzj)FYY3~luk%g`>^TaRHATOPQ+@P1GgO{65U*`S<`NuGacknLjks`Ds{10>$ Rx5VGLGRVj8F&V<`{{Yp?=FAck2pjRUy;TVdO`C&{Rl9 zLCdcxq@ZUDdZ}C(_3rrRK4)A zt~Pj+uLL#Sc%nmJ?D*1*>i(`mu@Z!vu5RgC-E+6~*W22y1v@)I(|s(>JUZ--9T>$D z8y+o|PS#gQ3#48H$yVs^uxu*ycH50eVQ@Xtwe1JG)h$hq=<~j5>jvwO^`m)M-STSD zLg_0O3#wa3nTVaFDTwM`$dXaeV@Dyo9<;-n_k}NTM~-$rD`eUj!@P|l4BNax2G3FF4U}W-BjFVGVL0V|m_P77fe$Fhk)$lZq?N%roFd^P zn8X?0e$fXB9}Aug zHord>JpHf1T#PvQrx@uxD<@tcEyv!_65Y zY1ioTF@-esvV1>aV}^{+5~)iDF5xO?Fo!FB+|0*N_{kw>!|Z1mCGsZRoM)a181`F& zG%QRh(xcvTNU@QS3)$5e#h)>XHK+UqdtF~~3ZMD}6g7i6Bk hIh?~l-tJ&PfORm~!Kvd5aKCQ>Cf;pNiFp!m{sF%}5u^YB literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/config/AbstractConfig.class b/out/production/clients/org/apache/kafka/common/config/AbstractConfig.class new file mode 100644 index 0000000000000000000000000000000000000000..6dbe4a27a56c840f3aca920a1aba611d81f41e8c GIT binary patch literal 8093 zcmbtZ33wD|8GirGCbPR45=c3^Eu~w|WH(0-Xdna%1R|7>+J-}@wN8>r*k-dE_khw` z?|Ro;Ypb@^YN?7^4+0V8&|2^NzVG|KwPNx8{+XSf4I9#@J`b7s=b!KYj`urf`OIS< zJ`P~H>WyGNPRZqC5qumEhjAS~QHw|L$y$60pRUDc@L9Qh&cNpld_f4mD6hX1!j~h6 z;45r)9g&Q1u}-Go)t8*3S&pKuy$na+$4A zmC6h&H1uS$!)>|L(DIwwdNRYqc6PNwu#mF{73zC#vi4bRg?uvA)@SGWBHWi8PFwjx z)>c?l{C($O6AJ=ZL`MscCn8KcPj9R3zM3CR;pl= z!-{F?eOg@8f0I3s*ILwcCez9MT7_#GwY}DuHL%xi8?lB)thRy7=x8R*TY4xt+}5RU z8||Uh<+9vIZQq7WCS_ac)lEBzRaa(^Xw>&4({^uRwBOEdxB64uA+iHjYNwS=%Coy! zHDVu-R~lH&%hi&M@_UmskzSxf&Y|G?{#-t54dfj~iB4qD9uo7CnY2QDy5lAL6siX^ z^tr-$K7ER|DafeGPTCth(vwLuAQn$^MyMzw%mW)U1s(3wA>h#Q zkd~6nQL}SM(XrJ-DjrxlwK5)Ag~}m?BSHn;fSv-}J&qNQE2j6O=Q8m~SIWvUW2f_o zCZjTqQ`?~{fu`$p3SJ@Yl;By7ZWQ^7aF8`bi=Fgp3Jl6wICVETkQrkgYA@qnxd8h@ z%U@90Jl3aD4F5(+f@(XR&yr#FCg*N$NvG|sHb~0$9EVADL`3x~M12ArbT<@| zsX^U`CY504Tm>_IEIb|Aj_F#exx6!i2`R-0X~G_icJZ(9t6 z&|WLI*O>*F6I)n&`Z9&=fZd&xVc@*-uH7nuVdCky)5Ocw`G$&`>H>A4i5=L$hW8ET;#zR%)h}nYbQLWe+!1qiQnoG(45ju7ztyZl<{$3Jn!ERYEnJ*oCKy<53ypia%jAOEE?t??#jgJU>8)Dn`(u+vPka%!P04} zcC}JrUP*dcH8a&Jg04CgV>uji=Gl&zp*l^q2Dg}MEndYaDZxvQPPvmaSD9*^_`WK( zCT6PjYLRSiFjSYRHWDJdQVy@%zSo|F?ya}+05AKkTr5`@8%rha!B~GGA4}%a%kuQM z-zBZ4+N8P}kGR8BPg0vrwM87abgJOmD#NocsIFE^Xg*oT(i!qj$C7S|jb(;n%=4-E z6l=0f$)?(>R+_3uEj4jFURZ|6`fS!ZAgOPvZMZ`rRzkXREn@EK7Zg65z!Xx0S~CXv zkkqVJg_fywa)Pw2JZVjQSVHlmIL=-^%}K{7enlwED03=@|I+i8yq&f3nJn!rcH+_` zz(AQNQvBa$-JJU~(r9J!az584%^%ug*6+zD$*qQS12eWvBmEpodV0()>Y+oss2>j- zEN{K`%{Etc;phl~V8F$K< z$+Lpl?|dNcl+RXTr_=o!#=_jjJ^o3e^sQCPh|62d7}jHYYABN(m8Me3`0IVPRshaJ zxXK#jH+m@SNXzN7K@@w`i)I61U~rcF0vB(A&fK((|Tv$Na{wb1{|AAdRUE?5;n` z6q^#ll43I_ClkXznI7VV7u;`U`O#g;O=o-Q>>Y{G+O)mDG>n#gOB8dtROq~{h$M5R zL-IxCU>|l0m;QYwIAF;~mcdw(m$-yKXwBzOZ9)Eg#W91c2I<*Q*v)kSd-%+b?>*nZ zXEuHB`9{pq9p(7X#PQXyO3$3yW zajQC7eG1{baltV}c1NqnQQI7?o`4xZ!;B-SJ8|&L7vjx_Q6EplKZN>m%sh-)@ndM% zeFWzn#q7hF(=v{^cfp|8dGFWu4Dfn6OkU%(ZUO4ij5hwL){a*6@_sF!cH&ZMeHq3` zxgS^HR{s1p$P>$HMH?=LjUldR4TiCocUdKI6D^jN;1bZQ5u{LsQLatzU7GJQ+_?zB zGlaO>z?gw&7|0sP8OQ^dJ&Hh+3v)q(xX%F*H&Wp$9<7ZhnuAAi{t-mQrYeX{5iPTw z_mx`Lm_zDmSnFw6>uOliOLjyr$G7__h9eY3({p$(49$5Z5uTNpzy$%^jq}AH4HpJK ziut>%5`9N;(Ge^-L5ew~xHy1aE-@YTkf&h-`2wpk1D#liHN<+Y)^x7J-_vv@@x6u8 z1*Iyeg;aM)nH7>_+JTZ>gNn9ct zJM3ym0(ARWg*{ebkJYpBoGDl>oXqO#X;?iM&zpkPqRFg!r(yMcykH7eizlk` zt_X+(!(^p~G*j?WAn73Gs;|Lw=n zLb61g>z~!Zx-D-4w3kf<56RmiH!RhR3mDSE6MC z{Jc5|J_(n0ed8uSt;VrxSHfj@@XT@Y3^KhtCa^kyQX{yM@Buj*nrFi*;6m)fB9{D3 zEPy>MfqPhpELNWs>rb9FV?V3L3-D|Wh!ogOYzJ?|o3xv5_n^5AZ^m0_co7K;-_ zNvlKPzqs7My{z_+Vk7D9E5bzH)tXm3(dpD(v4;g!o4=F`ZGL#07}Kz(nYVgk!Wuh) zwLu&_^UYE>a3be+c9|FAO1#MP;6;odsjdG5ajd@^jmNNIx9eB7+b(*cVIwci&YGwn!KM?@;3;(9 zg;~+yNpy?q6L?Ypn#X2|_@FM}OZoq5z-u@wyq2Bpb&RstGXmd0$J~i^gnBclgPm+Z z5|Pr4shDepten0ftNZbG8c@TTLO(+iGJq{{?OUPOjeOOT0(t8Y7GpM(A1Q~lAIAv^cy>`E>9Jh7UL~D*> zTZeHFO&uX=powUW3tPiA#4B1O$UT`C4O!iZsG-r08UlByIU3S%H~HX>8dKvPa`6uN z@s84KWV3k!*Rs?cJhM9*e2A8{5X^RtKaLZ+S#W@RXyd&ckM85!`}zI>QoJ3j2atP%e@6-1@NR)`vZ}9%l}JfV}E)7$4-yajOJ`BjG_$f~-B1;digho^GeH}i+#^H>ctSd$xe`%IqBwTHl^DgsE#L33(65!>N e&j~HSnO!5UfB;Tts*`lNOr465KZ3hz!v6!Lufsb4 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef$1.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef$1.class new file mode 100644 index 0000000000000000000000000000000000000000..905e40e175fedb73bf8617608cd1698001ba7ae7 GIT binary patch literal 1667 zcmbVMTTc@~6#k|y-F8`MEuaE|h*f%nauv0;f(2q@(T5Tqd@vc7rCr+YHruUCu{C&41v6EXipo}J@OoV%HQ~5qc!97=w2d0KsD{6@g(uo8hO%?u;ghhG^U9|&(Hw)T zyuh%0k!TUS$lzA1H3mkh1`0M;%iFxx@rrN3V76HOv(2EAQ z8-j`#=UqCMhOJkq)q*i@n^yWFm@~w>TreA!s%`0_xq~OOGbgZ8|2GY_V%Bw|xUJbG zb6>(81*^ELU;u*(0=TE(K2i$O$VhmoU=8aE9w94XLnxaJi%!S&w0#P$V3a1^aU?Y@ zc$Qp-;kM}vQ4A5MX&L%$^RZp6bUQd4iS~4(U|U2BG3btn10$Dsoz5I@RP46tIhpio z&9rQ8s6?%t>rqymck48aN-a5s9zG4DdFmdl9g|_GcM3O+Ml~fS&SUT@VGR9mdp*Q< zw#+cno21J_G>0j=vid>S09Y_6288ZAH$4Fa$$E)a8Mx>V$g%i0u-JFFzL4fZh~8fE z0eg)hT3_yXVVG9pC%3cFxe|lX&GnglJoGISZ^3;GB|+i{eXgchkS@%mdz$gD@FeJI zp)WIWi~#9g(!DK6&50&uaJTDE)9nlP27~M^g6th8*?UZ}4@j^h%(IV3vrlbou};+! zY#EGUTwtf&U89wYuJRBjFiCl@lhp@zFz^TCEVxeSr@H7@>8s#oAQt}xUxJ<%`~u8* tGKfy$*e&S6jkeb`Znk9@w@7+$8#5T8)flCfNs6D3tXD}(kbJN<`VR%tr!@v~g(-Tk(P$hwaiXEW6FN#dv38 z{4HLqi803T1Nc$KbGC$X;RQ6^_nl|`C1Y_V)vy%-?bA zdLgn^XGgDoY&rt5t&%`$$@W~|Z9f-?=cYFWltNIWmjkBjJLS&)wi8z6wnsCAW?)Ni zQ-&_{@noOXmi8WP9x)wVAY%oc(00~bPU=6|bZ%E3NLsh(+g{Le{rbAoZUi+I_f*Vg zVc>oi1JJT~fP5AYF)uL6DTXKgx={|Sj@?*u9j{gl!ywG!5r;iydBU>5@|5Kn%Odgu zGgJ_xTM#2Eh|w(w85hJjIh#(w*^=)&VZoEFmeUf@dV5#4cOAPekUQ&g0<&igJ-!O1 zdn3sM>2-Mbd~SNnJl7hF0_sYoViuRn0>=4sgRtiM(u=x)+9GLHH%jG~gr#y-AYr|# zY>+Evg(s7%mA5Nqk-22?>w>v#S=u-?az8b;(1!Ucd4v=$Vvzh2!4Tp!0>1o$IQ*0O zf!KHc#(zRlDqo?zjxb4>=%GrO>|u&f?O~cQ)x!*-b}IS;^QrJGb2t7Q9W9N^xQYaZ zF@Z_)DdefI64X}9SS2sw4J`61-oqxZp@|lG8=vr*JWlyQGWt1FGAEQd zsm!TXY{zNkjVf~{i!qFwIBQ~pV0fY;37e;v#a&S>3vaPF69%rgILd3`y;W)A9DBFf zBUn?OCxgkds8(f_P@MLI#iCdh&XO#yh=mnVbo@%i_ZYkdcdink ziu6Jey1vH+OL9Tf%Hgai*EV!ho_t5c&Zr_j>WIJmc~U}0(qzA2C_t;IYRo3>$%}L;d|llqm93XKDTuR8mAJ! zz}npLbJoqZZtI3D=6YvY50ie0`Pww?dh$lCGB1PMV!o^y^c_*26@jbb#zP}qa#=2! zNDkNJz5imanWgkI6AEYLx>(hfwT3USKv~JGjfl%7QDwm?Q4x}bUZx^DqHkY{=QA#2}fgZvPS$L-A2so<7DnzrB|&5Qg1*w{4aTL9CTD^rVzIc zW=uGatX6x5hK2~8h2C{4Y+L~BkJMM>6!Z;zf|-P!PuJ06 zcbRo$-f@K_de{zff1vv?=m9eH5IK5;E%X?9dV+3xibM1a!}J^{=mo~;B@S}Vt*8SW zLV+XhMNdZBQ?n(sr>073Pt9d$PfeE6o|-+aJvF6?UiP~|zc7pz6Sj#y6a6LzObnVR za;VA^hIlQAQfOa0y_=xOln8q6z$+!GoG+LV91ijKNTgpd!qh+;O3W?kVagie& iBybd)8TWIa?fl#6n8@?LXbkyu2D=z{bKJ^!7}?+9aGsFEOls7&PM< zciU3ZS~@dm4t3l6L^X~U9bvz1H>$S(j#pj!q6NthY7B*R@tB`+)8(RWmRi0e>bcB- zlH*0s;%=Gyj%4Xlse(6%G5#+Nb#2`dPHTf9d5eldrc5Ji9XZs4Sjnji-fH`lNlD8< zblq~hmv{S8=S91*K9KwknxbCv+P-DKab%B1u1CEfJ7nOlT)3n`TezZOm0@XsRWUGy z*9IQqk%3Xf4UEepfjIH?<{+zJ2wScvNJt``>8tRddTd)Q5}W>CdfMv#_6ED2>&tjO z^_v26n~U`4jL>hVZca$C09n!-Sdt8kP0FT71Y)>{`(T(L?=e~V=pC@H2wg(?Odi7| zoe>#hGlZYQbQiyjP;f(!tJ!Y|UqY4m&<*?p`eIz6FoW43PF*CFWL$(oFQ8@Tr!Nrs z5`_Edde=3QN7C^mz@!kxQ>3~=K2jm_+4=#s64#UqM9&cm99<2Zp};Z1fioQWNjRlL w>se7d7%El59SoPw!K8qzdUa9=Hj6o`S?Q8K!yLs!NaGG!oscS?V{CKcFL%`>nE(I) literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef$Range.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef$Range.class new file mode 100644 index 0000000000000000000000000000000000000000..2b8e70d0e5a9c10938e31fc1e457434c83dd0b07 GIT binary patch literal 2019 zcmbVN+j1L45Iv*YO0r@H5e$ieh&T{QmLF^L2g;`mg{&-C)SUY3%GV>ynMIBv(W8poYD?#8iZ z;GTi|0^ycv38X7e<=3**l-6!(z1ylQr>w8!GlAGG(=y#ffh*aQMkh77nm}aP-c|w= z71L5a+GAO7Qj@OO4cV+o$JFOQ8FBZ_jzIJggVQFpW?9NvZpuzaQ9W0&o!ydb%f_B6 z?aQ5gS!&p=mTgg3JLYa_*%K>j=duUDybbAAl6{!PS;U2*)+FbDs|O$Ol!A16lSCTR5jePS87YPyuuFp7uX93`*>pmH(rp*nW`%r z`wwK>>jEc9Anw|}ufVx%ZYW+KFWRsuu!%={sitrh^E#o&0#nDWEj!b)9hGtSq?O^|J}DN91y=P-{+{q$;F)_*Qa?n; z?|{Jksq5kmgmi7@oEX}Nv%2}&)4~XhZQEUbavl+W+1cC*ziioVD~sbcJ3Q4vvrqOv zC>{spn0;>;Pt41}{^kh$A@cxXNYl%qhWGf+O@a+W2x&cBd=S%mz9^&AabS4+4o_P= zg()ZUhY5@H_X;u z5E#*;%^5=$xd7{LKL2C|r|>&s>-nD%`UCT`2Qc)O;=AuLeT4YdOyLNLtwSW!V+Tn2 zmuC(zeu#qiCVXN{3M*Y678uP&P2PKqG#H0o9*qs0(lD?El2Y9--F; zsio80lht!gX~^XD0q69&z@+CseD}r(s6Nd}rU|hQLu_DLJVah>VpUX!nr#IXH3_zO d`Vup~#(ju?;g=}W+(mGMGo9c(-#{gn_z(30)l2{Y literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef$Type.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef$Type.class new file mode 100644 index 0000000000000000000000000000000000000000..b7a23d9897f350ad2e467ddd4a1e96698b0def7e GIT binary patch literal 1460 zcmb7ETTdEM5dKzHc3D?Nyx;FFU=^>uVXcT*X*ZC#XhQn5(iK->F$gC8L49vAsZECBNRrhO<&G9TZWIWzOk*|WcXfBgYq6^0Koo@#i8=L%*OtZNv+3pN`n1~qbS zaKuX$ql}v>#u#HN#u>L%Ofbe(Ofqh(m|{$*m}cBj@JfQRl};OpSW1GNd4Fb0(37rf zm*WNN{MsV$)H*t~BDqqrSaJ#6BPSn;3)r@gW_W*8g4fLKB~z5z zlT2kKw3@r=Jwn+?^P+b1!!37j_Xu5B=r5Gelm1GQ>+|2q}yjYdDp5`%aq9zWQV$*E7Ui~ z8d4|RYH_{E#y99tHM3ML=jLCPBZ5U8 zgBZ~<%4UqsIGYJJlWeBgOk;#vxH}G(MDJ0qP;#lao?xg!Zu;=V&Q+-Y;Qw|cVXbl2 z;vPgBGTt8k{RX0&OLV0M=-&D^2U!I`NJ0dX7J(#3ASn_^q6CsIfg}y^5Ai%HlMEjo z;324cVpD|tXf;Tm(Q*yXM`98}WWDT@hA2LaxqAF6TEq@5Acw!8^%^qAx7>?=L@WAD z6y~uY{B#Ty#TA#}p&y`x=SHsJ{UqFt!^t%9@*(%}`PQxLx*NS7c1Rgi8$di*zWy@K`C1N;5zcdF&7(Y?sZ zHME)8)*9N)tha_vGpp3lWoFeHy3MSgmU;3(a#3FBIs?ii64daC7>ZuD`!(;Nd Q;0cBaHHwrG#n9&9A9qze?*IS* literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef$ValidString.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef$ValidString.class new file mode 100644 index 0000000000000000000000000000000000000000..303c5e03204944ebdb32b882546f08a5b997c49e GIT binary patch literal 1935 zcmbVNeNz)h82@b&a>;R`21ISE4GK1dmunRT4bo!3mlCmUAmycwvm}?eaJkD|E{=W# zzk;8lovET_rtRPTOpfETcb9MkI>V5e+h=#5-QV+j-ZuYz`s;51Q`k_^kEtZ0xT#=T z#SCt%w78=n*Na))RWPUEdlgAcCGdlk^9kHjk;j5uEUHN0egY2?c&Ol!f*%>Q7u*z; zqU#8&)?_$S*yb;|-g1Sh7ev#|F(iwkW^uRW7!1Q*>REqna;v8M7`ZvR5}y^8aOW8M zGhG*Qqa}t|-mVbX>4LC~8kW5A}ofb%;w=xjGmNL{}Esm0mEui7CZ+S+62R~4^eoswQZDLKin3td%^Wm8$Qp# zY#H@^MR1K26xCBjmzQ@Q^M;oRDl&$IYx{X-7|4uXiUDI52NOlV&x?vv7n zhKnd@IEO(6j~S*;k~9U&8lK=MiS-2)>^^s&f`Xql{DLCGlDLH`ko%(=*O4g z`mwlm>Xo}Tu}FHU_vg`!ti8-}VcI0T#-4W^{)+M&=S|t;3?m1sw_Q4ZcEe9WpBJFc z4wVD7m&=7kVCRB{@#?D$u_28#8irM2mN` zX2b28w@s6ZpK_WzT%m_elC9fEa0^`x+zIWp`Ij(64{$%I9q5BBZIb6{7QqF?XrwBo z@k`oSBN-v7W_Q7|V{Z}pgH};|MPnaHunol6CX(zoT%x@}|1>U>4f#_8^eZ&iJZp02 zLdV`!yv8s_Xf3VXB1FkZ+e36c`yR1f^!!fih-WV@b?ljE$H$Uwd5!e+vLF*+yn#YK z05jwNBEFpc6OoAzP$GDP%pMZ!W8-^BuJ1xk^}d59)u~hO(MR)lIK7KA|Lpshk~pk_ z9%dlLs+eZNvz_(vJ3h`}6j_h=48|}{YVeZ;f^oz76B1w5AmUx(U! q&UW!hrF>G|0Wd=W7z#*4aNP^j4Wvn?h(eTt`8EB24v<`mCqDy3yy4LR literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef$Validator.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef$Validator.class new file mode 100644 index 0000000000000000000000000000000000000000..821b859405b196e628066cdeb9d84074a51fa07b GIT binary patch literal 293 zcma)1!D_-l6r2~+#3Z)(5kmFif_Tf(QYZ*L2;QIRON=JFvWfk;9{K_OsNxzyJ$iXF zJZ6}gzyF_K0CPNt=!ZxMiRk8NWzALjMR*-Acl^z&VqK_JYfD|Ecc0JAPO>(gtO@bT z9Bn4=WhI1{h0%Fgyh%Pi+d~WpGh++I2hO&l_MGooWu~r;c5v3Jt7X(>b89Riy5oZn tt;H@XZW?I_!&|XS4Z`@*h=4%w+;D(=LJz)U9-g`zAa)o#1tCNfL?^N}P;&qP literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef.class new file mode 100644 index 0000000000000000000000000000000000000000..e4b274882ffedf8cf21c8a4c7c1aa616991a28f3 GIT binary patch literal 10278 zcmds73wT^*nSQ@yl5;Y1nxxYU1X`JvwB*{9QZ7wHp;w!dwn$1!xlbpPWZKD0n3>QP z5fEK=*$b$9!QzUjNEHwZv}sB}6joSSMHiLAvbw0Nvg|78y2!o0-+#``oTS6^q$|(v zKAR_VzH|P|cfb8-=E-*+{3?LO^3f_J@y#Fx@uY=+2%;QM>d3c3_%@yj;yd`R-u*f?LijO$qL-gm zaml~x#Lx8Z=X&{tUS72EZ#G_14F9gr{}IA4ere%XLDb{rDx~mA5WDbe8^6(!{|sUQ zerw~uY`m&0{LaE_HeOdr-UwnX-n8&m5R34(Uf!{B*b;JknuuP?I4I>oER}#I6$&dY zv4S{L?4VRhP%j}%91@Z#)PhvoQlk&GL76IHTc%kuom6CoEi)}S#+GA)I0w&KQfJF? zREr#M$t+uD+wwk3P6%R?L@b#TM32l>bSK(UZ_7Me=G$_TEe*Cb+R|i8a|p}vf-Nn! zw5qUFZoMq9rHu-ag+c7VA-y}efpyIq}sy}YY8C+`SqPcX&k|k79=ipE}lZ&QeoNgBc zHtiaU3u-rPylmC#HOtTM+H_gZ@~-n&3&LI7qE|-SlEl=uaqG5tEVq=~V|l{lqQ+e@ zoJ%Cz)<=gpUDciFPepUX8KT@gezwDh&gFDzS31+*79EPl2I6hoqkY?>ZL##=U^+#h z(zUrlu8Q~77sOq(64;se6~l>4 zyjLI>a(XD5$;Nx>-^x3myK~Xlb_SC1R7SqwjsIu9a#XJJ`2Vi(K2!SQ&2!eH66D4A zK#)nN%Ayzlx4iZb6m{SKq^R$mh4%lSVf*N^X@yB4f{4d|p%nY;lv|oBm=rZRmT!;m zI;yl-ti3FRLz(mt3t^=}#+#YrODyxZS7xWVj+AGpqp3-Y;HeWvkMuU*h*#Q{EwzOI z9qc~^>m~?l@J9HV0U^r6N|4&XuDVKB4j;XpoVksL0s+N z!?>P}o3W&kYi%?;z=&}0AzVYpa^y_hXvs21&XThoSuQIC3nrCp3(6N9Q*@MZ`YG@< zoG~px#F3S<4=zxgCjEe1RjYF-C4>jf1aXuVB%nn4qx5=2*AC-;uS_ z>EO@s=MMJBI!n%Rq)XN_fE?K%8yz_p-*E63xXFX6dTs3tIKFLh7K9SAx z7!k?$DiPN^@)FDy%$^(BF_4H2LXXE56(&=P8%C4)WB$8+(YmQ4Tk*&kINFo>6L1Cga(B%?!WGlwT zA?r-#;{EZABQfb!9?9DYrO~D$DRTr_zypa-@ByB=9O;vO-96w)LbmDMc6vy0RaZL2 z`J@bvQ<|F<97##qk|Eu7g(DeZxVl>l(dtM}?K>=2GNv!iHns5M$PU?Q$u38(k`L0l z1*W64>);@@5N8sMs)~~g`0-quT;s^q@*zQkZ>JfLBvM&(j8q%M^k5p%)}9znC`#0s zOvd}8NjA2*xG!F&oS)jBO7BQTwBaAWqht;g%wWo+d0gE_(tT#938!l)Rgqxn^vHtc z^30qz>e6uVSNLlOe}kKp!#0;iT6eNCj`K`6BOi^17P0e~EGhS- z>gLnQ=44HW=GDj1e~r_W8}WA2r--Iq6}TWIwER+)8p0G zMkeIS=+IC+#bau#e}QX?JW^jLQS1inTsm*Cak%qa`L^ci5-2C%Gnlr@*N%8hPaZ+R z*zeuz4+>R2G)zmK;TJ8>nX#U~11#AT9-D8_ObRO6nH|>jQub%`CJL@HbKK^<3vsK( z3cVq|ldSPgnHJiV`_Qj`HNOP6C3y=HizJ2x+VxUJ z_1!xXxtMlf^^BT&_w8T3dHYxIzWpQ6w|`z+qY>>!P1!}Alqj9)O{AhpBN3l#`9_7k zLT7&}or$lEYSU2C@hH4ydWel|cxN{7lvT8g*C|k6OC}|bZ>!I*Z7S6i85WbgJ&f7S zpc;b#@>aygw$|6_tunK^eU+bpJ4_NGL!!8uF003r6vfo#%&t?C{Ee9~Q-8Lx35Z=<< zQfJo%>MH6gXIKX??I5O?V{?^KM>V6fm>#jHHyvY7XYj_fv7;JvjHfwR#2=wlSHs4w9JNv2>a4NwDWizL z^%OD8af`DxP0cN>F7tsA%$TUx%MFuj-K7O4YYI%(6qu|jFj-SzvZlafjmMn@@G8CQBqb{P^FWyk}gH~Mnib9yR<;Kw9?fJ zl(4JdfD!ce(t;|v85Jdre9KV?ny2TArz);BUZ*PlEI#MPtn8(as%Z~-;lJLE-I;=e zI7YCc^&pNF`sgTc>I7dyxINHZ7ubj6ypWvJWS;V4$Cp8$XKCD3?7?Pzm~DcxO>f_~ z8z=c=_ymTo*{^XNDaLUfwLRxnOf`FMMU^PBph4DNp|*EGw#JUzR$S{_u)3&k2|oRGb4Ba2k>`HUQbgA zKfr@V=^9y^Xu!`iTq~%+{N3F+$s21X_B-$e6ZD4)&Tt5KniOpF>-;^uD_$b)#6hxBSr_m|^O?Ji zhHe?b!V#RDH@U&Eh?vRuqP?CBb)H0Dp6XFL&0Tkzf8FV$IAh{E;;UACBUmzu zc3r*H-MQ4ivqR@RS@IOlba$RPf@PyP%fI?;eV(v-xr=qVx9)iL-W9uHjbf$hY*pC^ zR*zy$e%t#!G1j`<)_U8jRNT(|qIJ4x4CibKcOAg`N7@4}&AbXXQgJlvxu)*59x;8M ztKoCQ-9-gsKCOI-JjdF0DUt>b@{Nc-UyoZ1{3ISX@CiI);353Vz?bokfrt6-n(Jk@ zG#J>(pD^h266rB;GvA@RK64{Uka5kh02#g<%^I(u8qr& z*x7)z%NZv@d=t~~B!A=bE%pcB#uhw9uD^o;e3x85&7X~Y57*)Q-6TS($|+HW+$EOWCpM+3l7}To zsT|?2X((H*JkOufye46JL#7qdTe&>LM0gVd@;sB!{Y--ac>{BKw@;H=*{pn0Cv=z%Rlvdsmoc|l z%(-G=^iqpSSD19i+}u{Smb97MWd0a7Z7Ecz1315{c?>;U*t>2X#RXnmtZxZ#p~r7- z318@fi(GKA3odcNr7pP41(z#ehZ*&iyS=`0SHQK^tT#2r-5GO1aeUSYdPfj{m)Y4* zzz+AZ1SyKy3|_K%1d9F<47@vv#5mN3^F4%4@_Dq&9at`3;BTnz#74HtJ@Q3nhA&~K+=Xl9ZrmXE;A8S-M&G@-O}>JA*(dCm z`|yz5&#!?V;8%V7@MGCe-yESY9-!}y(#HiNQ*`#SHmAg)2+9rRjw)WTz|iH8wKh4CA2fE7|* z=dr`;DHA8k%%)Ghj&-#*i~2WQufi-s{bPBgY*dtqboYp|koEC#MzDJjnMa@2zbRv= z-xBUSfb9=}su;gz5t)C2V`WScX~Gpmks<_%hIuR-V&b@hFhifu66Oep39qcI;_u3m R*m-sxQ%4e45%%al;P>!h9P|JH literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigException.class b/out/production/clients/org/apache/kafka/common/config/ConfigException.class new file mode 100644 index 0000000000000000000000000000000000000000..8b93e6b83bc08652e15826c91f100d5a4a080a83 GIT binary patch literal 1261 zcmb7D*>2N76g}fCHYRCH+NNv;m(n&{;{s*rqEb;nY6sVk6$F-i$8+b-x#yf2|NQmk8-PWu>PTZwM+!L=c^xqnG!*4nl3`v$ zS%axzLC)@}xUb>?!$?PXHg9eVuVcH;`_1PJY>Oei;W{0kJN`Crc1403$cHf`S8d1k z*BHigl|6pI%_esmX4Us>r?HaXW{7XNb-^%Pu^sWI+o}og9j`Shlc~5CHR9Bc=20^4 z@7f)Pg^KGnOy1_!t}yrc&OSFSx7BhT8k`-wVGQX&$3RRSZl2gmz-*2}9CVpIiH{c{`A-Vo}9IIv@tU z>UKR#ytJh|V`mhWC5seBF(!k-FmX&P1b^1Gn|0yIqa2MNZ~P_$gspP zYV5e45gOU`xb%lnf?jUzNuaD{#yG8AJJ z$0@T$Ifce#p?C=PIjG4})Pw9K61as4nrFi*T*Wn-Q9a?_APk9hGs4k%L79w?5Hvw#WZ2-Aos cNsx+ZDNCX6yGO{JO2%YTt}@KfHU+c40mefZ+W-In literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/ApiException.class b/out/production/clients/org/apache/kafka/common/errors/ApiException.class new file mode 100644 index 0000000000000000000000000000000000000000..9354d483175a450f049683f1234c5684de96b6fd GIT binary patch literal 944 zcmb7?-EPxB5QWb=i49Ro)0UF(SNen6)F~JV?t&_z0#YF2CL#A*vrU#b-e~Orj|Eqe zxZnYJD8!7hq7ae`Y-#KnpZWIecz^!-{sX`>JS@Y-9)|}U_BkAQIP`ENP)(H8GU_Xv zXcND0zZ4K}1U4>AoTM^N`!brU5^?wv0`7Ai>-1c}Z*)G&Pcn#PJP5j})$yRUQ0Wb= z`7AFZ)jI796fevNC9v7iv3fThUn<+eGSvn9pm>8SZ4~GRJimE1gZP$-HYt`# z4D9j2LY3b3|H!a~ZOW(tJGe;@e%r+@QZu%eW6vq)e%Ei#;4D)Q$;44c@N7Vcxu9f zAHWY~ymz}Xv|{vuo#f1)oco)zH}m7?^>+You~9_@D=e!lYb;Matb1q+EX2xb84i?- zwKd1R4+7$oz+BgwIFTk9$S_r9;_xQ~D(|$>$*w?g^Tj}*)U~HdV7{-7I!L1v<&NY@ zNGFSZ8xU3!oc8%)DH-Wlpx3u<=*x2%jFf*Sznn=wuu)`O7&iI|B91{#SV+{ZN{rf0#IJhC3Y0i&PxTVV2|%gFxq&vKyeI zk+YY4RmLg@GL4B;ztthId=oZZT0OL>3)Dx=J$d!N|FzADYp7w4Wr?N1(iGUgqt*$$ z{s-x^qHa#L8*P;7?a(_w3m)##e4lm&R8gWe#U*_}CvRvLG?zPLh^x#pPg*4_&?IvU z3$&_DOn8Vz+A&~>lmWZsWN)RjHAZ2Uuu3u3GD2hG$_eDhBh-n(tVYi2p-2PpI$JNt xD9*y#RJ4)7)+e^#FfWHuDIQ~)u3~I6$4*5o%~GFH#OHsB$SjW7qEH1q`2}4B#+3j7 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/CorruptRecordException.class b/out/production/clients/org/apache/kafka/common/errors/CorruptRecordException.class new file mode 100644 index 0000000000000000000000000000000000000000..3508b3bcea737c07cc681bcaa043e44d0a55e4ac GIT binary patch literal 946 zcmbV~U2oGc7=@4fl~HwUbhNDd02eVaI{L;~t}72+dE|k6Xfv!^X_S}@ zq_IlJpHAO0un!EYfsSn=;$$EusVo!4U5ugfTE!|kVJPlD8!(gtJ(LVGWKh zzBr%IO1-N?!U}@ZJYOs&k+KZ0y4s9B&%= z;kz%PoF!!Ln4z)Xy%e8?KN0cBKTC{?N5{^FHT4{3e=8U|i>>*NOQp_6^-QNGl<$>GV0-x~4xK-SH+_nN=eBtyEWZ%cHsmUKYI)EL zIE53AZ93)JaFXgIlIBWT$te_<&S4ETY`D^NrNwacU$HX0_(xw)QqnH5n@@o@%Jdh} zAAv1+SfTL_%?hZZM9(#>QoTznN3_Oh^g4BRjl$PVv_`c`FMv&gw6IQ3wYd!UP^Xzg zY~+X&QaWv=bFhT4Lq>KpLUXRl38cq;G>G9?&1J0OZ8w})?Z2=dkk>h@m9tJ!q}v_s h;Q2L5w_#7nz+ML9bJ>kCFE>W5a9Een7J&-b{sj&q$ie^s literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/DisconnectException.class b/out/production/clients/org/apache/kafka/common/errors/DisconnectException.class new file mode 100644 index 0000000000000000000000000000000000000000..80a6c42fc07ab51d59f784f3192e4bda1b98fc3a GIT binary patch literal 873 zcmb7?+fExX5Qcx}v=D`Ih7F}{Qh@{kv7+TJh#L+PRm26*+?&Ns42g~GP0MR>6^RQT zfQPC&vxyK*Q&ruSe%rI-Z+?&W`RC&kfIWOHqJUMFcP#H&J_M))s0z$@<+SWHmGjz~ zuct=>;)}r4p*22{CTYq}Pvwcjj}R#AYon6`f$YX+Qy_O}ua&@bLmPG0>s~2$DX%&d znQPdXu#(`k9tLwsTYG`MhIOq--pRPFqFedxRz|VycCCq&bJqFjNc-3tqvGWFK2~=L zIjalQHX1kbyNo*0w4#f|Y168|id?px{lUfRY%J(1?`11>^2aWLd@OsOJ}!<0P<%`+ z#)hxszgH|?*q)2kiRR;nV~()HK1wKIie;W9WGM@L`u~Oq?EYiZxuPC%LtdJQD)Mys zbdgv=fC(BWX_Y|{IhvC*(l-?OOrxMNU+Y6Wq$$&+MM?!iN-krDW~D(AW-&)A2F#N( z;1pSUfuOdv-A85=vC2PVil_|Ih6vh#1uUl6083b=f{YD^*aKR#g+gtskL)P5LZ_>y i)TKez3$-w$GOIj}l^bQP5o`S~EcV6ySs^DGy!{1buB_$& literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/IllegalGenerationException.class b/out/production/clients/org/apache/kafka/common/errors/IllegalGenerationException.class new file mode 100644 index 0000000000000000000000000000000000000000..48994d15e954b903907ed6a15881939048402275 GIT binary patch literal 894 zcmbV~OK;Oa6ot>^Wr$K*8rr50NXn|D2n)dm1$9#%3W`|JlKs>Sj;S+KjUC``v5Lfk zAHa`7oa;nY38|2wfFBH@pg_UDxvs~MJsMCq5k|UQ=s69_?k{>CQ4;~B^D#z}FQCJ-$*1VmJ&x}9S zXDKu50~Zmi31>V{R`OAjDf9-;$DzK^(a417`s29{BR3v98yfGO&%%D1npmf=jWu59 z^!4&-Rb4K%PK(iCZlY%4tb@9`)+{>RyOh7*(V zT#gj`sPd2C*ToJ3+~9hXQ3-WaxTittEoPo@Ra~pxDb%;3WR+THDbQr`9jtNJm=)nR z))@()NfkgJW!^y0J$yJtX%VqS7p(%(nU&2EJV5kDfeo;UEjAQvYmPl)EEa0r!zs#( n)Lp*aUP0ZQRb5hRbE>emm$52~tTtJlYgpn-=Cea5CG7qN$||)7 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/InterruptException.class b/out/production/clients/org/apache/kafka/common/errors/InterruptException.class new file mode 100644 index 0000000000000000000000000000000000000000..46c9f88f85f42af9b2fa8a1e35fa08909c6b1a6a GIT binary patch literal 1055 zcmb7DOK;Oa5dJ1F)8M3`X=w?Nrlc)-6f6Xsg18hAg$NEQkvMLhB{5CxXl(<3iwi*@ z@dNl#h*>9fEYYgMhqY%s-(zNXfB*UM3&11X%i{`e=TXJBUTQkrF>oi3Eb0b!4eSw0 zzEqAF_oeb3&wY3Fj6kmlg+tHv1K|dJF%D&x`Se4C+!M!jf~SOfqx(^O64qF_L+i*5 zq*CEDkb~!6Y&i{>tJCZgGKbzk5>~s8E8m2Z6RD2H$(S>1UC(A&!Qvb*l9^!S_=JaD zPYo?G74}G4r{cq@uxxKK@mz~VJmoLtmSTjgEkd7}YnS;VJWhJ=1grAzUDGf#J+*MCK_m(XrZk$2ZZWUDTXG- ztgG6D2md`X!u~~=*Lvo$J;GM9vEz~QKI=`3Hn_BO$3LVP@Y%6LrKB62#p{h`Vz~2c zhp^(D9U4Nqq36^}%KW=6Nnz~3;N#3k5*4f<&+n@orC=h>-$ktQTH?%0&ZYQmw%Rk$ zcci`{{WYq*#%qzQfg5}wDvZC0HIA;&>ruu!M~uV<=lEK1_2z=L$2g5FwAu$VWX`9p zB8P2;*CN{591znMu*JfpNg{z%Bv4rhyhSFmQEs(=BKv-Z+=ViAme^&9y@dO6EK^>{ K5O=5NbL%fY74}j9 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/InvalidMetadataException.class b/out/production/clients/org/apache/kafka/common/errors/InvalidMetadataException.class new file mode 100644 index 0000000000000000000000000000000000000000..7a26813ec80afa58a4c0605b3b1070b3b07f6131 GIT binary patch literal 888 zcmbV~UuzRV6vfXZn`{lK{byoR8%?BznidyP5UKQ~Dnz0Ww&wkIGfmcRX3FfQ`n7xt z_}~ZdLy7lpmeR%+!Cg4>hjV{>=I;FZ{o^Nq7kFGj8Ji+ak%uCWe6)PD6_yk2B8vOk zC6P6syKfZKdxgbAYm$^q+NXG;OU#i+DU@GFCQ1(!iaSsG3f`d|XoaO-Wc0_$_(Z#7 zI*EB_wP!=d62p0&Z+ht{N)%r8tQ!V&PT@!gXLNc-L1@QgYXa?@bxF`Q7ZgW>4?3j* zrS$e&sLxZ5x35s!>3yLK3Su(D;3##G8SYCEf2@;)hS`_@GAMLzD7ykW9y$9}f@Z8z zNRxy~wHqA@>sMjdODjiq;zIo{k{&c~JI7N=SVa|!B5NXbk%q#{|7?xIvwu{6(%daM z<7GR&HcEUK_`hNUJ{EYq!>a-+@OaMgSnu-Aa~>6srOp)UduCZ;t*{kXXLAF~Jgd%3 zSivf<1hB>`fCF}lx7^vCqA*X`IgCqjpJNv?Y$L~Bi|EZ$w>aYVEfIw!5jQwg0T2EFu8Fdf literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/InvalidRequiredAcksException.class b/out/production/clients/org/apache/kafka/common/errors/InvalidRequiredAcksException.class new file mode 100644 index 0000000000000000000000000000000000000000..b356a8b1b2171d216a3b62b7b2180f165c1b851f GIT binary patch literal 526 zcmbtR!A=4(5Pd~p;|hW(V7zftN!*J!l!Txrn2iTf?q#Vz<)?;oBRx__t#OZj~RbWosA zp$EnuiYQSo6Dq)=ie+vnZV=Nau0Xkb*nI|j%6lpl-JETcau-#?tF=(Z>aYH({`>g*4}b%F$|Hw$mJOComMsru58DEBk#bswE#)F@ z&1L;qK%5CoA6XN{(!?zp_EnZR{0M>EcWrchD3IRWYYAkI?6ne@ZD^y;``s($n(`{7 z%tFHkgp~xRc$Cb<9UTdLYgpIzWlshj<=@GhJLw0u+qK45&RG}vb#pI6eciM@9i03O zR4*oPHG#@*<5u2FKa{5JU&KzEcI`!`*>U!P+tt{b*Huv@+liO|P6=cK*^lVs@lfb=+Zi~g>h iP*;Xk&(z$A$`)5Au`=VV&&1k!g~h&jJ|%LJ!rBujUaXP; literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/LeaderNotAvailableException.class b/out/production/clients/org/apache/kafka/common/errors/LeaderNotAvailableException.class new file mode 100644 index 0000000000000000000000000000000000000000..b7a6a6fcd5c6405f7aa7424cdcab07dc57dfeb7e GIT binary patch literal 535 zcmbVJ%T5A85UfES5kU|IA1A%}NZgBwi4YQlnqXEgDEDS}Tt=1|GCL4|%ae%*KfsSN z_M-7(qKR|p?y7Wk*UY@Vzq|rC!DbOl*eIZ0KtrIKNUJCwNSi35@46QP;##2GF*hbe(#-^^?vuT*Md_m@$}sknq?6a~)y??~YE4{=w|DOTZ)bQF@K z@4H|R8U7u9Y3v}29Opcv3<_L{)ELJVX4=eUIOh+Ko*|wVJtdCC1-r${ZIl@|KD)4r R3M1}E6>E!GHAWe%e*od4ioyT@ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/NetworkException.class b/out/production/clients/org/apache/kafka/common/errors/NetworkException.class new file mode 100644 index 0000000000000000000000000000000000000000..cd0221de0539ae908693c81f6a0aa86d926237e1 GIT binary patch literal 870 zcmb7?-%b-j6vn^lzs05q0>u_lTcZgU!p3OgMGZHoNloPj?EPUov}3nBWOfR^mRC)@ z@Bw@%<2l=gV68FkCg03Cd**k(nb{vdzkLVr3Qx)?VOQj#$Rm-*0U80C3hSwMhT@)f zsj4lUz}3`X9BYK6$!1 zL^M(4o9A1^4g#$3c!#4L$|&%h&9UC)%u5~>kHyv$>T6cA##-i5pvJ{@tn;igE5bc& za3p|DRsnR7=NAZC`v+6x77@GhCuWGotZa_p1=vD0!v@&KeQqe&+8letu~;az_NT}% oQX9OxW=7qfRozfabE>fF%UFd))>CFZ`wL5a$@%QClN=uW0>$R5(EtDd literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.class b/out/production/clients/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.class new file mode 100644 index 0000000000000000000000000000000000000000..032cf1185745280a96c53d5baf84816a07bfe2e4 GIT binary patch literal 918 zcmbu7Ur!T36vfZ&ztyG)f~~X)QW6!5abxriLt?5Vn1%`w<0pXE*sP<5hazht>mrW=Yy{{MmJ;oanhdp1 zjI*Ei-x25oVWIDAk}8`H)g;qZ=Ey^Y+8bj{x=Sc;JsT1#eRrY>ivwfzK^7ltf258l zyt6!T5n~m@d0lK)(y>Vh{eknNP@Su2tiv;PdZxn2#j&%Y_TKpIe%iTN}bJJA~%e;IsOo!iln@@G$kpj&>wc5bGpSqddc3214(evP+<&v3Fl3 za?YwoDodEuyw*Wjy$rirT0eA|kMy3AZalsDL@y+30}U*QtcbKk+Jv|N-7dn*8yFuo zcTmoK(PDH_<(tBP9UTO?#p7*Wl~6~8=Ny#v4)46;k$9~3rl9Y+Ws$YcR-n!14wiV< zn3-@F%e)f63abEi*(u&yZ+nW;JYkJvtmlN*%vBKBk9%k`Ls+eXwU08tK+xNMK1F#R u*5#tj9JVpDUBH3@#-+HARsNJ<+XePY#L7JNF-LszkBGvOh#d}9!h=6B1HW|u literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.class b/out/production/clients/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.class new file mode 100644 index 0000000000000000000000000000000000000000..4a3e750e4a57acedf4fd128ddde169b36c55f98f GIT binary patch literal 559 zcmb_ZO;5r=5Pge4V+BDJOw@}v#l*(M8-|4Nk!TtYpxl?Tl$Cb3*|x^N<;lc@KfoVl zoL1w-gC5+&%)XbIH*a^|-d|n;Tw>qG2KFp8EwmWwu`rScGht$>)!p@gf!#1v`&z{b zSILYAsVGrnb{IqXN-CLj8JZ{VJ%8X%z?JVz6C;)1Ih`>S`+6Z5YOYjboQ88@ZuvYQ zrr~Oj25}lk_=_wiD;YD4Ty1=ZN8DQpXU&&u?sz&3wQ__p+QiOSCqt!Ee>D+NAUz)U zmWeREC=zNheDp+=koyjU6^c0Kem?GpJqCNKQ{#z|%nY^vBzm67RImw~VelU=87_X2 znF#vM?32DGBTso@1M!8AMo_I)|l>&kVv*}W70EuWhB z;2+>0Wt_P-1WO4Ccgc5m=4L)KJGX!S{`w8zJ+^A7VufXuWsPM$#72lsff=uymT5;h zudVrVbRZx;3ry`><1=Zpj!bh^Ar3!6p!!i8o$U#fw_kSzf_;0T1g6{CsN=kMuH2bC zPsuaewlQHP!O0#ugRHB)K&x$C63M=dyDGYrUoT}8+g{I_NI7SnkB)72Xl$NzPgFnE zvGj*OV%5*6!InUCyZudmmr*KB5}jsFo1}HabJlhCifgvGzp$sgmr2pktt^2`EOSpE z=O+TlKcN<58#n#`SFD}d+{NmX<|(Y-eUdlaP91ejvCOeFSQZ67Ja{|;Z|})HQ8Y0g z&-F;Li3;5a`ny;{h)1+OCaZ)R0@_o9q)*86j#fcyr8$83S%gfJ)+iK|8ekbSw5tz; z@D#ITF<_3A0Y@m)3xv&`-2qDDh&8^K1!7|uHbPJWo?*VghFHLJD#+Nz2)jo%w@_{F r3{W1YZqeyp6x4-b)HSs_qB3i75-S*Iy(HGFJ6PNo&u57`DPj2^2urm~ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/NotLeaderForPartitionException.class b/out/production/clients/org/apache/kafka/common/errors/NotLeaderForPartitionException.class new file mode 100644 index 0000000000000000000000000000000000000000..fd87efd21518e6ab2b73aa5b00a4e5a38f6cb084 GIT binary patch literal 912 zcmbu7Ur!T36vfZbKdVg<1Y3%r6!8IyaUV4C0mTQDq^1%RpzoLM*bZ!W$m|sTT0S-L z!4KevGTyu0U{g^a+)d8>$+^Eddw0J7`1%dNb3Ce`jCGL*A`eA20yG0`DlDej8H&5w zrN$Y?`Z02cWB)Q za^85(_V#n6FJ-c)P}}aD(-nm=CH?T&JCpSHBu+5YX-fU<{ND@;jp?$JfwP|^Tvl0% zXq0kYZ3_I$q0pKx%}>11Dlsa@cH|;`XrvQcfB%}NlCz2`=0%o8>LRNOul}n|3NQXq z-%hS5Hof=SQ;$4dHyE2J@g3p+jr$02i^to%nnMLeo^w>zJG}FZN5x~QHHP|+`uBwsuL6LVu@D*SY{Q#K0C!*ZtadSH%nOK80#6KK5^v)_TwIE%n(-nCRSmV zwZW|BA6P4#bHZB9S#1>f1%lS@lQD|3uq_VoID<7Ow(DR)9*j%50S%5VJ(S+9{REF| Byix!F literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/OffsetLoadInProgressException.class b/out/production/clients/org/apache/kafka/common/errors/OffsetLoadInProgressException.class new file mode 100644 index 0000000000000000000000000000000000000000..aa1c89d97411fc10f9c7a798b6e166b57032496d GIT binary patch literal 903 zcmbu7Ur!T36vfZ<-)d6?!4_HtX%h?->%vSwcJ~i>d z55R{a+`HWnsHhL_CTITS+~1tJyI0pge*xGOgbsU$G+tgAm+ z6Itcl(f3eYCX{hcpt>2H%dgTON*(#fiH&u%$7#F~8D*3n|C>Re{Xp3r(8<7>Z=5z| z6+=08M5;dM5LmejyI)#9He(yAcQH4j_1DY1;g}UvFvqgQQe&wL?Eg=b1m6Cm{IjCI z@ojG#izbTnYtUy#10EjH_?T8Xlu@8LWh8w sRMbjg>r>k;%*$X@ie;?OQH-r;*n1HRv(ycW`05W4nZ*$s6e@@3H@BF$#{d8T literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/OffsetMetadataTooLarge.class b/out/production/clients/org/apache/kafka/common/errors/OffsetMetadataTooLarge.class new file mode 100644 index 0000000000000000000000000000000000000000..34760c143c1076c28d56e32152dda9b713f11108 GIT binary patch literal 876 zcmbV}OK%e~6orqQ$IvLHr6pC>P$?Nab!ERe~VQl z7W@Ey6yn~=2%->##Ef+O(7B(FufPBJ`VGKS98^%o1D1y@J1n~a_5$n+tT|=19CVd+ z+N2-PUkQk}0xM@GbzY{vD+gm$A`X8-p!`Co+Mfy(jvjRdif86Z39LpsRqw~cOJzIq zazH!lk%Z}EilRZ+zX%LYrGWlP}Mf3-!R{g1krBmMd| zRwGX@jD3{o{pl^SjR1FOT&C3$Dk##NVv*jZonsmWjiuHE;!9>(C9RMZsFQgUYc#7) zO}K}3S}|aQlmVyYWN*3Eo?vO7uuU$ qt(ENjKXA3p|Jg|t5DpxpN+yoJB-Z7baZCi zsXiNWXQ^!iVm0Ah=bOc35XK5`+t&4ceW8Pa@y~VdT>F6?jcnu_=d6qUPOlf6q?0C{ z-m#AQ=HPo^E)wS3Rj6&Zzv!>pAM$~JlDIJH?@AJHWa3!&v!kyW6dKcIrvqodNv^E2 z6zDW&U2O_{5}- zlpY)QOWymkq1Zx+zXLxl9>T*Nj(545Lj^_7nU8vpJFhq@j-}QZ>PKc-pjK!FR%mWu zk+bT=g!@?HN&w4L0qoH!-g0YajJa9D8e^WmX12mub*(z`q3*=JsleRqHR55RY<=8?lP%V(A^EGrI54$1;kk+NF08p=l6 zgnthY1jLcR#J&lmScY*!wz?`y9KM7=?uQO_yd#ia-)IPA_RXmhnDlh0es|j^%GTvc zi!#%m@d+ylPVpd_iJLkS*!GMKT-lL+Q@I!N>_WP}X}3-2Dr=36+&{`U_H<9iezR_j zCvBj9Ui+#Olf{}qW!*cMm(p#?FmR7!tHYr7C{u4*bH!b2?9geeD3U>9>DeiPtS`F} zJ)9j0p!|qlj4eLSKkb-5HeKthUz+Doe(ha0*-!xmOt8$b6j>GoYVR14K=qB&yB+;7 zP?V9SKZ3q47U5uw)^Unb$Rk60GDrG>GF!9?TCwUxbgC zricMEqzpJjnr^_URKN9*8b&OUi{%7S>X!`=bO4_)n_wNxQL9vfEe^0d6tjg~rP@P! on7Tr5S4yaJ{i+9QZa`&=3!_+>Vb)h-t-ZivUp${ha+1Q*f5djQZvX%Q literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/RecordTooLargeException.class b/out/production/clients/org/apache/kafka/common/errors/RecordTooLargeException.class new file mode 100644 index 0000000000000000000000000000000000000000..461e026f1a4519a310dabeeee4f33a75d69a8efc GIT binary patch literal 879 zcmbV~+fLg+5QhKF$utUwatx3{pj<$L@I@S(9X5yBP1P(l716OvX-%{?ayuOvLZ#o?ly2@H(BlleS#$MHp@uUsZ z@q@3rF}bS=RJOevc_-br3O}JhN^iWz9(yUEfJv4)mLkit!2bINBT)U%!S7xDGDd75 zOTPepPOQMe1g#$^N+FL7?a3GE6lK29Drn7C`Vha8l4;UBl>$ptT*M6R3WFki#4JS& zm?LGtDbjQUPNllrM`{$YN-jzXqC6-YBIp3-v5;ULEK;jff-MfQhZM7gT&3DadX&0O lZ&yyJi-W2cYHmnni_7C!nNikfVr{*_VqZL;6>^fo>ND?!uDSpK literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/RetriableException.class b/out/production/clients/org/apache/kafka/common/errors/RetriableException.class new file mode 100644 index 0000000000000000000000000000000000000000..052f7199407cf7df1e1928ff6d82b6bfbb838318 GIT binary patch literal 864 zcmb7?QBM;=5Xb*ZuRU#wAc(DkQt$;@!^PlR4T(WbFpUp_eP52XU1@Kdy({vycxvK< zAHWY~{LgtISZa)S$=uHD%y0iQdp~}D`wrkGwyUUML(CH~PsMD8XolEQSWb<%de}2Q zwJ!N|^hQCwS6DoBNt)>->*?Xdl*y4tDO6tB#AXKyt)1={eW9bFPWsVt=55mN+=!eE zy!)z8ho-aJQwR>-nNe8k+QfXAj82U|kv)Z#u8VoPrUv>&FvtcrRoL%3-;eZI#{(0c z>(A#pirr}BlE`@Pd>VZ;JXY%7UdCpeQGZ7v9GNuL{hWPEph7v;la!llbG0%)r!DEA zaWnpBbEO?gs2;nCkIg$PZPjl}+Y>Q0)UYU~E~X*ok;04r?o*-tkDAwGdp(t43uWFp z?+=d=;ts!e=`Em&fNM&ky~oHAN_>H^)qXyOy2_K5Xsb*H>V!0~%vJMu8t!9-9t*IF z2ZYFK4G)_^I@X z#0NisAIdm$UI-$*2+6QB_n+VXXMmr-zW)I56h~#a*yHek!#;-t4~HHe3RDwiwT$}8 zCfdaB+b;#g8-cZR6DO&R)4q%*sze-q2?6(oj&*t_;5Rx~@}mqQ84rSPYIQtl%~g6s zYd*<~NVQJ-0>yLlK?$sPbgbS@#uv)=_?f^)$AomdBoCD5#bP?tiNN!Yv4cQf%W$ZI zk-QwqAT;B#i34S=u}RR)tm`i_CF`#~hw7RSEdg(=l0*(N{y!!KN}-%2^l)>jU#c$| zji;#3*Z&@{&|pFJQb$oc?xr#v^{fo(j#}en#VzHonb=Uh*1U(E<+D6y_Z3vI#-YZc z&S6X7*?%oZ;CNNvTi)<&i#R}uekb&uV;dfB(C1Cc9F(ap*b&8BRB59?Z@}}Lr&EY; znP{D2nZ&>rLAFt))BYbBZexQoYQP=bB?zB3agWrDt>xG=%DLb5n^QQ8)Lk;Mmr=Qo zR5PlZQ<+s?#wskbc+kTYEcQiz2;O+kdXnWp$)BRQh&2G~`6ot>^&m~G}X-P?cplQpZqzDVaE~uLVQc=W$hU}+iNQTCqYHSx?i&Z2R zJOB@cIM<1)5`qx6bY|}Lo$nss`SJ7HcL2|^Uq=l)BKJiei0lSv2iQ|sO^vsDJTN}B zE_r|YQbD~@SUzz{n&~7P=y7hUV z`@ggzYybcN literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/errors/UnknownConsumerIdException.class b/out/production/clients/org/apache/kafka/common/errors/UnknownConsumerIdException.class new file mode 100644 index 0000000000000000000000000000000000000000..eb1f6882e00dde1164f96a6ec42274a9fe5fc167 GIT binary patch literal 894 zcmbV~&2G~`6ot>^&k&`wG_*~BAStVoA}jajp|pC8R>amd?z*zVqGVYybT9{Re=Tcw9vVdm;}-9*Hyqv;yoatfj`sdfYcY zjh#K~zEM!`6;_U&O*3t?z8+6ZnH;&4LgjUA2=}#jaMlMO54UPBCr{S3$S$Ac{!z3}j zd-3*bWG*ur>?pJkdY|a&G9}Ooa ztYamMLi-e{RNDxRRLCgXWi=s-XfK(K*Nt?Ypn8ZzDN1jQ`-(nSs z1wVivg}8SzDp3eRVl16&Tjzd0WB>U1?K^-McvwaW`yvM-_eCBAXar~yHe&6K8uhe~ zjkD*cZwd5)uy*2XoG6?0)M%=U%#n`>rPs!q#l8A`z!6= zeTnoW;c!QU>S5$@f5L%0^*@-Ky5~JL8Qy=N~Ms8ql^jY{zbAn%OD%x4dfN z1wVivMLcJ_VpB0(xSPB)XY!tB-#xom*FP@-yu?lkMLc48%(B6<8DJ~Gw!lK7oR)D* zxkOuYT7N4bJ_yVmTa$Qcd`redRUi&uLZJ9c8|{w-a(ho(0{LUxRs!=4ZPce>|4g~2 zJd5dMv0)>^N`h0J9p-&kCjxI8)^$QTkWp8KJ^8gK!^rmg)`ZGA>yq%)^o+eQO*_!h zXZJ-quRW!E_dQYr-W&=n?=`;3a~a0cbixzwwCNmjprEgkM0U~ye;EX-lWonevlkpM z?VR<$OQKJ}|F{HdlkM44Z?$ri(up0qNWIrQk0*D%({oN+Mj3M~ODq+ZRe{(4IWB>N zI|y}iE@R#PsJv`TFK-5oZ4~Hlp)Zd$1h_}*ELAfoAy0b>OM0J9UeGFNE!0L3KT^v) zX^E^rh0H5hpj~-v!UHT)#egMJ1{{%-y~W!82s6`!HHxvG5-MX?Mj$^PVwo7ss@%rP zO|v$LwfPU$3aS}vHDlG0qZfgc6r8n_xCt&vQ=rfSrBXlAG>R|Wh@U&L^Umm=d?Z4 z8`&}16Zu8jsNju_5-b~QC|h_d(8>F~4+2@1WitXb&Ff5MFmf&j6Xo&_m<@gQt^eW0 z9&cWe*U>;-kH=MAPsnV7w(x|}=I)vZleyab4$(ZhhtX`^!@Oh3z&c}{M8E;#A={7e z6b-(>yLlU`MChlQ%7}EOg$xRmu(^T+GQ!Y4Ic_wC8{Yq2RR7SkW_P ao6Y5BXGODt=c&XCY%o_yQ@}QADE|R+t=_l* literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/CompoundStat.class b/out/production/clients/org/apache/kafka/common/metrics/CompoundStat.class new file mode 100644 index 0000000000000000000000000000000000000000..fefb0596da3cc191b44e42e4d2215a27ce8129c1 GIT binary patch literal 438 zcmb7>!A`BE!|XJuhHG)~^V{xQr_>k7p?&fm4#^4peGZtL0_RqNo&+^e~* zceXL>Ofn`%B-2L;NHW{FTtAg9+4#_Y-@*{?{u4nM?N_MS6l%HJ=sXZ6rzW&;gxM+E mA>8pUeFC8)I+H)3E+N9Xbg^_DB%+J`X@I`eL~gpG9^zloM}X7- literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/JmxReporter$KafkaMbean.class b/out/production/clients/org/apache/kafka/common/metrics/JmxReporter$KafkaMbean.class new file mode 100644 index 0000000000000000000000000000000000000000..bbee68c3b2c4750ff2bb3203e56ca38450184a8b GIT binary patch literal 5120 zcmbtYYj_lA6@DjsnPf5{5E?Lr!a{*ULRdhc7}7|&*+4cRp=m>{I@z5h1G_Wp?ktyz zwne;Od$ZP7?M1D<(+UY8mDXCVwrai9yRG-T{_>BHKZ@`7&15s%-4Oct|zYlz?m@o-};j^cgda!g#_ui^t5 zYVko0AHs)Kd_=`ZHPqusH9jWp9~YNTXqbi@RD4oHqcHs`jfQU0FdH`u$t@~w)o>d= zU5(po@fqA941ZR|o#K91Ek1|O*WwGfTU_o@ac?aa<31H%6!$OH(C?Qu+>ft_FdoqG zARf{X!^0{b5i5DL8jopMf#WJ3uf`J^R*Oi!s^Wx-lL|Daf1jCjyN!&g;G(X5#sT9{ zJY(3#pqVjkH@>s@dV>NrW4bvjnO9IRHRA=>O2-pMmhWnMtU=pw3pv`^Rr^I^`CzEmPxbGppLkJVn-G&aDkG5O3<#s4&Tv~2z zS+9cdW+z40^<9>2b{8`JW^RwspQfat%SjsPUL$9TXI~h0hb%5b^W5~ChOI#Fv~4rD zIc?szE0BwNm@8;~T~i6zU`a4bDg@)LCoWU(QHk)> z^|(fIKWQVAp<+bEDN2LZi2vH9*OWt70iPlqT~^+u^>CVZ1+j7@U#qDQ!N!1(Vp3=& zA~M1#W>yG7md~{;XV$HULShs1-DwXv3g%VHVe!2nhcS`A2umz?nBKN}cR!l(vRWj? zA{sG=(Jb+zDdC5-U0j+7f)f;r|D6hEm6HE(cDw+3y>*v*`u_`6g{5MYoU-f#&VJ(C zRe=;iNPSWSMShE@NS$cb$QhZ}=B3E2P~I9;FyHf8f>IHn4TAD>&q0n2Do&G)!9t@y zztSq(l*~I1YHB@BA(u3_S)wmZ544XJqVMYHz?gy=V(fL+D~$XQ%S1;TnA17uXwG8p zn{}+mn^Zih<7+sh<0(9?6h@axpd?mm5-Sd*bSenA&10eyHO|__2( zdli4s@khL%<4^driofXit9a@hn2j(;eC(EO)@OC}s#Y3B=BxnZStX3d-- zHlQLj45i0(WrmD&+Bs;ZRw!6E6>|b44(x#?%krv|h?jx5$Fwz4~ zE@P%V6JkRRggd`nU$o(X#-cKf-R3}=$DPu3HwZPa;ySsP8?Hmb#>?h*+6RoZl`8VT z*GLzF@r0I&)>1cNjHRm}lr50A>w|63Z&!hx$Cp%Wd9_K$b`8tU?=ZPjlpKG?r|qR1 z5}Er$1veFqM!1v$n-F8(Qj)AKPC=fURH?9YKzQ5|G~p?*9Yu|7*0fX1ThM8R=Ix36 zPQZ!PuH%UmT;Ok(-T}d%OUhQTSOG1f9WqjkV9Nd$U?~q@MlxyU^NUulTpYmb8CT2Qo<+tcKfZ9; z6DXq30CS{HwN)hJtP=Hcl5&nZWaPWeL$t_Dk_KU0K9u6>(}XD_NhFJ^Igs?_j6Hq3 zwlbgqzVZl6s~E@xiOYXh}`n|QsM-?O)14sOLJN@jUBuo>I< zL~D2puArYzj>Eif=l>3L@w$$&#ViF z)jZ)(AG{HctBu_i| zPU=nS>qY6yYZ_X>R|*HXEYUPH#xV^CecOj{*rNbpo67b}lh_vIB)4##T=%<4W6{p4 R*f4Ly7W!LE$Oob|{{wTbai#zO literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/JmxReporter.class b/out/production/clients/org/apache/kafka/common/metrics/JmxReporter.class new file mode 100644 index 0000000000000000000000000000000000000000..5083f39b4245845e4afb1a04127907da501ea8dc GIT binary patch literal 5932 zcmb7I33yc175?93GH)_@5i*dl$Py4GnF&nl0vG~T76HiwOIV89mdE5}GBBBmGcOox zsoH84>|$#-rO@4Uu`OctFNMd3;139}Qs|J{H8sWygc^_=G6a0pR6A|Dh+A+*-)-eJbKS!PCqu_lwT za?8_ZK5ykURHU;>4dM1|E*ZfQ zW_h-YXO3%6Wvq^V0?chNx1~8l+Y@GblbK7&XLr(X_oRq_-njtanRSh)+aQNmC9;|B zR8m9ozn4u+%hwMvqw zbbLX>vQwEjSKY#@;ecP>!+c?MwKSd+zNSq~UKStfF}u31v>D~w`fch-o}->Ko;pV6 zuv$0zJ7MM|rBmoed;EgHAYP)-_%^_m$oY4uteJ9c$^A`l2j}@l{_VP69aIiGuC+mh0dO%ripdRX`7ez zr_x=jLrGX#gCkB3sXEjU{%f<>}4Hu0CK>;tux`s)m_UXi`8*Un6D|5MQ zt|5`LOq-3fK~>HMvyk)+-CQgsV?(QfpW)~7_=SOA3eMH|6@#PIW|LTA;5nIk9=|s5 z0)C_8w=C@?=M*!*!0&|r@u#9P@O%8hz#nB%Yvw;Sz6QH={K>$d@fTs{uZ%APf5YDm z`~&}FCOP;^ab7gSz`yWs4U-*#LO2$e8_vT29uMkx(ZF&1rwp}o6(NaZ`Js?kPTj~C zt_f`TC05ojIZZpu%yOtw6=g$DE_?{Vb0$7>MW&0Tve%nkkBx=h-g5g3aLDf1th3WET zPI1{2UP>lFpD3>f(pDyE_oyIcF9>pelSCdJR`R#!4iGP12d**`OgjcnJ@;IbHf`3r z-m)-t**)BZ8MXQJJuwrMc&q3`t#=2o%oYtr-iyqYtYQI2;3=#j8*2o(=0p|yO5 zxBy&?04_m0-((xb5Jv|`QL)Zdu@MzYwk|RRf8;PKw+tch7&S&8L$IVUOpG92KVle% z%Q2D2MRC2e*;NeGDctN_xCSrdnxI_8e}SWkx(W!sfWouHqe3gH(6C;a+(|hHx6Ewd z4u^z|eA|T0j&e;b4N$V{VN`92AB_$pq@iQpFse1&hsk^~G(3r#R)1uQe+cJo9vpct z@+kCSjMFfT@gCtx1>ss(I6OG=eDkqEgjy?`k4>pO@{l@f3$EvR*JG<1{mPyh~;8wM9D!%WZn?Mj%9vY z=65ZdKzE8ur>fEMRK420mfG9tjW~br=-@9W>y$+e^rpdev7avO1gSGc97m`v>{EWS5UUbb?R!& zL@zR2rAbBiCOx#A58p{db!2t)jSz*;;YY~1f!laBcA2@t_26FDns_tq4$`8{&EYv} zplKJ%3&)FwF<0Wc(G%BO3vt~f91SBP2|?NB{UguH=Xr(C`w$o$d8Ro$UupG<%2~1E zY+*wKlW8A0pM8K~p5 z#lyJ7#~N4Tez??!L5%B=xDQ{(B)NcD%^L?pYeq51ojP?hXLr#(ujTx0T)lxyoe$)J|-m8^~y-4IpSMdw`Zy@mD&_8L^2!1+3EUB<|*<|M!0xtRA;R8%aKveQD~ zQ3v+|Chs6FD9h1>3@=&rv$;8TF6yOuCr(tassz9UTM{`H*IGvXKmQ;Hf+oB3I(rd zWe=<#321rG*${|l*6l6Z%G($D7&Z)?^ihH`JYQ@c$ttDE~n$hiYwPdg489f~s z-RD>)i-t9^QO7I3b5{jM&w!IT&MSL~oNms(}bEenhUdT;Q4DQoU0N}cW| zQ~yQy#s|_4odWSqWv9`1WXsN1nL#+2dM?oZx?(#US$k6vZnP)OwB&+(kIJsfG6k_9 zFqS+YtJJcN8TQ(1uUc|)FI?GcolTjJ$~HETz^s9GbQp-C)4(NMHqeD`0~Zn3F=ya0 zo){RwkdCJYo?%{KIA~l>(33Hbj*CEFkON;UflG&&rL~gxzCI~Iutr=T?pH-=%$6Ap z0v)!NfKhPsdAFS7Y`vx2tPTY?gOG>e8gnV2`y)+_$In?ORvKgcMzw)F5J5;*gI~9} zEQs1!3AM8}+(WWF+@t&@aD}KIo<8zk-VB)A5N5iDkU3mK*c_=rGkdfeB4$sdhUiz? zgm9H>A9QIC4L8t&LB_nv>ffTb+vvv-uJLuw_laIy=RavJISF-=EJNxfd4<(UGSk#a za*puS#W+4gi@QWd%x{Q%;lZa7RhKCh&hQQD;W!^8avU5w3LZTXJixqs@D|xge!Yww zev{cXxkrCM7x<0~!+W2z!+Yr+($UF2LAwO?k$YI_)S<8we4XU&R8y?pI?+Me#v%6k z9MW-9r-(wQilAOYbq@2?MRJHSwk8$m|A7__pLx}+!?beWt%Qvo$R5KTBeaV literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/Measurable.class b/out/production/clients/org/apache/kafka/common/metrics/Measurable.class new file mode 100644 index 0000000000000000000000000000000000000000..9e0e25baf186b328edd2fc603f53fb0f686baee2 GIT binary patch literal 204 zcmaiuJr2S!6odUrX@CWe0SrV!Y>XfVM8&`fUZb@1Ck;wl4#&U&I25Ak1z2n^{_M~9 z^#-uRR6qz!2*PM?I;RQS<;A*8@`Ke~%aYTQrLxA@S{mKDTs1Nt{9x-e&-Ullkuba2 z&MAG$tDZ2A2Noxl-W2>~!umhaulojJ;XAV8I+NF=(5j6HgrNTmP)Had^anxa;~DJ4A}4rRo-yB*&Bnu7cmNM2ZYr?~0|Nu|VcwtT z3&0Uu1*Nc7*eE1~B)6k8a;jP(?9al(=(uPl_w2KpA56%sD>dKQ+(b8gZCD%Ib+*@? ucvqQ0|ECDMDW_ZROMNM7F`jUk>zMWk>74C16%znlTQ<8GHq>wa z7eDKacErx;jQ#G9QlEQwAu*;C$Z8>415e8PnjhdcwXniRS7X-9JyK&&VhXSG^Fzg<=LG;WEo`)}~ zSrgDBui?gubsA+_ffor(S6|piw$+L}pMe9X)ojww+M?}K8Zhp_qQ}hxfpoF7O}~}! zxhpVQ^#b=vtFhxYx9lCCnz3r=*#5TN^yE2FW}-ch$;y4JSXb4ogu$*?XVOx!)Ys*t z@Mp#UY1%(z7YbHP;^vEMXZ9R;nJ}Mgvo5e%TYd6$ZyK^a{hj zLO;d`&%?*a|E$6plvB_sXOO`)3}F^|TqlSdeBQ-v+~gZTj}y>hmM1}&*OTNHSD8F( z{({s`+@xSo4k)GB;hgPozL7XD(Ks7k`Q!IM9It(L>bdy z(dGn+_G1^4J3Wz1u)NrL328nO$+YCG8_7K)x&Iazjo=8eC6C|5V zl_8S6Y~1u*(zI?s580As70FPz^QYA05d&fbdF}XCaGGu<(L39f{-KMc)t=A_oPwC7 zuNh3Xt1y}Q9wWbF^sBLRuW|m3IVs0YLQJdXkEsGGEM$#;s$Z~I)ei0or%(%O%mwC@ zrx<0LG8=x<&h}vC+vUfDwb>JC8Sf>|{~E0Pf7-c4XiqyJX{V4h#hJ`JugH1-9}-u! A#sB~S literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/Metrics.class b/out/production/clients/org/apache/kafka/common/metrics/Metrics.class new file mode 100644 index 0000000000000000000000000000000000000000..5fd276c9ffad082c37a744967b97e97268594b9e GIT binary patch literal 6409 zcmcIoX>=3U75>JOJ+?FuY`}>j5U~^+*+zllhGI%eTu2GFp)r9#XviQBvOtzpEd)1h zDBZF&UDIaiPSS>M*|CM=me4JvX}T}JdQSiJw14|Mhg169_eQHNWOq6C8O^+T@4Mf3 z?|t_^>5YHAasj|b{Mp3)J{UOV!%g^ziPJuK@lkpBn2C@3V9J}7IMu|FPnh_m4}Sgd zfQ&ro!$Wx3#HZxp5xIQYj|KRQAD_kNMt8}S4AeAb69JSWDVm&*$#UNrHNfrfA*el!{}u&zImiiGT>9UgH)$L*uX?NB%| zI+}=wMx9J58cv4>)MuZX-)q2h#|^CRKW3k>L)lC;#x?P9HkESXnGk2Xw*z*P3z~+a zk+_}7rW^zJ&SJ@ys*E&~L3`BctvV;qdY4Gi3wpN-?X(k5CsJtx_soiek`}Y$k|=y>C)9zT_J%< zhxT$7Z8mI)#-o|720R^W_Zjf^C5E~9n*M0q8O)9zaZ-EjBQcJ&^e4i0Y@eNq%CkG@ z&5T4TWo=bc6+CILy`$>n5?(@Oef62;ux<*G4ZmAa|WLH@8ta_`Bhb2g-4nq zPDW?Gfh`@{uch0jcFwr_hB+DM&Xx?Ew)dF6di$zKPkVh=RkHa|xHjfM#!T8$%_Qq* z4R7Gl89Y#v+(2o8owcz!Xfddwb_TgVGBM5$#jGc0fh4e+Y7|n%&Xf{R-bk|%|+Pw5hfJXuY zeugC90t{@dE=M~y-@1zxNEpmdIT5xYC#C&r;KtfGrF;aVK^rj8+_AQ#l;nsMku3pZnngwB>SU8NZg}ZP_obodZKgVkpMi4j9Sqq^gp?E?n7WCamjyd6sftA$&iWy@dSZz{K zW`6ez!%G%U;9d)_;}`PyhJ|0^O$*2HD+|BIn>1SFu`?EPB6e(BDv}*#7ubF>>?AYM z1f^YDoTLwiw`8NSVJAfy+XC8fK>FBlpxwf6@LRe3j(fg6V8>F9Jv}Ip&wbVJ9?wx*)nRRooSq-&~T1`lYL45-dnVw{p3O3CUgc7P( z3rlXMrPXpkQ=Ie3;Qu4N;e;K_I!wmv^68P0Rfmn)JyuJQfgH0aJe=oCzHZJYsm zkkGrUK<}nn4ZVkEsowBXeFJG64(8Al>>P)W^!vB7@cy1BlsUU=qk?IfL)V~**Zq9DlXnM5 z_8@LxD-YrjLU=#jc{l$KD*+qn(gB9+eYl->Bw6s%VM-J!GA2P5#~)Un^I(YgI!yAe zkMH(!WGU*dFo)_*++kv$iTx(-G-)4ik&@1!g9PN6!gcy~E)>}2;5p3er20v;)luG( z7dTj_v@|FNkrHP&7Z^5E%iVl$kXIh^W~vrQf3VXqE@J5fuCD}3mVo&SV15_uG{Nq1 z`RyQ>hhU8a>zV=SK7|y}a|%d}E)v(l#u3svL(H&?`A`1nD30C~?3}+d@L+$NSJZfvYg!e~1H^lf!FUKAco;o+q!3ox zC~(?T;Iv8YY;n;h#m>X^>oE+6S|x~k;JJe3ZV->EI4(62L46}E{;Sw>cr%NJAk2%@g~+sZu}sNA%VkC%ZUBs$x^|5-oZ{J4Wws|M8Mo5&p^bFnk6c*xXtiv<>75Qw5OM40~?cw246Y43r zv`10%aGi++lCDd4%Dz|)ukzgoluJ|MPb%TOv=+Z=_BrO}d8Hsoc8BOO zDP>+^S!1am zk0%~;9}1E?sn(jdayo|rfca6*1XUDrO{aGDeEg< zO|k|J7rqWkzZlEZ_2E?YrguI>9qyVR3D93QkJbdnZ4{P0(tNqmiu1YoO=Rlb$`o~w lORM_=0v)b|V@|ppE_#gmj5 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/Quota.class b/out/production/clients/org/apache/kafka/common/metrics/Quota.class new file mode 100644 index 0000000000000000000000000000000000000000..d6510efa609c020f8f384959eb4a7cad5f66d2d7 GIT binary patch literal 959 zcma)3+iDY06kTU#avP^6rdltxwN-Nk9SVI@L^Svm6=@r2-%h5-WSYrLn9S5q@dm#7 zBB)>~DEI+>l(_fF5VTb2%f8L5wb$C`*YEE?0c@gSBagKbEUa_5S->ru^6dg@0=aZD zQL%v7qq-lZ;Xc(Jf&6_x^pgiPY)!xN;)Bp>jv^Gn{D0_|p} zb!MEc^*@>kFOH*F%}DGZdZprQ?)cLkwwbkNvqf$xl22QDo|+^xD;90$I~>X&C1(p; ziS$NKs@Z$)YBsV6?$wzcxR2l7xlIET7}5^M(r6a{K$a?LPDpu!YxAj5(Cynmp) zL`&;ogWf1l4``sutW;Hb1?4;hLn$s$w+oa01j_AWh<4)$MtkK5=2sdSI8U*vKQB|f zP6R77TSb}XYXl@1D!4$IE2bB^;)N#lRV-#}HjyJf!2SViw{9IH_l3II1{MUYYgU#; zf8nB*TEeCOFqo$>)c#>u(hLHZaV68>QfBsC`y0$pC)T(A0_QSQf-GE}gN-@Z_WU5~F-=gn{7G$!Um#CmFf<*T|S;u6|?o8N+_*fZ( zw&<|k#LoWXh2Zr~kw)@d+3o8y1{_#n74z^2NB=ZI*q)`lvVZw?i!C^Od_E-BP(qn= zg^>f7E3zgyE-=$&&f#3y+kXYUOna&vy(zoF%1zW5x4*lvh&m(gM*~aKSxrU`mcIa< Chkj82 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/Sensor.class b/out/production/clients/org/apache/kafka/common/metrics/Sensor.class new file mode 100644 index 0000000000000000000000000000000000000000..5c2083c5bee7fb0b0dbff55dc20ea0131445253b GIT binary patch literal 6638 zcmcIpd3+T075~m&vup-}!JwLD;4Efb85Mk2~e@K^-58U;=hU zxcG1lK7zY++^yrD2qs}y7`tWssJ#4G828HJz6k2DM|SS7!N>6lnLSX0PvTSZ__U7C z)F6fjW&0r=pN(J|9**DnWm&FqiG`hU^3$wdw@FYGj zv!`nCG`=8@FY0(Ef`xci9$ykRUyk58d`0g2s*bPe__~g7MDR^~OGDVPdhBe@*`Z{Mb6CF84cU3O2}%;g>4xMalZ zWz}yW-}dDqm#2n?Gme?bc>;&4zTy>1X3|^j9u2Kk?{`miD0)f}$k~G2^ij`==GnwX zQjt2e%ud_6M?K)FRkF_EHW{-S#(0OP%&62AD$L$Ql{+%sRJnGIownBH`?pxm zMsrI_xlbl(rZ$<5Eo*Nxkn6SS2i5g0xz6~moSE!fYYr$53~3$TW`OHhspC63zN?{z zE}iTVkQi9qMR2n9)87A+JfuhVyhm(%MDP6Z0 zw_G+tnqxRDav$F~`*4HPaU3g|ak@4583XDMxDR^F*o4rZL4I3_Y6mJzDWJW`dSf7qKN`3Kf70=11AoC^H7u^iM`g@Sb|z&CUM1iRyaRs|0RFC_ zLFt~3Z7Wj6GFufEDlN;xz(4R$q5hYc=8en*V$GWj{2TvaW-#zy{LjEkIKY}yO`xNt z88HT>+JTn^_Jb_8242Cda_J4j9CZDbE3725@zl|7D`y7{@zIFfL?g{-F>UHNWH6y> ztVPuism@hH^VlfekUxsZ<<_PcMXLGT*z07j@|v1*#B}B?$INBuNmZ|}1f2D&y0k?w zBqM8>HrfGK-V3CsWN(wsEOkz_2z(t zFKc}zf%i>N$$41(Gy8>e8Fy<0@8VIGPNUyWrR*#lnshg{U`MlB!-TS6lD*ibWbLb& zK>3zkoWI8vwzKl;l2Y?WuPSA4tEd2Oqcn%Pel(bmc#I!FV>nXmmZZ&MYMxQ;3PsPV zVJ0QLx;__Rdq*S8Jj!+DSR?4ejC2I^k2F$ni*#8ET#7h_w;k4VdKq0TYM0!8^04w) z3$n{a~dNdTd6uT?I0u(j4KIG0;}H z(5TeqgrYt}Cb-|@mg7prrm4EGR#YLyqrMy(9U0c-q!bkf&cuAWKhte*RbN|f&`kA! z5-HrzZO^jh4b0{52JQS6Mdz;?>_s@Mlb%D3>?YL6t^pd(<=lr=e9u;*v|i0M8wy3o zo`R#nf9vr&Qk)NH_3-lsU_*QmTD)m5eDUT-;eQf=y$Hrzo<+p}I6`|6XxR%rz7OHe zgBZ6Lk@B=A9wn_2j|TT*Jhvy1qIM7yA0k&jHt-ooL_Gs&!ZE6S!c*VIgO_oDa8}VjY+0cs(@k9U{q*-uV>$#k_)j7N3L&kCVAv)Y%jt!f`&_iJALQw|O5XQPbn= z>I;}0FW>|Ybtm#sKy(l%?f0KPCH@HXCw60!j8o<{5tE%hZO@?>1(|?C<|L)*V&b!e zm@MV9jruL6726dOFS~v1(7RA$W7g{(UO=Jba423|z{#SFdpe>- zTtx$PR&WSaaIifN=aNPct&Pv30ON-+MMD9njAXTeKm2tmR+AO$+LHFQZ1MWYOvFmL zOtlHbGEAnu@y%_)r~HdT^&v8^8N#U=c4Ml{>qA30&Bq_@Vq6)s8ivpi!p;zF@_f8u z5L2VUr}vjfaOVZDs?!FjV9DCf=~PrUFDWeU>I&g7kEve%Qb(%&!T z+siSTR!MMNNNcsxTB~TI3t%Z*_0S&T?+Zv7rv1grAx7&IH4E}hZ%N}`Ph;8ZN18W@ zo#{WXAgm)pOE>cugnjE+AglM8}VY3TQ5%MFl|X z{+)iJbgS635|w_`QuhA@{u9%7z`buB}fui*}? z=463J7hKpD*oSGGnLG5 zre>O>ZKOGFqmH*LO(Y|N(Ne4pVSc5(lU!tP2d<~oVwf8|!-RQG;%L1Hp(zy+O~HO=mHB=~nT1>eK}AJ|Q3@5N%=#}EBIEJgR1 zz?YI+6nT9~U@C^L&Hn<`UkWQpBbsP{j8}LMNC@o@3DKC}KXGf`AXBkW(=- zyNT$WPqy=jjzWwoMIOcSkp92#Yy88Wj zz5px`D=38lA#z4rcP7jyJL@u*tLTg2Qqodb+q$+5*T%bC-+2Gy$~I}9E!W%WkuXf` y-D$JQD?^AAFMd|YLkQ!4W1UMFbv9ShWPCUkM*E0B2-=STg@hi$uA$#%fanVdVl-0# literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Avg.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Avg.class new file mode 100644 index 0000000000000000000000000000000000000000..f3f3baef1d1e21f8b65763f156486fa5db4e24de GIT binary patch literal 1615 zcmbtUO-~b16g{tB>72a9P8=K%(HdPH;^in#pbn#8$i=Ti{5^aqU};>b70KE4M5343s=eR<>l_ zQTw(t7Cdl#fn3R}mrYrd)&twzllS+eX?fME=bBYJs5_Q#`hg65^G36rXU0ZtM+SsU zY|3g)VH<0%Yu8sQ()Vq`RWm)}wc>X+2|m%X^Iao1LPjf|d*3PNSJzd|3BS!_xodQ* zjz8ojiWOoC%ydQR;2;oh%1VPWh*fsM^&Zk%7tko-HoG8D%#?OzQ<{yyshB0l5B{;r z^(=t}g%u9}YlLnUOi%woWU>joKvuX~Uc(Ip20I)7sP?1zr30~ru66%ay=xqc!)&9q zw;^lo+0r+ivMYl|oeDfTsK9^a>dLdDKcRZbO|Mb6?3<1n*F&7+8PyR3Iu;BJVOYZz z1BAtpz)?rzbFJXN%ENruOPJt64QNeAzsmn zz~D2<6&U63>>Q&vJQYEO0X;@hVodY*YNi4qHoJr_zd&YTlS;5<1^Qm`rH zkii1?c}$1w@9^9h-<~GtBt@KKRcE=9&CiO8JaI0psQ9y@VpvC%_}4KWikd)yxkf46 rc}BOyRgsg7oT4X%$XDpmnD~NOl2BdA@;@C~ISg=B9gLIEQ5fF=%pHqJ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Count.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Count.class new file mode 100644 index 0000000000000000000000000000000000000000..5421b5ab2c38e269c7ef568846500e5a7106b2c9 GIT binary patch literal 1532 zcmbtUOHUI~6#j18=|DRaM2ki7NzpzmBZ?@Nu&~4gQgl&6;_`MF%0N5Qbfy?xxbl~{ zacN9622G4>|Aha;#K$+&N^I4%#wNXU&b{Y+=R1#kKYe@m0l*?|#-JiQ3I(}1^0?wJ z1r_rubQM$ozD8)=c!N+s%vr*n3t$#$Ow%rmFCE8;06LB&rh=exzdNMqpH4Hp!t>LZ z;h9hPJ%f+%TSzP`;UzVx<_{4Tc#fHrl259Kh=lMF3aJ>&$}?5G`2IPiyre@U$ZIMf zFiZF}j$w{#8aZ4F$gQ$QQR-)KHpr-OM6*Ep=L4FBBbtSPT$o5XRyRf5i;U8#lW#vu i`!si0$*+ip#TV|(Q4cX%2$usXLk_+LFE1lbK>Gm<<$s6( literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Histogram$BinScheme.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Histogram$BinScheme.class new file mode 100644 index 0000000000000000000000000000000000000000..b4d3db1cbe41d84ba5ee225f9571d4683e5f18a7 GIT binary patch literal 312 zcmb79yG{c!5S&deXYL3g`~yYO;D&}4BE(e``GSq*98S(QvJD?c!3XeBh)skLZJM2Z z&Cbr(_s1uITU;qj6)p+Y#)V86FP0r)QsTV}(p;<#rP`+Ge;eI$MQGRYEgAdh21}Sd zxm;qO*x&5fW3k5_&RdQZHYk6V9MpIz0)w8=! zdfjWM9f87WFZ9wgEb7fhW8xY|r=qqaFf#3h&gEXva^kD9<+D?nj%?{)ld-4H{ic?7 zJOZztiozt7VS3IBXY7s>aK79OLnof{Ws*1?nVODb*OGIR|F*jFL04LK6a-Od1x^}! zc484AO{|Mvl16SUgPnh2o)Q?ENu}MrB2@{v=MY0Y;ePX}wt=1B!}vW(TJl(9yIwOCiiC@KQ0enU5+9&I#0QD^_# znLm)Pv}PSU6*%#)uq!^^z>1m%^Ifc~PyZ%A!!H)Pr&Qni$1T{-2Rs5YhK7oAiEp@~ zjLI^-$};U@);PBw8yGk8^cFe_%m>Vm5LNvRhCU;Aqmus!?W`g;NX(bQF;Zqq}QQjk3?tr1pRE; z=n~+vxN(eU;%dC1Ay*vY>Rs5~Cos>RQe^H{yPm&F_TMn)?LLRdaB)5 z5?8XAqty>Fh?t%vWYAl|!dzxyqE40~vy&_aN62!5NI8Ctt<1{wVg5jFA=)bDZot3WS;B5`@VUhKvD5q8&z ztI}4gq9Wh|ir}KEcmSjbi3kJ|+;hbv@CZCX)&E($PHs^x`JCCAIWxbRIlJ%wdHoi^ z37pSi2WGP<;+R~%k;`!%C$!t2Xy;Emn{0rNZ-?+*4mo_Uz5CcxYV#j@LJlWU(=n&x zw2mJXs13)D6ex8CX;0juw4@*tS;DmyXr~?DiO*1B)R!u&^j-{_)Q>DXzJ0mtHSF+; zXh_fEa=;_2B6Q?)U#Z4zhp`iH5uS7W6{~G~^v>6P-wqdD5k)o~7M6psWr{UHv)K{X zI>NL9&kKChv*XaQA~T9b9GMrLC=Obo@D9rS+ak^>(EYB*42+gGDlA6776wv5W}{wV zO+#HjsjA5m8w!S3VqtYIiM2j!CJy@EOY*oj@MTMKD?vB3>-sSse>`oWr6H7B>Gso2OuOi|wbzb=x9t?Q?@} zn{IZS%wh1_j2p{aDONaAzcnZIg)QwR5w|(3=tjuws<9+TW_~6FCizb6fPyikDRCye z^F=w9WanhKtF%v`#B~p!I?$*A_Euj)sg^d8sv4U}SB+9hZMHM+daw!gIUQ2i%XKF* zTz}?~+sNV<6!0s<{DythmYI}+upd+0rF4M)N+L;w6y?bz4OIxP;0;y*ddI_{d z`Fj#D-*e~(RB#AiQzNP4KJ{cto>rSrp^rh^D6N;)N36~mhJw05i`vB5u0F#HC&tHm!~ zBL8lhdDD;oMy^NpOg^i@k7F1nR)uUM)+>Bz-Ipcz$R-#Hc);}`M)4~Wk)9Gn+<7APr>9b)qe1P4+`Fr+mkG&E^=l#U%III(AvvCXEc zsH$wb0I@^rq6;?NRHBt2kUsbUmHGqvBPw-Sp_cZ$Gmf1os8Uo*zIX1q=brDJ?|gUW zm)AeM1aKV107AG}j{~?=hi**B;XOILFKsCemwosCq$hvUa7DwU@7^D-LBo_ER|9Co z2YyUzxE6qh4+EINtTcWkhdDp42YB>`dhc^UT5ihWmWGct+!hF?a%R@B&*#kP%$;l@ zE1*x9X4W3d8^vO_sNoZVJ!6(xEE#60>ZQRDsH-mdoUR2(q%3+t7+W?LmW^1( zDika;R>+p@T&5Ufuu?I0DOW66i?&fXSPhgAQ18$zpxj6bcr#YnOYzy+jAhRg zF+47Ty?2d#nK*LEibRh82?76VncU`N2A*QMK%*?(F$y$HmyFEvWn-n1j?bZl38~?J zm!dq|$CF`+;4fLzq}W^(XbJbHmW;bbEN_^LG1tnRL|UL@$Hw_`EfWhtl5)!%V1C2 zYZf=TN)Al%Q;Bki9P|wyi7#vmI+;zGv`68lORpM4!R?y>s|5$2NF5c0ffZn$U$- zf_0(?gU+541~J6BVn7F;Hv~s;v;uaW=RMr(jXWj5Q+N+;z~{luxSqZmnR^N~m?3Bl zuhH?|hh~=5!!Jq=r?-mJi(@QH;vC~A$}Q?QX6Pf%6fxAsM_)l}f&WQ;^3cOms%OaC z>XrB})%nLb+$SGfz43Ii@4oc5+SU7yzDAJG^xHr{!IL$w_~xHqTcAH-Ob;Xz5q_RP zdh1AT1L+Nt+-Bx+fI9;u_cX^O*D2;Kvp&ws+c1O^{7Vj}s`*dhq?3OT=Wq(cjNgni zJU`4D67+0<+VC2FMG>)AJSs54ZMV!AXF(UIobULcIJwCNgPcoT-F2v-@q>g>T4du_ zi6lGL>lx{C=y;4KId|1xT@$SLcSiuS7jDIF?}zw#K1oRpVK1}oVV0XcQCOd>PGWC% zN9JD#>2?y4eVL4g_NYH2H0Q{q+v&p# zG*7=k%Q{+TqfMvR(KZ_mwO5@y-SHRLTXPtY4r>Z4`8FErSVw2|kOZj!#a;VnPOx=! zNfO<1kCHl&=>REfCut{``v|k2!Fk-GErW3^;R0z);tO2Fm$>9mv5)6<%sNy^dLLA% x4`7Wa$5_22MThb?bPeO=Wrl6IK%2ryZ*lHqgdSe3UQ)M@^M1|;IKS-?_~6!`C-&0ZikjhKtc}pD6R!flj2` z(GH^nmyqeiWn2k{X$@C3%xJhK5M8hx+g}t2C6a3b;U#y&66ns`j&-M2Ubm|E~GngKq~K6i-xR7bIUSzm-lw!8pfQE(tb)L;M1EWDN~7l6 zB_nTp{y#>!k|eOO`7IN+9xT(`(~L^*Ww|FXaJbYDDnH7dZ?afu8R=j1wa7+@vvB54 zK~@^w)V^vL9qHGqjQLS>%>U)9CC@_@QTggvb!%1Ax@`vy8e`|CgBmbvsuU&Z6pfr) z<5ke1V^+s03~QLvF^+i+Sse?wu45551ZIzRIeu>{f#lIl$xa}1G~Hj^@U|y%!3F60 z8B;;}TkOb0E*CUS{otVQXyDlonGe1dY^6}q@&)=6%gK6QsHEIo-?mIYi;Q}Gf>$U6 z-f+Y@c5yYM)ii~^dbI+Ythfph|c3Or7CL&|MDC?Kf?r>8MO3qk9%JPSy_qx%@= z3^~t5LNi*wmfD9V@B|}+kyO97k60Vtqn&ifW-jtr6HmW=9E?1pQCLMcph%2z9AhNo zNN`SLLJ?dqGG)0_Y&J#SZXDiUZK>Y-Fn9nbW bg~VrSCAn_HB)^oBrqHi61pgd5(FgrIwMmXD literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Min.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Min.class new file mode 100644 index 0000000000000000000000000000000000000000..c983cd60cd7b9c710f8e4319e33f979c9b6a6ec9 GIT binary patch literal 1616 zcmbtVO-~b16g{u)bf8QD5fHyX1+k^dSVaVisL+@|iY_qhAHz@v+L@*^#gBz<{3UK& z8WWA;LS6eG{0;6+6wjMh6x0;sBz^C`JNKS@?#G*c{Py-8fHBPJFyoZeBN9mBVk_E^ zP2dtP#~***37<9^UQx@qj;lH*bW94wrybV`W(1#Kc?CM>) zQlh4-;90V?EUS(>HWP9t=-}GP(|C9WT1L7Nd5zc$>LNJAHNhP?3LVGLMaiHC?Qj^~++*lLUjxH} zMk^qh$-IP?euMZcX#0Ws2`~@>jnSOO4Im{QWe5eH#BrPm!zNW5NlobnaEcy+DV*Uu zMfyF%w{XvPL!Z>5<9d&t*+#3tQ}p*~nI3%`i3mQRjdaLnQhTC{XFH$zwC8k+h1vCn zBnG(-k;qx3xQ}8uBshaO;fFCuoyse_OEpW&;gD)}mufa77$wqqoFi$CSU;+z?9;R# hp@#MRik7JOLS2UE2<;=d5K5ys5?ToUS0vF3<2%c8m7@Rv literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Percentile.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Percentile.class new file mode 100644 index 0000000000000000000000000000000000000000..48ea281b6fa4c8b8db825329c4731d32ca815740 GIT binary patch literal 684 zcma)3%}T>S5dJo4Q(L30@!zUgP)RGeBHk1)ilduj*Ch-)WiXDsz>AL=Z2Lb9r{y|Ho`kmq&sV*` ztquH!8(oWrB($obBf51Fd2&B)rs9^@XV_1q_m#mu@5dtU^UE)TWOv&xC>rrQ2m1 zCU+GMXuLBuUH+SCw=M34i5yl;7|58IK$c-I>5nBDVkk%=-W5TUUo_gT6El!<9VL2^ zG(D3BOA1hBNe#+6$QC9knzXy2eTs6v@&Z=b8A7YrL+DS`(lABQQpZJ#s{~)eG}dVW z8w4a6eFe(0k}^zHQb?oPz|2^NQ9t`7DG) zLE?cA;G+=7OT|SzM8T40c4p_Mb7sGN|MC^U6SNoL;C3dKOPI%<;`&?q`IT@Bbq9AH z+;ebWz&Vyt{U}HUL{FfYj>4q=NMNOJ;-N2(WH3_xq1-=|eqhF9qy4c;<1k44B$a96 zZ>u;^It?S0tTO$1sKazk;9288@_Wr)fkM~pDS>i7)aun_JW%l)If&?~_Dvw8T^Wa& zed;V^^9r~dTC2Dl$s|!UpU&RA-r@MpSjl7(v)SY6^KV%kx<>DZ!`01Z23}wdf%eq? zeSE4ALzCZ4Hr##E?k^-$?h2! zi)gucfQMvpW)kZiw#{1kk<>%~_258dyWveMv}KX|$y8@r@IR{~FlT*axzX&~_7m=u z-w%_NyyP3%X3TL@Cq!jsR&ILD9%_6E^AK1-u2RWh0;q`@pG1!TBFen2@YF>f1?sZb zI)U&SC&+o9k^f)=&ruc`2&|z>dC?}9aUK_FG0B!BmAtuhT5y|LnmaYH2t7!^7q_6 zaLgXx?{U-i`+Z-SeJ2WBJ2b9Xw&4{s+@*TtY;zG@|#bl1A&4`>ix(Ht;j6GO2sJ7*v#Cv@7tQxr8kW7xdf zbQze%a|3sOaGASxKofM&IozVBg!1moF@|@U?6P3jFi-VA^K>3juc*%mq z9b!H8f*xaiSD^Yb2;69~uj1H^1s*rY#cqtix=XDUf#PDgPTZ+wFK}P&_-k(TR;~5v zIk_A|?RHyWH}7Hnjbv+FW|U>@wI${RlDq_tlC_jGah80it4nO`b@Xso z;Ql{h@ECjH4(s-uWfQ1nGR(bKyQ&HklT}QhqAgp%9<$$d7*B{SpX7_oFgV;r64d!1 zAIzY6*L(c>8V8MBJ}@5U!#JW;0`vJ|voa{Y)O?nHk_-RSCc5CAvD4>{$s8H#y!|9^ zL*5m-__PWzbT7~|#nmcGyb^(FYqGwNg7qE6)!BWNzQXuSw}BeRNd^HSp|%P00VlYs zr@m>fPjN*6oW>b~YU?b{!Kh8(JT45=*qrrmCF=`}Pc#O=< zXpXPRu*6wfv*alL@XCf;sSG*dBMCI%BXLFZ(Ezwg-bLEiQVKIT&UuEOV|=S+{a4W> Pe-oGJ*N=pS+o=8qDu%*? literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles.class new file mode 100644 index 0000000000000000000000000000000000000000..7014548867bd8056d6070d9d0572258427d9fdff GIT binary patch literal 6038 zcmc&&Yj_-08Gg_1W@dLXO+#s#q@^~{Jh0i6ByFLDY)RWBtxZVNKnhqY>SQzBF5O#q zXWMO6&{{7jDqavlMJ-ANF9li>N~GXjQM}(#!Ta@xKY388?>Up~ZW6+7<%emXnRCv3 z^IhKWJ>U6uo_yia(*Q2PKMbtHTMb->w*@$QyS%(3fG}>+@h$@z-Yr+&BUj$5<2FCu zXTZSwb$r0U0^F|SLk1S&4nICD?{~_}UGnk~9UnDNhmRS!8z0y433>mdj-v*a;vNGH z_*6AMjeDi}W75QDg)7%ja}_K7f_@f{x?b&Ck(S{z;JsovtlZ))Nosrm~STZW0kE&@_K=cb#r|nF{PCBu;&6Tyg22+`6yOp*g zd+heT*6zJldnA=eq>}9k+sVWt+4ihsIobB%!dqd1z-TNv!WR;@j?Oiu;E73wR5 z(Oh_$inaBzWX$On$OrrTD%4c;EZbZfc=&$^t>sz)UvFxRur3^oCGDYHV${y;utp`& ziw08>D}Jq&iOKoYqR-hAlQ^EevkbR78#`q8CwmS$G8f*2m1mCnB)%$RDtjBxUh_;C z1P&6+{}-CxL@X(=V8pQ^d$(C>g_@%CV>&hpR4H~5+hEJ!jn;mvJx5};50c}A$D58g z0!ERr0&C714eIeEh;Hp%{Yz#h^k8~AsSAg!ggv&+wz9d5oBQEeZ?7-wM{g>TPUVtg zjNECSr7zqa*6}#=JQ9m0Ehm>DyRM#tz{|Elc=ucJ9F?yPmi2d=+U~unf5Wg?oj06{B=CqTCi} zD{Ep#n50-Tb$~hs1pNDQma2+GEyBdnhM);=HkaUvdZ^dq#kil8n%BO!`th-dVmrpnIh!lr5Fc4qKNr*Os!oHJ)ln)|AWP3SZ6H4F<} zpek|PN=DoJ<8eD`#W!W5xdfYr&H0F(c48@(^i}eFv@%WZQPsqgjK@_|#@?5UW$bJd z%YKuSY8thhI@&tgq=~N!Y&cv0T^l9dt5ey^6jr3-ZwS;EJsx4#l8pA`V(~FMBOiZL zpzfOFUT*Z{w6hLNeOHr-Z{bOK`L>Dg;3gOvXAWk0GhyyE?vS;(B~vV12m= zUaIVy_yK;%Vq8wg+>NkWy741<`7sX;6Fukjm! z&dO}eWvGc~u#dzn0@|0#vE4E8EPiX^cldq9G%VqEbAghsMH7F(A5HuTe>U+K{ME!5 zXkjz{X5#OX`ftQd0;?;Fp~OtZ@07$%CYQ5Qnp1)9a^iE-SK5{eTwa-i0%XMwS)is! zQOQm=PQ`_O$Ie(zD#I2w*e6Sszjyo4$c{}zI|SO#(k2V6Y#OB&ZLoi6^QNo0i>6K# zRT$iD7M224l$3XGP>EaVw4EFixcJ4y-7~I4!_-FQfi**Z`{<2!#0gI|g)b`acT%ns z0*j}6rxHPG^acB+M;(>LUgVJqg=e!b$u>~nLTaB+g?m8A_Kx;0tMn);n#zV1bR+Y> zSz=*cFA`Hf)ruXfN{rVUV;*LtZ5r3>30gXriQ3yov#Gf4*zS(t!MbLCSBhrjFT?3p zQ9|sG6}Y!#Iv1*wpXMad(f3%{Av;gbk|?zAORCAt+)eq%uHW9z^QPBr#C-C1l6sFn zV7*bBI_j9)5;7Ivvmy~Yn{Dps=n&XnK9;$gwXpA8NXWtYl0S`ytE2E9V`W3i4w7j4 zf-SO9)puu72_9N5CrK@*QqKxni&Y>zP@zX9G;3b17@=kCNGdai^Z8ZbGVqvy7mH+% z!EX;!oC>Q}HiAFJ6rs==WIAYD^8(Fpi6JmPvLxl zJF#4jE)aMEE4%#lTBH9Ynr|8H)Q+8bre3SQ{VwQ>apCdmPQAf*Tc_WIVcOgf5P z-gQQu;i?P@97UfT*BPhK;-R)sS9N3cBWNARsxGr$Z#3%y9iyFky+>;1RijfE8V z*9A`DVh@fVF^J{KBLQBIE=JpNy6tjmfGh%0)QDh!D22FBI?0g@K-)F!J`ZL6dP=|^g`jN_7GWvo&5qO`k* z-Ue?;k|qzr_WNTmS$7 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Rate$1.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Rate$1.class new file mode 100644 index 0000000000000000000000000000000000000000..5bc44c8841258e27cbe21c5ad6eb7904fa923d45 GIT binary patch literal 978 zcma))YflqF6o%iSmu<@e6)zx)AS}=#vR(j9G$vR|Y;3oNEf~WWr_+%xYXM;k z`*9%TwxbRecX%XMRU?$L?g`fq9q*teqrTtMu&m*RjuqV0F^VxAt1xxk!fl2rb;NYJ zcVJq+9gF_9Bi&v-2>d|D9Tj$0$(oXNB^yfa!DLwRg9DQvbJ3UP5r1{WP2mp)zGn_( z6gVO@!-z+rsTwn=lHTw<8Pr@JhB9O*r-j?yLn-KG>1x z#;VXn~!dy+kO%L~>{n Sm&vo3BCXIYQA(X0<-&J-$=Ndi literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Rate$SampledTotal.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Rate$SampledTotal.class new file mode 100644 index 0000000000000000000000000000000000000000..125870e09c50b7e1b0a5cf863e7e3e6aaba1fb6a GIT binary patch literal 1635 zcmbtU+fEZv6kVscfwmMxi>P=@u!nhto7UOEr zHQdl}Q^PHR$h>7+-hx0dmEI5tEjmS0pg(Wf=Dm8QVAj@Up-jnO-Z5l(L)I*{Zwf=+ zw&e=M*JPzyHjC?yC(GO}+qPL-EKAold78^RwUREY(%3fjUAeU@b;GGt99yrLUd=LG z-SwpB>JOx6jvhvuBfEOFNHG_U1A$7aOSmS|8nK(Ga~&;LeL{-}S(D`{d2i{CO~8}ZV~KszdX#CD#LlGmv7P-{y(t?su)zRhyJW6*z1-2$7S zk2N%QS7o)?b-im=$(CNd#@L^?$NpcwI`V8Oj-R*anp3YC=3Pr&l3vxrRAot1ou0-SS_q8bEYB3-PjueHIh)9ZgHu{k8~cEuz)Oq_hsToI z2ZRFnf-tEVx$p~3y!!QRIQ*IhAwRAmpMXL5B4aQy$~l8ApWFg-6rtoC&ifH%4`^m_ zff-bJ${x_n;u6UP`On}o35RL9pC_e%h5A=1VJ3ed5)|L5b&dM~QvA~XDuaH`gWN|j Mfds3fayf|jZ`nAa%m4rY literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Rate.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Rate.class new file mode 100644 index 0000000000000000000000000000000000000000..e19ca5339a2150afcecb1bf8e827ac8f88a2b907 GIT binary patch literal 2865 zcmbVOTUQfj6x|cTOcDpM)KcpeZPXByR7DZ-0s^){KxzO*TiYRwFp|uq$pq|8+tR1L z^rbKT3w^P5X;`bS)xP(S^r?TLtNVO2hS=JIg*D%8&bQA#*U4{x{`?DoUVI(LI3^NE zV$#6X1e!1 zmOl9HAk_f+;2-0`V(7~1pgI})?)#D~49pwIC$JY&F)YYz%Wy}A#RT?X$$(?vu7PC( z1r27_b?x$S!Kzg33b#3{SSr{H)1GhfZM5n-zJ~qdcdZpGQ}vxfChxiVYPoE?erDP! z+A|awpkT$fD0p(*D=%iOl9gYwGt1VUWsCB~qUUCcwqJJgm5j($GNG0nH@vK$bBJ<5 zLsP12mRQ5yf~}!#+;Q#6YH{8!Ph0Z^p0tm9d8;sMl^xlKhf#mYA@QSI)O^kIZK^z& z+QcLwayqp|&_;9Q;>>2~{}VSR;h40F^u>-;S71TGau+i>I@etkSC;L(S6*NwHs?N} zLNe^Rcbvt6kuf2zGrHVq=-HIvbGJ1#uULgDd1)g=@tGJC42m|U!6@2RrCR2z6E8_= ztczGpuD41MyI_?nHtjOVaK$b&*PRm*w%vLlHTm#Fth z4jU+%aM5j|2d6ajZ7p5pda|)Hx{chSGk>%B4;~IJir$+d}-n<^b6HB4e70$t6}?wuB-ER?L1xHv)TO# z6>F(ZVa}@D>_SDg1Lfc28uluc|JG(0{M_j9)ya_@gYT*;Xy6$5gUMI0-9>*%HIpnu zU#ZSlRCQ?RNM*AxteS1UH}0*n4O$gu%kiy|ZYX+c^%#Z&;V>d-mvZB` zu3c)6bByP#M~-iCcbXEKoTeW^TSw$4t|K_kF~J!`GH?(lxH1DdoaC1q3gc9WF^Z^? zZAo`OMbq3mqU&h>JJ2Bl99BRbfp7=}v?zk?f24IfgdF0J0Z|_pTHP;z>!Ka$2Do?~ zF0Skh;QB(iIf6A2t}VR|J$>{EjCA)C#5Op#(**EhlrD;sM=Kp9Mt26+>fmi5S3)T@ z;=D8H57mex#F@L(j}faO-W1L?B$^j$*j9u2BVPyp)s^Up2H)TxwzvLy9O{Y^dcvb;Ni-skC&~ z(ApnOcglwAw#V3Ruj7?|eP?v1?metq(5jqsj@r*-AMu4jFPa6X&LCN5ki-b}MkxJ; z;u7T@9iwEW^OVg!u19jH@I~lpnQoxCiqu_-vcX literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/SampledStat$Sample.class b/out/production/clients/org/apache/kafka/common/metrics/stats/SampledStat$Sample.class new file mode 100644 index 0000000000000000000000000000000000000000..1029ebf0ee6e19c620452297449e3cede828c2c9 GIT binary patch literal 1134 zcmbV~O-~b16o%j18K!hP6{L!S9~4C?6*_{L7%|4A#6$wRsKLbDWw?}~?M$7S7C#by z1v}llabqHyxbO$~qm1vJHi&USbQb5{bLO1)IrrTAV;0v5-Yo!0`h=@nvgMwz^6{tP0qwr-Gywc7ue0HRiNroILY`?Qnl178vUF zbFF*2_C5pU)wQ`z0kalvQ)6V^57gssdrL)6ZR3j@^jNe@kqlPOx(%V&TQ|>gS>xJ!h7`SbfM7|fhaUzr0?U!%Y>CE)^D*{Fk z?z68*#foPz%V~SIAJ;<7N=REOubl|kNI|tQ*zp@HbI;jp;0l8{4sKy7i)Dd3 z|2r;XHOg(tpy58*+EX4!W|Z}X&Yz@td_ufIJ)Yn1gc^K2`SdmQqXLU3DgJe-nC8RH z@m-tX(?p&-l2Axjjw4+;5v~URjN&xy7`-O_A+8gZuMm~lLu4wsLl~9mLzvnf`oy0Z z6lo`6(RPS zXJr0Bju8gDWQVUP_L?2PNm1i$iKR!VB4Gsz3tYsd0bq}{y1Fz)SyL^1fpzd6BXkVz zvj^{xH9x>KJ~GEh4`{Ixczf(Hr6WQKms0}uFqIy4nz-UwJJ-gEP*G%%HFxg4bI&>7ced-_|M~e>07q~? zga|GK@iwMpm^LvJf{u%__#njLQV3yOmf5TfSAtw#4dEKDx8Os}$=(f_eI(~THt~tP zKb7s9GTf4Hwg{y~C}SdP!Z9(g!I-u4rJSR|nk*C?Z!Bk*%TAd~x8PP>J9o{_RrysL z*I;I;pQ=jY0t?Ny)4C}o61isE@Q>Q zyt^Z~Evz76;V$lJIMlRax1D;cM%)bg>?7%Agv8T}I0&57Y!uUxEMmcsNz;QoC_8>?VZB(%XQS_^C zK91dpVSsO$2@JL^3}UEm>^qqJ0f|_b{upM`h!|b^z+-Ir5>{-WOJ74UX~ZJNKvcem zBF5Y!w1n(cvwa2!z>6Ac^*o{7^wt@6|^x=KAKF>=usqb3L`kn@dcee zo)VZk_k#5q0W~O|YfwC=i1`#9KeVG1VP~i_86{5%b?Z}v=ZMqavxcpL$fq1@C?}0> z2xe=;LaWj%!5krmTIS5sEjp#cu>q3XChjq0w*7~O%8>H*dD@*&YI~Laa6@gl0X&S; zr0=6@w!FH5J5(=t2ODnw3R`6!>!GV_E>=rM;t4u^_zmH82e&EN3$QT7T%6uOYp`K| zuue;&Pj_|G@R^4BKWSc4Exn0({|nnQ`Z2vzC4kC_hv*l{P9{SQTQU#{JVMtHpD(^) zmIP)V=-T)H{xP%;x{vV%E)n!)f}SPlE5vgZDfzR_-wm#BVNQ{eL{Fgu@2Zp^C(UW8o`JuMex8N=V$Bnyi0Y2 zG%07;F|w>`hm_|@CPVX!is~5#LaJSgz+^ETS5zf96Ah{pja-_j=aR)_eP7kNkZyth aIofsYAQ_Y0q~e%j`+(|*VRq1NSpNaip7}=r literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Total.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Total.class new file mode 100644 index 0000000000000000000000000000000000000000..b18a94537d022333b1cebaca7127b0ed52f331e6 GIT binary patch literal 888 zcmb7?U2D`p7=_QIo5W_A_6vok5gSnNTD;k%7t2;JS}I=7PSb7LBqK?-{wxs` z!3%$YKT13^8x{ni>_w7!XU=)vkIC;pKYsyugPi~^Y|LX0Pkn6q*z&P0;MTg9nSkgC zct>fG)^7#o!u@lBR##sn0*iyRNZyU}VN#ySVaA=6fmSj*mt|_=X0ugarIoA@Dl96LeyM zU0qzJqr+a`U_50bfrHuHY`M^%IfcG}pC_^!mn`Phj5YN4ImF;pk4u#tr)E-tZS&w- z-pUT{VZH+ozQBtaW+bp=>~SWGQT%>*ov51a-32(YJ(%k`<+!oSTW^75=Hr+e_(zzS zCGj0FI-)}G8$>@KqJ{5pzA|#q=6iu>z#1Lv)Ni1}s54RHK2Kpuiw)_RYNMZ!_WncK zBx#GJZA)5Z7OESPSuL?X-UXJi(l9@;a*wKZ^zj>jR$y0 zkl8b6ldg?NShXW~j3-T!E>DcpjhN#L7nANiYYi=W$9CpU(e4!OTA)Minsu)K1vB%n AHvj+t literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/network/ByteBufferReceive.class b/out/production/clients/org/apache/kafka/common/network/ByteBufferReceive.class new file mode 100644 index 0000000000000000000000000000000000000000..7e1f4facf56562754b0a86f65907114e91f9a849 GIT binary patch literal 1322 zcmb7DU2hCg7=F&ov{Tz%YE@a3YK#41(E6-2R7fp z;UErm!oy(~M_e3rF(lBw6fMWT5-3j0%N6N0WjO0iC2G@Hs6@^}?lDkp7@Dtsu%2&aP zBJ09bBK?KSa?t=9J%(Kr1iH(W^|S>#h`HEQ2?^Qd$_+YVCGT8}qn1ED8&pf<&&e=U z%_VQjmr0_uj9RVpse7a{PEE5MfvnR%)Nxwfj|vkYcj2C|7LyuXQ3y?Xr(oOiv$2GN6k{0*Xp{pe=2%4ipMVk^7byw{nrjaHvD zLt##uqlN7}_rUtf0>?$k#SRxcneZ7qn3uz@jI~Fr>)NVsV7=nOGIZLFvufJr(}b*< zKrhLrBahMQ;sT4ECvcr-aO@X!q%)-;-cKNf3`DbQgVlLf*Y80eyJ`t<(WxaY)d*<6 zN4te{Mz=?O@-ap0d&yu*aHCORm{T942FFd0KI1EWSEWQf``MtAY7S6Gi)!ju6V#Jy IVzA2J0DAx&J^%m! literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/network/ByteBufferSend.class b/out/production/clients/org/apache/kafka/common/network/ByteBufferSend.class new file mode 100644 index 0000000000000000000000000000000000000000..ed2e8d1dba04f481c65b98d62cd14f8ad92375fc GIT binary patch literal 1585 zcmb7EOLG)e6#j0{qce{{2w?y*&^!V&lQ112$|EsCVnSgBvq&lyvf<9mO{U4Dd#a}c z>dsQC`~fa?!vC_r1WK`!q6h@dI6Qq1n=&Pk#k+ovc~B7pvma{OP+5&+}^62a8=javQSHy^1=8JeA3!Fl{5A0-NdGUD}g-aiWL1Y}?Y;bYz8>h~$x=s{0y>4pG zDhYAAtzf&|^v^|3OLscPuSw-ah1&n4NnvPrEtl)}Oe0b_vsVsj_l3x0p*fQOdb+vX z*AmI(>LqV4?srTvm-=iRwP9rDd9B6D93Sz|0rWPN{=c%hHwmT z5J`&nDe7-hiYBO4tO;%lqtr{Vo-!b9q@U6)AOZFm<1fCUS)CVhQ0oS RV`=gQf>4;??>u!2wP$1OQD*=E literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/network/InvalidReceiveException.class b/out/production/clients/org/apache/kafka/common/network/InvalidReceiveException.class new file mode 100644 index 0000000000000000000000000000000000000000..5a8f33b5f7d55aea8930a3e176f14b4264e220d0 GIT binary patch literal 618 zcmbVJO;5r=5Pe%ds)(Q{s0k)|F$%_BOpL*BF`5t?4??&vWrdZtOWFedEl(sK`~m(b z<7^?sC|umbY~JjfH*aP?zurFpTw*VSBzAPzHBmHCVo2XeUq*F?;;H?@$J}zc-?N%g zApKtLtj&dTPf+Ezi&;zREAg*D*M z18zCW^OSG-BATdRU^VB8jy~Xcq6f<#lpO^<&CtRAbD8E67_{d_DG_+Q3uGOF(33@Xt i)$=Kg-(qDdI*i4RX0(NvITxc+ivN)f^23!!ai=yC?46(eqoW-w1;7S(1&f;}Bdc(kPa`-KN zXW&hRv=@{V^20$`Dn{Pap0mZ_pj6UfuR>;0N3q_X zPB1$g_(fmG7lLr6I7+>}{Zz9mu`m?t(2j#pVcjxSOQ(F+cbDUzV8w7MVW@o~-}Ydx z5#MCsYjKnC)oDg9axW8VW1s80algXWuBuDRW4O%8Bud>UC^ryHQf~FI>+9o{*$Ewv z*%Ka5m>U+NIr|`#O89;>B(fOog`F6<#VYDP1)h+s@1}yPOyy zRSdjk;HrV&D;&E&j0cXhz)g2`lQ+#mGne#m12Sn{SNENe<>UPErD4(w!tOuUUhn%IRWOq|3i=F7yNFl^!-ysOYD z<$b>z6Q$3Zcnn<%53Q{E!xh(?)GWGPvu-qN$Ii6n`j$XujI9XR~$8P&BWjEzKQ2B%!2zdH=mQfP!FiW{xNdi z3d+(67-7p{)`QrZ&{mx%7LV=v)(Z8x3VF6u0!edw@N*N#aNNWPVz?kIi7cTMsbSfX zRkp*}3Z|@CDH?%E8=hxX(fnPZV@w)g4fv8=6j)Ps$VUpB25b0hPvRiat`iOa!21f_ zKL&QG@6{P#;GZV`g*o=tTWJ}YILr9nc2w=oU9~L-{(aznf83T#Flxg|?#yed(q2uDxsMUe+37dCihoY3c~69E6uh z>XMGDU(FS*e!#P%sJHvhk1MwDQ@ssr8pvXklpKFIq@|A5nl8&HL&d87*Id8B2PDfG;vp|E`(jfL%t zFcjvI<+G;h=J<|9P zIK;JCKaq$W=E(-6Zc}(t*9z{xt&V~Yk{F{qZ;(B9p zxHoaBYNDa$h{6z#)`i*YYr+lxSs~neS7A1`n($TP6X8R&oSs83r<6GH!5nscf|f6^ zdi(~~h0GZ?Ci@{PQ?R#?D8b6E8SHjL*t-oU1>(g)N1^y1;3MYN}|FUfnV zEyI+j+OKkc`=Hz|qC=s-V-AMUMwDEK(Q*CeH^j4up9y*PE`zq2aM~AO<+t3zm4DLDYRde#8e?3AdA4ygTXkirQ@v_!Fs&F3i>6PHHzIBBit7mfD;6M63^olULw2R zBY%Wndk9ggZG^9dU)SpVy2iIJFdVITow`ODlJ&SkZTo0h!qtq1wcjDFR52$8Bz}rU zQu2M}O30+Slay{{FtW93X)9DW)ry(N!ymG!GAuc%lPacs;>l~|@)>OAKLed`mZfa# p?J3)wVUHNEo7|EZ1e)@l|Hal%(IRU3qnBfq!;f%xn!6NU`T+>w{8<11 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/network/NetworkReceive.class b/out/production/clients/org/apache/kafka/common/network/NetworkReceive.class new file mode 100644 index 0000000000000000000000000000000000000000..dfcc7db953920a719efa0ae2de7e54652b56f64f GIT binary patch literal 3055 zcmb7GTT>Hf7=CsWk`PwJAm>oy83GzSp!EP%PBt2e5)Puax`YK5l1-W{D81{2GyMtH zai%vNy=kZIOo6t})agZg(V1Sf1QDx76|ReBR5WofZixP-A7e=PFz)AMLQE#bXsQOcFzv-{KXxG|mhXt>t{3ll z@xC7c#C*6X`un0$eV7q_(vLlu@L^V9XkN^DF)yKZV)WW*d}{PwB0e!TI4mI$TU1xn zNJ=&4BMCF78}k<=>>A7%c~dpaNj0^sNvNI}jYY3V$A^a`NKw}t{z>rUGt0RocinmY zLv{@|lA&iJ1FNPsusk=XFg=fZEqw7W8(9M1c;g0R}6=6Cj$upRlCC~1d zZfK*+=@~6IuFj;m5{PAzYHCu=>0<6Iy3GYWFQF%v$<0U9teRZVB1`Jrk{U^7(&>y5 zF*Nf*CbtwBwfiwGsp%^k18%4gl;Cgdh;ENoY`8i^ySJlr`?drh>9Q%!WSq3S&EzIb zHMw+M%{qbla+*3jlFOtebUVxSh_PpVMZv)k(r24JE&oQ;03F3*f0=Vt(V3UkagICAt#o#;!~X<(IC5DdulqUNuFr|$DnlGB zLu_>r&(Nj{a`Pn-$yvP35QK)#vlpfGguB*J^94ucp;id;>GI$l{UltVwVVA# z{`fXp1%~vPeHC|Jx%!-dKf$vuG5ijSjQ;kgsCj{1(;KLL8En5_FnWTIZ1deyc3*eK z-Qa$TLocvr`UL{CuMg~9M?(X1&=_dqXdTTPXn9FrUaJWTcam9};&m$-*@+{cel@e^k7E0XvPb5^eVDT)E4@eT#)ChB8c;>bnYI%mYp z7Ax_J9G97W8Nbu(0DDp9MtJ_lK}mXzA*t_A_+F!3`imH)dJWPF7~)SzaG1S>5yol1 zE9Gi+ZB^@HnYK!5eYa44WEr;gD6#zOXnTUX;3G?0>c@T0-~3UY!vIY|2bmwtYUCKV eX@PHlj0&f0LbP~FFXY#drTQI?b(It{uKf!YMupt~ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/network/NetworkSend.class b/out/production/clients/org/apache/kafka/common/network/NetworkSend.class new file mode 100644 index 0000000000000000000000000000000000000000..f4721fcfe3de2d5f3e6710b49683ca58f070a76c GIT binary patch literal 1061 zcmb7DT~8B16g|^jSXfp;P(-QH@*&%$bP-8>C>m0YNt1#PHRZt*dO4J@dudrai_&tV-sUGJ0Ew>+A{eF%(iaEqMp=Zx<|t2y5=aU@i*3l4Y2xh*l(h9?8I0p8w*rI>E0p zXm*P9PVjo2N*R%q1v{S_*$LD>&32UcDX-(9IFriLoIQ*kbPQ!;q!&`AI-kWH`BAH@xJGDGRqMiY4`Y3NPTSQU#GT$7%q}|d?Q>{H zyRT#H?T;@`Pe1wueLt-kU5pje+VAjzZi?Vw zjXn0*Q)7P&nx+F|}C!os1O} f2J$;FjN}(YfZ!hD9uthDzc1ZLa4c~k`2dq2n72-( literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/network/Selectable.class b/out/production/clients/org/apache/kafka/common/network/Selectable.class new file mode 100644 index 0000000000000000000000000000000000000000..c3ee5986e4874867efb4cf3dec0c102b24e68f7b GIT binary patch literal 832 zcmbVKO;6k~5FOL7VHdi5m5-LTd>l}gLw#sZO@)L6NL#InM5Nx0Gb|=vJ4zhrZ|k8y zpg*eWI7t@-NUeJD^Bc{3Gak>+U*CTKzz5h1VLOB!2BEaZpmYr0N26rSM*d?B-a%#EugrygCZbgPw&fr<( z>8#YYL)Mz$Dpkqdc+JQJ6h;L=Ve)^KX#))pC0C^7cZAh%i*M4vtDQ(CUk zVd@h~1}o8^c|zrs<|9?Qm_avDV7_)F+oks zATBh%^ZpDD{{9tz6bzQ`dOEwfuD5En4k^orXVXpTFkmo4dkD}EXbB)7K1;Sb}AWtdPeHtdjD@J-$YK{T|;SzWIQ^Y%9KMajUKP+9Nr? Ish!{c21ymt*8l(j literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/network/Selector$SelectorMetrics$1.class b/out/production/clients/org/apache/kafka/common/network/Selector$SelectorMetrics$1.class new file mode 100644 index 0000000000000000000000000000000000000000..504c1ea1fa9bc9c8a0475c9b83533f3bf3c333f3 GIT binary patch literal 1346 zcmbVM+iuf95IyUJx=9?8l>5DaDT!O+B$SIt1wsL&Rw;cb%KOIMCT^VF%63|bAK-Hk zQY$VGd;lMX7&|Q$fk>v7?A;mfoS8Xu>>oeBeFw0NCmHCt6Fu%`F$*(;Jnm(&fcrU= zQPE-PSk$q^klo{+8TQ@4tT9+^<@ap9%bmWk2Yh?LZAT4|Yn5?>Yg4u_~qTTOJPrLFTi>$mRr8lxfA4Zup#`S3W&V;>c#D!;o&N zEkS*?T`AsorZVPVMOJ4WKX9Zo)q0QY>06ReX3(kDS48wpCM3ZS0s#ZI196(PJ*2a(AfFuF8OLeb zG+pT)=}6O^&UB9y6GBUO+NSB=d+*(JkEGvwciK`Rh2KxQ``-7y@&CsAp7p@P_niT- zkuGn+DBcpm819x&Z`JU&2%2zr1aHSXEijJ@9_?nKd>-dI_Z|eA#hHpo( z0e9>8j*jnY_+A8Cqs*HC8MCWJG&;K3WqxXafPx*Hnf*xpnwe=JeJh0O_X%QzWtJK=C+A8NKW zR8S4ps>6h)zFeB^Y~Po0?EcB|qjrACI+|rlbYCuIWe;2VjC>D>AnNQl*N?3m1C)3yw2#FyraTHequU%lkwJ##7nn4Tzr&|A;#Z8h2h zwsMQNv)Hyl&q^I*xD=$S7&aVu@t2a#AuIJ}(ImPE`E#qMUrNY_6 zMX;&ThI2-zJ#VLS`ScE1aejHtspITk>20&BI|A(`gq++I;k>fVrFyI0KDVfYt z_Ap^EHhf zJlCKY%{OQPU7*oIgBHy*2!j@26d@z zC1+eSr5az7)5@S#)NNoK4q<7dCR$2+&aILhUyPaOE=X9@SIk2%X1S0tiYf(WD?FVX=g9|2AnjIljhGWt@%sb z;RE9~_1jbN(y|-pwB@dqamVa*V=GymN+HwbK5laEs0PmJCZxLdph=mTT%cumh&{B9 z+j(o$jytMX#&aX_fG?h7I4pVWxbZHwJnhDF`S?^m!Hd|q13B4Y|>T`pt zFE{CU5?$WXS~5OKSa~)9{t+khAz|PK>0vZC>5b;NlbTXe;N;HwP8L|>N-0O(@ydbq za>_l|&NDnN8)xKg)jR^JiTY9lCbwE13+N~aa5FdArgfZ;owemcBw3;_Q7ww8%tDs3 z>TIa^8F-l#U#W52;v;2))W}bn@yYRcQKK_m(CFJ;sr8(d?w1`S0q_xUbE#vt$GyKTA9VQu_+^Ez@3PH)g4Y4t@JANY;{$;6-<2k` z1~k9?cl)z}L|ihts@nG?4#cz8G%h-T)e1ucp;aZd{oD-blOc^iB6z*?aPg7RL zQ_4)o^<(Rl$2My_quy9SS5cKWrhIlMWb#Q7jE~Z2#eM(TMJqk6NL2+W`;%iiJPvY~ zNXrCmE102bmatwLFp{^_;CN1|do)^W&^lT#>v03OdX^Xf=t%%)1@A>4cl-|@C2$fn zzbVMMIL^~ow$^T`B`{fK=ee1jmSrv#aGIC;q?wZKgcDX8mYU?zmVqrbHi^Y?UMMww z#x2>XqRuwcEng>b+Gx-w+HBAkdERZMi})CqV+6PU)*inUWNR@fN!=VHQ>_K3Md8d< z<0g-*oNrfH?if!}QZP?Bn&Qr+4cbN*OB&D3?Sa%n@)I_eN1^0wp>E3Rq@qFVJy|}c zoD`3}JZAX4Eui=rTtnr)VbCRXsYaI>w1fH#+DW?zTmN4GE6)D|N3Z7`;j7KLN+Vp@ zh_!GU`(pq%k!WcI*~1g4#dkCwvAWdEWTjHJ%NLooe3jY$C@v+dYDZSS-dGb{%seid zb%wm!9i@3*_j3LvfY0!*T1HNWNMPtXX=v}GH?(l6krR6f4o)^Mbk5NH^q|e~cNUZ@ zb|zJ_D;}LEfB?fe73>9UW$KPuZohpaOJta@;Tx2rUMhX_W6s>+QQN!H=1kQ2;jopR zcVv@tL zwLC3?Vk=AZfKuLViWSMJSbaCEbNw$1a6xmyuZ_~LgOA3qh7ic?a8;JclxYN1iHnEz z@l2L4Oa5%6S`c5U!+d1&O`H$tuAO|}Mx6gjIL3cTZ9!DtfcdFYUMtl%-_O)HU!2r8 zUoX@*&$8;92TJwLa}m(dA@9=s-UPxFZsgBte)$75{*Ta`yH}q8b)SUjUULGW(`Xt# ziRRV!B0K{vs`JMTTKWTWvWXWWr_f3`1%s!P=2-J7v=Qz^Qa;Thd>HM?aI7g7K7|hX z*2xl%#G0Z}|69_}E7ITc$Va=o-(S=kJ)fCkO{~yYJA>ILFy}Pp4oByi(U{bI67!SI zyk$!|%iHlenT9i9K~J+N%^@X-4;zi`gqJ zs!+7wjP}=XlqphT4w%t{XR6l!QgvU{ocf%XOI9y3tB`3vqURa2m?K0 z7KTJS(xL-d(TSTy6t{@;ahI5ld&C@kOw7YiMGSuv^GOpI&?>Qzdc`6d5OKOiETI>P zrF55AM)!#2^nh4F-w`Y6H{wEiSagY5qFeNe)ndO`Bd!ys$cY~D46#<+E!K(q#Cq{X zu|fP;Y!nZPO`&G7IW$jf4JE}zp?xA5x>odt9I-8QtJofTvA87kCb1JwU>~+4j3IR4 zW;}_XxCh}DJele8N%lH{6`@;kE1rT7dO|nisdyT9uY1HpcsibeX57ZC2KO;v#xq&Q zvzW37;t?rY!?QI!N5gY9JWs>(HM{`$CoX0F7ve>J_Y=S1J*@zJcJ&N~UO}JYf~0$K z)tz)E=ycJ;9C2RMqDT0HJJNpHr`F8i>VET+^zwkq-=;d{u+NrkI*n_FeL6R*$H)#D zy|!8?%R7bZLO2o#oy`rFo1Dpgv-u<*n+%(=a8(f{xQDKltMiMi6$PXOtYfD2^t32F z-1TQCBTCQFlAegvQ{hLdYO&x(sYs4vAs5MRbc(&`7FS|}*oQ4*KL*4B92Ez-I0o=c zF^D_F5Z)-R!U=I0pB6{(Epat|E{5@UaSb(#YiY5#j=IHT=_2ts+ASVWR||{AMT%|{ zX}Vq5Tr?x}ZZS${#27s&j;Z9H;KUlBn^gKX;V4~?+wo#f+B;~IO5ZRo#yjy6PHv65 z@p`aNOlIE9cRrJs_dnmf1+apvK3rU=qJjluE*f*m z!=i^P43^wts5Q6w9*+{PI#DavQgv=KIGIp%w$w~@C*nIi?uw|#pZ0hZr~Q7aA|>)? zsqRHBs-t>Oi&)4#!Ej@2%le@8@t}j6-8)jr{4T?6{co1#O`=&%w@7-rDV12??YD*A z;O&GmwPqUgWRq)Y;xg&vU70b28=9-EFSAUhisImbQbMmLJj(=yx5jd|L?Yrm)r-Fg znPs~?GkY`G_2nltsg?7%_mJHzzbXUh>O`3(mgQ3^W zr345uAHc&zfGYd|lL$O4G29$~Eu+{OW{&RqQG1)#W2pbH78dB=ap|wJLEiuq^u`QU z;faiVg)$+g$WBwV28$lRZ@dO;T-%5Bk|GN;WT!|4>;oM35mojH$0#>9s74L56j5I> zg^}7EPL`VY$ul|&8?Ru$gXcVlW51w;UC6rTkbN%XfmOC$oUu1w;_>m6w%qyM9!Z?--+nQkk|o{g>$rp Rh4VN;-lv@^xQyfQe*won(OCch literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/network/Selector.class b/out/production/clients/org/apache/kafka/common/network/Selector.class new file mode 100644 index 0000000000000000000000000000000000000000..86040208735854fd3f6323292ae5fa2e34583f61 GIT binary patch literal 15681 zcmcgzd0|VRK)_!eWzu&p{%@!sK{iC+rci&ykJ^MNLyvf(z zd-6*}v`}4BNRRL)X-=1Bvou?zIm1Jb@m3FS<1^*;EQ`;!_#6*;_*{8EFCW|SU*7BT z=C^peheq&Livu1i99EDAusQcmrjert`#Q{d1wN6$w(YEfV1H9~y(Ifz7J9xV%`4J;TV`LC+3(2gcLkg8S^RyAf8gdHy7@$`Ib?_4FJ$!8`|Be9jS$wuzm?{97QbFZxAX5U{)3SBM|t^^ z!2GkD|KjGqiWvXq=D*9(8^S#B?GFBj@aRox{wdAF0_y+dG)Vqmvh?4A(SO|hmYd%e zm4_NF<9DQa*W&jqe&3@=v89xy@|a3ng5h8w5sftk6R}Vr&SW=6BEeWg*dLDv@mjno z=8wd?Lh*Pg8o_&BINHuswk{fLuZf4-7VfB77j173#_E`gBB5v!9Mgn#JNy^;YcN(5 z=)0(2wo zb;BM4Y)=jDs#^|-C4p#HcQ}{`wzdQ#tvGZ<2H16>cmivd3>jLc3G{dS10BJdPJdgc zzXnjcqLG?NFwq;0b%K^is|G(Z1%6{N5DZ-)Tw8g};F=9s2ZRP}w}#?@C^#1Y>P%CQ zfT+_!;&Uah}UewD!{Sg!ALwB12MJ3ZbjG#+f|2nIT3mZ@M_C=yC6XG&Bxt{s-^&RI20#$@Y; zU6>)yBYB?9n!0T>Hp3_zqOCxqbX_PCZ0_mW9*k}BZx3Sx_8#zuH~V8Dd3OdG6Jrb* z7h?gY379=Z}8Auz0ZSlouH4)%;n^W9;46F^uFxlA*L1vZo;Oy$`K@9It> zUEGrJ2Rflunh3Z4*l`kpmFc@r|IS*WVfJ=?3OiB98ok_(>!fX$KLvl*dv&^cHmM*K(zF>6V>5ba5T z@9~5`mN2m$2}UgFZ`}nxK`#oy{N#8(`i)Vhc^Tkvj$^i2wq&+AY9}H0gp55&6pmt} zrEKK)PCB^sNW?IUnAItt*LPuUn|sXzR&KM+9Fn>u&yT&Cqm<3f$gs&h;v z+j7)SKuLm}Cm`#N9b{6j(2FFf`2s5WvOr#D9}8UM;K|JkW%Gt(**vU-1kco6$-OKP zb~1NQOSC5z2(H1s3qc_%zd0fco4!I{wds5GeVY!_Q#QRM%{S|NWZGs<@y9!0eaRCvzw2W$ z|1NQLHXWkFHa#oN3(`C%&5LvxH{{`tPRG~Wmq0;gq)lI=uj6W!#shbZ&S0y9jjSoN z>09&^Q`K+>8J4%~LLSm3?!i{0)M%T2K(E^LL!olca66MIW_wi}Nuq?oT*ad zrd7HTnLxWgjH^N-m||)Y`wXRD*%N}xgDaV~sUsK*`a*G^Kdi%}&+kinHD6ma<_op@ zY*nGg+RCfO;b<-tagVpv1a+KE@2H7LylhpeuxgT;Y^y2KOvNj`p{6m-(%5BL^Y9s| z&Bbg3Ut~4Y6Fw(z^*QNjt;XYcVfl2XiATcCjLtUat186j594f7(X$>+>}w7HyZS#2lMP@@QL~sFq8! zf@%7Ztj?Ks+VoHO1a*=sw|TiZhJ2e>@JY6+S1WDRphjD2m918*HMTlg%|#Amt5d*N zdW$jAD0;P4oocIf!nG!8nq}W=(TFkW@a?ZYPpY(M_3UHHG$6{j05np4?dZP+yyhUj)y2MtUDr~DRFq{_(!$+Ul6LA&= zTYV-?1w|1RwN$sQ&R0>JYk02B^U$c6!o4P8s~(~20)=2@!ay=8nv}^_y22OJcNPFar02%^H*+7mQtFIJ;hiO&*;brEg? zzAh+tC~m8ZMY}Fhm)hzw$b?sjU`|osbybRqIbQhBgGJrP7u<-pvHcaDEI)6mT0~f>8q~QY?U`8_U^SV^o>aZv!?4%azlf;AL{**pCd6Ub+ z2Uo%rKA8t3d5UygqzreSyzgX-N+;R+f;u)cFW=M=i}ngH^%Z~fG30o|rg%eGuK^cS zHLlTxxT1)Zgo98r`J#Y2#adIJNccmMI4;LS>KjHW>SiDrE%0}PKKK)4v6@#I2}O>? z+Q~7H54#gl6ZGI#lDx^>D_6c;wT&$81{{2_h_r>;@r7k&7#9NA3M5{&E$3FvZDDJ& zk^`5NIwZ5kWXFSvQ-i_o`Yy= z1;c)DaeVTklR7L}suNyju`tQwBe+*cKTXnUtC~RSZE9yv{Ltr~&0B z_$}bB72OSYtSdqAsaac-MveVehdI+4p$7OND2@kd>jOl=5*05jE5pX7IW*njVe#n#fLG_Z=0}~bOOU(&ayW`C`g2&F&vy_-045oNJ>@yLs{NrI2%{>M z;v9Io1u2o?sim`{O(?F;%ti0Xe#wz?kc)4@&X;c(XIy`+EX}|QiVQq+VWhRPP-A3F z%nHRfOL-Y^di^n2-_jg(WQf|)jw45O%`p@;wpIdXB?*b+(oqUbi309EV+L$B7e*}3 z97pm35c*qR3Pypp37Am|*iiqL1U6(GIuVMm#sv^z#|_rRjOTRY1km0Cdn^ni(RTU; z7}jyVBos$=E)ZND5n+Kt6S7UwXni^cNrf>0UPxBjXYh_vQ|kT8ct^o0_5M8GQH9ES zM=dJr9VIn=4$4dViE^5LqPCV=_cGw1w3K>(1@9<}>2;`)=_jgkcq*wT=_d+Gcuv9| z6p1kY6C(a572u~`M2D*fsG#}@viiuqk4kF`50YoAw{U=p4poouBfHu+nuvw|jo3OsBl{>1-7<7X_0i~bcTBolp6*tpyJOMyZmssZ254L# zjmO)Bts_R19f$XceN>5eAAnBkr^$UZrMmh6ngdko@1v>JW88hD9tBk{Dxwl9#Pg@1 zXd9KIJsWT5VthMh&Z80JhdcvRNv$-Cf;6AnXbH8`a_XQ)3ehIofz_R8!_-4vbU8)n z7V4(^=zQ8wF=#*>(oWFN0QoGOGy=SPjed?EI7q*sU*ah}kpS!7ufTJqd%vdN02i4R z`N~i84ojhw3Up6oYw4ay%}u{W-*#ZUL@#6RcNnQ8^*#u3Tl6}+Ec(4gf3WC}7X1mo z%dqIrMUGPbMT_{aM#OXl&=b(5)v9u~*X7Ok7J#eAKZ>C|%_)z@uor-LYRZ;toJv#l zl+xcoh0s$wFao}kh8{X9rES_8}|rz zKb?p{>mV)PI&*+l^wUXOG@4#(km~bj%cCG-6P-!t&|>|(5B;S_mboqg$z-Zljawqtr|v!>--fe>Wi94#m6!%6KPTNq5oJ;KKE^7d!7k?_Sym zZAXoe`(S7(Z8XPBDOyI4|*I1$~~T(1V)jY6B}p z;M5d>Q#A<+JH~0m^OE?$7MR@(ti;Dr6ekGr6^^E@R4~0|O|xDg3u@4iHUxvhV=x49 z{0HDb`ZNkYvpt1Eg@b}P8{wGJ#_1-0hsrJX;P4_A>Vt%LbUTFNEBfm5Eqj69DK@nWOM^)8LI zxYXhiZ_&6$?YS7GEaY-&+Mbk zV$EBu%lqjJSp0FduIkwbXd9>p>0=FqK{`{3)#3^Or@x4{;sjB;IY6J_u8Uw5Bk>t) zES&FT1kC9Ok@G?BA_U6CkmCxhUJpWr0SkfKVyt|PL5N_P(u)mJ9>*C)wA#^&k+cHU z_;Q>kgpZ)ScWE*-#NvvL7LWBpQ@lLRn2qZo;Fdx+_97IY1=M+v@YAi8)s+Ku&I#g| z%)sJ;>T(2LiGSk{(7ELWlY%#x$@7@{$v-ClJ~a;mP`2&0Zm)Zg@I#5(LT{l_5j947 zT1S@EdZdr^uG;GXcWaCEGgxa2Y;TcfVjKFEx{uD3C#c=nt=H2}9eb$WYxh%Vi`OIX zTSolrQkRwjmX_;1hVKII0PX0fPSL=yGt-qvvMaKO9`_dYQ*+SMEguBnurK&P0D7ZLBS>_Syq=28urtSk+XR`pNG*-p2)qJxrTjwFHho6@$vZK(R3c*Dt->1P+rAHl;87A{tM6I zcX*z1@qAUx3)BQYQC0FHHHlA9GkCFD%1iJ8s#b00I(05DQxRUS&gYZxv7lbv&MVa( zZqUBq7AoON#P!#BJQb@M_)~)kaEm2s5;bYJSWKNNPx}H7Xnj-rf+A4-Cmp(MTBoMb zLGaT}rvrzHfQ3|2?M-?VwU>Q5^(s*}@g(%}Xbej8li{uM$)_&lDHtnI7t=(Zik?Xu z%i&K9+-WJe#{;f-peTO#2Hr2mPTPs(> zJHJa_w8#QVir+_U9=iKKP>XaXpyW#xoS+21tMQb`I|XUnyrx+%QbE~`kHA^q^a2&h zqn~b;*IVR?WyU9ulk}6x*Eyu#sRu@NRGlm z2%=3x6XlXW21?#_JWeX46P%nTh9b#*FpD>5@V(5QTgY~@~>%sZ)?ccl)U45c;+CICCfiy0S* zay#i0__W}^#p1RcSX=@uE(I2s9UB%(J`Nl1awjZU9Kti;h2$wyci%??cvcS5$0a(B z%DTYsqr1)AZs8e{6^Bu`3-9ibWcE(5te@_}Ds$scL-k|H9+U4P+6~fP0lzFGL%17B z0g}d9W%r-~fGGkfh&o1>{OhNC%kDF2!yw(S2n#B3G0_oQ(r1y7$&QyxYQ|SWX|JNO zd^P@z?HZ(Z*Fv_}(PqA$w(|`X<{PP(Z=&6NGd+aoBYZ2I`)&B8(QeI4$(+Nqi5Kb2 zc?Dg;C-4%mtP#7Ga;>(kXB;X27AFfk&8j-|MDAtyD)$~;Yw9K?KNt2J) z(*ygl(3nCwRG#=jD&U6@Tldpw{sR8;@Zpp)0$|Dxr*XZGj=~xjsgmN1FeKL}$H8$8 zW)V{;ru|zJ4y}aVd`76ots>a0?-sQ#uj>VxfXO_s3%QqB;KlXzB5AIe#Zm=;`ZPc@ z=JnHO@mR8t%4_o_&)TQ=c=HG8b4ayx67_lHyK>`LL#N@kpR5n8!99Zjs(?{0<0q&B znb$;qlBV!MF!m{0%unO*+78i5K8!owv(&_2#uexVsLgXwwCAavU!?Q-E1DltsM!p7 zQU#FvQY^)pnB$fUpm4^4RG``*aWW54auct_oy4V8Y82(YM^Go+NpP>P*n&V`2A#t1 zh=97t@oqk>+O>YaWCd!QIa)d&n4?xuW*deq&y8SDPgP%1Bw?wD~1S z7==v!7S;1Bnj#b18&Y&MICPZYCN|`zT`xCnCBJoY5M73+1SdDp_`+dYJ~SNuAVY`T zsT16eFIVzgqa2XFc$gj@3d!69O(;A9(g-)7jI6~lFEic99|8tRv0NvO1?3~j`6X_M zZu~vgsW~>gj&Ae1oSXBGZgUei=Qvr3ZDwRi!=SyMH{{sNM{zQ_`LvV=n*<>8Ut!rJ zPtc=}Q?s}u)^~NCqanFayb)uHTTq=LlzwoHM7jO|a9oaOKa!K%@du=f3kv@qb-G8@ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/network/Send.class b/out/production/clients/org/apache/kafka/common/network/Send.class new file mode 100644 index 0000000000000000000000000000000000000000..94822e708fc0976138c4ad5260678ccf83e724dc GIT binary patch literal 325 zcmYLF%TB{E5S%TcAuW&c_yi7#knn|DZ=ixeNIirTH{&dco7AonLxE4@zz6VA2s@SF zvX7bFozZ^(e0~AAz)_5Y7!$&{;Ld3yowkN>l4kb{`6@*tO(}BMYEw=LiL%R8!;TBW zR+>E$;&rPXKUl(9y4f(=3ROvCxaq`=bQOo?%Xi0D8+4l85k|8&#j7CJ#W*BHo&Mnd zXdrX@(ki~zjfaw)O>rK~5Ke7d3b~T1Vo}TIS_*GyX^k-K*0!xhm?cd9t{ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/ApiKeys.class b/out/production/clients/org/apache/kafka/common/protocol/ApiKeys.class new file mode 100644 index 0000000000000000000000000000000000000000..bdeb0e6733fa34eeeb3675335a5fc1fda24b766c GIT binary patch literal 2633 zcmb7FTXz#x6#h;#O-|AQh5-vHtsoaoTOoiTmtJT{r*t4`N+ty&YM3U|HZ;w|$%KpI z9mTt%Yxx6w(MJs;EL^g9cKI88)_=egxX(#gRK>NV5Bc`~&OZBmXYYNczuvob8^9iX z6^4Rh1SL2j7$M9>uov^`#D@;);JDuydq4G;~+;} zm_El4M?;wBIEFdS3-bcU2**WXzREGm@tQC%aU9}!U6^lhjB&gv%(pla9B&Kr9gf2s z?+WuWN0Q?`VZJZaT#4WVd??(HLim^=G@Z?i&m;|oft2gb$L*S3TyWxx_S~W!FS<)h zZZ%%>T;DCam3X37KIUAk4>5F37`fymgPJm}Tp=?tVHr7#fv1gJVmy&cFsw`&iE$%a zNKB0vOe;&FRxUGL$QsirGfC@fW~M37C_LCvIA-J-)+IAjxojqtGR6zm>{`e;<;HQP4xg)-gsI{vj;R=jA>O6u zmKsGztj?%-9{a?jbNu3hiv7X~W#;DUj$c=CK)5{Z_;$(mNy|aut0~7WIbNb#GV7j- zA;EQ7zFW&WwMw~2VZ(x2Ia8xe&cn?kg4L65)%V;=#VJ_}4Zq}`uc{c8XBCU= zRUDFOTU=Y*n4mODP;V?b-owNR!H18!3tx^lt#`>2vG&6hFDf-l=z00RGS+VQ&p=Dw17rA_=$V&UV zm+xzbnPRGz2}s-i0*x6B7|Dp4X1_VMi$0}$>7Ro>78P9r2&q95sWTF(BNC|#5~=hO zsn`;!v=XVD5~+p~sb&(XR1&E|bW%i22w_yzhj6u+9f>qPfYu|Xk^D}CMLEHcA#XFG z_=_~benQ6`1imMaVVZt}a;=-=i?@hTTTx?YX$~;Zy-3(_tkZT2BOM!@E>HTI9jLjfkrk+BPwQ6#|T8)O|+S|;)J}i zuqNrUElwPz;3Tbg;1r3@63x)Lc(vM2NhjV>{ z^n0b>rvc!GH8z!yYj&0XcfJSWoskXoQ>S z3G74<#;}3bdT|DQ@bDxqV=5R@XkYEVHmW^UrGLj4^*+d!3V|%cqaYiEy zG}&~s?QVLc?KVg6D{Y##NwckJB2u+!x<|6TyZ61@y}I|=J=)!D(*JuiBM-*>3?bI1 z`M&qQ@BhB*y*K*iYhQkmi1x5tJGIiEDD z^f&awrS!M-cl`11W%`Gu^pEsU{NbPZ$G<4_uk>#+JtEP+%k&=#b0M_iUF)5j#XQf3i_23=N%OdprnDw(ZTDB-d?W%`7~*2rwFLdRUzI+>o3 zSeMM!D|FmtZIJ1c61zrbZ&2t?mvya7pOTm+vyBSfvTwoPW;3Rx~|yG);v*bbTPRLF5zyJY&T%zEHp_C}dLC$ruB>y0u! zEwP(qwnw1{+<|*#dPZjZxb$Y3J}tzQec1WRz*=-0WJIp_# z$O3jmVttIH(d1Avbt1`VTf(v@d-NH-FlF>i>l4#@Pr)jcta8tcZ8=uKD)z)|+p;SM z7_}x-$rz)K)WAR{md&R|@i&l9N0Wo#uINjp)1xEV{BSIjiE>VtXC$9Zjl}!%sdRoM zn$E_v@f7T?O2nf5v2;F}%I2fTqwz%aNCI=#@@G#paPf?l>DaN+SSFj##)o75+&ZJx z>DWjj-WQ#(W~B!Lrc?PuG|l0x^I#9hveEu%HX4+zOvbV&Qt6?5>||eT#6`Hq8=pu; z`}6T+ek7e5Ok-_~Hua^FnbBdO?@Oi9{qbZp3#ju}vVlJb<$O8*L`G}k$>Y&Pyx((9 zz%JGVSN7)4FTrilh*M6=8*ghlk_YY&WMw9P2Z9|c zT9a6zNbX8;V$Z3bgf%&7*vR5=p{Q3XSxclf@-}fwKdtvv9kbXotj}OTd&ZnB>rT}+ z7$q;3_PUQny*$~|$JO+9A3AWw6?6#7mlmvXqiKBT^6E2FmhD`f?pCgw5kqX-jWY#f z#xbojqn(!pXo$-0iO!fYe>fmY*CR7gX^gt>yzJCVyTg*xda-I$812})?P}(=_6gK0 z5@y**R!d`soz=&RScfDOrY0C2+#2wE#<9)vV{7K}RxS;9n=p)%2Pz6WnhC_{aL{MMIQRNh z1&ldpTYMLQ^&tiIv>dFlu_2;vw~5npx87AQ*Gm3l()kdn863V z5~j!KQAYb0!IPSps2EPF>ZB&p=$l3$Q;u;+o?ala#7NDoXl2XMW=#}L*)`4QLF|pF zXeHe&JNUz-v9n@`yRcnhdYofg%tw8eZC7WUv{A6^@c^Kc98d|1)?u$;l^vt(RJ3u+ za3Pg+r!XbV`D6OheVS!o!~%Q^{<3Gy3fCbF0oVIDuG<#zl^46LHDeZ1_K0pfChy?^ zggsew5V|o;iXF)^g(XEXA$#Rvt(FRC*YM%o-1DRY7WFm;Hrb=e2DAYzKFiy zp3CTvt81oc7IZ)V(TLngc)o5LgQ0ng2-AnT&3%j5bkjX+Sqa^~s18pAmtvL+#p*bW z=*>zf^6@<0=oQ?jCH+iORx!^Rg6XxES+iKw^a4M9z!x9kxE8BMw?hmYjy|qCx{r?$ zLdvVrWor|rQRF>B81q_%@BUe2onu*=01C4oy^@e74QJM}rvo7s2EBERn-w&GvfG}= z4CFhq8u6~oSY;G8w`dy$!#r(_!{Eob!QMsu@2)dp>ErS8h;228z zM13KPix|usu)E$0n)SF=(92sKzdQRGSOJKSa}a}zfanAKYRSN+-!ycYvfs$vKMxOY z%+17v$xm>T(M3!`f4~~**T)c8{60~>fx6|L6VAT~K(px}GGY-x@B~M&XAuPPGR~6b zxag|^qep^yuRD%Wn&F$VV`<<=koe3An=um}LR^p5cT z!_V&kj$9B&v;O=&W=!ll^J@?;m@p@CNg2m(Fj>MzrHN7mNC$Ci?uVb^@NQWI9wHPg zTh+;_v>|#>FfE6M-Te^BhZ3-jQ*KG%Tu=~&&+7tz6UHoBgjwEMIa@AZn5zd^;4)sO zCXm8gPjWc@R|_XP;TU#wX2vK7ckAJckj>gAB8b&n!U4Dd?$E}nyy0IdwohM4B)wEh zdrOKIri0Eph`JuH!ZChls^EZUqIHRLmY+J}Vk6qR?-_K=X7N}g_RyHrAA@+RJN ztXoQ=mkJN_o+Dt2p@to%&vQLHTY5Dr5E}2kQ8GhQ;~Sc*WdM@ydf+#zk}<7aEocmj9jx_t{CqVRS=K zuZJ=?y9r$7Jtz1o2)AAufmNMD<#KTw1q$H?ycfO5W3F#-ja@)H zU-z2%cn0(WHvO}BcLSB~+bXr!PMue-ZoD$5?t(=1t))p zzUxgs>RDdKEx$}MCtu^s3<-}d7tkLdiQbH5J{VB8%tP`$F_wi9gzwXDdRot6S_=eQ zYt-I#fs_|$DF@Y>*#0uLZT~`z!jWYze|ZQwv4StzHP|U!{}8G4Hd;k*r%rk&e&%{u z818g+dTFE5TKX+`nGuH|6ML09CHigP_#OIP561*%a2y+Iq;@|~E9Y>$LhZ2o6t!-j zv#p4+kHPli0ec(h_vrVbLxlt8?1ic2RqBGh!`!A!f8cwx^&Fm%8=+?!)S8P#LM;vI z$VFDwX?2}C&(jLQUeg+4;&H8bT-VmxpsrkGeVveU7`j7w zowTGu*XJUe>vV%)ZxQURvfQ9;xkz`NwhQ(S!QQDZQ89j3F49w{HwwXSA-GYMRH;EX z(thP63e=c%Moo*F^147WNwyR2m4(1|<>U5hB92SD8s;J=x9mz%d z>eMd;F(DXGm#WJeG?q#7W4TDWP8lJ{ z3c;wlTvZ!%JQq1pr;|dE6M{R`73#_c-IPoevK}IezQKv~Em=Xe0ji{>{bSf8_u2WG6NJ4m=jDUf9~ z%7QGfQ4VB9jqV0fYg7PPS)(bCNR3J$9W}B+R@G=0WOa?sfppgBevmaadJtr7jou2f zu0{`mbk*n`ne)UVtrzH>=LuQ%aIQ}8I#0+2?_2u?diPNGi@l+P8`eKT`?@#Wm_*;% z`8>S`t!L*8^h+T+u~W1+m{s{Vv;vR+mKOHZ>))Ddc#qR&)cQtRgN^3~x(>gAZ>B75 zL3i&)``dw6PP?#`?WXVGH}oIUUV5EwW^3sdwwn&H+vy-H&~5BKin7P3k3CB<_6iNM zA5fhAgoauoG~BYCQZ4;-tmQ7sw4A2VmUojDO47;Dak?XP58W9$MR$kpqkQOLx+nA~ v>7gg_{4^E(#tYdt(S<^y>!Cfr7OF@TVr=)1(K%bZ-Q^GIkFbgs`s1Gh*9(cG literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/ProtoUtils.class b/out/production/clients/org/apache/kafka/common/protocol/ProtoUtils.class new file mode 100644 index 0000000000000000000000000000000000000000..5038584b1f38a71631ae865df10d7ca59cb9d9f8 GIT binary patch literal 2456 zcmb_dT~pge6g_JkWEt^UAgS!8CTW9hVw{+SHjt12HHix`#0FC8evB}SqaaJJEHmY| z_f`~FwG3@W2e8V)9GDFU=!#7oendr-RpbLJ@@Rr%b$P$dI;biz6oOx4ow1n?w;=@4dQ%;Gb#WWyN1oLHWU<(aVM!uTBX5*CEzxrDrg zMTYi=x~V%;3~lkmDnoGA+TaY`3%bekjp{nLm$h|+EPV@BSu<8OTNh{F7<69gb%xY} zWmnQ#O)J0R=`C$@OG}rnYSl8+HQREmvSp-~#Ae0Ojrt^mTqnn>HfPxk8}U-<{BUQd z#_Q>#8#bBCB`z7kAbBZ??A=S}BayV4KF@a;SdO5!xn0*SlOg2q2=&foi(u$1I$C+_ zxmI(xH>*=6&3 z%`)r!h|<6@_({7`)9gC;Q!+dw<_oX2Z7pr;R(fW~;WLfRO>R$KoPuLF$__PA`??#+ zFmNUysphBl2r)d3|6i%YMXO<#`J65Wqr2JKqoOheS8Fhr z_n$+K`$4@&Fz`$7v=@Cp^tR#BZ7?jzSLgCxhY@wK2%#+|Hn%7Vv`(h6M#x=~1E65cv@)+`vtjdI=xVExqA}5AIpo!V}gyIAE F^dD8KJ2L

4LUo#8W4A3TcF?5BML3}T3H zJ8*^<--rY&FKL(L*xxloaw~|A_i1r0u?jVcM}1mS+YxkA+RmVx)^-Km-D}+rZI3@% zLoC;~g1r+pXr%0%?2o;|8@Z^qFZ&Yvhhtr_8V+#c%~f;_+8_ zCl{^Z5J%t5=4gAEy(8IG9Hl{`D-j*i-dn}72)YunA?Q5|EU$699Fy3m*|!% zL@5j~iB3L`bB!n1JIUL3H)ZVM`g?H&eYis*vs7Y+I?YoZm)bnTMZDnb&qV)tyPsmCtsAY(Da46Zv5id5fsqw2#s14!zxFKZ0?t1~|&m!)#?- NhZ^mHMtjh2{{=IR)?NSr literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$1.class b/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$1.class new file mode 100644 index 0000000000000000000000000000000000000000..4e730fe55b04e3f13726d8752e0fb2eaa7ac1a08 GIT binary patch literal 256 zcmb7AlS;r-1bx*61KmTNOg*HEe0T%GTSdJ9 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef$1.class b/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef$1.class new file mode 100644 index 0000000000000000000000000000000000000000..a0ed341dca1fe60dca3bcdd075cf770804b962de GIT binary patch literal 3821 zcmbtXX;%|x6n-WN41_^32vlv6wiF?NfZ|fHD&iJou?4YitwR_>B$-Jk6O``zzVEiR zdux|Z?WrjBoF2b_*El4o${H5DH@AF)3E!Vst_%Oo?ft3`ywX zVaQ08;sKa4(m~YVXbCLw&ID0~X&EO=a7qkk#NvY`cu2e-4&o6!Dy%&wtUVs2<0oW1 zDdQ;_PfL)ahHh!6ED4+X4KvoJrqt-9);6V1OsQ>Y%hc3lT#vP-Oe3nL(}vl$KRfDm z6uW6-YI@i*Oie<&myNWgT3XvaZDv$WWHeiY%B92cn66qGp?>f>)wlGYP^Zj>caD!NoCJ;=-TFN zM2oM9!`HT$E!Hf+#Tu9K9Agni97p1gpzALb#*NU`b!R^uo3wD+P&6+gE6+Dj@6trQz!gfERpb23a zFDiHmdnIi2fa#ogL?6(ShB=cBtlKxs5~7M*Z5d(sWd+S>k@1RxSMeI5<%+;fZUwL7 z4FzxFEd_7mfP#1Mu7dXjr1x=}@mKHxK2-1#&PZsuCN4X5WPGgP6MQP6Cg;zI){aa( zF)p%NI-=k+e6C=dFkEq^oCIG>oq`^m5n^9Z{+NO<@s*6P6`aL43ceL!hXu~N!2a0d z0c|n8w!WhPg_AE5)_Ry-pjXNGPQo@1oy95{$0* zkrdL&c1fO!JgKG!?awNjJ|)x$$|tPo=IC;lI{%wugZ~*8~ofLQZnTC z7|j=}sg$OVOK4pv0C_G7s+Wod@(Tp{O8FAYaF!$#Vs}<)vC`&Kh|s?9i8rDD#}P40 z)*b5j5xWMwrQwH9v|xT?`$Wg&TB|)?TTvmJG{?pK>%<1`Y~*Mw z<$Qbx!_DU)h0nwH17|*L;#5g?WyG|am~!6g=DpR~oP;;_NY~qC>WY6S z_R&$Wc@AYhT!J##aslNRu%s){dKn>mYXvbZ^|RTow5P4Js8|u;<2c3a9f5QX%OnV3 zew%tN|0e;dE+M@d5?n>cVYc`l>_8uSZIGRg`+1Q2xWAp?2#Ipr8Hy#;+CgbQZFFKM zM*-~Oz5?H0;+ove-)d^?!Csft@01m!Vgl+MLJ|gB=TIr(G)6gCE|FtZSKvofUqsEJ z^SCLrViqd}omH1`Gl!vD+QMxJ_`h#l)j_4!1KP|C|+s19^n^@LVIP zB*j|vV?PEF#SkVq5~mWTY%K{yxG&BvHcvf9%KfAe#4!v|%TFyQXB_$cly~;-=AH;a z8GL^tDC3?#aD{=3B%t=tQ92ITa@%gLZn2pcDseK#C*jz^Le9v=6veHqnir zR?5T8j7y6hn>%uOm!H{=k#d(yz^#{29HhDr_u>%W0kq>V)^QZ#T05RW2l+H`&Yyn) DJOmI8 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef.class b/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef.class new file mode 100644 index 0000000000000000000000000000000000000000..0f9de50b0529c007e73862e120a8cdb4e79e8ab0 GIT binary patch literal 1180 zcmbu8(M}UV6o&t^+jiTvLPZ4xQN)_GCb|?!3@O17h>4`4Nez+9={l5w?QS!>HTYI~ zWdaEg;6oA4lmZoN6JRgq?3tYN%|G+c{5d&12C#@1F4CBGk-@Zs$1ZHlIC$z{*1z!9j`X_RA>==Vq10=Eh+`QOGRrDd$qK$L&PwLS4&c!6NS^+dDo>c2B9 zRR?o~x20j~25+}gHHT{*%o7%G(oJydQAhjoy)tIp>p#oaMtUAD?ywWLG@@8Bpyv*X zgm<@O5ek=w`l+!e{e(~%0v%v%_r@*^>G{ub6V`9w(v?eTEStaX+L-0nm;u4z$7#bd zzHpZ3%;QKL(=UI+`o%vM?(%4gfigyT%%2rt6a~)MXpC`{X@0{nQTjcC&;+7mK#lYAsgY14v6D6|{j`AiPK@7-?y5l1s8=vm19e4EQVb zr}RsiHq{xO@rz&mKaS^SlaMqalRDiGdtc9U&w0+vy?_7v`gZ^`I9Bl`mSile=n(Bg z1>Y%n)QuIaDp->dRj@8&L&2tuEd}4p*j5mev7^F|$1-+R^kYfEo{Am;@PmT5nEFw{ zzKka-uH&f~A1HXH;JJzbmQ-|$Uq6_Pgo;~8ieE|wmvJZ~Eg`sKn%s^U%yGCQ;nMo) zmTe`u<5>0*Ka|jQUpIAkL4t2+cvpfyVx_o*-l%T!jY2NL?H!gdXhw@#NoMRaTNmT< zq~Fcx1UenH>~x6bSu(>zS$3FZA;+~j%jsr1#7|f*Z*V76J$Kbggz2Ol(@hBjL;KMq zcFaNsqs=kbCYf+a6K1CgyT8e@gwDKeY&hgZX zly;MxoG9oC0$CFjRaGF#fqHUJ^A&>c z)Z2Hzk*b=R&BPH;y5R)^RF_YZJn!n3>Bz{?IA#^>B!8$2S6;65#DpL}h*6BuLdg*& zG9NJ~Lxf5gY2n(c2_B0CkgzP5SYY`Hhug>877=HNz{yySJ3lu$H!VXC;s{w<$nsOM zO=pJwbBXwdqmouGXPKeOCMFCj$4IP3uGCwU>%fFsDPB2KD0Z(s zFrtHkcd1TP^Hi(Hn`;lRtXw@{zpLlC<5Esd@DpM>=~|@WVgn+1;q3(6V}_9=Z)A5W z5X%(=5demSnHDe?Mn2_SZ1eGUWLZMr+nG%D$F=f$a@;A!Ul2J^&dXcdn~~*MtWCXZ z!*#H}ehIy;AoBQ$3QBBcV~s++e4uV-gRnsnINZXj`cxEfZV&~L43jOSn%1;B^SxZ8 z()g|@gHu%V&S|XXv4o!U>?~!bFJEveHisH3K;z7C;}n(jvS_t>in1GEsgzeNB>Yv2 z?L~LEv92{u(%PW?UouU~{&3^j=TUxPcv3Z-9cm@n)R}`*rAsJCY*fQf!Y^$R(oCr; z2wl42_i39f{6(=Y;h$#8ok!BU`R}|Uw(GqbiT7zAZ|S>>z23^>L`B{0FXFzFYO3wx zCT`O+PJA7pTO4EbAE(h*)JoKLj+{YyQCgm$zj*6`1pF9AhrcND@F8Bcc9$(f5V4ypY~DgC9dv`YuAdn9+%GWae4j<&3{7kH?&WS2$0-o zMRZehv%G$d)<54X;#Pco>~LH0BYe xGvA;K_whBcy+C^f*WUg`>U~2qermrZi3im0K!kJwH0ljei#~-O(6=!H^*{4XiUI%t literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/examples/WallclockTimestampExtractor.class b/out/production/stream/org/apache/kafka/streaming/examples/WallclockTimestampExtractor.class new file mode 100644 index 0000000000000000000000000000000000000000..7b7ce05340eb763872d400e3b92b93c9f8139e96 GIT binary patch literal 723 zcmbtSOD{t~7(LToy;RY9zaP7{3wI+CAw-0QTr4WAr(@K#b34u4sqmvLtcZml;75t? z_BKI?jm`Hs-<*^0F>miLuK+f%l81-sJZ3QKW6sCCj|GO@j%q8n%aAQsE*QK6b0rzZ zLe-Y1-KZ|@Ij?I%#zP~xzTj3Rc^~xLjfxreLt`5O?{IM=gBHJT@gR0q@<_ECfxP2U zN6R?4-Z{&KGj^N7mubSVRSui{mIs;>yXGv3Y=1${>P;!!FG%H> zVZfP=5=pl|Bxu5xy#HS6YZ333>n=^1ubHkD@<=5kul~{TdZJ%I4!)1Y0+z7Mu=md` zFcd$#PFrKx{6n;3jgWC{Z18iX-^6P7v6B%)Np!81ZI?VhQCcU*qq`qt4O#jev4)(a! dw2qWZFENHQpogZL6pfQ{XI%IR zuKW=fet;j~SNILa+YgFH38?~f1*fWRpZhp<>wf?B^&0>@frkd%H{bz6zU_J;7YJ{Q zyMgGKY`~l@H+!tzW9Bdvob_G5WA>82=1aj_uERoZl~j&bYw;H>g!P)+@mV+$9KqA0 zvg=i=-mofKl+uW(lMwetA;Pgf8z3+x2dez2!MqWF!3F~^wAd-&A9l6;boYS0W2VO_ zgL};b?u1rj)?#G=f6@Szs*nakCS}TGnNq2p%&{S*lTsMxWr(n^v|_vyhXAI^=M>U(vjhqP8w6&ZVKi*6y|>MjfyBT zeOu}Lq;dP21k%Zp4w1B?ixzf8FzR#>3&o5+=p~_t$oe~2C+PPB-yCw`GS7X_LYGi2 zMbx+dnZ*ttbE>mU+30A%LVT^)cP8y)(bPpnsp#h~>b1&d*@O^Z)N2(BHfFE-AZS*M z4FaLtUa75q*3c0@mgJ2JN4Og2bDIPi6K` z6W+?~pCNn}&dKxVb01(EE=YWl_AkL@iLVg83fBPQ-|KSy2H~4D+0Jjnn6 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/KStreamBuilder.class b/out/production/stream/org/apache/kafka/streaming/kstream/KStreamBuilder.class new file mode 100644 index 0000000000000000000000000000000000000000..d3837ce1ab3fa72fc2400635ad3852847305ae4a GIT binary patch literal 2434 zcmcgtTTc@~6#j-5N^2G6Dqg{BFVt1R8?D94C23kU5v&s9!>}Fd!rrpon&6v1#s}Zj zhmvUGn=k$m|Aq0)ZmD2f10^w=%$zxMF5fwO&Y2&-zI+2PgQW-t5Dy^{L66*%{g{$u zDvThmMWEoiJl>GY%?OTRIzksSlDZYb?GWxT^kp(}^=!v{bH_xTDE)gBf&RYgLBnqUBT+ZgcguP&WDcCReJiBe~<_VU`(Ev0XEwIj=e-hDxwE|9D~X z1w+zTtGSx4sFtbL97mY0!d=VIR3+zadCE}5n~rhM;C_UunJU7~ndQ8xI>I2E-Z0NI zoQdV*Em;2y;C|Qg;WG@!S_KJun3v0bWiY&I6ZSz1uw#13SE0=qtTtbf&YCR&rsX^K zy$#wf8tuP~+0L2kA6TJEOffu?_Z#h#m~lDj#~_96j>qF-%m{ z7s3ce>GTd`I7t%HfK4(1+7n;Vv$~DIU~e6LpAo8~zm9Mnkw=NCWF1GIe?@RLF}028 zN0Rov9P>2e7)KHlNO_(nd}|F)WS4GFk@sUs+^Q@p&f#N-S=>wEbC}x9+ z@nCW)RmV{u$#}2A4PW+|& z$5h9aRBi|8|Hq6M1u{}|=?`_~MXDi2%4PBpi>>5_g|PC14kxMu^LJAi5NoU1NY28t zwz0l7Ip=$}wcD#r)ClhIt3Wv2UK7G)|92(`2qUgNK2{v_iNZcl6P`xkaJAh5OdDkO E3-BH89RL6T literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/KeyValue.class b/out/production/stream/org/apache/kafka/streaming/kstream/KeyValue.class new file mode 100644 index 0000000000000000000000000000000000000000..ca6a32441857600fbc1b06db9edd0fabf48051b5 GIT binary patch literal 1045 zcmb7CO-~|05Pb~@Iy;J@;^(fqnuwszW%sfYE}M9P=m9dx?ybRIY(XI7V8Y42qz7Z7 zi9f&}W%l(9ubXW7b*10d>U9S7zK&BU&yFkZQNau%6W)UkI#PQA<@dEb4k3x^`Qo?H z!MXqO6$X;VVfd@1{QH?k1qoQ?cGs(^V{QI!dNBJ#vqUzgv27!PNgE$9DUciE_e}DP z!0f-<$OX*ox!B^`#<|lG%ozgkYXW2ulWT}=Mo<@KXp@Y%p^efevkws2+&v=M{d+|3 z=!#;NXNrY^6`rdsxyHnGmIOA#G``Ohu|S{J+zep^3vZ$phfx*h9>iCdyjPdv!p2sEU_sDRm&xB9Ib=09(ic?#Za=7xMLL*`ZXWX87s E8#dSLMF0Q* literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/KeyValueMapper.class b/out/production/stream/org/apache/kafka/streaming/kstream/KeyValueMapper.class new file mode 100644 index 0000000000000000000000000000000000000000..97216dafcbfa19e46364134b4fb0f687be4669ae GIT binary patch literal 344 zcmZvY%?`m(5QWcB>Te0be45Xvz0UeQ|#RXv|%qV^{BCnIiQ%>V!Z literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/Predicate.class b/out/production/stream/org/apache/kafka/streaming/kstream/Predicate.class new file mode 100644 index 0000000000000000000000000000000000000000..55d6391507c09470f22cd1ea9f900e4baf881b96 GIT binary patch literal 295 zcmZvX%?`m(5QWcBswLtrR3h%iO2yKKghXRuH|nbXsOBn(SF`W{9!gv!7HTocnJ;rD z=iDFH8-P9Pl=G94T07pv^_K37 z4~LUM(Xsr)M#VFNy*QkdFUMh`|2F4JZbF}P*?XlTbPp;JoUy-?BIa$(;+QMR`%tHi jKT>=>gw}7DDuGba)d4aHWf*#|=&gjRo{ut7dlTykRw7UD literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$1.class b/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$1.class new file mode 100644 index 0000000000000000000000000000000000000000..191903b206bf5da232a81c0aefd4dbbe8583993b GIT binary patch literal 2184 zcmb_dO;Zy=5Pg#bmJO>QU{L%3H6jFL!EXs7ifBOvJwU~a9)@Irh0Shiv%!z@;N7cN z|ASV6R(Y^I`=c!1Y@)cBqC#O(>6z*2dHuSlr}N|I*KYu>;l2(Xmz5Y(!b~EGaUB{a zl(?edT9WGP8g6K~DR5*NDrL!>6L=W|(Of7##XHEe$jE?qJb9 z3~$aV{;qZ9^I!=3Rz`BktNI0tUYL^Ok(bM-Jf_|nU(}piaJ&j1{h}4Ddqo42$Qn3` zE(7f_3>;CS8&d}EsM$2`YM2q2X`*}VTWbdTF~By|1kPUS(Jqvp5V+F>$}XF|dDn1H z;9e6D`xD_yS!J&k$TX-Iwy_3{fW_5YH;#5a)${a#U7qc}R2FDqn^)bW>)%Qgm~4Xj zZuF6w5xCmK;k|KVfPYI#{)NRmJ5`kc99byg$kI_e{B@%TtUS)jTx%RtnUNic%()%J zGO3mw#51E`koXkZrzl%#4{V{Aav;P=;RH@{#j~m?!~GQca0)S;4z;mR?ppQXEcX(s zyf`ZFZ@E%d{h4jVMz+!N{sVf{$aS3aHqKi=ijOo&&<*HF(SJqPuW<%%LOQ7$#2-5S zIEVA3vrD2svUh=dX`;j-&fp>^F${J-AgfE)6U$RFkNK4`G;lF=Q~pF@X+@;uvRr#O%N=bP`ABT75KJWnB6%OjcxK literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$RestoreFuncImpl.class b/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$RestoreFuncImpl.class new file mode 100644 index 0000000000000000000000000000000000000000..6917de2f94cc15bd9507e472e4b931b65f653ec5 GIT binary patch literal 2396 zcmb_eZBrXn6n<_t$tLO2hEz~cDr&SzK$gBhEkU#p(a_izf~hrHyGbr#o6YXd?xqgT z*iZdK&J@b%j6a}1rK5gv939`!O#Bz~9;|hVZ7?EJg4Z-j3i3mc_CX#yBz&yn|J-yeo9?2`L+)gEbjfW#nYM z&u~gNy$pA`qicr#Id>S+Im@Z4nyr<$xLVUTYnocN>UGOh_pLNfr(M?cc$GVym(vWg zYgpd3MxB9W8G_!H?#9o_xW+J)H}r~bR&VKM#rljv$(km2Rt(K`xy!K9AJ+98uGQ&^ zT5E}Fmx~PFjCUz0d~i`Wb#IAbHSs_AOBNXXD^`UQPdVM>gsa4z4XtF5CYH0xno-b* zIoh|CegT-Q*SYIi4qt7U_l<7xZGq(l%2s>TxJNAwmnhNptZT_a?*x9y-s{+b(^|kjSdWMR=q6- zsm94mEO$@4uc?Mcg}z?8$IBie3*6U?2C*zrPGT+48p&G?r_5J%LH|fslGF<`!i$12 zOei>m>k6L5a}1XUZe|4|7-fKh8@MTBT|pijGCojHz%2zI;I~6dx)^4U2xB`|nY*s#s2zo5xN^jmBOS{2;GlvIA0lZh8A3FW3hmk_k~BjWj*%$c;f|>puG*R|`G##< z4z-?KbMWn-+l#p#{V6`a<-$`?WFpN{bz5T*be5xK56p;n`Aerj)~5W zYa4cC(AKOa3z($O=V>rJ0|`;_deiq6Vom_jsF)nI^U?1NUL>nYl1gMLG~=nM9kA46 z_)^jiq}12&-+qk14uW6Osf3qkjnQ`)_!eP2#Yuce{=Ub{q*2HhdB7`pm7cG4Y0lD) z5Fpu#$1(`eec(YV^$5Y3Jhh9^F2WR2TAH5NL*$bGp%fC*iTU6|$v-_Y5+qCZ2+AIY z4y2QNIJF?t+1=)?BWq|ViBSAZEd7Ewe#Ke* zh75j37JuRfLCiA-n@RYB`J1@c5fVt+xP_F$RFjJbm?p_b9&31=*q6v}1v5~I!8i@o zS<(f`_X5u0JY?9!=LM2NP4*_qzU5c2*Rr~JU^PcpKFrgJ50~%)J!2$|(|@1mDRzvc HIDCHrf5??9 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow.class b/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow.class new file mode 100644 index 0000000000000000000000000000000000000000..f884a7e9ea8968e4af312cdab626e2859da2d845 GIT binary patch literal 9359 zcmb_i3wT^db^ed`aaX%nwyfB)7l&9*BuA1Ta%{(T|Wy^LFId+9Z>+98(wAN}@ z?5>fY8tYB?Stl&<6$DLR- z?4R}Z+qL)(zN>WqV&Gq+SfwWYTNJCc{azGn@O@qI?=k!bUNG>37}n#zbm4z%?|*dR z|Hklt_@R#dzkwe`aH$qsRP!I>C(7?7rFmJ$F6)D`4hr-khSWqP5XBw|8WJKZ5{_b@ z7*UBxtszlEVuqN4y53Z>KbampnoRd+?iHBb>9n2Qk+O0*J13ZXmz~RJvi9z=bfSA? zG$jb8GKpb9L(fU;l+}^4(t{m`PMow8`Aq^NkxA$6`|^SfJ(=uahc#*?hU|`EYhc*w z$mO%PH9}DxquETt&gC-MjywF`4tJsQ$fYv*gJUBC=~ijU9N9Ix(#G=1R7X!TJ#6>) zaGcb&y~)9}l^@I6g2feyHXq)I>B^In`Zp6$YY`Y7qvy6(kXs1{ z`f14FuVx@pE_@s3;EYQ(unRr;>^MuKP6olCstD?@X$nFzr{!OmBWPnYC$k zu+?4VQaEXh38$JP6l1CASlrr?QM;eOL=$8A%)o$)$Chj4;{y~-TMF>}R?i6>$LP&x zdBRXsXdpF~8=}bmWHx`=Q$6IhUDZU-Dw|V6P!B3W&9N>)gxGhT&g+R8Nn}P&FrgB@ zFm27*q>0cU{pqCVrqsN$^M>qnKFhrb?Q+`%C1ebcW@ni4>xgi#%@mzV-md|2Te(qG zWh9gC$az#&KFPOko>GH$wz!nMt~){Hw`a4~>Ef`U!l74@qnb8jF?rS|GN}~nf(&)L zyKwoH6lp=rd%oDc8%ZO}rj&Fy#hW!a8ggBCcZHEox7bDNE%>6K|56Olg)D6ThWW z-+_0UvP^C^rB&KYX_vSu%Q+z{)X0^x#FSM!)*-76A){Z!FPXAN*0S_3Q>CAmqtc-?^X8$K8&lTFt){G(4TfwqdcV&i@$4k}i7r>AmrtFkm%6GRZdt|GLH{)(o_DZ+Oa%MkEnPSLvGt~}S zDIT?XTaWiqP4%oZ{M2vvH>K?KV1B5nKV#>b(wThIh?P$asR#53R+lN9NF|v)a(Y+m zA4}NIRn6(Rnc0*B^bIaP$P8r4A-O}ay^Kk@)zv+cmHI4rTmd=Nv*di|JPD%zx_+O4zFWTtDg{>a*cdK~a19;m8I z(HzH{_H)x4o!>jhhH9D`c@Evrz&m)5KW z{qL!|H&;@3`jNG%r1&U6xX&r$Ygvxk*&a*3axAE%6Uzh|TTCjRV&x+hwGUYRta%r< zc9+vT;)6NXsZKeauhxBM&!lTYm^OT^vDtlGpo^+@Iv3l8VaNal8pjOJNLhuv`{ zw(VhWIyB&t?WEJq!bh%O03;#Uy#Hn3U~E#WJLL-=X#uj=@SdsNx#E>-$%*8JBf$Bg%uJO)rcM#cQb_rG7c< z96#OeC5^d{H_I{m(151L?$V9+D5&+cFC*n*eL_dB5A8aI*5D_Go=kd>z1Cx0-TT_~ zgDc1M7OMltx^!e(s_@wBVN{c3xc8Liiz~}6N1=Q~c?NU*Tq1b5!k%BHAgORlU2VV1 zpNF(hpr(BafxZa@&m+Wcm|f#MBJ9?(8$FNM6C@8{kk3Yh z_}tH56MFbt!67W6g6;hEVaS)gXW6oR6OSs9vDFgRj;!iz-`*50bx&r$yM+PfbQkjizAUuN%D zO5}$N@;&i(Qw901^HRe{zbIEAioL@Crk8LRKQ_T0f$9pI4G5igHn~^P@{KS zZyzS8he51bC_A=%9IsN33dh6GAtaAaVR2u3JUou;n;L@?c(rbh>sSpAc&}qzr-m3WZ9IldL!dHwIf8HVCR%v$dt3hyjn*otNk8(LF9GgUw{ zr59Ak-BZrS2OJDFjNO%rm%Gw+0bQwR>e!p9gBzeN2)u;t1|B4!ml-Z018;TS9t^U9 z7WAzMWp31ta1mI^|19F>cEO|+|0kN55x3^ z+m4>S{9G|n-QYOkyD<;%L7eHW1;2|nJc8TsJ|?T*D>#3gsplal^jqk2>OC6rTRepP z)Z|jBy)IR*K&5D?ABCv=#u^Uul8d{1Ge4}n3dA7e)i zm;VkCceiyXTlJC%ji-mNoSBZ#IZC=mS)>H@;)t>r?w;o;foFx=7^UEZ)Y9zBO`PvZ7*Z10S@i@SUtoZ8{C zG3(8E@L2U#kUKOqHYR%`Wp65 zp!?#=DeUX>B(X6k!Ea1dBp!>`O`vB5W97iv%llOS1!rIWLD7{47#+YQH~J~Y>(k6B z7kH3gWI6H-dhv0F>a)zqpJKFsnvU}vci}Vmef~R*r|?-kj~DpobNB{6kMH3NxP&iC z4PF!zUy{}MvaH2dWFx-H&uw3GcKPi*&d+f5dl`dKwr3cJk0M51r}*|Lv)l7b1@C9C zj(ihz@IJ1%2VZxxl;P;{2b_)Y{Q_P8gA9ZH_`a9dw&6$kLk2-eeZ%vlh^%8s{0aT1 zo*zFygby=77t2z71b<3r)#%>J5OsC>D0OmWJwOk0B|Kh`p!v_0@I*m^@Aw}<_%f4= zo9sSj;Li-4V}$}-W+fMOQebh^z zubMJmI-p$V8mNg=e(_G8n0G)?TJ4Ngm)m{!7(5kDu+Hr?9Zt=8+( z@=PFsgoOkW55Q~i5dDWZch_sLvzD?k^v~>_@tkw-J@?LJ^4CAV|4Bq^^fW^k=m8JA zJbcMRnTI_d9`f)gOF8;Vp~qRuP(`7Ag$@*|3VN^BGL8-NF}Vq z1D5Gmvengx#FKH=JualRr~4suVrvoO3Zf9y<^(W5p^I)pLBdJ-yrfbk!!(>7L4Oy* zh2F13jC3e)@tq-pIY;#}f?2Agsms*1U_j;+f!4ujV8Ol{;a?di`@=c!@Ccn4v=I^yKbOW)DIGOEj0GS$aQ5SLtg(yP;Bq zkH~2kiz5q6^hD6=1Vn;nhK~$SJV7@j5DjDz6ff{~FzLCgBlrt=C*koz3S@yqcwf!3 zK<*c6xnJmqJs-l3Q69fS)WQ5R=1_Vtzk)gBAIujh3;Qa42JGixcuR4-$cS#KzfoGf z(W9ig&?7~iOY|tE=94{2sQFZn;%Z*$QB2LJdnA4VAWn-Iuesn2O3*!6n>0mRC~6TQ zm1u#s@%s+lr2F&*_I^WYc7(O-__DaUdi8HSf+%3UnGNpN+r zp`Qf1AFi-_T=jIjukAzNd3WXy%6tcXv!^K^2PiWkN?u_~FH&tLP0Mr#gj_ZaF8oN9 zpRf>f%i%?OJwThJ6_y*|RS4yShiRsMj09e%yJNtvMrV88Ky!pWCs>WM!JA{j@o3;J kv|`wEEE>2p7AzuwQFpIs+h~7I@;*)D%O{Wj8JZ&TFY2>lLI3~& literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/ValueJoiner.class b/out/production/stream/org/apache/kafka/streaming/kstream/ValueJoiner.class new file mode 100644 index 0000000000000000000000000000000000000000..409d1d1a22cba500380192f96c4e70c752f8c24d GIT binary patch literal 342 zcmZvYy$ZrW5QJy_i@)GwXyc(+h!IN*5ri17U&9$=ViL|o@YO7Q03S*`wGm^nF!RCe zGJAhqZve(hi0wIXzSt{mE~ z9NB%>u?fyh4yVPC9Q}dLA2u&$3C&HCYfmkL$Xx77j&ik3f>`Ons{&z=XkT#3-d>52 ucOi>YrWGFpQ!V^R@s%<=eyi6Agp&CbKovq672~VMl~6PDQ5NcN(s%;6_h4B7 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/ValueMapper.class b/out/production/stream/org/apache/kafka/streaming/kstream/ValueMapper.class new file mode 100644 index 0000000000000000000000000000000000000000..3018114c2b2505c17d71fba4b0c3a303fc5de26a GIT binary patch literal 301 zcmZutOAY~15UlnagN0KF8!yDlh@};=AaCC$&zKJ;&y2*?EF8e0L}MeyqLZphSEah| zkLwM<2rUN<2Teki(=@pdI^B81XI2Rx0=3*k#?QQ7Q6jjjFgS3QTSI7c*LvXf)_Uj> zrV~9F7ae-|N1PV-ByDxft#4*wBGGNlNp2R>&e;B|8KHl)f#Q_?ol!AwV^*ik8s3M} kIQ~HO<&uowi)sWyNge=Dg-}LCcvV;lHJJ}`P=71#6GF{VzW@LL literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/Window.class b/out/production/stream/org/apache/kafka/streaming/kstream/Window.class new file mode 100644 index 0000000000000000000000000000000000000000..3f62f607706c71e0527a8bc88d17b8b540ff5f0b GIT binary patch literal 627 zcma)4%TB{E5M1{an(`>GGuPyTFPxgH5+EVbRstziIGf-EH)$L>4)ARp_y9f%u_l5_ zX;g8r>|M?5&d&Pt>-_@&F5uLHBMXi#I3dsxna~7A&RnXPvYds_oF*(v7%jBoY$dXo z=1PXVD5RqIzprbVY5t}?H^3T^$eswe&V0#U7)=>c?jDys)ZWysa-|k2z13VXEtN;0 z?~6ENx>TINnX{NR1ILTNn;5ida2X+;fNkDaJd%dm&Px>UG&F9Kf>LeM3#(3z?$vlU z+QbD7xydH%e}4IzY3}399LKPIS*noVh!mv{9&ir%dTuO3@3y#~REwY$fxxitG6A}U zzFyv1OpgG87M?bKh;9d02wecYdthO92eU2cV{F&rK`q{ey+8auaw_Hl41X#degp7e Bu8sfz literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/WindowDef.class b/out/production/stream/org/apache/kafka/streaming/kstream/WindowDef.class new file mode 100644 index 0000000000000000000000000000000000000000..79f3b9e0908b7502e35d3fa98c4b66fcf0ab5d95 GIT binary patch literal 386 zcmb7Au}%U}5S)dR15`r6M`$9v1{&`|MQAXgID++ZJdVe~JM(zxzft%Be3WtV3QeTY zVzZmc%w}dzAIEn9eY8T{gt#RHmUo)axx0Vb@GGk=+e{_i8Jk6fy0LrDHWfx!7=_Ce zzi_(ND(6+sYVV!qop4oNp6c1yW$`A`Mq)C{{@_GA_>VT8O`_R68p@A2o;?1rHC=3U z>ZADuZ1|f-XePzMrFv|(;-OVK8|hW=9F*|z=QN@rw7=^qebfkqinIw-0zwr5Ao~h6 R!NW6(x=eXB(ELKKKLLaLZF&Fz literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/FilteredIterator.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/FilteredIterator.class new file mode 100644 index 0000000000000000000000000000000000000000..b8f43246e1ebbdf2b0c56791c7b4be94a1022ef7 GIT binary patch literal 1639 zcmbtUT~8B16g|^!X=xWl!TNRhs#OK zq>;dM1~#t9a8w8x!%xDH}sm$gU}6h*b2%D!!>Wk*I_qs z6%G$LQmNO<^K#C2UxW*~+2y(Y=3Gnaj4??UO2CCFh3wzxv-#M z-EJ*0$lj{wyN|l-i*8WYi_Iu(tD&0iitA9m*+5UrY|NlZ)$h)V}#30T_Kp~!7)tyIF zky_;o≦#p>C`_)a^f1MPe2zUl^osV#A1^$QO*#fxF(?pll4r;HhQLUoz1xFLc+X zAT=q-awLU9u9sYY)dm8k^1Ns;o>b~{yK|`x*qh&dY(V`*D`}>5t5vGg8suB^A*j^iN5ET1}X6c!F+7!wSCoFON1DDp+u={dp`1NGpK z{^gt4_Kj|*a*P(`;wXvKIV;X1#dnB(L(`R2y7Lt{Fk73d7PHE)B&!oPj*=6-cqvM=jICNl>{{k&-*q#3zWxH%0)X z`e|BckmD%S?hsb8Pklz>=@y2*(l)4u)pQ@L)Gk)CBN#Dh#a_fs-WMv*i@!4ZiQUyy>D6%tjHiYQ13Aqu|0SL~Q6eF`IvK|*-hNbK iTf)o#4YFqt9`wy?bTpw`7d^!;9<}$iQss~$vi=1+qD_7P literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranch$KStreamBranchProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranch$KStreamBranchProcessor.class new file mode 100644 index 0000000000000000000000000000000000000000..20b817da500211e3a4392e32163b433f1c616318 GIT binary patch literal 2147 zcmbtV%}*0S6#va`>C&=DMa9o5YL$M`r3h3h7NrJ5sER4+K~KZ7z|ykYY_|vpZ~hZr zy>K?sARf5u#s9|`^-WuvG<>9Kv+2y6*EjF|-tWDc`T6(zA%F=y^P?9r9}RIod>Hj3 z0j&>XNDAF}A0A*r!K8vI2CuteRzhP8GdbH?*La!h8-`ZoYelYAT*u&BrnRmW>!M~_ zuHjg`RM8gmwLj+^Zs{9o@-t&vrki1y4R_-xQeg08?Nx(eAZJ>}%j(vO;Vko&5@`l= zw$4ih?wDfTPdMBIU^`l~ zHre_{0y%jG$kR7c>^0kY$DLJ%a9bD6LZpCUdTV_29zp@0441{QWb?NR{2ehOuMql{C zZ={K4i6(vkKa}y@Qgk@h4fmqCy{G5=&cpxq{`~dr2Y|;YrVz)9js)&#xT{0Os)n3~ zdm8R*c)*Z;X4}Hu>~YT%9z(X)@Lj?CPhD=C`#Y{<3eR&~27Ta)UCZRY^fKPm9k*lf z0XO%B(dBzxZg@eQWp|A3=+)R6N0b>-4Xa~wf9MK^C$mLYnzeGXRjv{xQL${xuQIIX zXJeELEr!^pvrFmc>Xt2@5BqK5HhH^8mifA4@?ML(mfVNNn7?mPtCh2KYT3SUZQk>Y ziSbdz*-7;K!5F`F4(^eOBrV(XNh<1MJzt1o4#rxNE}^PapUzFGaIUnr#?a440mibP z)NEl!Nf?xOl+>xzHtGlbEjN0c4$q7BfiV4YYl^LU+%2l!DGhIlJ=t@^8M>y}wgT-Z z6ni8KN#g?2Y0M$d@bV1j{DgtzCdhw=F2MzcjsFKTWd5BOLn(qceG+uGBVtZbJ6B>* z^YX9yaIQlWCE%$g&|i%ey5ACD(CC?$!qIw>u092q=(77r!_JW9GFe`cO`%mU9zrQT z{0jD&b}B_7@CB9-!)>Inj4Nczj=ZsetF)uIke;P?nMGO_aE(@mMO+WFY!f5+_8sc$ zB0EIv6M2lX$U1@5fc}>t>BpFvByQkl$TVn`k@4aeFo{!wqDQ#JBe*zj1;wi22$rZF U>Ag%%fnpRSX$8Sm8k-K~HxrJ!(EtDd literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter$1.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter$1.class new file mode 100644 index 0000000000000000000000000000000000000000..aff4288cf95310dce4443e24661e7578b965cdc2 GIT binary patch literal 277 zcmbVHOHM*T5Udt-;7Q^OCt$&v2?;CW&Ol;}alr|ip@GL6X2{IQ;aqqC4`pn0<(BUB zr;_Tb^w;;tCx9954U9pbP_5#7a~w)yu`S+@OKZ4*=^u+R3N?08BTJcl6Z$#c$<3H8Ovde-B&^+8-A h%&#}7ML)_-!|zy!)AkJ4 z=2k;ns3w=<+>jy!&s`k#do-7ajMqa)h$E!@<%lyyHd^A6RA96&d|5RttRDJBFAE_ z(i*FHT(7<6^>(_W6?4PpUehJJXv$}puF9BY_!Ow}0TPwWVkkn==~ww zlz_7Wb$0qJ&9v+qYo^XUgL-EC40-6-)3fFExdm6X)ba7m* zQ|Xd_kx0&*Lh|H`Zvanb&Ck`NNF0W|wbS|lk%yOPQNKq#{h z>08StPHTj0fH9n>Rc)OWByfqOp)Sp3k{GUFn2-eKSA@w6Fq-*}&`*T@tuH(N3lRyQ zzhICwvJm?v2zOx!cZ01<8G^|oNk3EJcEks2yCq0m#kCfdQCz1}s72+_N>K5wZgi|} zlGQEJN?=I>1c|rFUkrE2MDVSWM(~Yfvio4i$M3s{bcjTVqMwrpk-FC=a-c~-5Ruy6 Z#*k`xP2(c%39^k+co%5DLo(w1{{h4XR5Abn literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter.class new file mode 100644 index 0000000000000000000000000000000000000000..bf165233db64b63977c3cd037e051686d7134cdc GIT binary patch literal 1844 zcmbtU(M}Ul5IuKWSX!&ass$AV6}2tOQeu3d4L(3f42@tQP2}yeTwr0@ZFaXtAN(U9 zG?9>K;s^Lq#&fqVQY{taWoPc>&Y3f3=I)Q5U%vx*hH?snc%(m%(=bsmQA{I=l8Gr3 zPfR>DF)fgJ>3J$xaAg>(P$0j&76nQkEIDqZg4MvcRT%n#K&la_EytFT5}2;}!Hy*x z(%x0pzTDoIRv2eF-j223a;??Q{R)%UoE=X_&49_z&y`$VU#_gzEAtv;JLa^~j0D7n zz+lz!oM>KPws0=U%0{s+kX-P$$R=BJJoT!1u&IJ|x#=>JtNFHc>oRckxgAVKyAIW< zo~4lEQ8`b#q1B}&JIHrKzi)Kl7cYQ&LZP|y32~em3o_(*VI)1963iBg{RlNWTfrH( z_k+!)rLA{mvOF^*@ZwBNPE)GL=^F0{)4`n8nC`_e`*qZ7XZ`B(Ty0Ojm6j{H)+?KP z%8n}a9!ki2&_#+S7#$++;LQt0N=0r7y*|vd5%u(;FkUh^Z{a)PVFi zB;S<85e7anqs3y7cYw*bawhiE?aI2!?+B|550UA38`GlInFFOSFh0^T;-*?~qbJ}7 p*bp0G@;%(|kt{Tqe;HRLRyJPe1HRIR+W0Ida8|>xk)ykAZipbLwQ`t_2 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap$KStreamFlatMapProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap$KStreamFlatMapProcessor.class new file mode 100644 index 0000000000000000000000000000000000000000..0838cd1f3e95ae2eae6425417b94022bb92ce81f GIT binary patch literal 2516 zcmbtWU2_vv7=F$s&9=#6V~ef$MU?>gXuC}XK~e&=Sgi@AkP;NHPQ%hJS#~p<4YXJO z4u65y&fs9ZaJ=>pIQ|vKQJ%B;fDD9*%|*}0+vmLR^L}pr`1jY}0Gz{jaZKXe1ef!1 zB=K1s7jQ9w&v8jdA%PfH61a?_TCV80s({xf5XY)o))KgmFLbQy_);Ji?AguKlECG% z=kJ=bA!~bZOm*DOw23NLA`NDRUf2C#a%Z1|NUWJUNIyv_E=#A zF8m*rbhYCml}DA|YNP7{$0~uW)mdp_&UJhxFj290T^Y1|nn|R)QOZ#j1 z8&m4^wT_a3GH&SDG;kAJO!$j@Z*Ux_PBd@}x0w|K6>J;00|p>B#60O1lYzVV#=s{y zWgvx9I=(e<4<`jGLns-Th<@jK;oz)0iq@Cjw9 zG0ekEI@3n<<%iVtJ=xr}_Jc6g_c&VI@N&G1e|c0AUGw;{Q|LF&u+G5se6v>{Ywd@$ zc>WE+82pg7A9(%~>EFj3zuZZF@VA|tJ$Z}enB)he~7UNen;#Oy1?eb zA;txMK#GS6L3#XAl+eV9=u;SKId+J*BKYyYzZ91!6qhOPKn6KvQRI3J^C7P+nbUR4u;+!X5<_|rYS;@`%f>EX{3pQbbc?H|nQ$ox&?Sqe1d7|&@e z5UQN5QWD`F%jTXzygqnyZA6b+42yI~1vds0OC4(M5XXp8gd!blc`Wm$;ZwZFeTt_^ O{#HA|eV%tD$Dacdu)RqD literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.class new file mode 100644 index 0000000000000000000000000000000000000000..5acc280670797211d2dea1460baa87f22ff3e829 GIT binary patch literal 1894 zcmb_dU2hUW6g{&*`4C%dm0Gp6T2!EkThRF6A}=*04b+-INPHaDDc!Oxo886OC*Mu{ z1^x+5G)*+|2l%6mclaz35X6w>&dkm|=iYPg?EL)o?FWFz*bO6qI}t45ZV)>W_>c(V zUJ&<#co0OAA$nk0!r3>u>k5})y;$}f!J7q>dxzY9>9lm=x-ExcsmX0y(BM(2<Iny*G!R{2abTU1eDeCogd|uGc3c)GdxMogVReaRb}w+w_a0LD<#7cFWSwT za4Iq^hSgF_=Vp~VhMe~n{oaW|LO+{@%&;g$i<_=GS{BkXK8k&2_wkPw0UpRaREgok zKL|4b`p@-}dWQ_l^|+;z!r5eMTDo?>ebqrDlXMZT%cf3b_VyT@nUb4Wh^h1w4})QB`^f(S(W7@dzr2Y1l4jWt?}cX#6|uF;JAdg$?yck6?9%LC@?G}^!o`j@-~LS@qbJAALw z>>U12H0$!p$bp>>pNw3xbonI9aD`Z6A>72R9;QlG?hd5CfJr!|lSUVA;{sd&o1O4| ZJyEtOIk|hA(ggV^2$7ZGJ`yql27_ThkbyL5~E(dZ9=LN zD5}WycKHJ^L_0x}phZZR)yd~FEvyi_vt{(chpFW_=i`d)NkV_mXC`zeF1J-E+;%Q; zSFIDIgmLA!hL4=@#FV@#*@UNos!aJ>P2r+=$2ORE?!5uw>&H3qW6L2(AY_j~V_7{G lzkA6eVf62u>>Gp+{ryojZRl0Ur*a()#imZG8d65qyaVJBRpW326g@p!#+Js35{yCF2%tDxY>yQj5+cPALGhoF4`6zV9`yPRH{F zj1||l{E8zx9gFI6e5{#6BQi5H2jM<#4-~>*p z)u%c>Gq8d?I#vzTaF_JhW5=M}2G($o)iQ7&pBwlBbpz}8Qb)tUS2!o|?buD{@P{XG z^C%I9i5lFh8`zE+-YQW2qjD6d2YO|OWQb%HIc|;ZizSpz)!H(}MG~!h9Nf>n3c#4x7j9h%S`d*AV)RNnpZOZ6vI~Ot!CapfW_EZ^_P@wG)Kt z{e}(34Tb4}{{B`+?WV_1n$o^@*oFHI)AP+CGb%kpM!b5+yl{m%4^b`gY>wE@<3gxa?&D)@yv03*zsww5AhJ`m zeH-r(n?k>5`R>qC-J@rj8CqT%TF%n)GS5#y`v++qSsl5*=`qhe4TTh#)p^>f3^ceC z;T+Ewo%HTh<8P0Q* Hk(&4)sb{p% literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.class new file mode 100644 index 0000000000000000000000000000000000000000..64071b46f180db4cd15e9bdbe199f8f0d8871dfa GIT binary patch literal 1812 zcmb_c>uwT36#iy`auHjsimghm7L`ll0{Y7(BnCs$fVV(M{AGr9O1CTv*H44G!zwFMtGOzuA8<1RNR!g*0>V8B&p8fjR4b>I!vX*j`9gvJnW8-0tr6I(F6S~0xZ>6W!(>xjSOs>z9B z-xW6RnSwSdT8(~`Th*`(x5n@=w<5Zh?=pnzqqmgVM$53o^U1I$?2aU0*ldk-Zg#nC z$oXtB>>e7VSA7{K!=hv@ZaS);x|DhPEW?NKoMkDQg5j&3W?7T!c|tLCR#oSG?WKG% z!*X11=_FVsmk*L%;3CLO(+|x=*`dQp?_D}3C`Lhy MtOO5{kO?S%0FRmpa{vGU literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamImpl.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamImpl.class new file mode 100644 index 0000000000000000000000000000000000000000..e65190974e329dd4b938f5ede5f50cb0ad2f8b12 GIT binary patch literal 11028 zcmcgyd3+ni75>(iy|SX%U`W6~IaBH*jz}Crh)lqC93p%L%Q$hM(2cCvie*ViGHGaO zq2cNcq(DncLm`w>dO!dhNJ&##deFPH^uF(w-t9mAgZ7)9-PP)Hb``h3*Y&WAJbSuZ}P;&~rFk1wdW&WC0AqKYs1@B+T9;(9N>qT;JQEXRH?z9x)* z-HUIC?{A8aZ+Y>eif;>;-%;_Bf<+rc{Ug1@R}FRz^eSlV->F@r1>#y_A`s4|V~L5N zg2g?_L?){xvZGpjCaS^YBnlZ#X<; zT3ysV+%?#<$xv~OZ1rpk^=~k`7H}%PEMBQ zE8d+lm3LRHwR5B|I5HYsPwllkVsYA4aK*ednIJc(qvNrNmW>8moLDzJp`E&vDXCzE z6VaTdBg8@xG@EyI;n+k%%g&@ZL2-8uT3XyVY6;O{N$k)}RzdZ&mQtWO0?-%Tt!IV- zEtO(CyRj~&la?r^pu-8r1-2BV-AfrRyAhGrgih>+PUn=f|~RJmch2(c*^ce8HfOtYrMyJ)l*wb1iwQb0Dm5dlhA?yA49F&Q}`vlB^0i7EC!5D?w95k~&9jmV-Vg z=GL@U1lUMY!IBaQL!9Avm*{thGZ%bLIk8Cy+j|zm_EF8HK%p8JC`US!=I&WGn>Q|Q zcP(5feoGC7k8m`#h6eI1D!AL-f^l?tlxL#ic=L8f;>KN5b+ceM)!A3-JX@k~y{)1d z$KK*Vl=nm8+RMD^D6Z7Lu%1ab#>7?BF~;xW0%?WF9DWSpK0o>~;Kv}=DmbAqpBUF? z3f9g;oN2z~O%-cMU)RRIu6B6^+U~~>@IyZ?L7yK#!jDDsCw}}CKV#yo#0FjC$xL|i zb9z{aU&Sx{_$7WNe&5WEmxRKr({`4JO3ZQ7BXs>*u&KLc-OWqhhNdwiZcc|}a6T}Po*!etGYs?BR-YOQDnPR*i|cTBopI2@02KgEw%(GcQ(0;{Dz5Wwkt6nN<`ApX}&bz zJ!*3c4`rt`hvpOGI|Qjo!ST(-16hw&IXcQ(2dUsTknhe0dRV^WP-o)D!?D8*3b~k$-I#6pVB@64a9jv#EZWO{tZRq^uhV z-4Zu!m9QTyW@$b>W_ioET@#5X6D+Fvv1Al$5lfEF*12F3%-PCsI=Umy+wMSDI<4`o z*P=vpS4e+97@^Ix?a|T`XGRF-g}#MCWt4?ui75py+C|K>LSeP6VDDN?T*}g@}p>`7Sf`JiF1@My=(dQT|qOB#Ys31 zYx#UWU&~iPWqX~i-N<&()?UJPr>%V=+ZWi{C$qiY*6v{YLRX)2PnEdjNH_@Xcb;Ec^qlPoQe~K`b6? ztDZx>=UFt2RkN=;Jd0zt97OGyK41>VK1wwbY`v~>fW8lLo-efl?XiINU>hzclqR%_ z1Azf_DZdHE04kog;R=p=EqkkQC7-UU2)f4|^ijgvO6c1b6#DC+8LaO&SnoDjZ!E+5 zabsXLspI8QYlR$ow3*=W5d4|Ym$|85hdE7!i&8I|p z&QC6b{1k!wRW6WUOXP1P@;8-5?u_%Kg*?TltC2RMr6{|cS_b)P0{Q7h$j^|;g}iw(LxyJV0hlp|%A3@6RZF4y1&?856MbY+xFw3drupSeWODg%AlSlcXK zeHdrY;WY&gG&Z*#dI~Lv(aQgW)oVOUJZ*>W$AGDlf_b@iU5&`4O*IFxoVmDNYFn-+ z+V&+LGuJlNFxiH0^e7Mg=Rmul#iLhddb&2v#Jih`w}uttQe2BS=oMz4l`Hl!GOp7pyqWPK5_E{Qzk={>^$PGx<#%cr)itFG5Nyp?f(vWfHTZkeFJX znA*1G1}R1X3a|jlvUN_OgsMxZO`fJ2$zvONTxk`oCN-~OEhP=W1KbTfh*fxqlswEN z_=rx{u*7B&lftc}Llo2$-io);2l4cFK8Zq_!aFz#qFAT!Ek=Dch~Zs`#x{QTZ_4w)$W!=}V)DiZ+oACcX`HS{AmuWOpIyztTiw z^uZtCk20QFph3|DLYTdC@7y!@oO|czukSwqJV7-F19wbhaCZeuxR*sSi~CtT5HN?* zD|Q3dSE0aGJqV9&*_Z8(vU~EVC+#Q>l{|6%W4kvp?WXhw>Q$1lE3i~^eK+0{cwAaE zx!e>m_JczuuvTXQZwDtW6*{uz(X(C;+R|&v(AE2~(}+86Bw)StS?<0kqeyXEo9ANJ zk5%YPFS1`Z5=xpuahnV>vfuYk1)i4bT{)DtCrS38)m80ych*y$bx{P0I8Zyf>KTYh zbX%bCC#Tc?8864#VI_v+=z?+$_t=;5AfybdB^}cKvh$9ygV_t_VL#Qg+60hzdqgxF z!60m_7p_)4KPAqlroz@d_uF0&aRT0`xDy;^@z6pEWeZs>Tgahe;XKv_p3N~k|LFoR z{#!SJ%{lOb}g${ebp3ZwB@3$zv=<^|h8<O@ z=`=oapJvVk<3ob6F*-NK(3lr+F~PNPiB<{)W)zZobnNn3>fgc5Pj>qb!!3*1;R(6EfR->1EvxOR9it+rAlccIK=9GlWx;(+;z2f5d17o zNF4Y9{3yiRxJZ=blw5W^Z)V<`+427R{o^NqcX%0M1=}HfYzKG|U`Jpr(MFeTft6JsPaE}day3vkm4l4PRxcmQ>{MFq{2S0OFLXhUV{Oiq6{vNLQT8a4MWKiv z_HsLlI&4*zxn;>lUyqC|Czf}! z+a%T8#AGTEU{~NnbMc1vW49ac%UgkD85Wi&-GkX7(>YjsHccSx=My_r$J+f8^?w%j z#$8+ut5^%L$6=RTJjwsxguubVr27Kz7cKVp_-1@QAJBSkLI?!(4rvAL>g(_DzD>jH zJY5*r!3NJ-m4Fa;>D{}9Y{IeDai2L`jCgoZCm9~mcl2FakJizP`Sgt{;fCH7H}IHc dGeYdVl6ZorQ>kZc4a7V~p5u-m;vs!t{SVU@8K(dM literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$KStreamJoinProcessor$1.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$KStreamJoinProcessor$1.class new file mode 100644 index 0000000000000000000000000000000000000000..44fb267b89630d81d7d76285152399ebe274cf52 GIT binary patch literal 1689 zcmb_cZEq4m5PlY}Q0UoWt@T~8qViIEQfp$QsUJw2lv1lH!OsgUu;sWTcSmbK_@6Yf zG5WzD;EyuS9tCM^nv;|Zdoy#-JhL-9v%h}-_z7SQFJqX(W3@bqBZ8+hNFx(PHi~72 z_#rn_N78E8M+~{L?X(Quz!lQh3nc5gT8H9!YP{Eb%o2Y7ldD0o^UK~y2co1YD^ySt39ksOL`j&pVF7i z9fa<*6qzc6R21-L1ocU2@3x7b<=jKr+0WNm zq9|T4yqTDESh3xS8O91Snk4F9HAf7uFC3%ZaI%e&XVIo{{pb}k)hN~|u?zi&SIGJG z$`c9oXMn^8bI0&10QTHil<1_csXUT&l}6}l4J8t)VE}ql_b3bfXE6urJCdrThv}Wj z=1#z}%O?nB**Ap0kS&a>v_=RC6fsZhqVJu+H7t-uUOp$ob&_=-_xBrhCZSb~CNz2o zk^PELF8LjrhR>vr^Kbh6Nq;Z;h+oG|-12#K+{PW^Q;`O{IE%VVsC)D+g!>e0ASLn8 jw=Cfiy*0#jtKKMiQ z5BT6y*AfxCmipv_zrY{iU+}W`OaejDtgI*xnLX$1v-h`e-yDAX>&Kq}4CA;NJ(vyR z8s2AVj-~k)EMQSVx(Q1`G-8?GR}`!&_#lWDe8`#^{#oOlkAk?4^%gwD$3bl15lf%6 z;8Q&2-Om&}QSiAy*J8%8HMzfF>E^0!Wwk=VvIW$6)70!~Llz2JK_Gafn_Je=vfS4M zI@7!IP$mp%<`Q&RH*+b0#%4Eyl$&(WkNS~ zXHsA#Qoot22_0A$2uxdB6m)x9H?`%1{Y}kYlbeQH04pnvb!qGTUR(@tI0U0^6ydV; zpLYU9*pt5G?PeaX?+v?CASi!IArU0kJ^srqdh}A$SH49NkFF(~5K~n_}_P z z3GEUm@`T0NXimirbb)~yD9!5t8paxQuSCvFTyYk|Nyf3O;#J&Gk-(6OaqOuuuuoZ6 zVPac>r6P|nRM;pea8w-NP{k4M3M~DvU&vQKSE88O zn$P z6Nu<3e9^%Z_@B9ox9PXd-5y3WMo7Mk2(9(dAv%HqMCr*AyYwMOvPy=%y+L{kX4s;u zZ_(h{ipEY6h@GK9;A^x#N8?7Pa*C#(W6|&6KSQ$*zaY32eU6rm=-?^T6SOj(02og{ z-D`mq*_tFgQ|QHg3S=5@W6s4ED&j3+8)h1G5wszRJ4KXU1j7^oqr63yl4QY0_i9d| z=%oWA6s*t9(I4cDoz4=zwllN~JRqqxcJq65j5qW({EB#A!x>%@_zGdx;XuR>=p5wb zmj%wy<)@!+Kfd|<_kV|yMi5Kr#xjMrLX}y?3^Hyg!{nE|o~L;hqZlJ(y!H-BJ~}x; z1fL?v-4wtC?$HS@LX?u_o#gOX8l`lVPEOHDPyb!g=a7T&{fYj7g8R{8zdRtDA=>d# fYTt98Gw7pdKdtxBKMTFI(?ak3cb(o}3k3fGPpXTD literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin.class new file mode 100644 index 0000000000000000000000000000000000000000..101a28a07f4ef0f79b0cddba3caff16bfd36044b GIT binary patch literal 2375 zcmbtW?M@Rx6g|_Hep$XMq9Q6PXer3H1r=$<9|#Fh!N8`5pVP9yLfMkt7W7ek3r!>> zVtfD}%6RW?i;HZUwlrzy?%lcPo;iDVroaFE_yu4FuOkR!B8mtmIeH$#R1^UyA*7-R zA{{~|gy|4w1O`?&tEMIoS1P*k!K{=t%c__Hv1P;1%!RVFEX@kxg+OpgH?}pp6+NM# zaZ)IvI|7BckB&NGjkZs^r{fyKmCnr&id#|r|1g~~SZ z>dh0`wc24(GgVnE+cH#2vRsg+&e!#1pt`490&}f&(v7NS8nSFDuIE8Z7|dS+{(RH` zU+EY%r>?R<4Xs$HjqB@kS$h^MtQ%HU8YPk^lNfJC_o(4J<-&nPs@v3#HPfnFL!g*w zhT-L=rcJ#HIhdFG@VKblw0EB?c8>W(DU|vqMl!Js~)NpymDTW7QDotzxq6yImyF~@3b8v z7~QPY%#yaG^BdpidQYc#4~(H7@fdF5WemOO6L{ak-sdUas0;eyg<#}Yw3&1anr4Jq zf!R(GXwCojddJrS=_Y92LJGWXQqy=Vy@hEULH-19FzeUwr^^!cV~Y`|TS9anSmOnZ z&*#kbr`O(u69A`67?k2@P4A9vNQh$)*+Z5Gk^wZdzJcBPeaSDHO@)Y75rTn-< zzj3=@jDDXI^fUAzfxDC+amp}+dvrwrb~|B5+|Y+%in$Hn8^V1WIqD}0zB-u)MC&0T z37g6?_2ByMa|E`M$?x!=q3bIp93nyc=A_*`=1i;soOwd!j(nqiVT_Ry4r7HzhVCM=f?<8OWDe6|7i<>^$1?fr eFbRy?vy;Rdvz`(SsL)wh%Wi7Od-e*+`;0au%=I?sv{D%Vsh;$p aP?PW8Ru_kK0UCkG zT{2x;=8HyVwfu5P7SePwZP`|7R$w~r*~>syATZ}_m;xP(mTfNY@2#8enq1FwC%Wk5 zWIiihOUdV(0R>&4vkqm&b#i90=(qy*}ECu@Gi#zf_8hObQy<6WgbEWj<9Yw5v*F07%UQ!om zIgt5%QmonL;fl0K-B2?rWX$8NG9(uPgZiD*s)YvapybY>F^z;1k(TENnpW~b*-gq{V_|bLaWZcn=|JvbvQce zeKM>HtfK=F9aqq3EJ88eZy{!z+Pg6DPe|UmdS8uj3ASblgRchBrFiqFW%- zgp-D>*3s5-6PWSh?HBtgFZR5=Vl1mauF5>&(F2PibL!TKrAF8&w)Z|AXr@7`bs_eG z>9Q44P2j$0#4}IhJTHXF<}Q#%*M^pJ_$@dp{pG-)-&Kxl)QUA%F(mZyKS2MzA+M@U z$32wp2JZ92rt|wJA`#W@T>0Ts-4B7Ql;$blb z;TmpGx_M#e7A1k(=%y#-^8;f9IL8tv@O?%gIe3N^KfWQj%pGm$1g(79&JgmW9<7~L zzyP91pclz<)I`P4c~p8-r>u-5rj8opMftnOEd6Cnb$x}YqW_?#|FEXtOZ^z%`|v0N zve8c+g*?k0g(8?po?kdTq#7-WK}tRhp^I~r Pt1$nCT<6?J8IjhX6YOWX literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMap.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMap.class new file mode 100644 index 0000000000000000000000000000000000000000..1fc74a83fb8dc2486b0804f93f312aa6e759d7e6 GIT binary patch literal 1806 zcmbVM>yFYu6#k}M3Wy8tD!N|Q)kOt1v0F46h45$1CZ>2vASC`A>MYx(6w(%TpT(EZ zM6-z|K7bEpJi}GAlCo*qnK?b@`_B31%>4ZI?FWE|*i9maTPehGTg6Ta5#&|eQE^wr zJrxCp^mE4%?!L`EPk0O)M#Fan@4euI*KXewp4WF7;ypeX2%0>o_uY=h2i!aoT9+So zx#o@hEvKV($5YJ^CoOId1z}K7g(2CnIu7@Tu3+$2N2oRpz0z#yl~qD1s7yiSN{t?> zplUgmUt@SwSOsNLLb1gV-S5Ap{4;gS5if_mws4!gZIfla-Z#13;;tpv6JyjrvPjM+ zVaP1U7p}u?Pn(NRDd!e(PX<$L!3kAHpwvsL|J&5c2V(tLHT0+TWB!h7Hm4)-s(ma> zztRda8s#9*AY<)c#1++fCJv=QjsDOz#epT;G&66m`!eA)&LN#f21N!V#HkLFE<;H5#RNtn>v;!YQLN#&DZ|z{RL2nSNv<$`%zTy|<}MP>g~E MjS@UULZ+bn1{6&L8UO$Q literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$1.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$1.class new file mode 100644 index 0000000000000000000000000000000000000000..43aff8e6ec498170eaa55bab47204136b1113a0b GIT binary patch literal 286 zcmbtPv5Epg41H10)%Cz_>@BPv1Y5BX4-OFv!9K20*BxhNX7%GN?gQ?l#HgLE3CT;~ zy*%=NyZ!+fq7@;E&?Lmm;_OrT$*d4Mvt{Xpk8?h)*q$Zy4}4)ld+JhKgv{;cQtpdQ zgqScc{Lb)+)4iCSw>g{e6f4W*uhC?#6z|vu^M1KEAbkHiCw^i%1PO%XVP-6=ez87C iS`tPdc_dvBD*Epus+!QPhA+h$>e?Gxv2sWeN#hRn;Za8b literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$KStreamMapProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$KStreamMapProcessor.class new file mode 100644 index 0000000000000000000000000000000000000000..f7b6100cad887e566bb7331925fd97848fd3058e GIT binary patch literal 2089 zcmbtWTT|0O6#h0XO)pU@2qNB4EA#>lMG&=61ZJ3FsHjt>?@L)Av?MiYxjg$X{1?vX zAU^m5{85f)Q%GT`4AVR`d(P%N=X~d~oBsLx>ot$YR3MWsl<_yzl6d0D0K7FOi48dhNhST=*UYMR1cu5rf^1S$rYSjr$8Xx3QMX zZorbo`>?quDvkWr9a-Q&RUFG|UV$_89LSzBgiUei zwwKx=X~>0`xXTnRk&>}v@`zoRHs9$RxG-mj~WcK*U>w>DJ)y}+O*s< zB$hMRu>8LvgSu_mhuq%62z_88^bun5xEx!O0#M`>gW)!5E2Js3M$?%eP|_y|d~-3q z^b8ROc!4-QHCK+}4(^gRaA{|dG=_T^B2UuiSBep&bvXSU%1;ExPtl`bowQJ9;uLBC z?HIQS3na-dh5JamvC=K;vsmPZ-VD3(hw*?U8AZcGk^(N1HeNDm;U9VU#~%I(;YaAb zf~S;{hUbKle9N>WS%lJ=6R!oDpAc1OlsvR+5+amjavp7r@{>_h T=%=-hq%i#fkn1zj5fA?Zb45XJ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.class new file mode 100644 index 0000000000000000000000000000000000000000..38692d6f8c4c0526825fdfd310ae8d4ef1236c45 GIT binary patch literal 1710 zcmbtU%Wl&^6g}g-Luf-wLwS`_0wk$~lfbG;!=k7XO3FitEU}!JY3jzYgY6(#Bk?^T zfhvK-2k=pdJI<>u!AjJHJu}z$o^$5R_5A+x;}?J@*iImVyGcZ`6~nzGLdeE&KZXY} zJd7d7ka})e!r3#q>k5})wchj`!Fvb1|H`p-;kvfN5bg1PU(lr3u$_*|`&>T~YL_2& zx$64)hSgEK!>Ouqb07$SDk=<#rqQvuH*f^Q&VuFDmR72?v~s0JK+&pU8D5Ryac)7F zN?v0K@7Zrj!g9l~#EU_%Eu0o_n-p1T*g7{g?ig}C3WmKSgY0-Z50hbe!m+sNs#EiP z4_s26Ig9?WKV~?%7<9E&m)1<}*{8#_CD?7!X9Ek)jFzj9km?3@MONLSa zhpu?O;2;C*vq&qbVh8R67@(^jfOxLY~CAwhg zjzOB{S18h^QK2zeIDt~w{tot)Rw1f_p9|bV7`KtY239GS8D`@u)@Vg_BlOIfZ`UT@ zmL|;WG`o%)^p?E)gvw<7PlVnS*a^a4Xf@=O%7HDPPb!xzLq5qejSx#Zj+>-#h^f*j tvm=FXU=mL8w=slUKLr<|4N3W-ktrLv<7aQuHbFHC;xtO|5E+?-@)vi7<6-~+ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough$KStreamPassThroughProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough$KStreamPassThroughProcessor.class new file mode 100644 index 0000000000000000000000000000000000000000..e58ca9119b999cd9eef54dcac4dbb17702d1e22c GIT binary patch literal 1842 zcmbtUZBNrs6n^erS*bEn5kz?tWo$_C1yP0=oH0=rCE2ns#t(O;Y-OdLw=K%gGSLwI z;1BRe8P9F^f|)jkWy$G1r_Y}AoR{98zrOzf@DNWj7{RiF4D#|+7{v}nqcX5qjr|gJ^#@k$P8d{4Vw7BMa!r(`y-OyU$Q8R7N5H`15ZMPD<+~clW zZ3@S4G>b&GW!k2<&F~^OG~05GAyIPn4TiC@X&Z0+qq-rgylzosyzJ=Qs&QdTzY|Qz zQW&NOrR@nvH(b{d4C*V}HbluH-3Ec*536%AA{%vN;!mL`?XK9PVH0mvcZ=0paeFmF zKo>(o((SMZ3`@E4AwTAt#c8l_>xYK!6$c}-{qdG@B6~dmXGk7%%cnuaNrkjEvYCq6 zu({_8if3}6JV}$(EytDEob)nn-_|{!dlX6JUI$~ATf?v!xilU9j?jA3EIG7}pS+^v z_^`FRIViODm)Q3CI^SKPGd)!*jxTiMr72hP+Ta{kW#d#R7*(NSUZqcSJ&i{yHZZAT zPDU1ROT{8?r}3EK-H=~k)GjhSJr{eoY}z0_Y&XM3R0kvG#IPQr;opCx+y{>M$i+Ux zvj~R&xuq~qH)4jaNtVsZ$_zhS5ThTUCNGXLvMIsASJ-%=ZseIuS>2vXsqOkBgSPB}D P!)1CZlwkz-Fb?(`HGIm~ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.class new file mode 100644 index 0000000000000000000000000000000000000000..e5b8f0d7f7d40e513b4d1fe8350cad04118bf435 GIT binary patch literal 1069 zcmbtTTTdE66#ixxSV3CtMX^#Z4}E|>bkn9!LNG?!gg}%8vWe-_u#PaY&XU>1#D7(v znrPw=@JAWXf|AlDq=tt*w>k5jGvC>B|8R2$;5&9QNMggnhYSpCTF6`YXkm*XUvGt4 z@QWr7f_6{)!)~vseOCm5uNks6rG!58=uiXsqK_1RAfo2AmqxW_k1f9=K&9%39~2E(fxEwb`=kYngINA zeXnuOFS+e;dggKGT)1J$nGvhR(_UD*D$b~SrsWT{D}Kso9=SM+U!rcZNF&4WYr%

@Qe?VfA177{1N{oiP(4hM`)RN9TY2e?(B(b@qyy0(p1WM#)mc#ReER;MCw$fF&?JCb8abCjLFg>gc#=5I>Zk(ox?AwPk@l0vd=c;s5{u literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.class new file mode 100644 index 0000000000000000000000000000000000000000..8ff0ae3e677eb9a71fd79a76d8f18c5b4743a6d3 GIT binary patch literal 1205 zcmbu8&2G~`6otS2z)S2|(D9^<5^5%POguHQ%V4@;Aid9$ z;Z-|~1`Z!{_ri0A{Ns>2v5Y)E62ZV3N1^M*aTq!0lj*7yU1NZ>?wPk1g=cDB(`p!Dap#B>WUF*?2KW}^~0EI*=QXtJM68{n#N>?;)|T&K}U#cfomu4Ef<12<{5zMyOXI>Rk&(w^J2 zqTvozTd2~g@aLpx^fvYnzCoMutyHa%nVqypHQSZ>=*-^)rR6z1#3KS}c>D*9r5L6F literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$1.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$1.class new file mode 100644 index 0000000000000000000000000000000000000000..db5cdd0b74cd9701ece8851be0fb25043d37ec49 GIT binary patch literal 277 zcmbVHyKVwO3>@=1xFF@k7f?{MLZU)+5lASCC@4~!!@zONF4{fd-zfNid{h*ZRB5qg zk1fr3A(svvy)EZoLlhX=ke;dSP6>to?|A5!i8 zCP7M=`mi&6;(RZrSRu9D!`M)U9NU_|IHoRjcoIYxp&l_ZY5Ni77BkG#at%0Uu11;@styDRrh^+ksf(A|e literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$KStreamWindowProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$KStreamWindowProcessor.class new file mode 100644 index 0000000000000000000000000000000000000000..baa50236e1bda0ae8eab8a8e0425829b05c5fd7b GIT binary patch literal 2752 zcmbtW+jA3D82_EzwoMlri$)PIQ3K7bP0|*TCM`;_NE-^67)118n{L}B>279|(DLBl z<1;fd7$0y(-gxJq@j?Ba-CU@c7@JAvobNl|{hGY^^VeqpmheLmr;(fFkq=@TEAm(k z;tD>W!WXzI32RfhhE)lDIfZpxm*`g#x*?$r1)B;A0)a!r+_4S?k_F4&OKWwlvahGB z+HO@#H=4Gt9T?_by6S@IyAE?sV6tfJnOd`D>jI0TOXf?Pxl%c|N)3VLzR`$g1oESZ z8)j3tO|8~QZx$V|dY%OGhG{fc1+K)#Q4}u=_|~i)#xq?oOns|$u&vuAZM(*onSxc( zYGutfB;I-S$p8ex#~iwCTNS<0uxtVKhH2{dT1{&-bn+F)@GkoAHkifmanByryWZU& z!(6vY^8?NLWX=Vaho-#V%|*JGl{JfXJZk1-wv~#D2wWQG>kuu}(V+^=#tIL#hg!O( zv0ZO%KhP`9-0LsWuD7Z`k|lkOUBLfPtF>5fFWXd}WuB*sO|4SBsnwmbJ7aTG!4^j# z<~Fm`>95qRh7^p&%L;A@{5Ui--EkV!)x+}fAF|P`lU&acC(Cn`6<8flnZWfEDd?5F z+e}PluxPdHioR|*Gj*)F=4CZhgz>hDH}RH)7F2wMj}_cjaR*;3D5@yo8xTm6URsC8`te?`YzieHG<EJ=#SqmFH>n-lg*hq_N!`YC#mVKz%JBaXCe40P} z9)8&V;QDf~Gog^27Q%i~I6S1hLu}pQ1cHg=6L=C&;eE_kFV6A|kOxTOU7o4~2XPJ` z5c}}hON5w^6WGydcXUP)$*1ro5vzXoisxh-L4h`={E%2O)rRWD z-1DChT=va9KkNJLG39+Yj~HcRNIAF9$wb)EcbU{>TFB9hJknTke2g$#wAJ%5gIP+- z%w-TKo#*thRQoNlgKEq{V9;Wks-ZO+w^DpUpycdZhH1Rh{6w(L) literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow.class new file mode 100644 index 0000000000000000000000000000000000000000..04ee90712000265fa93bb5bf9be19a2484c0516d GIT binary patch literal 1855 zcmbtUZEq4m5PtTK*Aok(qSlwTiV9RYFve&hB&JOh1Mwvgj31Zlm0qEDX6@+j%jC%@X+z%rYMmCIG z80!qN7pAG(n_ccWy2FsF)?HiY2X75?$2xp%Tbk}Tmdy}799XvWU50$kvfC=}acxgm zJAAjpRc8=om~FM=sj7!S{v`^o8*P)jeOqUEvQTijQ7tu^r3!(B%7$sU6^3GFA&648 z$q?AIb|`tgW|;cR{y|H(8@$yeOR{EZyxZiqA@)OKz}+*5jPg8r4Aa$ZlXo4}M;#TM z@3xbb7`k)yKka-LZd>|5IjwSaqqfiAbG6H9*k85wbfD{X!)`NQt%9gwjHR5<;PpG&I(#OR`NClHIu3P`UKd z3uhdEfp>aYol3Qg9j|-sg*VPP{UMI;+0BMDfw)j-yx86I@t)^>-j8$6yN~~V_z1u` zm?9)B1mMNh9xNguU`aqyL^qcC>zaTS5q_)+xGqBE-w*s)jSSFD^XX zeqyi5Ws7T@Cm05wHejdX7_s!s?3;r4#s3q#1_LXGo4TqoeAAJN6aJQ9xJsFfT&gKc zq=b?#k-B^2du-7G2w!?`0Ya}tG?_iXR0W;p8xq?s&3AJ14DaK*}%ev44^h@ZPJ zZ6$Whje@qs=AI$00gkh3n`^bNDMo>^bLLo}*X3$8T{QGsp-9-0!_hXPx1p-3Z>h8U z7u~3A$pky4Cat5%)%qq!PtlQ<&P>u1wbmbRi*Y~NETa63LDK(Ci@TEK>c5#$&yySV$?m^()l zM`pNi90;E6oKIeY0xCi-9S5W-F*#O&;Z*BYSg#h2$O3L5g|}(El`RiSNWfWnjU^65;ZneYfiKd-guc!F!I_`pkoeowld2%Gwj>9lN(mv`YaK(dbE=7X~N_!+Qo#+TU@7yWP!o@$S}sJXV0CXIh)TotqfzrX2br1Kze{D zqxaGE3p@-fW52=cbYk8;bZ3KJV(N=^^LBXrX9&FK+k-#*0D+jW2ay^*1HHTG`w{)S zII)Z8b}&IFv9ldUr%5qTl#4|aQh#bDM$y;G@O46R$kfuPSGCXUj51ovme zpwMkC2>Mf5Z!loZ>_N&}>qK(&J+jjFag@9T@})?A93l@6lMkchxj4q@?=}pbyz?c_ z;#-R9Lz?-W{+oFmDz>K@ad?6VN%1&5A;~CtRKy>6htNEP`^Q70jNBrPdWUMT?*x55D-7EKf(DuGf(WsxdK-A$Y!oyM^(+bLzq zhTj7c6%q^9h#!TxW4nSXA}rD@GS}C4zBy;+dVc=;_8q`eY-f=|C5JTbtLTA-s)iba z=37p%KN&NyHbcr5rvXE{$E}HA$hU1~_=s4cc;4U|hgfZf8N8IoOPw=s{ zhsG!h4QU6$v$^FPZ=^kN-gPIg>v%yWL^c?*T{*OQF!6~0YHncjpx0;-b)hM38MGMI ztL@G)KjDVON!089u`q+iUad#ocAWvOx!jSqAj5s(9q_(InPSH=xz*#IRL^lT6&y*j zoIZ(mqBspDD}fvfKj34Rc5jcNbP@F4J%=SpB}1m#rZLo#cu=VxRS>b^gI@zY1m?TKJTZ*@N5QRhHC25>hvCjZMidp_@D0^Wx7?; zbk!tKP$t8OY=dr*FkVEG#w9FM#R^3q;`)NVUH*!M&d$K}jWZ-a)5|15fFhLu7U3j; z#x%{Q9H58!0#+%je$j#~jMi|K7Erko)`@eC;!>Ph#UeeacWby#xk%WJl?*m;lY}Z+ z4Vt6oYx>q9Df&dAas(2g78r&CBA|v_u>cLXl~V%U!QB{S(5Pml^)LUz=lhRFlF(WRd+1RvL`) literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueList.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueList.class new file mode 100644 index 0000000000000000000000000000000000000000..fe731f4743b33af26378337acc749a218399dc8c GIT binary patch literal 3042 zcmb_eZBrXn6n-wdn~)G_8f%fF2!a|A2_jle5NUa9Xxe^&NvDo8&LwPNOWtO8gVR6Y zPqAP8pffsj?2I3&<4^FnIF8TRO&}2Zp(exR+}*SHKF>MNIcJx@{{8(A0E>7KM*_JR z2CyXND>2Jati)mAmYCaOR-;&pBZiGAHlw(sU>?|dPr)kca>G_gQ{B%ThL;(q>f3Hrm%YqGr@B}Be!Eex*Ia)pfXpex zww?W|_8TtoH@gO0+9~FiiE8@VDKqdr$Mv5oM3$VY<1Z^1>FFY$uGaQQcctJ|?fZ?& zuI=vV-7-C?Lan6BMeRCr-*lS(fx~QLLBs{e^A$F_4peX$=k@c1PwOqxyFFoO$zkX{ zmT@?pFFeslI#bpx(1YD4cFE6ei6Bw0T=S`fg3431qp;km&fS#3cPh5$>q?!QxA;4v zs#INJaNF0V!+W|O=*H$!=uhYQD4?(&JFZB#mY(i9HIn6f0Mk-uz_ly2BYPtZFzR^g zmAZ5do8=pYp#$x$NgDTE4T>CYb6=h(7$PM2qXYx%yBmVBv2TNUx z5oRGxp9;D?2(!hjkX|ZgJL|aI=~9Sq*BWk#Is~I|Zd~Uir3Bu@^#lfyOyCVM*N}{Y z4YShidq9xK6WGGt1PZvP(6uTuUuPm*J)PZ_PZ;mB8@r7>d>T|~ zZ)$#iLE+DI*J|XtN9f7ap5-;K@Iw#s>Pb}Vy$Ga;34T)({Ms7)aYmqUl~;X8(Q;hQ z-E(rXweRH@3%aoxL%t%hMG~op-+q<#{7kT18>o$VA77#zRfT8I~c+kM(9a| zXK{mm(BJMEp7wipCL9gVo4Zc z4L&emHeEqHW)RO9qC}b?|9i}Kk{s{j1AK^Cr12%DgV+}NKL**FvB}j0cT28Wo|s}( zawX>s;>_`Mk|!jG@jz8MDLI7u)7&5KN^#dTy>S@-U_6Q$E7jEgBcjjZ<7U3McqeR< zW8crb2&Ie!&e=BktPc6aBh+C@=XfG>TCfJ;1$I(kPcSA6f5wmK4=X2*=jf+|Qom5I zuzvGY?Gx5wfjTU<1u)qWfOsST$wN literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueListIterator.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueListIterator.class new file mode 100644 index 0000000000000000000000000000000000000000..70b218c7c8958b7f6c977441250012b1a77649eb GIT binary patch literal 2126 zcmb_e+fvg|6kVrn8fXi&fOx+n^rECHUWRrU6wtw`pcdUKKrC=?X|j3+FyUZ{QxkDn=$kt*9#dJV~FFDjENW` z$i=9l$ha)yij1oaVbif>OfpCn-e{@}i3MF(t=R^*ZPjMDT{5gyg`2##rYd#*s?HVL zu~fdH>8ncJOBGFbR7>X#TX~}CE5_?`(=-js8Fe8G483J-Rp(CABK*670gKC(!Ze{q zi<+)E(+u59bT*7=Vr@K-QAo^X98;xj%|u@b|-t`01#V_L@jm0*rbsl4Wwz9px5~(R~np- z!%}r!tvN*grdXNku1|NDyh_v28cJ(=ih|l?VVzEWyutStr`0KR!ucUlE4|9m2rb1aJTJ~&XC-1z;lxhpIhqi zh+!t^TZzo!wVG<%qxpQE;cf6qFC56+PW`_y`c7ne>C@|$E{JcH;s9kKBq32uVUY46 z%BmDY;+gCYST^?=p^vl*W0>y!6oCiyWATuX9^nwJM!csf9-)=MKI)<$>)1QKx0l>w zK0(<@oT9tnU8X2{>&t#ZXcuzi9U|fPREW6LQFpWAQYO3yeM&*JUlBy7>5>qN&j>`C zMqk;suWU##lybX>XLDZ>CWW~Vdpw@R=k)m(WXj7ni9K8CZcOiI=+7W_#z9aH<2hvZ4%+ZM4*CA;j{-I_aQ3z~BQj9hR_!g|M61Y+BP+ zt5XUa29VQ5W_1G&gI`;0xDij;5^3KFph$X2^B>ZuL(pJv}!{aNR~`fx2LOGy{^|) zToCARS7wd6Q*v}$b1YlMq(D3f3q%WsX*flJ@NjlPK$*5y2z7kcF!i?@o6EXAuPs*@ zIWcRMwd#Uq8}iu?D$bfw7bwhHb|tUXwDOvs-_TY!w0zyMb#2oyEBOs?nm0^JFtutu z|BhjE-B0b%e{Vrt7xM<+vQaf++f>S|qg7`@^p zEV%R7oTHUD-qdQoq$t5kCO3Il#87JuA6}L4m$HmS8FP2Jp^^+K43Q}&_bg7yNZX2D z*O_nCxJCmm%Z1YHx^_j&S2ZR!=h)0|{BZ7NYje{w^ZREt$Kcb<0hT?UZGm}vd0j6{ z8=_U+tT<~-&#`JoSz3OGjF;RZ$`&|#6nb9va${AX`_VgX*7oo9{1H$K^Yi0HrV*br zDyHT%Y+9N!cVpM|2gTAW=SoOn6nyoS-)tw<@!jGot8!>kwQlj&*C{1Q%Ou? zM#bw%WN}eo?C1q7kUF>~FGGO~N5>oTyZP`H{Og|bAX?jAVCKInl5mgYI@hc_`X(=2 zvu&+ewrWj*-07nd|FPmPWTfoZM2xgT04?-*8jd1@Cqb)JfqGxIr2RPvNRdlp|#Xy^Qs8Mp*PA z!%@!q8ROfJdk6RlVUSS@{Sy&I{LT|`?s?jmwZ+r&d?q{*F7~VU(V0ob-p%#(FO7aM zu@$DG-!flwy;ae5V%{LidsK3RTsH$n3e%Q~!V)o1lnE5|Qc-WKB1KS97qOnfkgw=A zqHf+pDOLKkgFe@x!A^v3etdVHmR+Y+dk^-0O~oEpc8f?`)UZwNTg>vK7G?c`S3`la zAzvBDe&AIcp+6w4$`Fp7?+e@Zts00H`*)C*9%VYP_x{~^T6T?QQPV^9aY-_rPgzEv zQPyoD-U%dyXvMx+1A(N0K+-^qS+1mze|yhk*q3yfPD!tO=+VgT4^Wm0oP^~xqK@n? zNiklbw!8c8u?{|GZGGX281&8t)I9-p52-Ue8Rv^G$5D2`9dboErZYds#v-}CbR?W1 zO$?+Ix?HrFI{x>bB5xW2Y literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/Processor.class b/out/production/stream/org/apache/kafka/streaming/processor/Processor.class new file mode 100644 index 0000000000000000000000000000000000000000..9f5410c4dd3bd86614e4632dff8aad250debec55 GIT binary patch literal 424 zcmb7=O-{ow5QU$MX-I)mxCCZXZ&)=I8zdIA5=gDGH{z;%qE90Ea?M!q0-R zVK?uanWuU4_Wtq;;2b9jCJBxixOFx%%*4`%LaIvX2P3!Yeye01gHeyx6|xFmo4WQv z-hNk?-bM2qv$LEb`RX&AiseQ49nq@!d@<`==6|q@{usk(<(r_*l`Y9}{1a!|p3N}*-xC-nzvB)W l7zXrXfOEPToC?Z-A@PLh0DGiUj5@xLgN}W64l(Z3(Fev4av%Tz literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/ProcessorContext.class b/out/production/stream/org/apache/kafka/streaming/processor/ProcessorContext.class new file mode 100644 index 0000000000000000000000000000000000000000..ffd5feea808d13307eb67265024eb61f9a2b58cc GIT binary patch literal 1281 zcmbVMS#Q%o5S}e*oEAzCT8?s`by^8O5J2FnFO*6>5C~U!;BD)U%_g?jT5rnn(|F(q z@S_m3o6AZf2z?sO?0nyRvopKDe*gFh01eo3pzgpXgBfwk;DTGwRzO$UMSO4W?s?D<64k6MEMNn0ug6icN8*sc@B#0%^hM{JmX0R zhr|>idAryzYy`fTn!Su8@>k~qc7-ilVP!%FwL)`^yMi<=5p|27aZgBZPb5gyoD{VZ zVZmVQFLirK-n(ux;c6`@GJcZKJ<@FFYrH>cVW|C5)n3G4o^2n#9zSVLi0!z?hyHQX_T)~*zi$dO{=0yMRn0RI z+76CbX0^;k$Y8a<9Oc`i7n5_Pg?UK~sxpcS>>>~7NQ;yv!qc3=d>}h2M6*>4);{!S z{6=OPzv`!E>li#Ad$E&7F$$)=wM%Ptb?>*zfPM%1EC5z2)Cg1n$j(BQ;toY;pho`O zh(8Y(2j`dQd||}D42za$36`y0fh*Ro!kV>L;hMGA;fA&AaC5+Si}=>z_Q1bG{$03d Y^X|g~YajM>K7z;Ae_|{lHmqXz538hRlmGw# literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/ProcessorDef.class b/out/production/stream/org/apache/kafka/streaming/processor/ProcessorDef.class new file mode 100644 index 0000000000000000000000000000000000000000..aace23453971c6ee99d1d829031774839b22bfd9 GIT binary patch literal 214 zcmb7;Jr2S!6odUrX$lkE0u~|=8!KW$0*MnmqSw+Nnkp$Aj)4PkC`3BI&SF`zKHIPN z;|XAmk%Cf432D*zjvFI{)oi}EF4x?$xrnYft62Na315pQ*R8WgythstzivYqpX|*U x*%ftYnEsNjWQ&sTOj!O`89>6MM|I6%^>8l5bP0hFh35dpgaP8-VHh+*@&TnWJ{tf4 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/RestoreFunc.class b/out/production/stream/org/apache/kafka/streaming/processor/RestoreFunc.class new file mode 100644 index 0000000000000000000000000000000000000000..37214223ff9c8f8d4978464a5c4d2d3c0666c77d GIT binary patch literal 166 zcmYk0Jr06E6ols?uxjNov~evw0R^pzCN@;h%@g^N-Ry#iSF`W{9?A%v&EzZQo6P6? zdIQ*Fp`aAzgoK@Ip9HlovLf4@3Coimypgv-{gs-Ue)HC1FztS@xDkoLhb72={RXAOIAo!I`)6ej;P j+rybnXe%f*Bs>sBVv9%`$%OU?<1cp56@>(n+CcvW4*5AL literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TimestampExtractor.class b/out/production/stream/org/apache/kafka/streaming/processor/TimestampExtractor.class new file mode 100644 index 0000000000000000000000000000000000000000..8936a8a33d0f0c9a3750f3147d100e78c2d67afd GIT binary patch literal 232 zcmZusI|{-;6r2~Mt5$YiK^wObM69(C!6IJZxvVkY4ZEv&HVY5np~Qp~3fc_A!2F)C z`vbrn0|6n>CkTD>mX#+gC%crdEORbn8F`zO@#;(2A5*RTA7#2F3?p-~N^g=}6UK+6 z(9UyFt=nv4XH7C;VQeh9Vs+Ls<5R}cd8@got7K)3(#{zxf9-_PXIeX?M<4|C7l1-S M2Vuj|tua9F4Z1BvdjJ3c literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$1.class b/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$1.class new file mode 100644 index 0000000000000000000000000000000000000000..f0e7db12b837c0a7b1c8322a889de241baf5c0de GIT binary patch literal 267 zcmbVHF>V4u3>*XD$bpnActDEOtdI~700k*UL=iq<7IPeI&ZpfYQofcV56GhsrlpKW z_Q=wBtoghA`~sL^m?6vXNXS3*9KEcnAcXNo8x2~1FV0oiNt_({iwRG!wGW!Ax_p=R zpt}q?VXm=s+^|20JMsR+F11nk(^RgB>Ln$O?#}RKN*KM>wZvuMlq3;~+YcAm&eq$b i_?9sHk9F}3LdU}0s%wC4Jv3T&P_wcCK}w|n literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$NodeFactory.class b/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$NodeFactory.class new file mode 100644 index 0000000000000000000000000000000000000000..4e7029f1c2e03e7064d822bcf8f8708fb93755f2 GIT binary patch literal 346 zcmbV|!Ab)`42J*Ic2~C*JopG6^x|B4^CA`sf~cS`kWS;e?KoL7+d^N>gAd?CiL-(S zJ&Bk6Nl5-LDi51?YL$CEN;zDYj&wK;g=@Xt~K?N zlA3E`Cl)y*cmLhH)umhzst4Vemw6L}aQvvP2Ca8jy}#J`inm;1l1YEU{8~Th8FTiX z+Kw>(QBAI66q^N3NfP0xC%M4+>bu2Cd`Fo62U!4wLH<_OAz=^0oDDGgiUq2yRh}S} HI4G)5Qvqkp literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$ProcessorNodeFactory.class b/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$ProcessorNodeFactory.class new file mode 100644 index 0000000000000000000000000000000000000000..966db047a7fc88bfe4d40fb74b903d7ef915da2a GIT binary patch literal 1633 zcmbVMYfscb6g{&K%65w^3ohadRMe$B%1eBJh@fkT8!&;87(dOHVJVd9rY%N(hW|tp zSE44y&;BUmodS!jYf||#_ukIjbI+W6J3oJY{SIInk2A>PwhjY#ly_H05knd}RbZ-R zIExXCY8cazfu&(W!=#2O2Cc?@A;W;7`%QU=zvY(8Wz|{>eMeSj8B&t(35I;DJ$+Mb zInoIoPclrEJ-=%48n?HFwad45xfO)I;CqB))qKwuLE!n;s}t{;*dqA0aN7w6#~Ehg zrLTK6&-JPYi~EkdDSU-6rzp%byf2;);_sWa5F#G*nad4rka|nrCaKq1cBEL@->V3J zombpQPtWG=2KOBmAAd|KF&KIpFe{W@;&$ly2Mortl)_(fc@PLX^)eo%Ccy?ucV=Z;O(nWa??6Kc*(sKo2e& zDBzxf9J*-C4b0%afeXsZqg%rR0}t_tVJ_B8{^KsH#H@xnhUHi=Oz1SXCRXq5qYk-H}1arA?@laVMpLKsBEi z|6p?b#{BPysH;uK;@;E{4A4VpfT05^bg5~fw==IsD6%v-KrW|-i{b>GE1;LwOBB7M zI8AoI9Ik_zhe((sbtKKv&q)11x{fxpFI9*3G0N sphw3_WM4!D3rA_k1$7eO25u5)0=J@^L0lu7r#OpoTqWB<{9 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$SinkNodeFactory.class b/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$SinkNodeFactory.class new file mode 100644 index 0000000000000000000000000000000000000000..217c1f4d2cb9505db258d04627774efca0074c05 GIT binary patch literal 2230 zcmds2TTc@~6#iyuyHJ+3fFkvNK`qN=1raYODky3~)CeIlCMLr&l%?CgSjF0r`G;tGqaI;OJd#5En$I&SE=DWF$msQfq*=zTiBE}u)=lYY@&h(p&e zP6?#rpyE0LJ*_^bd|6fkT_v@-ph8!A?n@O4jL!#Q(UukItSP%BS4+}%f^s?V?WlH2 z#x7ZPAG)aPx%BEA0-1QtjRwvOOg5uMaj0aO2-=k}a8wipp}iPX0xu|T&TP2eiV6ua zbItYLcv|3VVQ&!4A-70T^P%R;{uKS+dkiiKq-KK^%KpB2*H;fV%F8NTl*=CN^vnm2 z^p+%5gzVa8it|aQ7hJ#efE?$f69?g@fN|gVRXFR(C{lD_u9?g((ml`s%bl~#l@PdD z7~CDC>&Gher5D+cepHo-PU)Bt!ubDcxVx-0^!32l6xUvRK`%plF~(w?#RV1_;yR9Rs&<$3Tv?9`qVGfIb5U z(I;@JX(qI4t9lRoXeh>?({Wee_O1{b*fut%K(0RJ9xksdC#DwN6Y_7bsPi{#a!-I% z&9(wpH|=SH!=a&Y9NG@JK$DPvqQ zDv2h>Cm;M##53Dsp-W=Z#25Q;_MDkB-}%mEroWEAeg}}lq68VkF%qhT4vYwGRKgg> zqeuuNC&YvhlTqB0AYm$s`%%n9G0PAs+a*JD8M@xCZ1X*?n%pX=Yh}l<3Ns8Lix+i< zOI$14&H;n`#Ikf}!Q`&1lgqN7P!lE2_<*VC;k>9n!(6t%IZ(qY>yE`uSABlc zOF@-JlR9_&KXk6SIyB60W^_h$Jg`qUAh16w#F!A{Lga*)V3=wZnn9{IZ^;njlWGV) zES>=wJxI&Agl?fdlF^AIy(co}@K{D%XkAFjxQGD+*t7h$u9ayVTf#nbXWGwLlPv*KAA?!Z*RW>VmZ&^klSa05i#}vet=3** zf3N@DLz8PIJKUf@m<}*pK$wPwCWQW-l45M=8xX_deQ92RL_!P}*+JT&Iipj(WPKzn zLf<}Rpn`yMgrG85K}Z?;g76PSDu^oSPz7yDI$S~Qvxj$?WUqJX1!8zf(bouX9oO*+ zcd&sgw4WnJgo>-gQT+NmRsl~FJcFJl7=}Dea10Zto48eD_RzC|)Hg`4l|$lmNHh)+ zJ3{*>Iun545g~8jjTck)^i(w_m7)ik8kR$TQSy_Ofx)44C>=fmtLIY1cVK literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder.class b/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder.class new file mode 100644 index 0000000000000000000000000000000000000000..6f2208c49788f175f2f5dc6e0b787c2442f6e343 GIT binary patch literal 8232 zcmcIp34Bz=9sVYJyqDbv3C9u_MINUi2a!mlZYatjhz6uag?PVgUXmrtZrt4IcaEL;U*G(X z8J;-!&>(=h!sWpdY*uiS2PN2CiZk#v1zS9DB?mXlnOhXx>S24E7hlKi3hq#Frw603 zS;bb_enZ7wvi+t98tzuHO~pNO>Aec>^PmFTC2bDAC5e37gYV$GDt4&2U&Z$nJm7(j zoeFk&Fcp1rX}|n>P{D2wW@57(JmkdyzAwLipkUC8YV7f1FYfiA9uIr*L;R?en;((A zM-}Xo(~p&s^L_=7my-Mw3VtlxM-)7%;wh=~(;f`r84pgy-3or9;8_*Vd2pH>J@3In z{M3V=;pYl|q2QMaekGtSi$u)WqOcKSg*jC=4ekWXy(j~ zUm1%A&3HT-%bE7(j7CM$WI~d(uW603&Zz52gu-duRHN5j#M-i1A(aGM#Mo+V7< ze91@3v%*+wshP#6*_2XD%bl5*9EJ_DXimh)$)Zsnw~Rqnp`cC5JuegqCFTn_Dra3F z;9Nv5v}02!Vy@^}+iJ$nGg`x(8QT;M8sQ6!SV)fTNoS%nMAI~kNJFWU(bks4N8pmm ze3>uGpPDsNUd{+@C-tyk%(jexJ0UGXO~w`ADDY?j(elLzM%S*5M(W}wO%x7oFcKk- z7MrQD8EYKLo&o}m*aR+1O`_Xn_GSwvFn1)PkXd#2<;c_%p_jVgGGm@|m{c}a}FVZH>HsHkroVDX<>gZ1DC@_lnW_SqH zAGszZLTs~%Xhbg^HwA7eWWu5XU7E;Zv%T?X>hEbY+hs7L-wB+QuXCZOnXt5rr{ zfyI?YI$0=^Fk=xT9Is1LN(FMpH>Zr*0`rTLr2s9N1V~3|Po=Yk#gR(MP$-!Q2+Cxq zGb;|Bu|8;aOS6&H=5i8InBfGUkOHR=F zt>J{+WYXK6N+>I{NJ(Zz=FY=ePUcM`izJyB$c)9A^!<2TdenJjsunA8Uwrf#*o6|w zEBK9q-zs>4RUGRy7GA8YvML20f-&@m49LlnW5)2g0;ZJ!c#^s-3CS{I{IHWbTT<0< zK3IE<%Wy~+SQ?gN1^H)wI@5@EQUDDf!MPgF!?}!ad2#3e=3)&OqgBC+8h(eD1g2W_ zEoWF34x1fDctNbAXDu^E+BXC$GK-QmmF-8;!wi6%PDn4Wey(s@mayd+QQ zLh)^wxu<8&m0clAf&L(HVji`ECd?!R4Piu?+%^1BN;63h8WD=Bx0<@4Cliq#ZP!DI zxEXFIfGe&1n^A+hTQ-t=Ec@uaW@0uYZ=N7Usg+yyOQl{HST-VBX)I*`%(Bv-H2fKV zk);2ccOV&9G`xW~=|md-hQDk02mUE=`iKO`;kzFHqJL_*6bI;}8vc#{Xm|_snYZzd zz`UY3942;Y_%Ghoa5)TtQ;(2P<{)VZ;xd7gkCcE7nZV?XU)j#bA~IoSdEUgHNLM6! zWkkPx>FqSSyUhrXZgFTAz#`?OJX=C2qC^u8;bc*sCv_37z=WMK z=Q*uw%wU2Sw2@GnJ|~A(vvrMRZnwy%%K;XOnF%YokIjsiF^n0!hvzI}@D@;M+Jq5` zFt_-!E6Jq9jk7LfjI`{^VyrfMK~juHI~`?R+DPhN_a#YqruE=4TqR0&7 zU?dVaSXo>}kb*76Ihi4|ITfgG^GO#BwAr_+5#1Z}`NWHObePj99|7Fv6+J9zh0s1w zOJvTe4MD!l1#Oc_Ld9Cp z+V3W$9Kdqelt0o16xJOp%=L6dZl$+5Bi10-VQg^qF3yNAq9$@XQN ze3sv&+&_Pc5+|z)Mhr)X3?VZMkjf4VY2j6R^`cJeUbmn~1P`Z%WnFT&9vqo%Qr5Mc z-kPLo%Q6SsUZ-%2e0mU=V+);*pN&~{HN)rdp=9XG1k^=QR;N}vq8ZA1B5R{vhqWjeoCamVGP zz>JHTRs9fE11M?fhhrC?}C!)s`pzwR`YuT->`umIB_D%_#iwHCt(K1Eu7Vo z6mcOg;%E~q^h<1R%el+Jxw5Kiu@|NCQ_}~}4o;Ool66~KueHeN$)z?KZes0pne}^_ zEOod;l5S0s7OPRpf57&tolvTG!&`zam^J{dWdNh7$>_0VeJHQyXY3e$^k96&`&Rkhz)Q!Dy$OdtHKs-)TuOYJIn*x$@o{+m!sxtHQ*Vss1E;Wm7dU z>jS0!Qi6Zn;m*XQ94HlksfGN_$U**iiF~O73BoIwg9A7VucDdv@;Gn9*Wh)!+X39e3*iHJ6Z?1}l;LeIEp(O*f+8)G24R01 zg#C2gD=p{SZ=Y}Ja!G@a&NsNgdh{j@?&u`>QGmEc24fjH`Y^p8Gnxi4lVR+H<#xNm zZUfHbno1eSc4L;n7BO4)ss#TJuUdM;Zqx__lxjasS<7%#( z`Q3Xlbtg^?FzgLBlsXzbK98?-D|El=^VA2tHGc1I%qbz!db$6kjUHP2#cddr1S+;5 zAPH3L#N6Qoj<*Ta`n`T-FFurHYi)mN8?Khy^({DGZr69Bf!m#aRjxcaC7|`gDbj@p z8$6B%ug~lANWn^d-ui%6HS!+X8;RZs_C@-ShNEIm7-mQFrTg?v(q|E;`h7EMDG#HgP4p%T;ae{;lxVe zLW^*tL#T*~Qd}=QxSj1iLc5DUaH zA}A)3=Ohsl$BHg-oahmgty8ywA>(>ROX>PvJcH4en|np2JtyB0&ae~0N-+)_a1}jw zsW=WF#mC?lU5uz7$JIPsJX-dNEfX<(g6Eu1Vqy&w&9ywSlSL~wvge>?E%+3BPJEhL z%UGUV|C#jq>(c8#n_mApu9Ii|PK-W8mvUSA;$fo6`9Z;#6kKn4 z2lzBHt>hs5*61xvDLKR-sU3a~GZyohBs~r4;g+$=cpFpRLCHb|H+ZSWCVZJH$U3N= zBUuModH+Ey9+uF=2^I*+%0{)#m#nyMB!#bVX9>QFGuc+r#7o(tT!l_ZmhqPgwr8=O O%Jyuw)oh#Gp8o+Y9dVZc literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.class new file mode 100644 index 0000000000000000000000000000000000000000..c443ff70e990bec917669f367cf402a0e91f89e7 GIT binary patch literal 2266 zcmb_c-*XdH6#j0KcJpIPOKnpsT2U#cNhK(iB56Z~rdUV|+6p5+ZIj!wB-xF-8x&vt z7kF`e*B5zkh95IBqdvh5uRi;qFyeRjZrWn|l8)KQ-FweH_nhxL-?{tzpWl85FoVz2 z7{#oCvuVUIo5Bb_HgK-*W%N^ql7YE2QkYdP^9IT(m^iN%3u^d84HwjKQ4Nb~s2Er> za7kdaF8!Kx>bA4B*4gmo*B$B9q(HWM+qz>FJAvISR&A#t>s8wiN&=}hd&{wcjz_02 z_H~-8y53gNYFo8ive>XT8&=T|JZZI7W0V90{wHgV+Un{cy8*tK;oiX=guIp$<&Tc0ZrQ%TVz0=1#k0Iqvhtsyh(%fwxkmX zlybdBj%r#8Yb6;+;Os$T2pH5(ff1NLV301Y0RpQvR;+fY40}~z;h^MnOvP}(lP&j- z+^$!`x4&aAuvL~hZb_Dqb9RvtN-66C&Rpf3Zr`TGzP!3|Th@Y7d9SHg!m4F=H%ym# zdd=;4HMwM~E)aSiCbmCX4#c#AO^caRtjJs#r0xs%BT!a1EcCDB_gBg&t$?Q!{~~ z9U59gfn;0CMwOos)-gA=S5bb%g04usXu5Z_f6Cq5l%AfC=Jp<)y2-;T{F3dd_!)gT zS{b%qX|;p9VYl7H+x#|k1P>1~(J`(}eUz(rxO(>$%WO9lEj0<>!}wt5x9<25%@} zi4fQX49<@e%n9;+k57(bPhlPfEMYnXdQ7K|fD%OLCFp;Oo;gBP;CH+fcepODeh!x5q3A81irqatfAS*in~ zA0)0Zd`P^fiEk$4kkwJQIpiY_3i-~SMy~c0a)md6*x2D+NO=*sqa<16TkxIK&|dq6 zb1im%XrB%9YLSzBL{=6p@&`n!BYsbeihHEq5QVK$Zg&IVe tjk~am=6IAY!@fIBfkY^frm6N+V!hX;nS^+O5voD}y5(o^5xrtK^Dlt2B!U0{ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup$1.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup$1.class new file mode 100644 index 0000000000000000000000000000000000000000..502579775c07a83044a464ab9a421d9567202e66 GIT binary patch literal 1527 zcmbtUTT|0O6#lkN8XAgJKvcXGv`T4_2%^kb!~sSfuy}*veF+N$(xh%Sjx+uUUwm}@ z1-$qm&iLT??0?enY={n*7dveyXV0EJ`ObGP*`L3^eFrd&$vko>+puuMhJl+FMs2cG zER0#GTBtGP)Q%UGZ!gmT;AE$;3JXOC~~aVJtz@TM1RJ1rTyB8o!kc!3f!;C|%1 z;8JZ`VXD}ASO~KIH@B;BHZf*#<#y5Pj^ff}4`|DhK zdY>8%od!c;DF}pI^m!Bs+Rk^zqdZAjS#H>DaY=#SwVl>0;f8YaWh`P*Z#B@K##Z(Mn^Nzu}J0X$YQg>AP-w>69qV+`M=-h_#f#{0sR=DFQ`p~<9y51isH#)GYllA6NO`V z&`~Ugr)gsoVCwgOj^`SUOr&_T_2{jV+@&7Y^|zX>Xqndom7FG)Wen3@V}YRytXS0b zrb{VHPd5r=?ICRr8Ttcuwe|(9`W2baw9DWO$sG9r6X+#5c<6;9`e;Y~2^!6=u6&C1 zo^Cjc&PufdWEaLs%$OsC=7*0kGy7Wk1k>2h?vqP42`~tg^eK{eh|*mGx`!*6NditM zXQh7uOE^a)G@%mCCv;2<;Q}raRyVylM4E|9bl!!`FANw)S8<-mT~5i(ke^Pyr&{{~ rW1OA?n3~GzTy<^8F%4}HCGB;U0_$}cBeWX0hU+*>noZt0TIu%(!vLE{ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup.class new file mode 100644 index 0000000000000000000000000000000000000000..2169a787f2aaa616ff37b35cda4ff3c05970fd30 GIT binary patch literal 4427 zcmcInZF3XX6@IQQc`bQuu(4^Nyx1X4Y}rO3O=yC_P^%;{7~^0U;?Sn7rNu0kc9q=~ zpp-UA^IDR$rJ-#|TfX*7GLy^%7?+v0A3D>S{)5i+mt@+c&)Ho|D^NOeUp(WxckjzN z&pGEg=dAwv`S1S#;8FZKja~SzhUdHRJ)F^Cq>;c<8g0m@a6bw%m@=G|VL6SBI48T7 z%+AQ{ybLQEiXx#T%eKrMSuSgMA^Fl5;h(I!UGU&{p@IuCT$JONG*mnBVjA1=eYtd5 z4qn#q0}Ve^(5G$NbaO?+^Gr`+L)mbB%eNf+nTlBnR%Xr{7mRGhw~ESn(&Ex&w0LU8YRnK&X!## zZ+f2NW-Z${UE3&n*%M~oaSH*aiAiG6UkKn$RTr!hVQh)Pm~$=1wft&$SmD%{?KJ_r zJ`jCAm3zKXDj&Y&yGGu3T!klEGfi+eUZqvtzT+Fksmj^2rfU`y)U-m!e#^G}Nrj`s zF+(I0|FNQxlM0EPQ=sEE%viQLTPZD>?t-yYwBv|6Z)UxS^#8kLE+?uGmZ{n05S?}d} zVVzkBWzet5ARiQ(xUF?glWcB~X9|=5Eud~9%&(B90fRKWM0eTdB|peY${$|0$#H)% zRg6O6gmE!k6lyl{|CbOZsAolkm1dgdMOZ#D!bD5eN}oc@!W7bO3`KvKqK(28x=!gBMr%q2zvOuz3C{$av z;(x1LoSCjUldiM!XDS=+RANcm6IDz8HO@W>IC$|D+Hrl;*nsa`)v zTlRoprXY^Ve9`e38@@5c)ALTn&6`hJQW`fkmb2Yrla7b+oWfV?_BHe$4X^0<5uPPM z@9B*HJ~110J+MaW93+8gT{-j_D2Bjz73$7O8B|cLt>m)lZBsOIO(~PPwk* z1Bt_ddlat6%L*Io1|DBJZ{~fL)OCt%D9!Arb!G(RCiqC`t}n7gvKxCrJ(z03*Y)6^ z5Xx>Ho{nW(&MDEO2DJv8iLz5K<2_HYlG-D4XPnyYyRi#L*!d(1dt$erEteSOkCVa<=S!GRPHMYFNrLtcQ za-RltTri5kC-8l1(1VZQ96d;4khb(fe?5@(b%jn#I!h4h%?+2=D8Q@QIm4S}N2G_O zG2Z3KdEe!?a1b+AB-3y)#2*N|_@g3$KFNJ3JjAy)4D+1@q4qw?cb0`x*cB;JET!LyBJejlPn?*eh)z1cw+)msJ5JN~CdS9On!1I)y~%;Z2k0G0 zM&@!mGIxMGg)=mHAo(H1Bm&ht7*@cdy({QP5j)|K7G?UTcdr6Y79xu4-pb`p-li9ZuflFxvsxSj85F+3{6u zRk)5JPQI$dwfhn~w^RH}vPAn6io? n1t}V#pq{|U-;koer9*DS1olP(+i*HK{T-5xRG~<*l7IgKJBX&4 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.class new file mode 100644 index 0000000000000000000000000000000000000000..b4343ea8cd03cf6d41218cc10491ba37137004ff GIT binary patch literal 10324 zcmbVS3wRvWb^h;arCF`UwzOC_vM^r221%AJzYUU5uw@wwOSZ9O8$*DMRwHTc)vnlG z*%pw5Jo-w~Hhqv5N=logb(2C943bdP0S{yP1hG>lu2ie!QERAc zWX|dew5V&PXHA6)6NJs{*ale@Lg-Dbpdr_5FP3!Y1hf4UPNdx)cSyG3}Qb= z)CRRWpf<|8I-r{5y(yqJm-D?vCT|VmnA&Ej?Ln?!hctG|*UiG`7H)!SmRT+G)hdr$ zrP(Hrw;1ZSpxUK&8>-zS0Xvsl!^^(JJHmY7#y^dEegJ9zk=#i@TaZR zNre_Y$r`aDLw3_iYv82Slu9RUYd9JoY#K=>B6cd3NH#^|X*(IWVyUJcJ=#lRq8Y<> zIvI_m6zU6cB8lPQM7+tJ($wutB~^JO5sTT8bRwy6TOr-t{14laM6$mH`{Wkj|RMCpA6V@rKX*3;;HSJ0!t+74!1EaRAqbE8T zx6-3YTj6NYNv(^P=K%S)w2{e@llEATos3$s=z}(wxTO%%RdmXoWu>E3dya-6+ZRmK z+Mt=AvSK+Pg=G|GkDXy8;%u8IvX_bi?OX(zBhoU8`i0 zt96})jOMm4kA2TY>P(}Pv?t;N(ZL0~QN9@$^16)s)E}kqw$nPe8(q=3ePDFB&rbGQ zeKGdXZjlkOZIiE-=7B6`INjdKC!>xbT= z-E?nMX4nx zk|F%V6r(lS=&@RT?_%ecoz zG6ufZ!$1UREfigX*)5)Fl%q&&F0K=y-RGgDfL;!rbPh=xo*9=FLdR6vQ{*H;w0kmj z8)~niZa36ERwgVwn7-@knV`27Gl(mP`kjj)%SF~v+Z*n8%p1!akww*uB6P&4Vt^mD zy?~h%zbDGLP)TTQnYrMxehzMh)kh#{57I~^#tanki()}RM}QuQPxFlPhn;A)cQhVp zk$_#QgPX$ULO;9$Kc+V~GzwVJ^j{3YD*$VVK|7sW15-hrqe8E9&oe&eV7SfxxHZgZ zzC5!fUOYN(VqhR;r?oxb!FZI84pS}G@Q8SKB6-?M(z|c+igK{;1j|C#lQfvyBRVXP z3`Jx8QWq(-%}D|vImh>KN(Pi~^L#pPjt^1oj zx^sz+7U^rNBRz>l^tyR1vQwr8^(s5#{q`A#fCY2Y7=e+|)R3d7UGuc?%V|nP^2fE# zjgiEN1IqZsZq@G^=Sj7r&6G@~@3?XXb1WuZWVc0|*355nZ@Lc9B!yM+EYRHQP z8tq2TXNP9im#*ohAmr<*(Ef9*^BUO81hexTBq#N;Bby zoyeol6X8tJ6?UqyFg-ZzM4qsmB*XWZ>aglDRIhB!)6zJ?-M^Q7@sgpAn(COkOQET7 z?fFIBemR6@Dmzmhmx8{fkjyz;X7fOuQ#(_-sqRzvGpU>EZFtmFmg-Zuu|S5R6^Drr zi3TI8pLKz$Y&D>8RgQ;Vp_CY=8dO83imDT|=G?d*!f)dxQ=L=;ri!UyL&Z&%z|*wT z{B1YYh)j9_<8#QT_as|kGYqy{@o*v@8w*FoR%}{=^JI!lh5JXPTnuNAWqNZqitg?* zF^T6*mBb6CN+|}9H92`Yc_}c+^QJ&~c=gEfC7c>cjK=!I@kBa2WQ~m2@vxN&>&mdM zJ~OBV>a)W<0>$k9@M-1?J8aR4rCtvcWW+h`+tll*!m-N(XY13&fGhM~9@)u>i-9%q z=lF{`rOqbU#lfG)*Gzm_oif#F#pKgkV65|W7)cT)Ud2C{Y7Cz-@sIe580mwidPqG? z4l?j|)n}@A$m5+d`Ut+N(7srPp6p?$cbV$l>OHdz-O19t1(_Fv{G^oeJ=U9@2R(MV zBsTQ{!_LfUb0J`E60(x~36fcnroQY!^sM9&rFn6aD&bUH#U1w3U zStVSdlx~kM8MNoJ>f10ck=%^b!t=zub}kpZ$&<-mPR0t?6calKT`ADsOHx)b(f9zX ziRxlPvbqU5yQQxnoL%D~w<=Qu>uW+90HvOE0kU0%RqJK1G!EN+7Iz|Iw=~R&QU`d~Bvq$= z%|2UXi*!R*LI;@`SIpjC*Ki3{5n)kjMdPXcQtHpzOczFONlHdW$?RtDR=cAfN6XAF z>>>cIA#gk}Nb6FoWml7bQV921_oX1Oa_2O4_{23eD1()H)nD z+{|7Oq};8b+T=`h88e{4m+){SlWdz2BN(@B{=(p#ZmJ@q$)t$U{Q0XN3N}ovlG>aO+YfV`rw32lCsi%6YS{M(-bKuJ1I*$E0_^>4 z!z5J0GRE%Q@n@e-eWo%Z4 zs`O~pI6`b*m1{1aM)mOxp%oKYS^GS$o$Xa$cm^WfR!YHI0qjucmHa+xjVNQfT(q_@}U0;W6AVUt1Jj z!q(=}#+uRzYb+&qO_N^mUOJB4N+DO-fBbqcroaBTGxDAy=$ zXJa$7ubrJa>wk)?_v^hnOj84IEdn%xN**Xf^yf{y+1N#Y zK1NSY(Qi-V4E=MAp7;pVU5Gwk^mz4Hg$&mR))C)E2JG`frtZlckCh-O;K zSxb2ke2iSq(TpF*@6bl5f7dGbt2_<)!`z)pd>6lqCuv6^d>g;V*i%A|C-M8F@Z%$- zk-c*;K1vLS@?%-bKOm(u|D&Auhn(Mr;0(iAse!Ww{>Z?`m4QzHm(Xh9k1t}~MfiS- zS_4n9`x5H-aR~u_-|!}U93wJs5%|ov*iuoU*WBM<)aXv(PdF=|cuBs+ey|9!6H7ly z-k&N8vMe9u(K$gL#4|-C3FU)4ep!%nl;qi>Aj|VXo}3fpIgGms-NLuTkATFF3GB(o z`IrYMkPRU*Ot|#k_+#V~f80Y{nkDA3-zARIwj==7l6m7}STc={lh0qy&4*VC zSKxE!EIx;y&g1iW^7#V!{LS2a&R>yFkB855=*{QzCGvTde7-z4pO+aNoub7@b(B%q z4recpOyH=F*9SJZ(Rw34bg(|w>}TaNiMwk2VtvP1<=lN1W^<{0)Rew>cIJ!X!VTEW zA_4s0tlv%?zg{Q(*RU2}H zAU>;2sRN(mmruuA#{^~U^VzX4a7=9ECipL*lEHF@pH77RYXkiE34{BNju2uKih zLfPLE;AQ+B+Y%4IP>btPPndeF*KbD)*U~`+x|(^U7T@EX z*ZJ33-{*|iv4wd`G_Z~N2-uok_txyXx4P?=(ZAQZryAF~n)(Q799_tKM;9_bql;DW z&5%t_3tYaVgMhpIFHvedZ_h_4XRC%0QJbQfC^5mpKG1nxT~(9ZY$RBjO-Z;;2EtT!)ujCz&JL*bvY$UBIe ziPwxPZ_CLlf1C&Yshois4@NF#|1-YoO1F#qE@4+fUl~hiD-T4K@GtHBp2e1D7h8_k lXx%)8;a{>MU8e;iT&VAPM4rVefm5C`bTeCDXP*!M`v3L62p#|c literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorNode.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorNode.class new file mode 100644 index 0000000000000000000000000000000000000000..075c28ad35f95d7f1256ab3774d9d5f98a6388dc GIT binary patch literal 2779 zcmb_eZBrXn6n<`A2qar-X-gGbrLAp4f`ry;DM{0&6x5QU4j4y2xr7zAgoN2F9r^AL z(is^Bo$&|wqa2@e?`AhmU}hcsa$nB9`#k44=iHP0{m&nN0eFbDEG$&AxPT{PTvcUw zn!&Ry(#BHD;W?Iac!6aZR%G}*gD*1pQem{YXur>*c#{k8qZE*K$*pWbfO}9jCNw|FmnDy1wVwdv1HXwC{DAPPg0fN^aYCytduy zmcH)U);e2GrBJArtE5P^?LDGT81$CSkJo*V7(ySyD9p#B3rYC-!higvvawp;XlSoS zDsJ2Ls|r{1!;Z}~==KstC|s<&ZD;LZZ`1KM?9G-I(rMbQhV8ks4~!{)$L%Uq<2iIj z$f#(t{ucP*eve?~T*!UGmrv0f4?Bvy#OKxdaYC;I?@E4VI1vgDXpU8ek$0@M9#(&; zH-?P=Q-P1#TU$%=$P{Y%c+Q?x01}9_ey|G9&jqEQp8~TTD|%-*q*3&1kk_KH7!xi` z#!`pf;b*@r7@4NoC_IYi)DQ|W_DzpbxYu8QUv0j2ntr*jfC7u~CU>2~xBE|U$XLIz zTL*m8Nj3%<8`6{Vrc0Pyv^rfm&&#e-SzR2aXq;lJ4qsEqu6GW+rnBPeE)t59W;rK~ z*}^qU%dlW!0+%gZ!BvIJQO~Y za`MEO(d^R(C1jrp*omy=CA1&L$Em;%b0#(ulAAgxw*uic&f;~t@H>*fa$#hTXydFV zHL}tHX(UuEBJ>ieZ=V8}8ibpV2{(?9g1Gg2ml=e67z@sbzC-`+GMFYEKXR6ATNIrs zo?tYA@1g{llrJEM$C|-}(+Y!bF(EgICy|KGBoa~rd8VpYeFd)gvN%c+d>gRl$eJSS zSOSBj71rC6KGIxBn#0GK3rI~uw4o>5p$bC_!C7j_>H z$RRR5;mTA~bgL)vKL1qqZhgvICd8l7#^%9M#J=P#ce3Ki336)4j~A@&<-QbV1Fcbk PiA6Yj43Dv>cjbQpSWa~D literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.class new file mode 100644 index 0000000000000000000000000000000000000000..028361c9e961a4d80ab65ac793d33cb1c84c4b8a GIT binary patch literal 7712 zcmb_h33yc3b^g!HXx?b_#3F-0VC*rQC4m^g7>E#ZBq14LA+SZj7~A-1^dt?8X5?7_ z)p5GSZQ7=5N}WJbmpHMNq-iz>;fd-cUD7s9(k+dfrhAjRP205TmV{!{|J?URGeW|# zzZSml&As>Cd+s^^Ip@FUK7Ibx&pZ!co1~*~F;xv4m-O+hg<=#TZ{#yNGOdqa)5mib zp11IVg%>0EY!o&4^(dnF4SoD(4Soxs)7Ed-pa;Jb#c}+uh0jOudr>UK7xeWPEqp18 z224fp`}+F_5&WV4{!tD77+==MpVZ(hc&P?|im&S9&n$c`inaK23x5&8U+Ubyief#! z5W!zZ@HZ;+Z!P>?6fO9=F8@Xi{vIz!@DJMgM}2%#AOEBi|JlO7M6n&;(vg4F$G=t4 ze*HJ3y>CbH?|8|=e?-xa|FrO5`upEC_zu1s!S{6KWu5jv7Otq}uju1d1?6f)XmyQ9 zjRYj9t&r~bazv_hB+QOjH4;&~dZaocQ5~sKRbR5ij!10{j>rO?yHHz;blPINKT# zHR$BrJ;|(KfwwN1iT5N^F4u%}`ApW$2^M;93i)IzzTX+=tLlN|NZQF4vJ`b__S?3( z+Z*MbaVIhA#!ouKC!KgMpLLxvsumy5W)f~Lm&sCBC+`mEmb==y#X@ea9aX$A=2F3p zbLAvbNjII(#S{Ldcxg5TY<&aM+m3Y}>#W>$=)mxB&dt;K-fPWMPs}nVkr^Ayq~nJ& z)z8**V!WhlwEIlL9ams;maG))`9DDA zJ6i!;)Y#|$S$@gRmX-(_BvlO+RrQkv5)<UOl=x(e?_yLc0o96B}pYgNc3#FYgCajf6xlXz;&I6#)WcDyQ z*^*WC&2Xxa8>NTc^a#_O1Bp~7r#bFxFhkS2u8Q=So#1b`_iOj2({8pa<>Ye8>`P~p z+gdY=0T9x#B6!9T{YIn3FE0~HB)p`*_Sj&Zq`T4mRn>sakga* zp0;JJTKR;mvt_;9Dp*}EDtc2Xcf?6`WJh=|r}Hz+EV$W!6*F-Y(@<`WU9(xEElnzL zSy`Z$E}eyBYDj4}Y*LE)G|L9Ts@SNLi#aJ43_}yKLDx;kN(}7|ZL(#fth1!WmQAwS zmbh%TWQ#3ZN1E!*UFTeiy%!MZun&Bv&=?36drj?*I#Yu#X`c&M>VD9dxt z@GOc=xOrQ4NtwT|6#hj$%G=(20-0mzJ|s*AM+PlP*!Tc0uyX4n zm(Q9rBzG_%47Md=_gGrcP&t`37_rb3!R+tK4wcW%huOkYn#>cL)*~u!QQ22s?%6V` znUa*fww%DzmYlRDC1bN@9vD19153(~lX?$SBt%_V#a)6CRIi?5kwPDCXHvTvz^E1|>FD?DDvccfBm45^RE=SxbnX5Wg!?!yIE zv^)p>4a{rL&|OU|qh^g)C{*L;dG0QkhiljwA7>m2S}Mhqrq40SCpsz`$!EM-g628# zc-_*3H_{YY!DKvjFBgxw9tC~v^_4If^>Hot_VRURgvumm{8nFr%e*f6lZz~Jald

r8TgrE#x?LUoD+>|gZhCkhCu7OMN}Rehi8zR=UXlNX|N znyjRCVMg<6d*i-&YTN4utjFUyLOizy`hJUO{7KERs#D&Fi@C=~Zk}%m7551)WqPJ4 z!fMo8z8Yp3VF!juW@<^94NYmt0;#|g(mxC7Ld`a<3z=-Bn>n0j(K}v^ypXMq!CWR) z$h!xfJgf9Yn%0%yZHQsgyW>@Q(+|}OO-lZywpBy!V1eL=jn-6R`Kny}bBA5+=mvWpv!l=4dUabHA)RmN}(2jMG_i1537Rjp=dB ziARVi%lj_HN~_W^bJN)61xqCiyKoD?uDSd^7r_cWP}y36Q6wQa!FB+(sM8{w?K-Wq z%`eMpzJJ)f&vKj<_DsLPK8x*{{%Q7EcUSZyY){|^5X2*t@J)oF-v}RUz65Drc@cr; zjTaGYo<`{CB~)ERxVhybta3k6ZdOktdUP5!N1H#5U|sDcEVzV)ts(ZR8$wf9RK(%{ zv{m;>?hwRV`P{@$ia?wnUpJ!~Td)9I`Ni`#+=Ojt!tH$Sz&`B6z1W2Z(1u4jdz+^r z-i9A!kGdK}#IzKXfN3cTL9ShbAHok){*Q3frmZ?ZhXUWdo$U}u9`mvLQMSkMV|*$S zasKM;Dz)YlYy^zLVKaD#QN$Bh~0M=gni^1wPlcNZ?!fySW~#;MZ8&mJNR2fXAqNDzif6h zYSf3$o+qMMHEZaJs~3z_T_vl0ox_|JX{irCi#;AZvo-4uKvN}TBJW-Ns{qi&6xoC2 z{Ch()?!*D~ktX(&BKmQby$d*qCvi6`+W~wCZ^1`#n155a7hl6s#`68L7>=yLplm}z z+TqFpB;_HTkQA@%6UfNpcvvQJTE2uc@^#*pUdAKx9lTX8Gi;-?U#jC%Yueg>B2K`B|;`dR#(<`7#yk9Q#;YvgA9 z0wn~g#aHl)cpM?_u^R7YD_lxjU)p;p?Hu=P#=_;>U%@7^aDM$|1TG^USbGJ#Exh*% zj)WFn!6Lr7j7@b7-0OY#B`+`0i2(lt3y8DkpNWi~YqR^E#;$sAW=V76{)8c7!7mf% zfFX$sq!FddJ(L%m!kv8^p5Gm8Uv=Y?*tTKSR>OT^WxTy6EYD;4r_fusPe<-5qAzIJ zaQ~6Vf--sai)B(=LAybemJy*wVzh(!>}J1x{crQoX0} zG*u%52bOXHfsmzTB_*Ir9;#51H{K#x4yZ;($u>5Ol1KV=?!C-3#(oj^1;OSq<-i`J zd`0hmHRZE<#&N62B|Na-+wwtQct7VF;Yl`(@Z+r^WwHh(9c8grJtpYtQb-l?kn(QJ zhK4Gikv42>s4~47Hi|kkX+H^@uZwU3I5K(l?0g4lf6K-x400EeWkM==p&vI8#rG5U zCyDz9i2DbzmoOeCkV9m<5qy{j-bWZazrxXvVHzLD=eYhGcn06XMSL62N*G0{!E>?% z&l`X3WuEQCMy~z{o$(ZUx%Mpm`#8Deqs)R3CB6s0%3SAP`#7#gEaOR*ma>JdkMXUT z{EhLQC+*{Gc`HwA9GF?BIZNYUJ*-k3bi9HU7Cyn4cm;zNKFPHH9wH{s8!96DVXg|` yQ#@&(nThOMC{6!W*V)fv=o$2CfJ(*Dy^c#h&9MMY|1f`}e0zj{y!=qO`hNj;Rx#iJ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorTopology.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorTopology.class new file mode 100644 index 0000000000000000000000000000000000000000..cfae1520021613baf241e9e2e2edb5a2725b94b0 GIT binary patch literal 3668 zcmb_eT~`xV6x}x;jD#U7MXDABTa6(BQYqC6lv=2^Xas5mzYrbb2m>LLl?*O<>|f|x ze}S)ctzs8_X#3QM{)@i#H?Z2;eeX;rlK`!0dEj!-{n-1QbN4x!U;p~yM*zdHW9Y<< zR=kC*9B#^Cx(zd!mBW`BZfVHL>FqYmVZIG_a90lZG~AD&9S`KNpy8q1UX%wHH9XSr zSRh_1+j+B6vCGr;6SE?4B)e)nGcp^FRmfzmiZd$EnzNRRhO<#N1!fzz$Fg>LIb)QJ z{EC@bGoG#)nTk_3jdiQIobjJ#|a;=*c;`%Ik9=$E?IfHa5%tm z(Pb>$Q3uA-}YKAP+cFy>5IQ|B7j_J~my zM@$N{3e@7_3e0#34^}{5vril-0fDF^i%*J3m46VJl9Nsd+&mE0YN`Y@Hyq0IcwIIo z?LxuKJC2oIw{|Ty9g$z7eq)Kq zz9KM^d^0Wh0FyTR^K)lZ^_QaRFXL|pv0r27>*ifhflF@&waXiU_MBtn*VqVEH`8G7 z-SP#y!fqE%N@oRU-2uto+9*2Kx_R|^-Yj_nYBj3{e*o3b-hGyptA=ZqY@A&_bG6wG z$|BHl8WTE>p-aaJoYc{clNy$EjGmyap(%QQ(l} zlXcbAWz`8b3~j}zOtVOo^eCZ7Q5l`p^YvVmscDpPhF@x({Duo7B!vWlw`qlNoPX4m zZ@)vE^78Fdh;e=zJzVeQjj#DX!vCGA?;%p@Ere42TL`BHwh(#A^)Q)0{3yrwc=9Y- zaSm}L$bTO1ay8}N=k8frMDqNRGwDf4tw|rkya}>~y;H~OiF-G?TKOP-FUzC(B$QGsu1g_L(KYA2<<(cM#?H!e~i*HU{TMfOA*{70lT`ia*|_6hP! zzvI%U^fp>T_!(NGUqcG5%RT95DUJaQ5=S6|_xHeyRPhcq;$8d~ybBm2m-Me8XRW() zHnMM|AJC`aJ&~BRJ&?xtMf#9P7jen+DML%XW0W}yHHJ130EyN;;YYm0Cc;q$_7%aT zXcH1-8!>)C^lzh0;2Rv{L>GLA_UVE2E5u*C{=FvS4s~maBD+qe8pDBTlF;3P`$kv9?ABbtVvO_H&1>2eD&JO>}`j{2j-41mJ9D0SrFK}oiaw3v_ z@%lIS>``h{NxDN#m|afNzaX0a=_PS{=t=@d=++rB3_<5O!|-$5nd9DF?#@%&caX+C zD(V3isFwETL-u?Z~Ilr&F`Td`te*tg-nGl*W zsA4Dt4+ev1LP9J_u^kr6M`9UKF&ab=V`4KN!evaTn5@B-5Na_M;+qL^cr}ENaqR@A zab29HRLrQDRp7~gtss)x(C_N;qHAX3bB<{{rn_~sXcUd4g5bQlX6bIxF%;Z>lihI2 zcGlv0UeBx>@lE~CrXDZ2j-lsFYb~C4?2J(;*iPKET*I;SY#}~Zv@&i{cTL-xr_X9J zYa~a==5zJJrh;U(c=M7iO=p4xhE2vtvZMUc$MUeX#5gX!Q?RnRsa>$ z&bxYMb4Jh0@MS}dvZqwsAXO%77nmn+ry$mH`qsWE9{M_`#c6Oq##L6TYDxP|HOmUI z=3dv%D+rakE^X#WqfGN)-Yz;BW6BgoYueYqu9Jz5iMH2p8ZiZ@-Y^0hPN79Z8{XD1 zhnpHcK}^Fu(h4pgzv(sH!bJ@WNUK=X@F|uQoI3=Yv2!`wieDFlb>&{h$cwpA(41z! zbrja^Vs^E|vfYk#J)bu$5zw;0`b@>=8ot1nDsF4|3Vj-MtSA^gL~!I`m1$E@FO!wk zIo&r_HjIp`ApGhmz3$lGl;*^%ud=#Lou!N8fJ3rtS$2t`J8X!&VQ}gjJG&0su}@r< zv*7AE7TA}!vsnd0RYvgYdLV=)XL^ieRie!Or{`V^ixj-=U2i+q0-AWGl`cAa1J=xAFWu2|MZ20?$iAcvW<%h^pv@qN3+d8F?>aWm(^n z(@X1!l$DaQcA$0(Q8V3pJ!Mh#2-7IcNX7G@s4j|%;w0f)&?7}@geG8Fl#tRSq=ey- zQV!~fqpzex8KSdRD-eqptP6N`NMcc8sM z?K9LaHJsc<-SY@f>JtI+DiYYiTViQw{|R9)gqwKXC=y@7C{a9m`LLHEhZ1V#Mr_Y?Mm7mtV#OX(Xcz1|&Ube}0rKa2bVRx}y(>D13Mwqo^&i_J;>mL}H Olw}RjWPs8 zqx<{%%fXTG&2&@dXanP9F=qS>WkW!beK)Mr;+AQ}DX$$RNqLf~1ETY!^~ql#UTtlD z0;}_n$hGPpkbg(E92SVqQv|R=(W|7d5e`sKY*j2`i8PAFGAcE$_p zjIYl4qa4qcARW^ghq0M#cTdiK=X~Fuoc#Rt^*exRtaYImITL*tGmtkijtM^VcGBrAfU zuk89_xl!K`YMpO*WEm(+hkMVta`iekW;787Yr*%0DtSB%1ugIYXCm1t;$=y{B~WOCsVG9`Tv-7JtQUkCT@3ZyXV5q!~>z6$K3K0A;-{ zeS2L7uCvNj^$H8VP;dEfD5NX0^&H)X}ZZ3~AnU||q<7)BGt<+LMcVAjH2%vrdH`+951!UN1( zc!);^9$Q$zqJbwC%2=|nj1`9Ad8MQp358#w;3KM8mA6I=tXg=AXH?jU&=Z-) z{^_l3Y*D#0Fyz`)liVM zM0Sy(cVF|e^lJ>pEABPKN|j+Sx7f-?Kc@XX%?*2i1QRp}tk}wD_wZU@#@Nyd7n}AO zPie+lKrMghRj0S?vaHpFZh8HG+emVu3z90NHFB(Yi?-{VO1{?pi`oE2sGIkJ;K2I( z^=m^-K1KH+hRAw^X0x=`7BfHo5iI{1srOBvVH!Kg2N=as8i#kia11ACMgC1RMjj_| zD#m(?3|&BR;s?gRA^mdv6Ebuk`izd{{0Fcd7?)|?xr44RFw=PP4!t|*95!x7Knx~OIM0DN2WHXyQJsG0>PvZ>jpoo9+D&j1$)_E6k4(BQB3pCFl bMc=VBvNTWOB5~5TOEey*+a&*9`lsJ-gh1{& literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/RecordCollector.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/RecordCollector.class new file mode 100644 index 0000000000000000000000000000000000000000..475bcf630be429d8d5b525e614f1204aa11d97f1 GIT binary patch literal 3402 zcmbtW>sHiO6#g>z8KWZJs*-a-gNASA3LV{6yygY+%h zT1u^J^#Qv2x3AOHZzo|GoM0WDS##!`ea_zd+t+jc`uC4N0Sv<~;xNW#__&0ln8;(Y zgbY3r%9IS#dE73cfQb?g;EoJ;_u(Gy%j`iOGx9&X51(RAhKDkImdAVvpJO49FBGVg zj$?Xbw(k3;uOMx^bp_qiu2bY24kOegTQ?L*qV_%u0-GgjTvgNA1+xYz+}bzyX2v_=7WcX`=21L8Y4y{bRbwMJms z+HHMn)ABwrv`LEAc$GTm0jVZ55Y0F(Hmo zXozn*OA3DMggt^)X}89Rg65mF-nPEiC1%=8ykb6f&LI$UUg(C-(lExTVE>x=cn3nV zkpF?rI6upV@Gw!NZZufQf&N#bOe?S{X?{@5R`4BPctp@;eh zc6T58Zt8j3YjvcOZb&Jg9vT`_a6iFiomG81+|+jzyshPeXgNXylG=ZjEk5dTr5U&3 z8RobpUFt}bcg@anUK+HD<9I{GTXy49w0nd(HEkagiqa>BaqmSNnE@ zW+wFaw^V+tv4s+=8ITs24V$2XYcIu;AcU5d+hVb2R2BS?;M_fr#|=NBT|D1=n=gRY z!ZKXPXm}ocZ8fGktbu)A$20%vLRo~NysD5;0;9Of(w zv8IQ6BIz707rTV>gC^${?nZ&Q%3U@Oj^b`v$?1|<@Dzs>2-;NW*HX+~#ElSTx(5x2D-@1s#4k(R^-TydAt>_zPh%j!1J<8biaeFsgA_1_=#=VMOu+GlC%uM{pdLG$%BSgfRz6X;K=} zVc5vX;H31N2(oZBuZpKE8p%EjNbsf`4S?S?e#<3G-HfuXEEA5(2+DK+&J)U5f zJ!r-iLK(wxE!VQs3Qgsj5_T$Or(?Z##!4KR+$my1){uQLXXXf?zVn1JYQ%D`m5gn1 z9OKM3^JLD`-D3@>4L9d-Q=jjqwsMP4!0sBUj5*kCCTwRg-cBSD*Gieaj*%EK9fd73 zRoF}EtSBw0pvLP@tHZ~uMA9NYmV>5ZFrkD9rA((q>mKbm6az!QiqMN*D{LsY=Ene^fJvEoh*@Vclk8ew zuJj2mv<-9&bV%{&^1<%I(&?!ag*81cE73k9<1t^uM+z7HnDZ1y%DtUt2!(TjjGas> zZ1y_}erif?jVUAwm`RgYT#h=k)>(43vPleRR(5+TEW#B79U8vW@f9xX=)grCU*ocdD>}Zxw=AP& zx@V9%iO(B4uHukH{X1#C*YN{>)Nu_z>FATy>tgd~8Qze6Z=zqJb%s_;lFXD(t#Pam z50c|m4OT^uExcUvOI}5sEGP=qmK0rYS3TKVF3r=Y9XGOF=4qFlOnaZY$(>Wmt1Isr z66B%f9@pWMiE1{?(A5Wbqr$8~Gi&nlOY%(YmHOUPdi)C}20983JPfP0M7A$~0j!%A zlp-@rO0C8K$Yn!!d=Qk>y+^(`n&o9-4-I8aDvA3j*}Z34YM9Zre(#dUKZ)XdiWsjOg&6UHZi-mld zFX2yyAm&PiVQ;RK8}G}i0)^%5S0KiD)`sGE73Wx+xK_=%HCW4b9Y^O;$$wyJ^fpwq z4wYjFMq9>E62&jg@IOa<8 z%w5!tV_vHVwK!Yv%|6>&wKP8!Y^a*Ra{;W=?x3On4(9Y1w>~eu`3>B08!yyrxA9_Y zh!(;Pp>Zr=r=}q^ftM=q%l*;lI2wN?xPTXp#R###2lLU%6z{`Mbg@Dnz&UjDBIv;t z^s>D6c@B3o;VW^JWqk`tvy)OC>{W6925coEG)mQB8@5yDe2iiT_XH#i#e`kMF6{PV zevQ(58Lm#Q*Ylg-zpt==7f~qe=Nf-6-h-y$z`wXu;Nw9qbrV`XQ8y?eM%IYC<5+kX z4V^82BdBhP+f2aA6_DNpUf~a_Ms|Z&Z=yQdNOrbJVk^BkW_6+`rO%Ts87Qv#) z6C``m^FL3&H3e@XWJ@o7RoqwW`Cs87)&=n#4X@wB!T?u!2Nj^M;#UY+orn6WWTBLk uxe{7c7dgrM%3prbRQdOO<+tKcK|V%nlLsRB+Z5v$N`!+&;{)EVxBdt0%Yg>~ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/SinkNode.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/SinkNode.class new file mode 100644 index 0000000000000000000000000000000000000000..05c27f9babc3a2319a636fdc41afad87541a7a4c GIT binary patch literal 3306 zcmcIm-%}e^7(JUG1VU0;6e?O3T4}=q4Fa`=L|U3stx%{9jZ&YM%?evKyP4fhZGHC5 zXCEAA{2!btWwbNun=}4Vj_2OZFWd}}GG-<>_kQ=@^PPL{_kA~i|MT1L0Os*k66bKU z4`*>pG|Qs7oj{=%can$)Gb{bLi%&%JX#)3>=*MRXtR|7hS`zoMp1?)|4-~|Gr)ug7 z2G^cwPqnP2**n>y@0#{bPCpd;2JC%xK zXTAC=%{Mt}?IB2C(cG~$zvdbWu6GTP-(1Zt2`W!Ds|EXdNF|As&>h=1p7|XyJl{36 z3QwP{x{hvmp5ta8G(;^2wQODMl$$6ih!vbNRUTL~ZDYMw z**4rwZQG({aLv&*tEBNNIj$RH{;ugMSP1*-q=HG&TwAleY|*s$*6D~k#B8lX_~FCw z&#Mg=%dd+qu6;AOuOI#&iG#oj^E)P>^OBViyH_ih3%jOOR9%sZ6G;>qks?nNES?aqDN7^O1V*~(Yx>@Oty+)UL_IYXTxk!D zjqN9f?&sRFuWk_-QC70UMXo+c;aOQ~4B|fBu`D7uEOfWJ)$pOAJ8rqqK%jzAMzCm0 zZ2K}nI0yG^IUe0bGh$TDuP%lempa%^7Z+{$lq@^2ZPtd-vSWDc`J;yL^PW(jzU#75y$9xe&`;Vs5P)|X=%bm?g7X?7UP%IDqcfMMlVY6!g z*r6yL6E3;#MeTmaLf`|wb!s0Gb$TCBbtJZr*e{%k;(akMSB|yt+$84$e^cyD<0@OB(RaiV{xa(1 zK6=#`i2jANI{7R56bPwaAXYadg+Zzold28!1i%dS`H;x7Ji}ZYvrLm&M&3=pEJd&M z#W}$&%Wn*k;j^^P5GA;^pU@+b$LYKGPxQY;YJK_@(pwxyCZ`T?Mz9pbUSgniiD#s= zH~3e7fg&tQDrx}Kq%uyLbCSY1FLsU&`$_H^KH}cim5tZwfh1>fy}AAd*9Q_V(6)#i mTOr}c{EA~KEi%DP-IOGPGxK?n5hUESEJB3_XkG$Eu=FoFVwBzh literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/SourceNode.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/SourceNode.class new file mode 100644 index 0000000000000000000000000000000000000000..16d8c1754659b4a36d1888524655a96decfde333 GIT binary patch literal 2590 zcmb_dU2hvz5Ixsx$HdvzX-Y%OM@b8%YdftWq#sFK15Igxgo5PAFXiQCvrV_Lz1DhD z2oi5dNc=?|6Inynz?`y5oQEg$j%7LAYJleeFK+>O1=J zj;^~ww;S~82d8x8(@ICJC``8e?VgTyLr-Dp1SuLDYb#CWDL>Plbn264YscA&ZqSRo z?;?d;Sxkc{^mNznZP)u@;Ch2W5Z1pw*sYpCBARIUJwIwH{NSuV)z5UjqZ!hQLPoA+ zvmQ^@Nn$E(h5Tx;MRv>Ue$RWj+uiiS4ZYc+XJ$Qcb*HT)R(y zVNZ7k^;WPOy52*UV(DqFdMo)r*1SChXAHDQcPpvs_-ga1=SC#35CwhTC6QY?(h9jp{N^r<4`=HN)2Ui* z-PtAiM;&jAJ~EmL%TBhzag=AKTiQF-inP1)rS1<0r#So?6t1R$4qM7LDLE`&BQl4{ zm?RAFgO)a|xIc@m^0IJ@&JtR1JHbHSJ7V=4Ys=Xz?J>UQ%e9;g_NaN8?bM)^%*5aK z&q&;@_6{b>isBj~bVq8}vJj1(F;igl`s&s614#=5!Xb%4m z6dwoSw>sRyCH|k4_;+U^C&GXuOe=@e{86KkJx!Z3@!bS{XYnfSbF_ZuGtcLAwe}lS zb$%bY>a~4XztU%6mg`LX{vryPW2H;XdKs_LUo~+!j|;Sz0k7i?=9Zlx$;CUtVBt;f z=lGO&=kXSw=9Hk6U70^?i2NdRyv@@F-#7G2oLzm6Ty6dZ@)mxfZLmo%z?B$}$#0MF z*u-k^*hKse^9c%JL{@{saP=;ISH@Pfj;L7qj}_lz#j9|*iUvOtoABUNYx^ijqkrO5 zwe|uN3ZlnzSVKoq9I-a}ls0v0vq1Yp=3I=C&6<28E}U3YuW6g_VkoPyI*`+@euI{nZ9O`MG}Mv=rcF|k2iK-fQ~PnZHTWEh(G6WqHr z{sJ~8HYP5NYk!k5o;M(%xawl=yYIaFaqgM>*l%g;wJ)`n%@d!Jd7BSsy8$e`#m*?Wv?$FP6e)&{lJea z0{OyXfzx6`z&r`hn4YWpfqFIUHC0rX&2G{$^klaoBVXSqpc%LQfxx3`7`0s4mtI@B z9r>ms-9a2F+4D)SABCP848q9ugIGm@><-+TrliiuFkk(v?A!jI(oe5gv&_LOENm6% zBqL?Ku-BK7s)qub|Czh^g1UxrTSWpZ$yuMNwoIyC=cVjVCa#nlj~At@EJAb0Nzz}- zMqb%kIE*~?+}EPo6Cv*FziDF`cWqc$vT+@@z{9JlDzN(RatLh9pcx&mNuk)lK9?*F zZU+q9(EaIWxQPa`ypu}e3({L5mg8-R6mx)F=?g^Z_!36RxkTy{p%hkW*OKQg+HIEF zVd5@J0S@1p#`kre+8ejio^E7dgJ_=TIBHV|B~jTasXV=#xJ9d}ykied#X0^C^D|P< zN;J;<4@euMvHKBebCen}#+>8kaLfa;IK&=~=D0aiZVt8*DVEr4d&P1#&<>X>jnw1#xF+667=5#}M^9;-Bf@$xm+~V54P!IX; zAy>7sE)3UEHPdja6=AFEzVk|Grd`TzX0{n3>t;zX3>0)jY*i~e!hX$n$`nZ!OpTYf zxvk5!AB?#By2FqU7IR!%@QQBisg`YO!f{Mn)eTqJ1}{5m(d89Ol>EZfx9zpSunTyj=*BA!VnTB49u1##qPAqP_x~rQ8LpYszLH1A_H*fb3!_afY%T;PR zYVPhj!u4{9!(P{{BQkslvewabbHTiGX4xSNlJ7*(tlFB`&^>LQ(bJq9nS#rBq9Bc@ z4D;Py1qHWoTfqb-72LsH2}34>DTd*HdZ_cWR@KWTVas}J3Z`Y=I-W7~&!kRenOaRL z7{!=;^_(FW=%*t->$ipO_Wz~Ry+}9c#TS65sAMqi>43fmQEpknC^5`7CfARi4eHNl zg9bF3-faIGqDY8JGdW(evfvUuWmezRSA>2dTtnouw@ns} zdauU3f_11qzaW$1XbaKvu?b?lB?L*UEkv}ri^-M{B+RxDv1W)&O9+znDMX08MlO0k zH63KOB?PJ7huEUANQjB=h`&8S;*{0C8kFP(G8E~rVW5VK&Fw8TU{Clwhy0<$Jh1o z_ZGepLNoqBU%qMKTLJuI2I5|;|2UMfEO+NNb&rqh5rg+F8TX;EyWq3tjeq!N&Ec|Z> zEAUJR9r&p({-1^axA1BJKMSD?uPN@=oACyowQwec%LP10gjl6cAN9@BAdNa}(uWno zJ_!(y1T6`Lunx}zq?x1G`j{1vb99{DEOTUTNLpl`K3Wy={D3S_=!Lp+Zb;hXJU!}s zMf)~ASsap!WJyq#$}-(@F?o|EmMjlpk96qP6}mM{xyVYgB4JB9 z^^jGTbcLi_F0tfNL47E44I+;re)>p%4vS}wW9!rdMr_#x&lgT90-LXX0Nhc!lO!tl{2#9h{Dwl|6a}h;) zL(a*OzN>3ST7kl#EgVkrv(AyMV9nW~-BAHZWPxllm87VT3O1e{k}`%1=yrKHh;Ozh z+mY>~qZucg;mX!qBDY1lbJcB0AjNPjXOccR%OZD2z+x|wyjb1DXxq4DupcBgY|&dFpQ zwY+!IE|scS)@5?gaYppRtVD9e8KSw3}?SFrA~78S(NCU1$AS2OgISla~ir z7->{B5!gzb%t~CXep?vkOtf^XxIv#Se#B2#&bg-M8G+i`(YHeXTR2&BkL5)XvRDj2 ztZ-TMT5=`hn=2UBDpZ#iyHhIGnO&-~Jh-NI6x9a!rYI?*@p9kQJ_4Vam#G+3aE$y~ zW3d(J(AQ9cWmdnpG`CO)m6cl!9oOr=fC)LbMdF-hykd;S?OPdFMF@}Q-3^iKpj*51MC&^Vd4uRSV&w$G%ke!!X)Cn#; z;Pri7uK#4ZQck+7nDcGvkzPyI+OiIxwK1v$C&8|6ZmAjB9LXGH`eDMcrBBu~+1Ro{ zuC`^P^b0PoqQ+O7wp=5dn6PZTU#_+BWqgHs%a-e8vtVA)-@DkQ6)KK3Rfzo@qq^X* z+3~WWSa(+7rL?{4#-eMz$TWs74c1V3SqH_(Z*kUQZtu}JFP0(K>s-nkJr`pC2 zjHhS?%d4<>avyiH=~y(=z0Do<$swIB5UdO(6HdG9EbV)3*(W#YV?R5EeYV`JMjWx_fDBp^wPi&5ZFUg- zHjc}vU@_xvD(RM%33Mz$kK&_|{zbHp-d`Zc;9j636H6kvHo~q_|3$CgW zgB!QBRkGv`Tke$i@LH&(v+W0N;Z22K=qnVeG&6IbRlKU2X)~>JY6H${s}%1_1uav2 zjeHb+wNR(BelKBW6lHr=dN-Ayxqxv!)7a;(gX!d9x9jfn$(1v^NlA{Tnrs?#vI7YU zZh1#pspxt@uU84XH7K3by)<+KfBB+YD(sHLwMh@ev|2SZFPKs{7kFX%v<^lxgY(m*EMDe7ev0dxa5y_yykLaCSu)f#e$^f#;+oe`oR0AJ;T;~ABC6_xtN~x zx}#usM)+G}8SM$rDLJXG(1x`hWMuRat)HBValp&jhMZa2Dn<^wEWPz^k?m|_Dw5Me zyR&0&wGw>mFhfXC9nxI)u&^QfHF{H7xTmOs^agiDaK~ASrD0`X@0*jp`KN@>o+FA>(33$1?Kqe ztc~We?JPUc{2C!>PUOZ5KTQc%`q+>E$$FW?D*?^<0((4iw9ql>D+6-Y+2&TGg3W&X z_&IIKDvRz*krpvg&)3xGDZY59lhJrC!>+Ki3SA*gmz6Wx+wbC?DPz#k@E|*`u4;ds z0gO~{_{G^}CIlKWuk%sNFA!mMv1wut?*^O*>+Vq>jQ*;wW_RY3%UoJXUs&m2*P1JB zx`swB_s3GfvT~0vWa>)DUj81v9})gwJ{#w0W5HRAcKK#x^K3>o{AOhHXht^eW@HOy zM)uccWM2tEhVwdPiHFUhne#fqjJ#1WBQF)q$a@7d@@j$O9KyaAzeL#g5y4Lpgbhev zcoGt>pFo{H>cg#Qm_SolCSd6!5I%nb!EomULgABW9_DU)0<*$xO)azYn3Koc3ABXU z8YeLCBwB|jF~7%ZYdC=g%zS~izzHlA+>g~dJ6G@o+IoU*jcvg^&dcNco=~_g*cO_^ z+j^SYEKZwG;DUPWIdwO z0Pn}I@Q)bop|0=2ukx=iZpB6T06s|CUf}wN@M{PX&oX?NHWE^MAmpzTUT12jVrFWg zb!PS(_)XNC*}bH*!OYY?8_n!v_$@S%pC6I_G%RxPyZCMV4gxd+AnffQ<7xrh$BWp0 zk6&HL-zVfBaQ_gjGZatI!XJu-2P}NT!XH`qB=K7KV+(%*yo#PP*fITh22XZfTvpqnEZMI+T6e2|kmqsu0q zl04r2`3m^E%iseZyi3G`g#1(dnTO{-%8>G;O%u;b8sD0}3pj206ix3qLJg0hZMc5r z&<-qeu3&tj}WucUXB`-LjGMJo-=InmTYzMacC{?r?1$*OkF-o*r%~PZ@B5 zu74C;+$a)eAgkwM-TfpZkFA_4RRei!-^Nq;E}kaVFBxsN^HiIE52Zd_$FD7z zV2r09;$=_6s$k^YYBYb%YXBkGPad?tB6rdF;$%h%fc`<6I7!1eQLGT_2xj%#z1$ zJ=RtBdBzRCgIQEpE51kP`aZ+^4{!;?tGc1$y2lV% z_6)CVpTs-z7_LP7It}{`gI0}`07X&H^+c^}dVsRWt3vO9L5|9<3YebtPL=Tq>}3~m z9P{#cSLqxR+Jqf})M1X)vy^Bk6U4dRd3E~JfQNfCNmp7zC$VpMW#=SrsxU8lfojF2 vI8>xKR1A(GlQ8R;u>Pt@@nSS!$$ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$1.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$1.class new file mode 100644 index 0000000000000000000000000000000000000000..a0d5fba6f52490b92e824c1791579ebdb764d3a9 GIT binary patch literal 1903 zcmbtVYi|-k6g>l23f;E!VSV7ER@9c(F0{VE+J@A`2CKFN{W>foOxXw7U7Gm8A7!Gk z`oSOIk22m_T8aTo8z9W?%)N8YnLGDhe*gLT3%~>9V;IGqIAXXPM+*1UusDV#q}AK9 z8djpXAH`}EI>U%xmtHEv@Vw}_HJvxPQ5SlHe{68w^IgF$Y1j0o>lniG99NgNFI=0O zp1vPgOLg+8<_IV+ZRu|?Y%UIkv{Yt@6r8GHm@GjL@k_3Fzz@0J@})^@rYQ`CmQ#luJ#l7O>D4+Iks(e;Brz6IGR*a7P=E~+h}b?!=6=hiYxK?&!a%~X{3_rx-S@dC z4xNU&&iKC5at-lPs{ESjPU=;4b(DqMwqZIRmBDM_*PUt<84Y95G>jvwVGZjVmEFTA z9%*=tCmNpOnT8xDsHTP}sZ^FB5hRbvsRG|r4k&jRay^g(Gi6UyIPEn=ftzN98;yRu z&tBsU;YAh3I5$);NM$nAHOhT%^6ZfRsg4tttYR^SV_6!Qr>ZVJy(BF`8M@FL+IBXd zdQq3Lq-jb|Q1Mo&%TP#mPD8S$#BL2IcBebW>zNGZsh`I{Bw&ezN<3=wVfu|@f>tN# zZ37|t0`c_nH?Z_~guc)$ganNvOV literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$KafkaStreamingMetrics.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$KafkaStreamingMetrics.class new file mode 100644 index 0000000000000000000000000000000000000000..e687001a2dc6d12de058c8f291efeacff4222a8a GIT binary patch literal 2462 zcmbtWZBx`%6n+liCWKgF1<}4(i^$938f&e!1r>|6wE|Xlg`!ryWG_o#lQ2mZ`PiBE zNAzd3Geu^~On*SX^bhoB^mC`@CSh6jWuVj9ncUpxp7Z3Kljof5AOHOJJAmtOyD@@0 z21YTh-kI=(Vs;DLs(4V=WZfrpsYF{fePK%eq|q~RM4 zj|~i{(|#=IC@Alujwd=6bu8)lRsqX8p6Ym}!1QnG~+$m93hu<#>?{ zJW&a)LVQ|WW1A9(lbLcnC%R3TZjr-(U>aW}bj|oBF2<=@$CL9_cSQz8u~K0qJL}t` zvM2&a?NhKzr9c>XsOp?V#WAM^CYTRAPX;p;5r&e@3jZh1;g-+f>`#;fzq+XcTo+ry zst7I`hr)5FH(c% zSWYG+R!!tEK^SUBlhkQmDD`FQobzIx*kjG z(unTocXr>Fby^AY^`ZA9Et)dt6?3)@su3Ro_jw0X%VyeatvT~{+U+K;DE5JalhQ%Y zo3JpUp+b1vRGymdj_ml>oj~Y4(Ep>Ab!kV0&N0=~nJ^w)gsHLS9yxkm*2dmw3-=a; z{KS!_^pu|JW^lsE0g4hUYpKQDN*xY*WU_gBG?30o5mo~=o(kL&lPdKa?U@g^PeHhw z6n}UCab-HzJqSQ@7{y4d^y{*3v&-A z+32k|ICX&c#&{p&*+cm081G-+!vLS1*~Oqbj=9gq+~@fIT*Cc!BX^3w)5x9T@3!ZD zFXlea+~*VS_ZzuW{6Zsliht0adno3<$lMnb?xFU!!?Eopw!M_t4!5%Xh2KsGKIZEJ z|78Ic(M?Za(h>$~8N>7xQ}hf)dXA7@;Cp(BpXe2SCV@Z6#$Qyz-z3STRhpnRx=9W# v(FT>MLO+m;8_au#`T1Sr6MV)yleRI5DaJZ*D}MHEe9EiA+8BYrO?3PVHO0cu literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread.class new file mode 100644 index 0000000000000000000000000000000000000000..158d301c541f465c96951cdd2651ac1abd636916 GIT binary patch literal 12434 zcmbta34B!L5ue#?@-~~t6(JZASRgoyZW<;Zhw>!(I9cPP+%%F0%EJ_mr)oUS#nXkj+)c$iMBZ1(-3%8WEw5*~`52xh zuPa?VTgECE&yjJii|5IBtXz(hOSN3)YrMcsGkBoJHEx<|?q;FJi`-Pn#|y<`7oQ+| zSt6HOjhDK3nTwapwoVkHlU%$ak5}?4jZe6l~ zk>D&BpDp5?~DbRJ(Lmj*Yj;cJEVI(Zx@Q?Hl18)UmT%2GGE z_?O}FB&!Qm=z4KWy-G!N7j0y!G_7Fd27OJ*BX%uroJI# zb8P84FsNS?Z>%#SOk-ko(g@kqo7#k{^)xG~`OR;r;o6f;Z)S|lC{ zVeNyPH#-~-8om$~*6QMcV7;t2MakuB^7+>p-u1E}Z#1zZZ&M`fH=@yS#Je=y6vjT9 zXPfiD!iA6Zalh~wm(If<3_$Fta8N|v#nugG=GF$*hJ3Mj#9*3u2vcXQn!RfFOj#O= ziN+hhXXylyx5}c%%mXZ!MxIj z1F&c$jfK7UvvIN53 zd4ZsObHji^p<9Tp@O-8`zc@qUjV}BGvBcwmSUBbjE;0P!NPTo^7}gUeSj60ewF9G` zsb9pX^T9}dqskWy*7^MFnT}51O~QrK9AioM=_2=xKqwHK$z&9#b4m%0aOfeR4m7L# zcE>D-V^oFf;TU~t0wKf1j-|f3Al?9I{J!9FUnC&U)=Ot>T>#e0u**!crFo;WnfTD$ z&3>aP7C=A&*ML2^7K=1UA9SS6`lI3h9U`+aC-^G1(Pe z7mwA4H-X~-8y{d}c$4e{3-NiiMj3e%Y$RN)>O=5&QvUxlO)EEFdy6Ww3=6 zHo&uhe{q5m)0`6IOce*RL&|z-8j^Sw5lv@fnv~Ax0RE(+4Dyx$vX<@NRs*+|EV9xN1{VP7mf(LCM8j`j`Ky=)fzF_ zz>Tzx{M3l_#Z-`)F{o1fB&23zR+@Z$m|<@dHd% znbmO!{6uj8&c^UY!>(ty&oAgzG=BVersWxe>kz;s z>y7ErAY%Vy22CtEQI@RyVN03;VC`X&Fr4+n7>Ze3Gk_yzs&Utb;}O3h`5g{!CJD2c zEE9j!*-208^bvim(|zyO*W-EZ|5C4-9*3Cxs}`G zv0dk#+yQES5RNZ3?f2G>X<%uV0272sZ=D{d-8yy9gF4+px9W5Q-H7a81u0?xbPq6B z6_6eiK0OWLh{tp@PfYRgtkQWGcK|6@Yy4ZC9}>a5X@VutH^I-VDv-q%cby;R-AvQd zrl`Do2~*lfb-I&&4aez3hj~ECJ+Y z0Ck;z&yVSJKW)?b5A*?Yq|~8hjZvK+=O;A&qs~wAQ#w6B+mX=C1X89vW}{D$QLB6@ zP?@1jm8|imvU(%vYZfN7L%tra@@AQpeam9uGAmW<)JEHN+9j9Y%H<&;FHYNvRTY>} zP}>}h8I4j2Go6q|n%cFs22MvkczDJlSEqK`$uuUVc9=QiG0!@#iEBNr)6MiNY!pQl zsvE>b_ZXnRNU%RIF?>WS#JksFaL#2Rzc0RaUCd)dB8a8>xYSJ&W&-VU8we#M3we2U zxku+`_)j`N%TF^6u0@@u1dw8_yA&Efr}Ojt6mn}yhGY$=@t<{iP9!Xv=L2ZfdtzZk z2I{|5l-m|Z9g=}E5ffd-9A49CFf=0~np!x9^onPP=HQuB1OZ-<9Iy%2h zAL#rF{|(tLB`;V(4LZNduj%x@P`u86*Z2*c-{iNDW;4OWK{*AA4xRS#i%g@^yvlOG zuEharFr8lEw{?Dp-_`j&ATWI*X8wmL(Wc^LQFs8>${CY{;#BYJ{DDN|kghiM#XNC9 zg(qTch#S#Zna=+ZNL|v6D(cx}jNWK~Oj-x9x10@}Kje>u{l`qlR;f}lf%)bs2}VZTIx+t0{@GJ#P9v~fx&Tynzw(4uZY2KqS3QE9>#wg_4i?oUC z^qt20bl%VZWg41tI`asJ(zMYROByK+oKc~|aP+lKdn6lwz@55-aOYtDR(CiYS@`O3 zq9C?yToS3_@p-JR^L9tJ?#OXy@cAWD;mKyyw=>QX@nDL>ZL?K$CMP>w{23Gk7j)!` zg41)*K?Q=!+nU3zJMtX*K|x`)+3}L?_39F=W}$-2p;W0#`D!R1?J^uE(o5_2;uxwYE|d^7vxF5QQ&f=9KSGKAm5Na71+eAv%0P2L76*X|S9-ba>X0e?* zkO89`(Rv6(5pbb;xLD5-jhxc8(abbCZGlYagp_OVY)iXC9jjC-PV;KBN6;%3_jrLr zSs;ly1zmz5fkn`%hAg!R?S80Ey8hk6^7y*bkU&6}8;#dR{gFT&R9#SPH9>61V0AT6 zM5B3+CQF8Z>e+=8jV9X$5R^bC6x&Gl!2T}cr$7yY&CGCk%~-Zn*x^QLz--{T(Y8rK z=P?J(&E`f)X5xC9RYr)=Hkxj(2G@s|)*E$kxWA^hd3*zkC(&G2FUJ-~+ZmbRR$-&E zY^3}?7V-IEA@r-I7D2DVyfIG=J@3tGpL=VK5Pnv{l0BA!T`94WSwKnT%~6Peu!zao z-}KRL-7@$jI#f{uiL(c!M+zzEnnJ833!(B5ZZRK}$b zD7Fse(Lw1ivqndxMXTjpCLAo2E)DuZkw!^0CW}OCP}I={6ca+GuOzV=iY1T?4hiQ_ zzlAZ)f$OKqkO?us!1}QUDbOrb6CeWJy;)qH&Zft@BU3`?@uxF$J1V={A~SupLlyc! z91sQO&zoivRp`^Dgvp?5GjdWTOjc7IUO3I}9M-IK1KFd}A_7!ep&pJPnZ_b<)!v2W zq#b}dZ=jgXUZ>3Ssao7jo93^rOe5{;FGG#! zc&HJ*3pJu)phh&wFgm22p+w~ZRBpFyf)I?sTaQWUQyCceOjq+$+%YPS2DJh@*kkX5_9@vPDvXLz>r^n z3#N6@KnG#s;VZ_qQQ z51q|p>0F*l=W`WZK#zg5$Fv#!fgZ;btS7xu*j4Vt3mo(WUOVxo0ssC;PvQz|)88-; z)~3}q-OuSs_w6(no1E7|6KZVa$JxlMO;eBUp!r!ecK1RdAGdoOmKZ^!sf>lJ_9B(%6Xq(x>j z;WHtv&^~J0eT{ART^t9)ZYM$mYg1b3WFN7F2f28G~&i|8uo;zo?7W#&;o?D!^7-vavE zG2>RuXu+JjFzZgL1aujIs}LY+1KkT}z7HI>fyWMTc>pup;MDC&C#|F+SnkizXe4V8exiL8M}D6^L?}CL;Ia!EdWePt z)F%Pfr$FmR<7+K6yAhyp4FKSF1pb}S;obBMEcGlr`#E|U*H@w0H|Pa=hh7BVzk>4~ zfX_?xCB6!{EJ17xhabFv?^)o~3*TR&9Pph8>X#wK^Pn*ABwAkqtT^!hyWsp9r2Pod z{S+WgcbknxtB&Q)99^EymV;vHVQdBIrme4jo!iM z{WMggcX9JAjoeF)ebiUY8S*W;XKVCcF8HAn3;vIS!9@!A2-u9^Z==9<)CVz=e_98v zcTkqN!L;11*VAy^RLBGF3xpsj4?C%`gF-;bdz|-xml+j9l=r=g6#p91=<5LYH-TAi z(F}T zH6eu#1)>OinW7XR6bL4B)5oYmK7kmYB2Yh5A_!E1xg~-d(E1@VKPy!OANSFCh$KJw znd0zZ@XEoA^RR9vPhD*DRB_Y|EmU5fZNr-g@J2pFQ7PodY~W)dn}i55Lt*yrmd<;z z$gCX{ABxf3yK!L~Z7dmz1S>v10*P`geB9h-2{uUdHQs)M_xteWTO`rFNTuIV9qk7O zbs|MF(xO9cy#$&qrK6PJWn-iRD@$%Bsyfsd*=zUhcw-(S@!KI>Q0r93sAVrhGpO@24mSJljcy+DBr@oFerD4K2&!;ruZx?o{clYZAnO9gmVP3hjMuY4d)^n z$0PAKy`%8AvBfl>$4~>8z;a{Z@+GjG7eDn)qT6^p-N6&k>Yqq2^JIFTr_hHyjlSdQ z%CaZJVH;=wYRZ>zcL%^*K-op}abEcb;iSwa}DBHbegch8B_@Kn)%WOP7kFfoGSu(G@M7*O)7BWyB8t zxQnJzc@oOXP^J>kgk6uJNnA-&c`lXnY^p#tzaI5^BhRBvd@P;L$I%5`t!yWnt*|VX z1)D5W))92yV%bDAJ2VAiC6pz)N@3WCSpi&17sP-fka8}r0tXJeMs*(MuI&I2YTgfPz^ImxV+ zK0A{T$YS8ri+iUDfg(8bUb+ZDF2G?T3y+6#<-h*%wo8OxEl0R(wo|U~VpV0C f=Y**W6deFkY!py>J;r=&c_xl<*I@+s%76a@bI0(; literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/TimestampTracker.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/TimestampTracker.class new file mode 100644 index 0000000000000000000000000000000000000000..ccc05954088b3b819bd3fec17f73fe2579dda729 GIT binary patch literal 511 zcmbVJ%TB^j5Iq+P2#VqZ6aN4rrH<9O zz?IHsa!zK>oSAujdwv0Mg2NgPYG^CeM`Qo)W;DJXDX43O=0&P=OWIB;UUKFAI}{o; zn+;-4SX%|JGfYkBQbxfey9;_)kege>ltemoGm{3K=c#ccZP{pwb9Z7X$#~{lboO!%4d0A9Ll9#bUYBm35pvdXgyut`s`G!Rc^LD{^BJo8$XA z2iEKVrexbmx-WBikh#=UF;QT{{z`sj+6iZ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/Entry.class b/out/production/stream/org/apache/kafka/streaming/state/Entry.class new file mode 100644 index 0000000000000000000000000000000000000000..1b0676b4a72d786a177b36b4ce41da0f44b4a5d7 GIT binary patch literal 1283 zcmb7DTTc^V5ItX8dULf<3V1;&qOdKOMa3I5Bx>}9MjvRJ`1rM5VQFcbWlO@7e@PQ5 zi6;I4f0XgeZlx{71YdS$PR`7nnQ!*j@9#eWY+y5q6wCx}ASa)E49iJGV8*c`b~QJ)}R3TiqPZZL{1AVNmhHmazA1ViS)!R}?>^gf*?R8w6w9*cd zv18rn_)(crwBR%yZ(Cu>98DiZa%F|cn^ukasiM=g-**lxwp-Gb29b2JRn?8Mb{&a( zV8q*Zjugyd%iXhdTUYmOtFAxQwRPmVwmx*I_Q=zoZM|)J?g=GL50^P<|1GtC#To_9 zSt@UB%lKyCi|&k>8y$%IFmu6PXk{aDueIyBlyxy9K*=Z+b~Z;Vl`p}zM`;Sl-B!n~ z+V33MS<;_+OA7LWyAu#w&F>{5bh4K?wzwHDIj1?`Eo_{95KEGJrVVak#NTH-w^sj zCy)~L!8yOZ5fFOqrBxr1R>Ac1Wbs8WDI306cHuwD-e+0f2fw1W#`2=p{DH{FGmNF9 pUBvQT#JfnOlYxIte1m=$*DrQn8t80}3?zsQBo8a!hgkIO(jSn-_3r=x literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore$MemoryStoreIterator.class b/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore$MemoryStoreIterator.class new file mode 100644 index 0000000000000000000000000000000000000000..82140c4e18fad32b73d975f1b791503f2a1fa846 GIT binary patch literal 2150 zcmcIkTTc@~6#k~rE-i~9Hx+N7C{TndC@NGGP!y^90BO`GhdRPS*^=#6?Sn7+q7VKP zA2jh2H8DQ>qm19|w%t{ll$e;NGv_jM&Ue0Zc7Fc)`W?VIEGE!~vvHUhZ^Z=8>E*nE z3kEK>B8p3TnMxprY3)fTc<{1;83R`gToq`xT;&LKWj5p!nJT$fKD9`TbPLY3Kz!L+ zv!z>dltA>l?K))vbJ4byGoP2mqAChZ)VY}s*>2120qnOQp*=v-ft5S=$e{)Hdvoc>&Uc7-& zw6I)_%*uNHL|5LfB7FFta@Kein5;);qh@HXjdfYPqn^0}O+&*E$i3|;$)VxKpuOg@ z@Tl_VqdU~lTJz~YQqLX&Kfdc^p-VA`&3?i@6F2LK=JJIiwJyf;20Aa<(7US}D(4cy z)AZ8RPJ6bFZD!}O#!OU4r8*5EoQdT^$;qh&OAl~w-Psz`fK7B`#Kc~-8<;n794AZ+ zVAQ}hj&ksc)_4^-)o_x%2AQ~yQ4BbEcNu6f^Ck3(%Bh`ENMU z$5*)x0(3cklUL57FB|rKx%7L#>yF6c4EptC5&L3BucY(XmXObTf`GO{ax77T+8X@r9 z%V+whD*1Glz|u!e=$Wh3QX*1~uHTQ@Ebs%-%s0gDk9>h4z_mr-Gve=f$WN8pp$zq?NLmkBNoRY%;GQy!K8EqiQjKP0TuuN literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore.class b/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore.class new file mode 100644 index 0000000000000000000000000000000000000000..0e0b09beaed6d2a582a1528bdf638a6ca870832e GIT binary patch literal 4032 zcmb_fYj+bx7=9*~Zkuem6etzZRz+yjLO`WlS}IVA)LuYR3oW8<(`^&F-Hn?~rFcQb z+h6b(@Wl^$j?hy%NB!)-@NalL>NB&OWSayE=!eYCzVpuWKJ(sY^ZVaF{{rAOe4D^C zm~BA^u4#BDfvEp-T}s|f;yt`C2P26&WRsYODTf6OiwWq+$swPh>J1G`QqhvbauPPO z32cL-q0o#E64(V-gQuaSVMQQ0VJ>Oid3AQC-VAwrAe-1dfe6?qb>~7}=beUNRPzjC9d+O=Fpc(*@Vbn#H2y zrZ1Ie!~RBMG!5qMyf-8;-d#`mDN%c->4#xwo~PJ4p0~}(((;_?PRVmh&p27bnl{|L zd=G>%FPEoF&ppau(K9?VJ!<=IA2U~{4Xb3T*z601f@f;qXkxWcHZ%S|;D`Lbv%9C> zX99_W=@#=v&$JntXm`&n$3>I#{b3hhoV#IWy@4=7Afoz`xw8HyH(&oq>uo4HO&1ZS!-M>!I9nsaIJV#(y#hE$tUL1|HRYPJ`|Ie!3p)1UWeqoV zyn zj6%V*<^HcVd?Rr7zq4iMsq*lN%9o?!e+k>avVCbjsjMNy_^-jZwo)WfgVsLy?8ZFT^%k zZ2Wp`leIbB#{rJre7nURQ9k#i)*w>7_Yg@vKy+pev3rQ8`tG6W zN3KTE!?7K4j{S(?7+P3bJ8^`_|DM#g%Fp63Wa3ojDnDeQU*RYhacD49-z%O9Fm?q?S)WYDogV)&RA9Dua}r z!caivE{Vjblj6Bs1SUu(A@Cjca*`DI4fqf$up+XWs5q*WrjP$ z_o)ikJ!JkmPOH^ag>?#NaF({qTz#$Kik8n>YMrCj2(2U$`3stc^M5kWee!Q`=>kRr z^4GXILPagrD@d-^gICdnnNZ-g3Ky*|aB|IOAlIsady@voFdi7Z15NSme1L5;DtNKJ z=sLE?v5q#uVTZsPcGhOk^@?l2$}Lw=Cni+(NZDjf7^Rj^M{4vxk=luJ9J>TAD(b0J7k2Pq{I zQsC_h)cJsN;sJKew6(8c_pj6lcLghinW)2*5+8HAg85K-ZV2k(^rFF;uiz@FM3_8# T`E28?14TRw`JcK2&mr+IuK%NV literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore.class b/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore.class new file mode 100644 index 0000000000000000000000000000000000000000..f6d7c07b0255d96c09217c5d5d04a8c7d788e5de GIT binary patch literal 1518 zcmb_cZEq4m5PlXYlq+p5XtnjFt)itsy)V8+2r-hFG*C4;Leh`Rb*0zd3*Ie8ewKbA zv57yxA7z~7XbS~V8^6rX&d$s;Z#zGJe)|sK2?_~pYGByXFpv8QJkStBb{-EkB=IPL z?F4cR3;V+H{GK5_UofQhTvzy2n+Jgi7~*BiwPb~%o~s}8Ph7XT+tv-~TW-5l_x!fb zJKQ`H`U(GV!u3G=4Y-*y{dn9oT z*}u_eNE%k#<+7{h$12xej!eqC*0C^UsX0YfnRI?JK4E3{g%rLx9BbxCVKl?*`lfhe zsli#F+WN<;-we5lNgd1E=mkk<@lyTW=H;$^0dgXIQSjAp?RQ$ zd4^|KbI>&tK+u|@kUNh=MA&^n`% z5?!KO3v-Clxs)#yPr(lQi1ZQtLXs$!iE5+|Y?FQt*NEE;wXuRU-N+6ZTqj?OyMa~G zzDay)C|SWRx+%Ohq6+UL;J#$6PTT-|ZuD8MHtU#Q|tOG&u*&oP%bD lpyUwixJ|)bz-LHb!AEh2Xask0Z*Xp4fljs6V^kCk+Am{_vikr4 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/KeyValueIterator.class b/out/production/stream/org/apache/kafka/streaming/state/KeyValueIterator.class new file mode 100644 index 0000000000000000000000000000000000000000..c6e62074c2fba7ce6e2848b53dd1fceeb729e007 GIT binary patch literal 391 zcmaiw%}T>S5Xb-1*2dOv!FSM$i{QzGUZjGg&_he^(=f(u*Q{(dLLbDJ@!$jaP~tX4 zs8x`Gnc>HL{AWHs-`)XS<0!&jg#8Ey5e^AExv`ZH+UFM;p*PV*$-Z`hye8?*XwEOJ z43~u(&*qZ*IQxSQewVFD?X^+&UL1Stl96?V;%CkuMJ@PY!K(63_(_N9m1i$%SbDci zrfEFQ;z6+W6m8Y*N+i#W#DxCD)-IPjZGyCw8n~&dOCYp@wb2PrrcU@Px*=;JYWyyGP3(UG9;9=V literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/KeyValueStore.class b/out/production/stream/org/apache/kafka/streaming/state/KeyValueStore.class new file mode 100644 index 0000000000000000000000000000000000000000..57365a0444dd5d6d8ab2d2e23748923089f4b341 GIT binary patch literal 982 zcmbVL%TB^T6urZv1$-fhZ?_5?W}_R6B*tK3AaPOB#N}AWS{NvqPEGhV7k+>rWxOrg zC~Y9JXfo%{oO91TGyV8{dk27XIMLxyha(-131l6v2%Oe?1NO?OU}J~gKMuI9j7G9V zplG_zn5jT=0)=|NYc%?nfhv;#RnAgWk|GTYWkM=4!?lLoYas|!zv+Q;1?{;$TEOkj ze6`i{q(j++*)N<9+4GQ5UrEkJE_(MBQ=E3jN={p|7Hy+FYxBefS7^JI;xh6Pje{NY z5>vo~4=D#z##wHs%cm9-0l!mRGUZ7O>?R5TJ=(l8gfwcLpEAVnEKcPmu&G7XSXfAyje^>;{ z^RJ!&U7(R>Od~)bgI^Z^EiFeN2YEyqq73MW3s8)43Go`N$9MzrCTzvHjCdO=F|H!s Rfm)1rQFEre2m4WngD(M$9Wwv` literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$1.class b/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$1.class new file mode 100644 index 0000000000000000000000000000000000000000..88c9f6d94a0a9c7ce5e5453d48b31dbca10bc456 GIT binary patch literal 1824 zcmbVN+fEZv6kVqmYRf2(8OmsZ>r>V{~P zxKS6Iu4Rh?LqfVAh{ki_2wUfd{z(R!_XpCfYSl8;wiS1E+I4%igHgclV}=2DPj`}e zhDCqWv-DmGSJID)oIe(?XvZE}{%HgX!KQENkmd97wN z7$T|d)$LVDHAt<(5ZGR22vec!L^8H%)oo3z>yq(kuL92O^FvO*?}e!umP5TrMtfEz zf=3Dxm{)KW!wM8!R&WJF3gS{Eu%O^E79)6~U*3d`%TQ_?xaxi7521TEDjJ zRJlX{TlgQ~R)5!B-Eh<`y($V`!BbMkxF$1^%;#y0H!eDZX+I5yIbSyW z4jRK?rSn`dqCT<5z2f+_zT?z zF+y^He1P}3L2|^4GlrWOqaFFnX`=gEv{pR3_8r4*+-aeGCafS?YmxAcv4RxQlU*Lyj^tPE0$jaxJSM Z?oq4&?&AUJ5dF_R#3-#~7Nex;^B0bF1Iqva literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$MeteredKeyValueIterator.class b/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$MeteredKeyValueIterator.class new file mode 100644 index 0000000000000000000000000000000000000000..9c4cf4f9e4b9f20ab05b7edd7ca8f2cc9e55728f GIT binary patch literal 2680 zcmbVOOH&+G6#niE&@wb(BndH^s6-4fBn*>>(F{QmL3yiCv#`p=rMW_fnQp4PiBkRq z*ZziVT$n1*#D#I~KXK)2;l{H3&Pcyl(f;|NQVHfE)O@8+~{q zgDh^z>P;Em%HTGNJ($ED*-rJ~dAu#dI~Jy8epiMW(RwdKg;@)87T&k;fkKBHnLy$C zk{@j4^{#fdO@2pzwxjc56c}A`y)CYEWb%tPo{lgdaE$nh%`-^*7_6u3^9U#n{Jj74Tav=S<)1tLV-Za6fd za4U(>?98&E7iP2W(;EFTLSoYO+-OSS>lQ3dgwft~-R6IBOde_N?`VnXn(;Ro?7k(} zGb?+Q4HK;EjWQ+uOTMGaB^|i3kA)o)5ruP2-qtNE*z=xef|;@oLqo;YB$bmmJ#-8{ z13K^|t^M6PHT`U+STd}b;?(#tCZ8y&?Gq%iIU5ZCNruPg#oE^4iuoc^NRNzu!aRA^ zO>Sf~iP9X>SuYCq8LJy7R&F54nhi|FzsC}wWmZV1Q}#pdde@4fuYAslpS2#lh zclAA;FKc%7>c(B;M6~8f{jHiSy;BQf>j$Y;Jl3yjM~WIn!@F_ms&OUHm(Y6UWO0x zksKaVSZ;f`71`r@K_S~5dWAw;6V0JhIDG`Oz8(0V*V_MPTclQG8GY?ICJcuP6TCM+ zJwd;;f+@Br^0-y@BDWmo*WHTTu9p%d6Iu#Ap7vP4j_+;qMq*9#X0NlnCM-9fm1VL} zTle$A!e|Mv@;#Y_;(N1$^~Q?h`V8N|DV*hw#m9BO)Ekh^N^NrAMV$+{NS&9t`wD5E z1qO3thfulik;+{;L^^l%5FNR(hvu_52%#JD;x*V$?;co<2Rhb?=<=ciTue2 ze9{~T`5WRaDC?=M=u- zuEs=M0sg95wrg6kWwCN35i3giC3bqAJ2}}~!|Yc037KyxXoUT{0k$^*D-waNQNBVr zb|*)nyb#03lzSA8f|n836+cQ;ssTTfz!yo4HXA<1z0cf9DCcUlZMt;8oE8hg~G+` xv|Ma4>FGEaow@IDsyW8LG+=ioutg%+={U;Q`DH1!eG{*7&JehdAB&fe`WMK|sKWpN literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore.class b/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore.class new file mode 100644 index 0000000000000000000000000000000000000000..0a93ae523ba4cc05b331683ac436f79afd6b593f GIT binary patch literal 11596 zcmcIq33y!9bv|bnz0v3y$+9h3#$Xv7ER8gl7h#NTjAhvd*|KaTVJsVhpJ$$>u}3pv zW+dALLRg#-vXIb(G$d`)l*Zj?<6v0`7o4_~CbXmpWNDHTlBP}fHKerJ-2dGBW`9~x zTR!yeo%_x`_niMM_uTvZ($AlLo`~A{eL)(eH!AV?kv#r0NJI3d^!`g8|6N5trk_af zrvbVkq(74%KM&A3X)gunCA3TdW@$YEdPQ1qfL@l?7ocxQ>kn{+v5SL!3{wQ(VKYik^o;W?NWoU z5TYvsyv*Rn051>lRY6+IR~x({Nb7}qklz;IYXZE|;HDsLqBjHF9N>__Ee5v+X{*rO z$YEi#s*2lqb&%S4O%=s>tvuGrqg@`?R`GhiP97VC>-93Wv5GhG<|@8{x5(p0gSQ6h z7VeOD+YIiMc6*Ted583N%Ht+^+-z`HkPh-L(Q~&vx`pv(gZBivhxZ2QPTm*fTew$- z`~0vRQS~-3Z{vQ0Z#8(o!M8Eh@3Av>%8u@~Pwltjqjnd5S(#*t$?Qrb>{MsmN~i5K zlRqXunO1ivQ-f{Rh!q*K+m2gDk6Uf&Ov<)~V~IiZtc=~3<<$p1_9mvvzSv;G%8aIL zrVVd#`sV)KoBH=}+5$GdOl%k&R+MvyB!`ESiMG*9ES_%b2NkIOgQ?``2vcqMF>B0f zi(3${FO!0lcxjA`W`vHZX-aOxHmD=%wmv(NPNrnipzTZwM(sFY(;R}})e(>DABL6f zRi?0_LsEG(KAIlV6jdoZEx0)J&MHjJGYSGri9n)TCOHy|0N4>Lm5F6y$pjO3VLTd3 zWlk|wYn=rTh@ZiXg~cO+sz4eTwoY!>bBstbk+Dx^n64|AByZu7R5D_x0Z7}vY_C&K z1iOmOu|zDhh3Rum<)}}`)y;3Fqn}2K*(_Y{FU|YmU!BP)JfgZgmauz9hmY8)e(Oja zBemVhh!x*&rDF2i8LU2TpW1GxZ4Aa^cS~fnmrLv{l%=x7RBeT`09y5!6)zkTM<0qI z;-Uh4feBHdCO|4%+ zMwr)^u_DL!SR=|NDy!8as~LR2-~odVGCljhcf{O!(~0~4;*e$pc{O9}1jN+irb}6J zn>ml<5?ZB7L_Nw^Dw89U_*E*YKw7aA$OIK>mHU|HThVAYL`g(WIa!El->g$-c}N~A zACd^HNRDV}yvN7jwF_Lxm@XT(P93p3qERgg(|xlSq-^VQ0#vh-wKtVp*`XPB_JcjP zl^#t=CV=;4Hi_I*Z%GA=5!d3>t~==TOCWtWHJ2mC6ok#vu{7)gw%(>@IJ_5|6w@_@ zuCn*YF*^c53phAKAV(G}-Xrs`ut%Uuz$VdzUNbgfkYlf7TIs^QoVftwM{XNMCi1G; zmBI);Woo@^!pIIYwOMsT8THx^l~Rz*#k8e4Vuw&dHbbVwq*STV!sI!+V(!W(7wk;i zO1Nw;_l)UTYyo_Sn0AyZI0r@wFpEpmZh5hyze>Z=v-igso>0LXcWz`lYi zVr@A~*D4}E2TY`f7{x}BFGgI7hZ)!i7*Z3?Wu|QC2H$S*9Z)Exbab{0rm#y4m*hKV zN7}VZVQ(j*QnLiDh*X*@R(5ODrg^@x`Iw@nP96fE)IT|m$L})fHAc30 zo$p62S;jOHkD>TXw}EAJG-9i}5!KP#y}=Kd{2)IhGM>gZY4W@IJtj+rQpGVZk||F)rYuft4df2l#43UsWIHIte=Kiu?U5? zJ>WZE<;TK7rdR&&N`z8ZDzg97$K7c}E-wZg%%+(mm0ciZ56Wp3h4ayA^~>8^j@US7 z%RO@Ws(H=w6MFUS>dw(bL>(kombsO?eVH~Fo|gJ_spxVRDVd(TOsTL&Mo_t`dN=R& zIXhiO#o1S`J#azAblcQ|=rEr14Nrkos1D^S z2Y0@>i^>ZtR~Im5EcgASu1L9TqJxIUkd^LHH|@TJYKm`nau+Zkbpz?QTpQH!4mneYtZkrs>*UectK9 zF?&$5i=i^=-lMqmZt5yL0Xh_zHbV$x2@7Jtu}k@ggn$n>rEtKHyO?Z-;=m=Z&7|~{ z#+ni)b*_%YZ42O3X!GHksA)S6+_W5DWrv6SpUmuhkLQY{W$s>QiWwK#gI7N;-O;@G8HoZnQ7GoEU32vaSNTe)=~!g@G)sd1do z&@RBchv~hL=@AIFfdW*8(KkX9DH?50_iSP-9^%^Q{8&$E>_)!P|G-ZWtkr z)KxfhCWy~a!^U9QXhZPgSzz$%^hJ6N`oBi6tJX*F!>?ns1kbJHqYkR1PO7Huw1{@l zGTMo+-Z#;;bTi&|(QevB`|-4>8`j=~)AL>&iud6p+zUSa^f=v0pQHm==@lN&t?;qj3cm;nSs@NT_0Y%BZzRtph-et} zgh8J$=#vKhlIr}jkJzAJS>*Wjr$CP@1jq$%q8G`)%0#`eL7??Kt?UVfTfNUw(*!lk z&;<&?uq`d;skJtY2T1V$D&KR|HsB5Sou}0seek?iKc)-1M>N(7M>DL_0dg)2@Jebu5WJbYj6V=ma(DNyLOoePO zW(pP$$i0E1(~p3c;QDB&>yAt0}K)gj0--xBRVnW9RZNr?iIY4Tacqs})!GpNU`5=@MrxKGx>>L=6Ds{zy z9;M$@U`X$`=(hoexQK>8=mi3g+z>brJOB#eRWCL?Po1rkwB18bP-ROithhr&(Q$Jb z2nm-)HMldX!SV(%mN1rVcL+C{e@kK9j9VEfw+L31`T44}` zJV*^Y8I!Z_QK8dA>q zekEh0GQ+kUxfjek6sFdJQ_O4UV3zc92DM9rNs04Cs5|^-uSB3wi4tXcyp9Kjn zHnfUdze?6YYDf5zO1#o)1g=qO>@jHf5hUryk-9!ctD#dnJai|00@P2?VWi=_Gg>)h zEC(4v3V#k53Cg!AjC4$BB!ymMB=mxjrhgvvf>srIe@gAXe&oeE2c?T3!aa|JQYTO{ zKzKT*Yt|9=cnARXU^DYI_$Fzuz`tC2`(*wt00X!+_|H?XhVjg$Z_9`qlJkpP=?pE| zSRnux4>wdeNFG4iB>W{+tC3u#N`X%SgHK}%dXm;6Rdyh2Y)3NeN6t8aTyYF>F^u*h zWQQ!GYq3H8A^nj`zNd58o~9S*j{!pv*T%9DX|EGRHT{b@dX25n{|V>?(OUBVjF$Qg z`ofQC<$^+BzX%Fkd}(0mCjeG9oWo_n_IYOGebr@{si-U-Ztyvnt;4oY_}iu0Lhm~Z zEuRKvC|TekZSbUZG>L-YSx`>Wod_7gN_*4c972a3?+c=c$RniVQm8a8LY0k6NT6%ouRtPP9y${B!ax!*bvcP93IA9QqAhTd=LIE?PTX~lJo&^e zBeJ$$=d0k0`$;?{+3rFs>jXvVNjkvTE(lmiJr{&5gvzTDt|Nw+d1Jtpr^ELm5S7g@ zpTxK76$YS$lf>$mVW=R9E$$KMi$}w7A_<0iY6Um#JXfl$@Fk3X87cQGc=swb(^o<9 zDgy6oO4fXumO3z-!nt6}&Zck)ocU6>wUP&kTW9TsP#w32_Qt>mI&4_O!f6`|%5T=IuqPX@irRR9^B+jV(J_1uCOE2SD$|-{sHH;Rb(0 z#ROSW9UM7x=>^RRAA9nZLI|1=1goNNB58jMVe^-CH44rS`VRG=7C%PcMVk2@K1zLG zp|Ay3=s|`2O?YDksG2cnLWTYb$4jbE3w|41aIry0!09_sd>K;K>*(oZ-VS=rS9i;hg{(0_}0I2c3&GD85_{o-E(^BVa zv=>rz_>nr_c{*C^i$6fh{9C1&F4JlpEif}zrqwt}LB@W71X@_B3OyvQ34qJrIcv9} zl?aeM!z+RAQO>U`qUHj}_tW3Yp#qctf&S5DvS)Ua|B3$DnS2X$7Og8oEsb?P?Bk+t z`RZELO1-*})4!lr6_G3HU)2sFy?;}xdYvHuclr;->xU@!Wnb`MCmMu%cxVNF%rNt& zX$r_9vWtj&9UAB{I}Rw!}aN5iqU zn);?qRZv6m=}ugE_(X9Eu|i?@*B*t$r}TAS`@vXbk=O(LSCF5VoDZU`qM z@|=0;w%a2qg#~r-WLr%*5pHg`YBq;kH-~FdcG3!WL@1X!+g43|Yir7~m$h5Xn-lR! zO!V-t*xGC*>X?x|a=hLpPOz2e)gW}kt-GEsG2@6^fpx`Dqb6gEWKVqrU-WVGsziq;2E zK;mp_&)3FmL7&2uVSFAHfLht@jcI%mNcva?8_VDG(ED}tDxC)jI$ojT{T`w^4kVbY zs|Y=!zeO@^PJVRXA!izibjDlZ*pXq&$e+X7=h_mEQr8wMYDtlBRY-_@cHA*p;h3yeIz5IQ;ctki zlg*aYL{^ETOVB*2nI<-3JKdC9KNTkG#~fx$(~&6MWa965lU|%>cO+!}4-KGF@iSc)-(osU- zK#}Q|4S|SsUefETOkrv+Wkf<7YNDwo;TjWn;Bzbs!+ty`oN8yyHB~^c;kqzYWva>Q zSOs2w(bjmf!)ggi&ITi~pi^Liyks+%Xt0%Pgo3lnQ?uupYKod_s%dJvXcRQn3^mhK z$Eo8L%2#A+K(M?eC|7C3N(GzJcCb~dPB0#ec99|{$`xh~ho&|fwc5hbvyyG;4j#*K z`ElpPHpk+fqp6gsETJItL(O7pW*xJleq|1ai8^RjtcCU^l2(VlP@-M)*yjv8hmOhm zT!XHbmv7jPvsyVw)BsxFo{V=oJxWgAWh;X>oJdfL!rV-O(PuKfP^LJK%yU-Qy#1Mq z^1t?%=JL<(XO&{DI-gxan`NKXl!`~wwzV#7w<`pO3tnzM>wag#FfG$@=qo9uOE?*_ zBKjnbT@4mXXZ|I_SJXboiF%Ju`WDA^W!YMp=Te?A+QX@})>d+K$Mlq6RfqdW+OpbW zJLcm;H=}F{qHIYrYbwYa(9;@Cr)Z~#n^rDo^~#FPTzbo`RC6+tU^FRusOAYOhAGq} z)fMA4=V^|{Q>@`WC(GBzM`>Q^Wv?Vmmc*UiX0J8bJq-0_aVA@HrWy9JZ?(HL`e|{v#S!62C87}; zH+vN0=Hyy8bpQF-2&621zm5t|;z)XhlqC(NC_l2*kWiM5@?KkO@LEn=^{vdue7kjn zO6;@cR#=#OJmdKsL8HWC2x?{de$T{%j-QfGa4gPT(u**IA7&=Mk6f6HdaUDj(uep| zfOGkA$qSn661k8$`$*%hOIjnJI^pIDSWwvu)sKS4ez+QY;jZd|r^MR>qX)iT6!yTs zv9f9}iYohIlCJimcn^;i;Czl{@NhhhtU{QHMVO7nn2*zOfu?EpxDX%V69s8?6fWZ0 zO;sxRe=$Bv#i>p~1q}H4mFKN|1jUVOp3KpRZ;E;_hAQRgwYDE)8|U_7T*%#z@l?K~ zG2rgSgkBsI@>B*qu3nUJs@jK%il5SDTuf5@k`5?@yaDe%Ojc}SYGie+!V{PhGPpfp z?88)foF=R3vI_QKMlWV|4?Y|4^x(LFy9dYjpj;kib)z`s)2o1QV6LO*?Cn_Sto$2? zUzPWxqOoL-=P^3bHJ8>jG`cf(HS~cYs2Zr=-wl7LuzG)>a9|HTv>e^oMPKd09z7du zR*vxbh$_&tTSwr0#sD~z-djqKE~ocba6B8U`Gvia<2HU?ZznKz5E|VC#ZE?H7w3J1 z#8U*rGxYw8;d2?F1Nu!rDN>z$FHf22uuW18DykWqO(ZnX^LNl|b zMVlCO4rEnFu{MXTAXk^sZ3Q_Z3#4(&h`O2Q1pd`klv5+3#t|$h$kT}#+0Kg@eV9K& zlWF1#I?zXHW9YsBoid#+n@yKhp+jp{N%xZfkY?3GnpNWpTuD8IdKIpwCc|z~_&BcN z8@j{grIayrc{RaS+*tRdTGNjO%$AcH1%16ZxfiFbA@0`pVPVz&-Idk7IBm()hsYOr zj{o8X`J13Ar!dz-ECP!{9vZn=66pY%0^X9-d$1&A1dPYMI}p%NKf|q|9`I)29WY$; znJk8Tejm;}r7&Q4xH#)nf56j^X^jD2X<;vxmihazY#)}pu+b6FJ-8#_(K%e|-hnxq z^$I78O5J@}$u!E)R+&DzkxdRnB=94(a(_HAl}*GG5DS3?#Vg5^_k9 z#};z25k#6Nx)`f%Ox4Sno|h9>A7ct$!8l(@9A3r9U(N9oxDMCiR*rY#dM5r2tX((a zQTFQxa5G-OC-E|F!K+-qhEL%Q+>W={X1|NusNL;q8t$MrpV!z)pont4kg}#}6#1!H4|%U6?$+U9ox&b!*-g1Ol2T&C z52J{$2C0C1=mbGH$lEQSMdu(DFCN-M+0GxUglsQBfkP8&=nchkBrP=R|efu(g4?#;>9$wT=% zS^J1B9q(xIU;%d)Fh1fzDSo_>4`TxD&(Tk=rR^ELPNg?#JPNZH%5Wi*-*06y44hc&(`TXY+M5T@E(OKAtpzDkQ1BkNsUL5BE4!=UU-Qr(&>tm@cTXlH zlfcY`{9Vq!0QzWW$TGAD`P?6Zm5+@R!pqFtGQN*K<+NaTka_Y!dRsm+;dxwH=i{bY zO(m`UcZQrW^kHRq&vq@QB|J^4O2V7t7$p*l>k;EvbzjS^ i*hCoI6 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$LexicographicComparator.class b/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$LexicographicComparator.class new file mode 100644 index 0000000000000000000000000000000000000000..9968402cb95a619834122214e19a12295f9fa9ea GIT binary patch literal 1441 zcmcIjOHUI~6#ni!I;FHykb;OHDo9I521FsGgoI*@#HxukEUcT$FtkI5PBWbv*tp_< zn7DDn#*HQ}8rLQ!{u3AYh$f2XPU|B)HZ?Bhp6{GF_nhy1XXfpX*9`#IFcpD{@d!c~ z*DxrYq=uA+2@Pq6(1KmD{Y3^jnObE~mfdxWp(|%sth@ECHOqUz*BpxUcbJ^Xha*tCc z!=s$*6%AhHX2~+je51^bn(tYB%dQj&ao;jpm2X=+tK6wuL5I=5+l`)3J;R`xK`&yY zB@5X?HnqYKcB~CQyW>-kk=xqmk5#hDnBnWSetNXT(;d?J0)&woA~EdAnHQeqHQFKgzNhsr$BL z{8`RIA-hNtM)GdmGp(Dp_zDJ&TgIf&A&M|MqBxCShKK)erwsAKsx$Nkp+odF7!28e zXTmVoUXf{r>uue{2>qiW&^Jr}uL4OZOJ0OLgDfMfPP~Hjf`EihIz=+@0Z}@m%>=~I zLmm?wp;KIbPuZg6lDsI-DnseUGt49uos6Cyn(4;DlUY?yC`}YfsEN>1bO^J)w=1(< zg*`ubmu~2d^*t&g2aSh=g8S6!BL?vaWB3dmUvL%s$lzN*=oaM+q8|zL(OhFhte;Sb z$jsvmA(^7H7$BqwaxLNlouzwXuqB)$Bw>hfC*?)k|@^f6bI=$(ke1fR-!vXUv|8{ea(d5G*w){MY<;AQqXLaMifmigDTSq UGER{sU9Rh@JAWvYzs)NBm!wNXKp*+oH;YopTEBS0B{fK2s&|7!>tGu#KMRm9zg=r zlADpXSxM$PVc@ogc@1|MBBoufbBBAj!!VH9eZWoLVd9HcFhfL;E`BXf|pr-9@irt;D zYGS=nEePioFH}g=m$6M=$#KV$a!aXtCCk;Yz@Yo#R#REZpS<<8T1`0V3U^)MGW55s z8@yH&hnNhTenPy?&5|(6e5cF}*K-7~TD2l^+!M_#S5wc$+Z?YngdcwLPpm`XL})62 zYSGmgqNm)MoOT?$%Al3FyDs)ThCnR7#n6@YxLJO|>#b`>vUbBU#WPD5Vz`BVBBeR` zUOM`4pJ7W@`J9)^h&{`+iw>`sEc3Vn7zY1gk=PG(=;+e$P{$)Ab@ZTD$90VBSi)ls z%Q~K5h2ixD`5?O@}+K~)RsO^I|Y}?(od2Hm43izV)_6q@g2%0 zJ3wF`>h#PP1QWCS2z~Ouxv zM#zfK$s7WtrRiv8kn?3=JaGW+9ioZPP>JoJGxQbV0Nz7WKR^w9qyPcPG(s{g>+)fB zA7Z=(J3-bUl(7}s!MH(!f=P_fnIQEL{X+~>eImp~i7IqX`SlvYT{;)Z4?P{f0d%pk AWdHyG literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$RocksDbIterator.class b/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$RocksDbIterator.class new file mode 100644 index 0000000000000000000000000000000000000000..711e12b1ef71894cf50c3cb60b6d71416c9c2946 GIT binary patch literal 2160 zcmbVNU2hvj6g}gwbvD~3xWOcp6jMTKI}o=afkHxPosSaYCJ?8QLY~I7;tE-wnLBgunRD))nSK2Cx8DJn z#+HsD-0Z_8yrWa|u8ss|HOy&vPscpo=jIkq-sVw}eHJt<>M(GVXO=W9Cviu|06yTw z6%8M1Sk+Jxh?$;p1TL3sXSd+k)w)}&6gIeDp#|yL&a6OU&a_N#Uf@dh9KYNa!7bP| zB`{DjEw$F#uPA3zRvI*uD%n-p*piOPb~qXH_Dom9nt>iZH~z0qxS6s(nl1j z^6_fJcF7tdwfE(LEHosQV!d)-RXtL)`rpak1lqN0*S*6T*f3B=-arm{fs8M-<(ZAb znq6*H_m&!JpK7%9psJdlXmb2^4#V^`u9K ztj@P`grS_XUMV_&2Yk&b+1WFW+G#Pfd}KA#rKiEH2|V8&Q}!HsMSPE$|3w`cdSr({ zu=G#y@uO!vN#kLRQ2QcT%ZShq(DO$S`H623`HHLvE>n9EQEGog3_qa{kMR;&>A)MK zxJnkq`Ka_T;8l!;m^0K1TIi02v~E zjKp^SYeWvA1zqw8eW%fW=|W3~XxIE5vxGu7gjz}No8TFj3Q4-ewIr@)bY#-smBi308FfUFpxtO27G~zV=4H|x@FWw(ii9c!nudY0f zhMNpgeqYBML~fEII4!|UfkxM9CJHft6GbF2$m-7YlP*nq zN7I`fgbDIz1aVULAQ?})0!0uwc$<j`|zUx_Nw6sZo?Bb2;(!g_$;23)?Z2SIRl@Mpc+pY_?s${qWXG3 z2L4v}fI8j#I|F|o!7`7<7o_z?Js;u<>3vCfeOX#xk=8#*>mQ}~Cn^3}imw{@S_B(B z7GIavzZm$}2sV2o-;mb78Tj`IF2fU2d^3!1N&O#EJSA-ZQ*{5fbiY%BEAU@o{I?AM zkAd$-*#F-MJKvL$?@RZo2!4S7Gw}am{4k6kMX(<~Ht>@$ej3Klr1x_n_(d4M6mXsj z<5$xEwG_Vzq1G8{y@ENIP8`d2w8hT$1W1n;q>B)>WGvy@O)6+ASo>W-otr^Rclj+b< z&uCx&(CFcKXNQ97R#?)WPGwyy=WG6KQ)SWI@2v9nnxof^7x#%}SKw6+h)iulk-PuV3_!sUv%duLLX z)ONIuc6Il3j&^hpD%AJfVI8+(N#YlayBQ}nNs=lzJ>w)4YKWQZ$YKQ}kxsexNmpUN z*2$W&5>s~Um^E?Cie=r5ZB27=Y=*j0!*nJ#kZ%!5SxCM{dPbPhZmWv+)$a|HB7DMc zE37Xf8O}Ja9iW?7fUYY!o1L1Ys~#Wgv!>}c#<-L5TA}f=Qu&@kv{8Qb0Vn0S2Nil6 z<^%H__r2qYLS=h;oLVgFaZ+~Q?DUwO8M4Nb9I5X~C#>WVE8|GtAFOnz9D>xbm_RzG zYsY-e57~EWKh=k1n?I;K+*4&rBT(y#1R7()nN-+Xgl=1s9@b`M?QyMvM0NH=F@ zIb(2!b2L-caXIaz<#=n7a##5Ko^_pMtcPq{G#Qy^$fH5;X@QgA8K_ezxjRFaI1;kz zJgw(ADy%Mc0v>JO5q3R80S9CQ2Px+I66Ob<9fN}{hPssJKJHAWEO%A{ZhI*L&zhwm zRy}SfZI|q-#0r9RMk2Pt)gJa2S2OS8`KZm!R!(Hn)3l_UR=DoD@+m-9VTr-_(W@ln zE^5>=y3{ywFNtyrmuLM~PZuC|+)!3#~*pmvzZAZw~d)nuw>wbN9) zRLoSnRZL-RkWcU`HSrR>)KpifD@%xi_o#`N;pL{FMcoDt1WHkHRfN!uBbNE74?nGU{Zbsk9r-&d$uFGp;=zok(Y*SxF;l zs5Q~7lSS9JEkojCp)vh{B+^sqnT*b>NYN#$#Jzw3x zcx|d~yvt-BdZ-*wbE#S2LNf7Qb;!g$QrwOAD-6#AxOCk`{WkB!MX9){dQ`6f(PyZB zQw`u@g~8vOtU?Me5^geaUNNKHULI;4H;aJAc{%SFr1%(4oA@aH(ok_z4dI1`I&7*V zklT9B-@P!Uu2|oU`$*?tytM~Weo(s&3b&&k>tSPp?MZ%V3So+U z2X_?uE7(64=-05H4D?O*r%TqS0{t}SnT3+~ddWd`?P9jTKSm9hS982xQfgf@tJXD> zSzR*^)iqO1T{CyoHM2z4A;||>FDK7e<25W_3#j|~MYfW4ZR0tp#-_6fH8!7@FKK5{ z+1Pv*RToe_at=n{&XuUip>~+HnZwfI#tT@+iMq2`-uQ7W-n`SEVgoNE629U*kv5M zoCBd8w%0eXV4HJ0a%en@rT{_Jc{D5Ju=5Od32lspCfeOdsw+mgZa4p%&f!W{%R}d| z=PdR`vDQ;nTE1l#vq`2|~ zMwz@@coW9(M;OQZIrCvU&*$hm-^U#)LU(yRxi@<7;|+Ktdo%=Bh}oqj1`bi$ckyPt z1r=;FulEdxZ*ZM_DIeVsx(8-+Qx5wc!d0g)VE>4ZQ2p~fXf#Y$ds{z;YaY9%GDvhF zKy)qJ!pNgtS9uNxAEQuU3BP)`U@1$fDAL%yI0Y%Cf1w9NmLv`~m(@ zwnr%Z?XmTCyhFCfmiVo3A0%Qc@y60?={9+mQ>M7q0?-HvCc~@X%sIZcc4S(g^sJnbu-bb-}iOY4TdxOYb z@1fPos=kedDCZ&9tbr4a!}OaxU`sW#H*n;Q{AuIOT6mOW5j1I_1d4qHp!VUtcpnuJ zP`J@<4->vLeC3{`asur@^OL9-1^K^;O17UuU7_2YL&qt^`hseAE~xfSQTqY%2cp95O? zt<*l@fqW06=m!uZUr|m#UeCDsA^e#(!R~@`yKyh>qXwr)wVFD3R3G-K?!}*TP7F|s z(63<__={g+JMQN|2p{1TpF1o?@$0xm>}NDKDXKIVLLz$jKAP~9)xe;lrck z0mV6C*3Z);ATGavK8aC#bLelPPYsm1{{4mRNm$RbT#g5{=g8O@jku@rgVgoojO(6| zn|zH|`2u?AJ2zdx;0U8(d`^&u=V?0f12D->@bvbu$`=@BbjX3XhY!p{Z1Gv>1M_;? zMRpwXeR69KL#O!ZJm`^!iw+Ov=e@9&fhB)>ym0dnP4Ngv&+~V`3)qEE6ck#9P5Kmx z_r`qH{D9I&Kifoo_)hh;I>=fG&Bh##cmNE~+05JXC#m_TDDKl*gcn~-k8xcqt@Vh{ zYd?V$Yb|MPqCYlkpS*bvKrcvL8DO%cz~rOU-}9e61W913(M)m}nbT%IgPH<^yYc8E zS_Is1K9e~u-i^l=(BkM~+wirZXFOiObq5*A-;y)9b%7 literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore.class b/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore.class new file mode 100644 index 0000000000000000000000000000000000000000..a863a1a687cd9e0b9e7b79ab99a7848aa9fb2f70 GIT binary patch literal 1413 zcmb_cTW=CU6#k}A7FK!@ytH0hMWsL;O*B4G`vAtoh^CQ-r0Ij1Zb!PZ7qT-L`?K_g z#3ud#f0Xg;#Tp8!NPL+&bLO1yJGYr1Kfiqk@B~!@w@omtnaE<@z#S85+|8n7B8Phh z?i+Yupv;he<#{q_IzlO_==jdJdg?`^t^GhUWVUS2*4qrNa_gh`B)B8IE^q6=_PX_! zA9T6s3+qVop7_ucT>4$p})t&ePq?b&&RN((J{68&p=+!4+|Mp@K}&J%6~K9&Q6Va0L% z1^=9c82bOOI^#K7zU%rPA86ZAeAjkmTtQ6|Lund&hFsh3dO{Dvo<1LU`O~+!jBlit zfjk^3IJP!blSl@QnW8tsj?NPUUymT5J~9;lK> z4HrpTi>*;Wkxt}?OSnu}p>zdT6ZDk?{XH`D#R0`i^#tj?YHf)5S|PJJgmIRF6Xc_a uq7u8DQWQ@qiU|b)VingYloYNL{2XrJX0+eJ1=aIuSS8f=;Yv2{GFz;>>U?#`xjVUhyilq%Dhn-9OqV z&64d0>_;s-EvVpJVw(2!yglbR=iK)9pC7*ftmADQBUp%HF@~3MB(N02atyCxScxGM z!)hE^ti|yfZ=%RWp)d^YNK1O}7($uVDnoeJHU-0ISz2PR-K+`cfY%yiNtJDbH>%u` zzTP#4y;JEjWXrZwS9psXr$RaBC+A!-8d6xEtLz5yvnLzwfpEQThB=SqK6hMEvdk)P zNRxY_WSWj}T|rY5EB&k8*-d`A^<)ag`2+=pr8Pb`f@)`fil?J!!O2u=eRp0i9 z6KXpRhKUEf4DpKHb`0@B`oo&M*LKc_OCX6+>ft^#LsH9c=2W^gy{M{c0?#nZFy6d0 z?IxF2&aj*0D5_NDNY}OFAc|q_D6guTZq|x=af281qFKd0yy+O;z74tQU`U zEnirt1@^&dREO|}ZYxJYbv5LpC?x10Z7^gWyVSm(KGx1?12?8NEnX=(4!@-B4RW_E zUGgpW4Y-rBz1o?1kl}~XLY5xGfKE*7|EFm&b07AfI#{3&WrRK-APiatM(H_5nnIG1 zjAXCCzSGM9#_2g27^jh<_e3WE6PP58;Rz)V1j!yEfltqde`4S;lnPa@5WZ0&y~<#q zL^@&7%3(Ma^DTE3(}V@g63IC{#dECS1-3BX!$9vcNO$>71#G5o+3a-LByR`|^;d`! zYhZ~gEcc)$dr*g|n<=_y45IMOKcS@fQ!0`4lZV!>5x>Upmpk4a)C}my6QHI$fn8Ke Pp?;oII-`F4>CXQJp~nsX literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/MetadataTest$1.class b/out/test/clients/org/apache/kafka/clients/MetadataTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..8438c6ae95a0c63de7623a740e90f7e4cdbad296 GIT binary patch literal 1619 zcma)7+foxj5IqBdg=Hatc)tJ=APMSn^9rIEgMtB6koRF2$jY)Cb|SDZ)b*~RTZ5u zHO??v@Pi$jm$)cOdzWwTa$C5L^i*gsOU1Xi;_EV0+GW=995u%osFS2g+bAdY%k&E)PRVY9?L;sR@d^IVgKH zruEs3jyc&7R{ph+CFS@Y?dx1qTX*(y?w6@`UX-YbsD7vdDal6+O=yO)u^C!o z=-{uoQ%7q_FD~hdKEj*ScpI_ts>zGO6}~6RK|n2UbLHkSl>LC_x6#JaG`7-4z{eRHPesm2oW*(4$SH!P7XcS=u?BdE zM6@fPeFydti3(asYO0|1E2e6;`jSzV+A3&2{Q135)jGy1uqx<0{Q2cAf%d`1Bt6c} z(BWKyLE}DIL$V*xioz7im?!xmUf08%qWfP$e-wPa7VrRhwh_+0fx#Y{FO%McA*5(d R&^KTN=SVWir;BF#{Q~fxsfPdn literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/MetadataTest.class b/out/test/clients/org/apache/kafka/clients/MetadataTest.class new file mode 100644 index 0000000000000000000000000000000000000000..edb8d7e54e2971656dab9cd954805fe7bfaaf26b GIT binary patch literal 4420 zcmb7H>wgqy6@F&3o7pTANK*>Z3oIp&3xrK5A|;3kG$0sKB_S9J=w$azGGsEd%+3&K z>s{Nbc%@cLy;sE>S`;?1C|3K$pMLQ_@S8vX2nC-rGrL(x3Tfas^S)%>Mn> zn>PV$$J-5fFD!YSmd8{EJ7A}HB!7i?QTHD?o7UlEkb$eCn1KOX$C(V4!%oAKcS(r8 zjzBhNGnm1-49??O;dw5N=jHtgA%9Xku!^C zep2P8%<(BRm$xnD_@&(8n6F2}iZRNZ@Z7*D^n0G`Y3Lj{ZJsrAfp6KlyzArx&m&XL z^xbJIpX&=>N7T3?HLob4VaS?rOh53HhQ4>hyF0pUo6baT$oDK~qGu1~)$g_(%ip7+ zuC0AULvo*6pyU+;mZJuP=`rOEn`1U9%>!=Uv`0+Ol6_1}`jZyJ(0YI0hgHd^wzRKI z?zlsiIu!UoDDZSZ%W{3=zVgUw_!LnIB@Go`4JqF( zT6r-q3}dZ#8WMi@ohQSSo-zv*mhiV}XdLpHjl*U!^dafz9) zF(-E9;aY2s76}V=%(VRB(5Oa##vM}3WgQ zGjeyym5>>7123=kGp0mL1sRf`t%3yu@5dekJMpA}9_%*IhrI@N;Ykha?^~*J`!x-l z-$iX|SbaAH_Xd_-P%QHGRX}AeCp+H*vQaX#yRrtpi|-luK7K%~+%3nz5Ah=%KQ{0> zexl>227ZRi27ZoT(D6Yx8-)1FI!YB(VXJ{(;#UTKjVlIzgWnQcLu@W~)~Q@9#5c2r zKxKV5w1<6;?ncn896Rm|4K)4Ag{+iZR)yWKVBig0kq&siGb}*^Cvl36qi=?gXLE^- zVm%qSir?w@y@6}EZlH`=19RA`q2*3>u{9;%v{km!Sn16l@TP`6Qb}3YsUB3Eo-s>e zlFIw4AkK)5Zs2dpjs<>}ECs9JwE6~a;Ex(|3lr~7PrFV|Ni#0xqP}H}wt+wYxR`4+GyV3o-Z^UzihKi4IzvdA2D2we) z7K4FpGj5BRQ`6mIP2$12h0%&qw9gJooCNe$O&9(1oVYahK9^~GeN-Gw!(x5t4$(bf z2%7{!!|tFu=STSDtZ6&c&bGEJ@@T(lm)LPuE#lH~@u1ODX9A+2!p*^Yuo{kGFCedO zmHKS!ELS`{zL?W8E^>mUqpV#{#HJE!%2vMXRO7H?5d#h`B_yemtyD2`OGf@_*Wx^0 zW=iE&j83&345N`8=giXZ&?36TS0f5Z|CxY6;Z|x*qVk7=a88W}c11RQW;Ay% zmXAe>siy`fG2TRTihaWKv7XeSC=6p@HjSMLyd~oSj9n})l=}HP~C^9u1?e?cG1Xs(rz{MQfS>AR;?>z zb@O{6|F+bB`gc~uVQnpq}8(NOwM70m>4U|%%SD-?SBa0Mq%3c4yMcX2HLi% zR$|4mwGzcTNYYMWHJ`1FWg8mNjuv!a3pO)WTNt}enAnN}a>Pd$JaprE{%ClCSbK$N zdmZoN@Y{)5BJCzI_BOvX{zi=bix~SidhtKvOOn4m1jem6GYR|`14O1qxtDMlgS2D{ z7x6ScOz3#HfRAvM!Vz*_!1CM3BB5g_2~EfFZ4#p$pTpY8g`^V;Nw?5SqK>2dzam>- zxsej7)dLYbtljGjBUDZ^a10f6eUuapqZM>%teX&BCzgsX$qo_LP)2KW+Z@^nyup>& zyjSS)8MpJNsa>h7%vB4c-os4w(xy?qqg?J_2Eo`S7}Z9`wS|#_oK zD5=NEcap1T`0rGhqby|rC#pF*9_Of+g?yaZstfb69rgFj$EZ$$x5-i`u?jIQblH*ym1%jo82zDcZQTLZ?Jwmeoq zVI9T^@<}>m(bcKY^;KabIS{*^B=jSA)+RY2Uv*5iw8{}m6JwihVB2WN=2<*(cuN`E zx4c1GA}p%q;knD-+0XK){&O_}Zi=;#JA^*xAp!F-0zT1{#>ZK1k&YbS(iKDZ=LJhg zlNLMQQd4w}EtSl9)>DN!YN^7UxKg$1Ud)PdsW8uL)|Q}|E>DCiCfN-@cQkg0@bCZa Cxhfg} literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/MockClient$FutureResponse.class b/out/test/clients/org/apache/kafka/clients/MockClient$FutureResponse.class new file mode 100644 index 0000000000000000000000000000000000000000..915621b32141b49b6e03fda047e010bfc0ec8ec0 GIT binary patch literal 749 zcma)4-)j>=5dQWqO-y>tk6LThMkA(BFR3e~Z;B5M6ojbINGW{~Hk)mG$>lce?Fsp_ zdEs?l&_#-+VK(|E~W01#p1ZMXX?}ggl-(^xPdU3aA(Gk|8iE zo#-T0hk7t&;9YGlxR?lesrX2oj|7)`Jk|-H7;UxGF}Kr+N_o$kOxh1bFo;sAlSE0Y z1`OPp`7Zh~dDgpHiugpB$k{i}oNIxhe3aSDsNnktGPEKqW% z!RBoe>bKO>nUU%!a`h|!NAYebeu@CA2m-9(Re%B>1t_9aK!aib?rI*Y#ZYyWJQm3X z|JEPUk{JA5H;*?J2T%XC)A`T-9)x;Mhs3@rQ& zFKo}?hxHk9zsT>QO6U6gO-M{F%538?_J|i$&3CyJe?qrIuBJCfuBuOXlPnKUDF*T| J(zi|d-ZdfZy=VXc literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/MockClient.class b/out/test/clients/org/apache/kafka/clients/MockClient.class new file mode 100644 index 0000000000000000000000000000000000000000..9e9f437c82c37e7526d20d4c0cef78fc148d8241 GIT binary patch literal 6787 zcmb7Id0-rM75-+E?lhZ8Ly{f@+oqJZB)e@F+Hy4|w4_(sB!vcA5>T0LrpYv$on?2n z2cV#+1r-4Wr6{PV6crS-O$b%MTMtJxmzyxMDT9BCxZ9l z-b%a=@0ZI5DsdkUSK@wrP%aXwTyexh} z7QZNWUy2avNgZFVz*G3jbbK{}75JKtuj}}Rj&Ew1u`%c7GS+r0n|4xJOM}stN?DoK zq?yfHSqEX?M|jIZl=vdzZD-acMq8Hgflqkq~bZ(PG;kma96X2%7l~2SV_~h zos0mIX72$F)$M!Cy{1BqcUdmqRCL*WDN{%ajZtscc#lal z)faC|xmKT*X>Os&^o(^?&dR#vUsEJ;an8z#uI908Yf7+7+E&WV##`0zb`MX1S7dx& z5#1*sK8P#yNg#QSR#4TuM>(H>o2Edm!$}Ny^cIaiM1>t*W2bDlMZ@WhC9E5Fc1=5Y zR;}u+s@v_9)tMXIWo0fgcO~ie+IA;lCU=+_Tb}*Ja<|_mRn z!ifo~VA#^sqoFM2>|+FNVMHVxMr1-lYCS`XF>+!6ik(yt+i2QJt5?IRCr~VUBfWOk zm((z;u_(a0+>D*-YZgc8etc;tg;+9j#EB|)Rj~S4bh+(X84WXwj=ha5rt4(rv)QBl zejuIqeM^b`lL$SP6e>EZx?D3c&|#+4Vd{9ARI*lzPKr(1B=JO~5WyN&O(qn^#wry{ zLTPB4SUR7Ce(F#p?Ie?QTjQ23O+~I~x3f|)nEv@{Rw3of$+;bj)bfOrX7rpiUZA)j z2*xB_8Dz1JZ)v!o1d*wLsLDB*PFk+DP7GUIBJfICdeLO|O^rxhV<|-~<5#B&TYOOY zN_>9fxYbF(7Q&c`CyczJ3bjSbk;ZaUoz7Tkv$+52S<96&IEiva$eDI(W76*Hcl{34>f}-`t7l^q zdvD3|#r3fCNm={zlABqar4(H;3Def4?XA{2F2&hEJ8m&>K3WW1fVFf`VcghkX8Spi7+8l3>G#5mb(xHLV7&|o1~y=$fz5K+ zB9~^Y(eZ5q-@!p0-!WbK#(y^O7g6+A{7uK-4g3TDG?2r7Q9WqjUpT0tK>>JUT{3C) znMt-Q*V?c@VWqv%a-knr*#-M-x1R-mLB8f^c{gaa3Io^Tda>_W12@QBJN~U< z`FQ(|7q&Y7BWL&mTgV`1E&+K_!)fC|#?3wKHU(3*?b^dJhC}Rxf?H*<_L?2%g@PCJ zBLTWWK#-ZEAOz2D4svL2~CXK5^#;v_x3 zn4Pan!x>zj$n>$rr;CFP6)c^~es<(uenHs$@=)J0qiF>5+ro)33!g=-`-65eX=g3^ zub0ka8RL9fvd)%D^X4r*6IwcPYYQ_AoV~1ZoVzy+rd<|YBcCD{s;*b>&kpM2rIquP zH)E^D7%l1DlI-Xx9$c%LOzz@DDNz#Ko^?DFKY?2vHTKZQo6S^jk{@sy&K&PlU-d*n ztm{t7hoa3(OZBiZ?EL{wI!bxgzM4{(9Y?BA_nN187^?F(j<;ycfZ?}0P6b^W<}`MB zLu&EILR-@goXSu7bD-f|{#&9}+6(`Q5t4qTe%Z44TPq!kds@}ndOou&1<#xK%mx)a zxAB?1DtO+C*}QK@2d|w(xE*EG0IZD;LyMLjMJTErMOpM1%Daa#?I^;$)pBhy_ z&L^@^Hq%My!X-Y#ZQN7NcQw%?2*-v{^$2E+pgM#HxEO+{409{!B!OH&gSa_>tdUNy z-q+w#?4X232j;;q^}*{#fu9t@&hi3$gquKr*I<2-in>5WU7(^aP_Yx2QIU9ft*;_f z@gSdy3AYuYg#xrtfEEhSy3ymKoy}ii6Dlk4^UUuo^5}Rp4VU8zALBVpS03%5(G?@8 z(a^DU1hpCtp@lbf8lJ$+&SgWGHH6u*`>=%jBF8YNdl_$x5uD7F(uPq#f>Y=yO1wJt2Y(+Oc)WhrLihGnY^RS#l#4FSx z_5?oZp-)~(_7U#85+*&ciyDkzCBcvp+gLe1qLoXf(86Lm1cHwF|y1v_AqAfsUqe*iJ4`}%=v@|DH^0#Pl78+ zumuT5S}&7rcVJ#U7NZ|F>8?a2_FzC6+7TGqp^Vp*F=2#Wz`6(x;N(OJ38tsHUo>qx zhK1e2YZ#}ko+dZGP0PJ|Wz(G1;rj3p77bzXA=LV}OX|ZTI6cGzd#ZlgLxl3q$unRN zs;I1<>P|xm)f6^@g4__TXc$71oGz0fs^K0~9Yd_UcIhydJ-)2qnp(B}TKfG}4B=NZM6WAS z-w>#8P)ga6aWYHrRWu-Py0?&Z@&CT%52XTL zii-+~3m$-nLj0RptwdtP&Cbqv=9_`( zftos1UY4q%fcQW+R>?RD6D82@Me%_n$I?4g&PeW$q~is?3e&`KCw4ba-4dw2@Iyb{ z7AUW``vMg=8j`To^Fy_h9S>BzCkFu|&0gfmpf6)z`+2aE9{P#E@}5dkR|dgAdZRZo z90n>Du-}HEirqjaiDJ^rVo^IP{S?KcsW#18n**d)D+=A{cpRvdf;QHRfqzZe?weQ> z=>DT(Rzi16@1u13L}0VH)4VzgeII49r(XGbbFG=2%_DgtZP>VEqlN_=7V0J*3Tzj? z+YDpgiUb;(>;y7AaNZ4$l$UbqLgudenCXX`uGEjh^QYS=teMr(73lnbI_~i=u0p^> zi6^J$zs`lBPFrS-OQF9=@7W{+usYvS>Xgq={>EqtOFS)FAVUQwXz*sUMn^B>3a!9Z zv~tR8X6oE}r$R#K@fj)?m_@b(K2y>c7Jr?fTGR9##lnd=!7etg=g{Z08fs9QxWMVT lv?{G=iZRYH25#Wyr0f>ImT()(v`yX&?xe{k*XW~B`vU?)`WpZM literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/NetworkClientTest.class b/out/test/clients/org/apache/kafka/clients/NetworkClientTest.class new file mode 100644 index 0000000000000000000000000000000000000000..dd738c6567c47aabd1e5d3b5cbd252df79b8c3bc GIT binary patch literal 7304 zcma)B2YegV8UH_9_Q}U($r&hwpg;h}P9hQ@1v`W|jzer5CwL?{P~a?`?Mq~xkxnFO zOIu3m-le6bbPwpDEh&}_E|l(;?!7lHbeAsZp!EOUoxF4+`1jL|ckkQ(@B7|+e(-hM*3O(?#u z;TsyhspDJtwvKc09l3m0M-t!D@O>Q@;s-MAhZ=sQbBe?PVR7< z_bN17?X2x?Q>boOx1XZgJW-3f?5x#Y7#*~7edb_>!ed=d%FOIHbGEz(gAsScW)#+k zC>AD8f<=)vdCM(~Da3mUS&7tsJ8zR`Yc}h+rfWM{vZOf1?offcC3jduPEPDwE`jK= z%=EF=YoWd(%rQ?df}U_uSUihIAbOc_=(V!xKBqsM6Hekfnw{sE z!p0Co{9xp)YYQaJCwu${;X#eGVEidv?)y3*BvI5lI!3iE3S855$h^)>X3XqxvLlnR zhRqC3ajo{FDQiqz!Rc{NXM1!kV+A~Vto#_gM7BJW7n?phYbu?l( zeIal>=fw1Z5yN%z>8l~4A%xk5@%8OhtvqDN~73qwQViDiChvbK}lam=-Lc*DN8eD4#bi~TuL zPLf@=2kzx_^Mo>jTxpp%cWa1-fwM}0@#VB^6Tg^8Oxwr4LPuC-3v+Z!hz#EgUd51v z*L$5pE@kbqCCQi1sSXmMje<`Dc`(b)dV1y5fQ2E2)5~THj^~t7@>SIahB3lSD4XsV zvzZ!7w9h0h9MLQnyN8Wtbsgn$4s09kE%H_;> z#DD{WC&bV&?^56>TZqDv8NE8Pt~G4sM74zmu0_sJi_~I6Em1L+7eBCxL2KB~<`V_K z0wv6R!gdoQyk(t4P}7+D8x2*jmTGF5p_Z!^3hjL()~wG7JD;Grtj2bJ#7YYnGm*E` z)&`3JPPqwZDB+SqsvL`XKA&J4OdU4V$x1hHvx*yPr8>n>tJJB6S}pNi(d``ZNPN$W z0Ix(E>NK^&PzklhP^YUi6t=exv3DhoSP2&96lIH?^mIL@?(^8tt_%Z48js;dhN09M z6ZsLRkVz+oEjLjZV?(vlG<>Z(lWnC^s93MciZT1@`Ia=9XqnooA@? z)fVCL3`1R@8Wgsb^9X~cq;6P+6;>A-cnfYbRI?9($kS15Zkg3oi=noXoZK!_*if$1 zQ9Ub*f7Z)ur|@k17ew z`iq-nP{w+!lx1IMiI8qX?N$3&W+aJ%hE;;zh*Lb0<;!ujWVsX1kdoivSTfsmmqz1{500 z4XG5(&~&IYc`X5Yc36|}xR=#!SlbL-U8)#CH}@89C969-I>e2}sGVU=wphv2)Sm0=PA)6_ z+NN-3WkpK9@q$s~RZ6ze%JaP*-ns4|<06ss7t5{O80F}QlVpdKb9b2;Stl+JiPAw) zwl3&5vKXbN+?p6ZVf(cOlCmj;Q2KC*j5!%L-KBz_^7z!GGM33@l>B!d9Whcd@JNw((B{G%OFI%v$-Q-a(wX?f2 z_V7pvscjDT+E39r=Xw~O*2nSVomIHo$p!^XWWCe3O7ySOqnNxz6NI< z@V}oOYmg88&0(-kzRF+`jYTxclrY-$@MxJvwV+XHoxEwnPqIS;Nh3jz%ZpLoh#gpG;kmP zjc^%F>p>s!BgQ0rMFL|Q(S#fELMpozaomI#5g1~G(!IEW(ItNIGMgZ!0f&pD=|PY9>fK> z0$sS0BhO?-yNc&kR}+BO;AMQf4THD?DU2hHkF&~snh<@ERqpGoa}V>-?QtAZ5gb+v zdGdA&MpYy0oWyv{4^a@~MQRnfy_{iOsg~dscqL=J5l?t=ucrLJ;Z=Awt40@o<{_Yl zv)_r=;I%9QxADvC@OlP)oT+g$>uHplICulz$ZGXCtJ5ub6T$svN|l=AkKD?U*{M(^ zrycw%b#{zJL{cHqBzYX<@2#kBn&ssKw?FyNjMo-=^RK1XayX3@dj`dcz(R79b3RIt lC#mvPkTi|(zM8+c^0^9cqbI?VS4FS#Z*i1~Z_=~+{{d3#N%Q~! literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/KafkaConsumerTest.class b/out/test/clients/org/apache/kafka/clients/consumer/KafkaConsumerTest.class new file mode 100644 index 0000000000000000000000000000000000000000..c3aa8b02a35735c899a967a43ddd5fcddd257733 GIT binary patch literal 2103 zcmbVNTUQfT6#fnbCx%!cURrN8wO$gzjIDQ2EJkdB02UBSZ+bF0gwe^IbmoLgAKE8> zOP}=tS#>RcKz~%*?@WTS46SQ5Yi0J?d-mRE?{8n`&%ge74d4=1GdPAT4z6aqFq8}3(+(y}+?l_oqa5sY?T*+V%Upn|IgDFg>F_Xao zR4g-_#+;?|4(>VlT42u&KlIJGKyRVAERZVeni3eO_@SC>HL5CFlGT8cTt#~_SeB7* z`HnDU*8EuDaz#gVS2m@$rralT^@()7z*nJ(T~CK`tDz!y#)``wZb`*vOyGbawK5a6 zJfov>pkrPo_%H0@Qa2hpY&)8K>Zzvjbx1FL%}6&H#i7b$xh~z7@dNiBMJh7Bim8y* zL2ZgjmUSyMbU!7qkD}zZj%?qek=~P8*-R2ku(?Y31Tt@KiXAKn9A2OgzoC}>*e4DX zVW^F?fiQr6 zWg_1zMXISIlh_rz;;}_^XEotVl;eetvV;3sEaO2I-(bbTw^@9L@3Z&;D*|qpm;Y`U zfw69jc0Vv9pE>-JRKdNyVbnwv$&EW|`-O@u!aoX}`8O9yUEUPh!NV*bK@y|8l7*|~ zjm~@9U7An&JK=3#XHm5oK7uDO60hl2P|L5$b(QyItG;IPGR&*(UL{k@N6NIKP_Z;M zuqQ5d$MoLGVO^4531nDz=c|vE$FAQ0R`ezlf&N&T&Ma&QTr0FkBr#lMWyAVdx1_j3 z)b`F98hGoz9LSgr@ps-&C%W`6EWMFd1i?{50gK&WWXZy6{e@HW`-cl zsyLQ)*5mBXGIuK2UDIUSud=}f4(>GQ)UUXVPjHkUBm3)tuuF>16XaY{LE3ljCB*aO zb)L^XiSQ`RfNWauu4*Dqj?G<`fUZS@$oa@W&*@W|8IG4`tZ67kZ z{!L`<`33fE;)9znFhDZ?3WF=d;uVHynd^|3*gu;39V4TuW2r~a+WOD7aPWA-Su@M! z4joQy;=?U`^gMTX3rC(as7si|J$@K_k!L{T7~t~+(&UDi#1RI6hJl~QX=25sA|86#j+;CJ7^8N>OT6tXjgQq)!08M1kjI1F}#L%RE$LNF5Ziw z2BT55VNAt%1n?}~a;B@`YT9wvlX_k^Hq7Lf zzP6<&jjUzbzL(r*N=`e*)_7TBNI^7Xt=qa^a7_gZRf-R%H}!2jnbqy}WX5+bdwu96 zWrRlRhArFjM-+q-9SaJ=@QSU}8nBF{GjQdLp{RN#Kz@q*G( z&~+LDr&ZZdM@m6=C8wOrS^hk)4f>H;oEc}|swkuO#Ry%UK@TkA)O(tD$IEv-U>1wP z*+fgizPZX*QX7AO!p92RALlo?tHGQ6vAv+3V}g`Xgt-FFb?!px+(XUcV}y1QeuCOx z0)Y;W^5OwsLSbA)Cs$fY1YPLnPQewbs}Xfd9udd`tl0T0YQuk^ZZQ;BGetzALTE;` zi25S5M^EwG9vT+oYA70Sj5qD#`E+?7i^t>VxStk*t|FRc?k>+UiqBjr;{0H^t2O)^ zS`<9R3*G-MW%Z&ETVo=(hYMn37cUiYkv4_)7%vy`%8Adf^0P!7f0HyvC5@nS7*pqvZBwAzeah0nY xYFS1vb%&_y7OwGq8^(3o?c`kIA2{G2IN-l=!2cHc-X?z?xdu72KX9zU@V`a3Fw+14 literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$1.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..46d9a7c75bf2ec5ae3f64c9a39089a6a175af199 GIT binary patch literal 311 zcmb`D!A=4}42J&-y31nXK7rxj*$D}6umR!$J?IORu`bIF9WpcEyLs>dcqrpwyn3@u z`=?3&eo0SXAD;kb7#AoCj0n}UKLjg_&I#eR@fr_136GNAedrt4iKZ8c>3+*cCfv-` zIv+c2SCTgVwLnE!`p_EQv)zd4`K@PTe|*u{-*cwZBth9l^C#hMO1OSdmC!hjQ6ix} rLyco?&C_}-HWB9k!m1}AT;xZbV~`Vh8sakR5LdaExvIQFSy7kYcQ#yV literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$TestDelayedTask.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$TestDelayedTask.class new file mode 100644 index 0000000000000000000000000000000000000000..4c771d85458783888cec97ff16ecf46e33cb8eb7 GIT binary patch literal 1076 zcmcJO%Wl&^6o&t?W86B1gj>0lQrav+7MMy%fQs0di&8~pK`3I!#2wmc+%fW{7v2g9 zgv5de;Gq!zq!u^PMWwK?XU_3C-}%q+%(w4fz5=+5PJjmH19+I9CI?b_KW zw>3Gfp(@J9DwT(7b0~|kK-f1%<#8&DLNVagAj`KR`A#O=DjLhTV;Lo>R;DbXgi5Mi zl}Fl?DmOAMqPTXgt8zcf$MGc6mgxLdXO*h+%20K8&5CzTw$HBCd@KnJx>G8;vTE7R z!5xAA)R6oWFS066)KhJzyKrpJw|C^83^9u!gpU~?%K|U|%QOYrwpNr%vlTrb?Wm*_ zxOb9Le;k2lC&@gv{mzQOlc|uf$gALiC!P=NEctff*wNC{q;;Q`{TB2-!|A!7;C`Ux zAhbTNKt;wLPIDI?MdA$3(&GKV`I-`w>eAQjz5WQd@#Y_xL*m{~Fmn@Zv&Jw97jO}x z4a&My10Qg;#XZUrzZ4zyQT>MTequaa#+6ChRVF&P1`?^!q5cK5SbmN++A}`@-yRud literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.class new file mode 100644 index 0000000000000000000000000000000000000000..160f7ff93686bc102cf8fb9ef2a178dbf88fd9ba GIT binary patch literal 5664 zcmb_g2Y3_b8GcWeeb#X-V`xJtAz(nZ2?`FQ#DvD!X4-~gGfdm0XZdU&mUZ&!#7NU7 zq@{cBz1ws*P%Js2O=&vPrhCwl?%k$)ujgs{es?F^vZP?z!lQrP|9|8C#{d0(;NSP$ z3t&C|rJ)_SR^zD=wBczgZqpFPtunb?#VHLc?vTmTH9P~SW%Nu9&%(23;W>D&is#A7 z^Hsb+#S1HMd)WV{VIE#2A6~5DC6(kN|N1p7!b@fKWf8m_uh7`MQseYh!sFEuyaun8 z>FZ?Q>!o>vhBx9(GJ3Oyx8SWZeVa_*E~9tI+B>DWQ^mVfyjw#f-lJg+-Yd=f@_a*b z;{7T$L z!HN=l^MVAMcwkQ4PNp*>0+%m;Yx!gOtuZS}!IK6NXbnzxneK7h8J^ItYKxV$+^q^K zqV)$Ueyb<`+zu;gc4bETOsB`_OL*wpaU*fia4eYzi(z-jqTP1$$1Y!aCuXC$H;8{6jFI# zX^M82Czf(-*N)qXm`kgsB^_nr;=h%q7+s-&~bGs#YEiJ42TFGiU4&kt% zN-%Wn!fpktW|q-(>_G=}Nb&A`3SsK=T|=0BTt_FmRD43mC-EsAd$CW&{W?C4&!~7n z$7k_51xt#F`4~#>((!ogXD%$>lV>=Q*yc<&9QDEvRKqZ>U6TFM@usow~-FX%Xc zgDSqL<4gFmj<4XWI*#GEj<4aOim&VV2EM7oMMlR>_?C_b@og2~(eYgof~Mk~9U*?*O~{MdH6p^A8>W#6t?&i|PIsz%X@8I_BQK zqh_4dz>fb5={n}i8K5_LEf*wrI@_fR8Lx6-zDN4|7wFnKB$RUokL&l+t$^t%3_jK#CSQLSUK+fm>469`ht6 zuI_B_X=!WeX;IKn7Le)3D0Z&ATwXOhb6RD0P83oteQDP*;;f_m%ejZ|2Elr(&3ecyb$2m71USF`Y2o|B;RjtFB97ZWnqkSzJMI#C*ON+>m2R_>yy0oo zBMN5u6t?rcR7uezUQwO06A1-dOOczYE<#81iT5}e(vGAv@wjRBo2-{Bj~NMBtIY?W z_ekFuDR92mm*Mi5Y=Lsw-DR;GN`z0f;=8L_ZuX`0CDf>7INkQlJEY!T7 zHJaD6?(%vTP+rf{hrJ4gH{@Zqy_}7snj;|Dkb}}t6BeO_;*7BxB4j$t9Ei{x~KzlHo&)#6b( zTrq~lXFPe1uw8`++ZFtaz)EPGEyQZnA&P5JPfZ(G{54_(89afzbYD_D5l`YwVTE>> z+EWWf<@qqzPI=tU;>s+RnLK2@p1 zI<`%?k$?TzfekF8H_{ZFsK#cPxE`Zu#_ibRDcRsF5-3^6JvZaYRMx~280e$Iqx@@6 z9R1{Dl7oQa?;7A}kSkT}huAByCIAAdR&ebOY8Gjj22jz$bi@P>0cjG%&RSe6s{xF` zO!fpo=8t626re}M1jI>%8fb73llf@Da%>|mt=LJUAEbj15vMd~x6sK?(apvD%X|$2 zWH#_!a@0d+ClMUxDCFU>h`s+!@bJG4aIFxTL>?L3PrO2;K71b6^(r~6?-0=&*;Hk* zA&ZSNjbyP&YdnX?ijAj!r$-xQ_K3GdjDF?BI_>3z+OC>QM-(9rF#1O z7Dj9vpDLYE%c1Rsw@4NO1!gB6NunB&lklf-KItPa^zcVEWx634c?p=&72; z^?4b`(R?Suotzf9#0~PDR?LUP8FE|9c#Trdu;=W)DR068O1hYCx|~w2q>D!BsP!12 zfd*;dAsW+~qQmY$hXo9Uyp#J5?XQRdQ`BGqLJyNvNX4iMN1{EA0(C?3V&i5}cM~c( z0%o7b*4~=Og==oiq2(?vggi+lI6%ZE^U3T$ijX`610wtAY7#DUB-F3WVOuA?w{?<= x=GkDBc5|k<=A3{M&*214l2(MmNT`PR{}xY%r_iy18qT+~$GmE$gf_aV_8;?OPYVD5 literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$1.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..31b943f8317aeab69cc659f700a090850881f436 GIT binary patch literal 1521 zcmbtU-%k@k5dQXhP+G3GBH(XDt9u_cxYWlfb#3b$&v4~n8^*kDR zEHO+U@t_(fzAxgKVX1YMFyN zmX10u?{WV?xLv;A<*pw{@{CiII0;4MN@YZ(co4gu*3q6+1RCv#*rY(~Qc1JH(5;Py z^dH&RcNuI??+J!-ODgdy3ELvt;q8ESW?I_k!7h(vs`oeTlr4j^sg#JkfXA^Q_31bX zSJ#MqQ7cb}y+9a>yjUCHu*G}L-)g{B$9?h67d<1jYNpBO_5OY=45d&Wa$zWl6H|t{ zkq8X)r#Zgwo$%Z|4YxjmpbkT=+#TJM{u>?{nI$z4r=?Y=xk21Y+lKpH;?Y0D!nW*C z;gg7{@*%2Y1R4l^`jCp-I*EMoQl?d%9csLlwAbnQUMN4%u~eNcVGi_O9?K5yW5q!c z(+;LkaNxkrW7WYL9#FTB(d^YVhDtVLSxH~F4~1_S9-j^A^#_T1C6vbFLC~h7WoVx5 z`q$B8n4&)K19b-Lan|tHM|8#T#E;IHx*}Ig8@T?@R}Fbkr_;RF-qv@qKzG?BXaHbU zDrtEDTB%la-&yoJDAC_@BzdIgXe~FEj=>ts$FLfok^7Jtmg$?qfc-#)el)dwssf5UX+BP{z9>=e!>bC3x|qZDk5t_jP^U>C9sN`GKW6zF0a yma@Er%Vd6qWT(F~kEgX<(%xIxOzOj^J#T5a6M9n6qaB;yHM>G=iJ&8;Q? literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockCommitCallback.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockCommitCallback.class new file mode 100644 index 0000000000000000000000000000000000000000..a65f3ac51cdcac3d973342d8c51c56b5fb4df892 GIT binary patch literal 1394 zcmb_cTTc@~6#iyyLq=f^045cWSOd&P(<4zR!$!p)3&7&I$JkkyJ|WeWve2J8X{~n_-k6rxEhHhk>s;k zGjTHzT@kiq&=$LG5rmPF+9pA0bkgg{I8fTkSc@nLs>Z}SN(*b^x=if+-vtPI-e@Y? zjie=f3#9?Uo>ft>F1nR7`8tZ8+1*W~B}|2p5Qdr4EI)UTn6906T{(jd@i#iCo308s zMQoMp`$-fvqnnjg!kybvO`_I|3APx4eTl^95cW`Jcw0IH@V~czx#Akf51ui+zGNc0 zwk^|(L-}60v6>#a>8EZ=efXI0F@~IvJjOjdVc7V;M5@dV zk^VwQ90gh@DF8f+H209>V rVaF6)CmaO_-2o+sKczCbPu+5156I5(5LZb~(Jq6>B=fjSI(@$Zt6*;Q literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockRebalanceCallback.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockRebalanceCallback.class new file mode 100644 index 0000000000000000000000000000000000000000..313d381220fb1c4b3c1b24e5a07dea381f48aba8 GIT binary patch literal 1662 zcmbtUT~8B16g{)M?b5PPK1!7jwIE6<%8Fu)0a0vXLQ=ngkoaPpE+Y)x?likwqCd+- zV&a28z#nD2)0PUlFSh$|XYQSud*+;bcYgl*{sX`|9%f-H9uH$04GJJ}Yc8{$z!2{okC1+S=95HqkHhLQ8tZY;ei0~v2I zn5E?d2CJeD=^DAJ48*Heqb95fhQ{5 z_iNm%6Z}pPh_K=lq@Znk2`qdmf;b{JLDXu9(3L?f!hrh`NutF(t4ssZCFs{ugh8<)vY~umL+ke}0z``<& zwf`3Cs3Z4v?Sy_78|w@^eIhL;wC8w2>`9;b+bz1rura`ShDUv#Mv-2fA<$C-i!7Ce ztV1uAff1TftuFagMOtIJkThDfi;7Erp1f`H7>&8|HyCB}45l{L7YZ4eB0CC`rsJ@T zJTk@<3KVlXM{or*f;irA6wHO7%#ka1)E|W4CD4z->&^I73&oaR*naiu2^_F9R2A>CA5md7h~N literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest.class new file mode 100644 index 0000000000000000000000000000000000000000..3703ea531e87e1ddd0ae92c446deeed6c52b17cc GIT binary patch literal 17373 zcmc&+34B!5)jwxQGRg4RLI4pIP>>LoxSB!AIVeXZmN%`@lnJ1XrAun8Pal$muLFuFdichm0q6Zqe;d? zm6vDx=t!O;Z{~V=UKv;Ou|Ar{$4SfaUOvIgCl>Gg*!bt889ZNJoaE(`eKeaF$ipdK zUg)EF29ZTxUhJde_*8kbL};Jp<)vP(@zF^l>N2_3%GYuq&F4DlwxWzz@+u#`{60Lc zmQicuZM`tsAa?;g#82(flpU)Ra^G+dtp}g4T<%@ij5a1Ggv3y-3 zy)G5zFB5{F_VR9F;Bxu;OetR>!ozLC{_Q@xj_>gD=Y4deytz@{-07p6_zPaX z%gcM1#x=ylolW|tP&5<^>$Rb1bWG%c&xfv2(1kzQjt_79>d7dm^F#p=17+wPp2Bd0{fXpCiGNd$1-pjZ*NzPtI;>1 zZ%ymTl%wyMge4Jp38nOo9ZW?j&>Dz|bSe@}R*RrQ2~3oPqY*upVw&N?xO!!DuT}RV zZ#bGxrt}2UG?x*EsI^8H^e&3U+rZF~1-l0D9{oln)0>jvM5J4=0T8FT3^2I~qeHq& zPgF-@U?UcSJVD3cU8?$BqB*oFsw_1g4n;SF5)pYe8;eq#BUq$U4|aZ$5j+$p z^;8;`KCvks6LZ@TNk%Y9O)M6N>KW==uIAX12D8=6^!9i{G;(YT1Oo9yS17tlhacFa zhhTRl?P)_j77dB7Cc_`BP=i5ye0`S0JGw2Y8xEeVQ$shzBW;UB8%H}tGZC32g4Jf$ zI1o?OYzsxA=2{qHJ3GK7OQSy;)jLAbl{&nzLe6YoAM1?8x5q5S72i5BXNu*KB&;H) zhk>92*VXYzY^C^P*rJSR(j6_wNQ+6^_QN%B!5y(KxWJ4TE!EHI24x-d&UHc0a0PlI z5l5T!J;p9kFo1h46$oF|BnGXdvHV8*VeeQx={b zA;HGcu53UF9GoCpxc17dqo93MZl56J$t5(~t&?M_Z0$Qx?`#ID-Do#&PQZ(wzEvgq9I!1;z>Fo(Ux!F(%rYU)C_3==fKejdz??6AONL5~s zY=Bn86PDpH&CKh3dL*?u5MQCE!ka}BV_uLw4CpMDkQl>$OX?ssgB8a&#htlGLXys(Q5zUhoFb_-HVq?e>oJyd`Cykb zOV;UGRIs)u9TXW1EJZ?V?pr&YOA5;$FimhnCpX|*PBkyI`-1UwBCM~7NK#$l2(rxx zUGro&tkGlixNJPppwZ{)PB> zJ)jwjr{;ErQrPinbRXU6n44zuB zzvamX+TMEkYZ`x@9}tQUYV;uOMSfL~q7^E!lg!jF>f^Bv8NOGeZ_qxbnu5b!7_#&w zYfCBAzM;{6`WEtzd}^*6XpMTPmuX#o_godtZ1!`s(0$h_%x?WALJC4 zo;Zp=%hHi(n-oAwEoe0E<9?05$q&iA4{P*odeqC9`rG`d#*gvi8of(@(D(^{QsbxS zagD#j-_`hg{Itf;@Uve2zQ)h-UN1kd@eBN-#xL>98vU96qVX%z@dtS1AIgg#3Bgz8 z?lpd0zJ4qtf1>dl()Lq%`!jyi%WrA?bN+?KZ}U4E|5BQNB^`b(cTezdn2yiKBWqui z$&2k|SL6Q%>-1Opo5sHtA%4fd*XSSouEu|Wwelb3?oa%l#_!AHpXKf^e80vY$gF>5 zK!TAU^4~T72mg;oA0z*xSs5z?nge+%FGl=+r zpJzCh#G`peXr7UtQJQD85I&_4VLNHCjL$M_E35(^<^9?ioA4>e^KD5?C2tF_@8g zIiZXkhw|0|Id@BUWbk;VWrG|%KycMO6U5LS^BjiM_F%{j(JSZrq2pxkiA;;~=N_~Y zB2Uk_=*CT3RDEN-8SWh}XpvvaUsLjClE_;vv3b4Jc|k3SB6Uq<2ZEyIYS* zQO!jKol0Pwt(aj}Qr}eEXhIHZsg~0iyq?3c`_4b$Vcm_Z+|_Yr7DfpqE3H}N+$$J6 z4!13Ro+*Y4&y#;#;j*#@cwNRpgMHwT&y2&;WZPPhl2ZXzyYwo|u4C<|=Yg#BpkTUZ z*MI6%wG{h=Kx3e8pc9sy5m^1AuqyJS;vYm+w>zPChZ4FK^boMz*4O~grj=EF5>*_p zW2FkQjvthIzpv7gF~Ev77k^>n{tZvW5o{Ona%dvI8GAO znFfykMhju%~c*J3NK#P)Ud%wa(O%kg9V%$aWZ32UIDsj zE2`Wm;1qSoqfuS@Q{y19{K|tYU=&VTZ46Yjb6?EhPuY}&G1zj#SBW3 z3EbgaD2a?A6(&Yi?>d+gv^z(3r|n7KPGm z5Or0t0zW~S(70knu&%kazOiO`t3S}XwyAMt6LcP^RJ#3cz?jL-{){9J$3INR4G;$0 zW46SmprAFgGU%3lWlUXpQH5)+)=8r;DTZi@HKEchv^#T2HA|=43g)V)GjGf~B{k!Z z@>d6_X($!-AJ{Hk=gLpUXd2x@x8g^5<>ZlCEbcr~fmK(ORn--hRdq#iRb5eERaaD1 z)fM$jbw!<0T~T0FR}@Xv71c&{McGhYQNdGJR83LO`KG#m2=k&4sP0jNQ&-f)aGioa zs2Jn>2|Rs^ig8EOUe!ljRqpAbVO8ZtJycw^pGsP)_R+9medO(-(yGzE9xAJ=74=hj4~>=kvAt9=?3*;sgXzZi&;(FAtfh}8_R!%EVkQqgiSOe{ z!}kIzqJ`w6MKqcg(^NW@j-e$qmrkP-X(`^+(h6Em8)+5ko-y7|9ZI~<=Q@EL`VhR+Fow5W#`XMm~#JVcE&oYvCe zw2msNiB6^j2(^woXhrRivG; zmrm`aC4F>SFD=FM;$Eu3MOZ7lw3n6{O||tt+|2Ez<(KwT-Rxdk(N8PUGPTMYzRDOr zx2%`^#{1Q{tm&uKBGNFm_VXYNQG~|P7Me_55Gh8-QXCr4jTKKouq2&HX$sRe^w_S% z8gH;`igheB=KJ&<1bZHoL??`v7jS2Z{36~ky<~~ZrKZSND}M2H=pxrE#7vQ$Q00i< z0P*({dIx|-V?V9M>;QOO7AgyABse|~sC){j>;x(oLaJRrwB#f7^8Y15O+!Kmy$_7gwLs`PAao-T`YaIo91yw* z2;BmNZUsWO0iip9(C2~Boj~XdK-ZwzQv`ducuH zoJs->7|5PB3<)~)c0f11mrl=8u4Qnrgwwon2sM*&U^WvWWjceCd!N{GknD ziK&HGLZSY>&bz3WRG(iGI3&$(MBj3`W5hFKLk!c0#2{PN`DM1{Ryn}4Or>V zfYX~4qPO7xe*u60HvIcL!0MOqWWRFYbEWOa&dl>;(vq)YKeF(lf2xp2vs z99F)dpY*-duKbfj!7}=ABzaX%JDe*Z$an}c0fHP(bv%jcd9s7i1_z_V!P1NPydH+p zF^tYkj{hy(b{vS?sWhCYf!m|O?R0QE6WmsU+gXF-_6?g`*_23j8U?nl3I?Ym*{OP0 zY8YxiZSJQ?A8moT?x)TcY+IuB=(8A~Ql#?z)P?tZDONSRkK(=5jdshoZ8c-1u{Al0 zN&)>`8prc!G9ODP^YOHlPoOjTL~4T>b@E9{AR`t=P57769QrBTTW$jSrY)6~bMLY^ zcS}I`Q$qF@Nnza*Ptr`MczBg36@0QSk-RsuOW*8$loHQlW}o{go#l1Z=7JYO)I|_= z3I1^FG>E#CKEX9~HrLV?UQS7_qYHV3L*!ki$TJY{-lCu9iTsO07g_SgLl#-3lkOs~ zhRACmasVPXLgcj&c^yO!LS$?|cs)ekFa(j`wncu`49+{P1sv=u%l69_+4QbMU1ltI z+K9{TEMr{TE0WEYov$kOm-zb%t;e|V{`g|xoT||H-mLvgJM~X)9_F`vI4G= z>)ZwA_kj5?0snh||Gn@~_feSdcVOOTV}6<|=JQ4K`x)2lGwI*=rIr?%(4bgta? z(s@?zj9!(0O4R|+w|4;-7+rUw@`X1U5*PN;E=1bQnjAxASd*_ptgk_=2aps!h@@yQ zt>JG_6YoPl(F0TKb*#}=#~Mj?X_T<=OFvVnH%y@pRKhAO)J2ABWeb(#V6$H$lNhwb zMyPoR67Pq^4@2TdAn~^$@uRevAEP)wPCNJsx{9B4NNts@vNtZjv!(vPl=^OKaom(L zvkopxX-1lC!3N`CvquaC3RUCe?*WRZ0mZX`;`@N&IY99|(u^1Ae14HG;g|4d)GyN> ze#HUD-L6XTdmD~k6OIEV$Ay)^bg?4b#eL+e*_^@i!mNzZQ0iA9*K3gL$B^qMkn0V| z^;0PI&!E(ABBOZAA&-?HJD1lZzs7}vKLKZS27FEtuA}zTr7e?@oL?3|f%((MQHLBA z_+&4@Z9YDN76(Dukraq%r;T!#H%39c3U>57(d zMU&>t?V~Ff7EeT}|07@hq6!06wvoq8a5yJBe-v>t?Ppk|=@u}1+lgZ_w(y#O@> zh08mYpfj+|zXxhE8hToWZGH((y#N_D`{*gZSvDD5f<7|Rx1Vlmk&WTa87|cJp8ut0 zH@12{dv>3>)$^ca0FZe!Dtts$SdimThrSI8n85W8TyH|V{JReSErO3G!ZFV0ZBCrr-<{F(gV_C8^A556{-rSg!(lO$;(3*gmAHtz@M^Ovh^>646+o<*Bjn3;rw2Gh}LQQ zBzYZI6>7pPrq=P6{aRel?>##oI?-&-N44QYlIt`}QV!<+dwz98ID1r;G!JYhMrihV qF0i)l`E4UU6YdU;X)Zw+mFD|7DvP4VI4(TKNx@0Ex_GECn#mtf8C8q` literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest$TestTask.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest$TestTask.class new file mode 100644 index 0000000000000000000000000000000000000000..fa06e21f6b2240fe3590695f1c2713f6c494d9b4 GIT binary patch literal 1418 zcmb`H&2G~`5Xb*(2fH;64Jj>@&j0}u2kJ&b;t+&@rbsOj2na!N+c-<>#$6+SK>G+h z4iZqrfd}BB5VOXlJ@i!Pvg;Y|{`NmJyYu_c&tCxUqHV#zm022U7ED~Va1GZj)N#Ya zO%u0FG#HFz%a7|T43B$ZwCVB>+}jfFn7O#{N2w%GjykcWUyX^Y2=9~zN*90Q8!_En}6gsPQtb^3rl@DCR#Rb$-K zE0nXhQ@B{oUY4vh@dNi!6!E>DpFQ6BBis}RAB`9m8viyO%5bOo4mD~nCg{b1RVu2~ z)Od7gs`BJ5XKhF$8?c<#H)zg2^e<#-I7OpDYk-%SqfxD=Rj6Q|W`;9Z$O&t-S|?xG z`3mg^O3rtjcyj=;GW=@-3^LGSW|wmiZNnnYl01;8r5xGKk%rUS2RmRL7F5d)H6|Pb j=SiLtY2X4b5<$ZyvNT*qmF6ma8Ps7<(|nF>RA&AHg2{xL literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.class new file mode 100644 index 0000000000000000000000000000000000000000..7913ad842ed86b6ef7a270f2555d2002f119f667 GIT binary patch literal 2833 zcmb_eU2_vv7=BKg53+1&EvZ;_YLy^q+7L<+fs{Z$K--44w6zfMYnvR(Lb|E@!Iber zubjafFZ5RafipTIZAV7$^&fccjQ_zK^*OtnCO~JLX-sBz-}9dHzUSHJdEc}9+uuL^ z0^l^(l6Vb84FkBCz$FbGxTrki2~22+E9YeFvwwo0nw`>c8CR6`j)r$Jt%frR%xdUH zDaMrQms0ar8zZe@^f!!{hB?ePMj5qwA%SZNEGDodpif(tv?ptZ<48x~z_N7QWy85A zkaQ|{WYw!lTcB8~+iQ7a!=NjF&$xZh$X9Bnv|J}&sauY>F73Q&xze_bnvB$1~%1v4ETv-+9E!{Ql8+p$)Yx!~8HXf8r$K{gblDTFXu4hYug}v5{?H5F$ z$n^0s(=y$nKu2b9MIbs^uX3ZVl4;2~Z+%tT%f@Pro}N;@V$@a)+f?Jw8FlZNWM^W( z2z(k8wFGj=<$WpO*0k0HVlIgq5twZ|d%B?>xAC){3h$!=eT$yun(K1KbWA?jxMkH{ z!!@a%K)1j1U5^CiLzJ3AxhU7`_sNN7R4USO(jzBN3Y0T#<17sBr7uB8Ho1YeRQK$P zykaUj`kpN&h3|x_ARTYwxQ;A_1jgDaPaSC-6F9s_SyC*KlkM@y{k6|RRs-uFc4vE_Wm)hNef;XRPe)X=b zxUB7Vv!+?X2Q9hjF0;z&9#cj$GlMhSOl1y=Yn-Qy%^6)CH@18E6OZQwYcw-6<2U6- zy;ck2*rL5nK_=-Z%Ai`4a(%<~PYrf!rnQv9ljL|NC@){N;Q0#Ad(?&x|A$&`yBpFY z1%;)IR zfg`-Wgf8^qWh~+ZJ$jG=d7R`(V1(%-D!up*_KkBI}HWYs=B0r`G#;{dCRs#MK#ZSznIjo{I6w^5LtP zrUJLgI?#_pl=(#pJ%BeT=23p$NAQ|?vDy@5e|%Grlh;%nsaEw%cN3T;!L zUvUb*GwKh#g}-n{Bym<8#yOG0d2s;+U)5}&T&QYRq!=?wbx#Y8k)u?YFLuU{vV_hM zRh)Bg`}&{V)xST`ACGpm{4aeKxuK%U3^i0H3*gl~Z6D54XZ@sTDPktAgyx~#E zn}%;f-ctB7zH`p^{=V}~-u>Xkmx<_nepjbk>HRAD!)iK4KdqvlN&80{{jpA!^mF<6g}i<# zuRoF3pH^U<{OgymKhx;Xb(%)+%g0}6^p`r-(qGBPUu*O$jsB+M=!ecfoo3PB%EaI4 z^!M~@>HULF|49E-Lp$l8HToAB`B#npt(tyA|E|*l`VSfTPmTUdrzOswzt!l!b!u=< z`yY+|SEuFlfq;FeGci|l8JFwq;|h#%rN)}ZRXVMq_cgB8X)Oq49WB?$%g|{xPZ7we zAc3a=L!J)&c}5l2>O7O{s(6;{Hrpe)tcvG&{qib4Lni0SYaa0BkI8GkY-c*6Mr#UnsqcWWRN?&wBZ6lHLY6a$^l|;&p<{#R9TSsJuizo4xZYWI>Az zwMuWZEN#=cUFS=Ai%ti*Lta~5@R_#@`jQszX~aCV%M97u@LfG% z*}VN>+7m4wz}j8Ec=>hd^jUWEp7#)@qVCvZjt>QdL0muI}(i_?snjXRZ0&TEsaXM zsIL{r4ox3r-G;p_ds~Dfl^a5lkiChiykQxP(buYUW?E+`Vg-^T`>c4cxi1Wk*LFsO zW_YI=56QPX=(C4HAbQ^I9h^>>(+wf7h5~)ssB?gCKH%d^5BN~5PNXqifW&ar& zu5C_=yY062nee_rNPMA8bwt|3p}`@Cv0ahvW-?**XIw1Pyv+I{lUb3po1w4;2R<_k zp0v9Ly5Z+>(sUWU)UunyVdcW1Qq^X*e?<;wKvGF#SH40-Rg86x5oR#&hzvxVHsQa< zyTLAHgxyvm1_K8|TN<1rBcZ5Y;zvtzU;wt(v$eE4i^ZdMG#Cy0ZD<3$q8(2LMZYWe zskKbAif#!5u;TG3%%6J@--fP1(28g9IIoBfc;4hd5==|@mBxFR<2=15nv4gncJLdF zb-hF{K+l!PYS7o{8HsHeFz7bA9SfwtQiWK9J}D#ToLYRvUcASkPthF)-ASJ|=q`EP zEw6j%-ji&TA2<#A41E>>u<%gV`ALLzJguC#*P#38egweM@R^`GHJeOyMk9l={0f5} zpwDT1rNLKmpFt1OLmFRgu*v%j4l-19e&NOr%OzSG^m%%iX=5QwX-{1cAT_oO9$gQ~k-o&#T)10l{5;Q>7u1AT zA%ng`U)6ZO!NVNZc*NicM>UQae1PKyJweBr)|}2!t~w%S6|Qo~O;`vp;m{G~>Xq#^ z$K&SV&6YPIkBYvn3gdSm8ub*8t*;G5_fgKy?r3_ixU8hjhyZty4hQwDvX zZ_)S;;m@50f12+y=zH{g2H(y17mSgC=CVV+>(FVeoN&(%`4~ zX@kEedp^U@8vLBl^L2jS;BW8=jlXH|3qneY(*}=9?_r^JOnT$|qF_3~w5kx1N!|&` zol0;$2c_}141S646QR6p@GJbP!LRYR4gLi z(znqOiCFR0u$f3$3BmGBgWuw}rS}fgg@uPr=6p6W&hfqQ#93F=wQs)_L=rJGJCwBm zACz=#Rzrt7r|5SWj99p~fRx~5adoMT4(e&y2-!?7CTX>0E8>}c!hg~W@{a0+|i z@b$$we=18w2nSBG_%gZjx>t|V*m;&$>~`Q7sS}2k{FUGxeaugJzzZZ|$oy?&6K7=D zoDT$IPhz%mG4i)YM`B2863E|sksl!g33LV8;L3^+;JJ`u3vtwnLZRlZ$h%-M8P-V5 z_U>Cu$>N#a>7r3O4|MJv1=qYvGZwQVNH10vv6Nipb*dh2Km)l|cGMYSs`D%$*UFi1 zK8db);V|dE?uCZDaNpF?(Xpo_6TH~#1m)aVTw1$MZ;fjgCsW$VrjvEF1qoJZtR`hs zO^!#&&J5cu}#bo5Dt>+7s4QlX1 zl4l~nExkgOZNy9D1Qkz5s^AdaikGip+1?f?P{UTa>nj%@3`ga9;~NNvaOIxk=~IC@ zSGyIVeE?ow8j(r$HaWtVu&iMi=_xN#WYIqs7Lr36tPnWROzPQ$<2V? z3{MU!pBJsejWnAEU;vk?T({u{xLmi9iFv++Dro|L6g;PkuXO4p%}8kCh?Rz9HgUV{ zKoZ5fO!*0psc5`%RCKbObF4=0RY12ojCQBw&rmqm>0r;QP$q?$okygX+F+?RWF`XE zA)p4F_kc?g^^x!FGUUz5<%6U2O~PWw6chv1g#n~hnhD7g#Q65+V4Tl9wZcs%<_3qP z9+Ali!7@}FM^s67rh9G{UQX3Qi8cu@xJ5f`?Trfud<}bKwjw6glFVEnDN2w>fQJi! zaqD+(^hgZzSao^QN~-!+RwL99;&Gs>x39CUd2?HLUwc<~-}dJ2-j3dmt^n8}rL*3s z@O6jlf<*h^lIHCQTo&ls73k~j+TPLHm%nUgS9^O;TW??24m|CB-OYhZ+kj6fk&`9` zEURBiIj7+lAXVg{*&Nm-yKqFfHY-bKE6fI|vQzY{Qh{6@aDCVdZ^&Hirq@!9?bySPO zj%rb|Q7uX_szpsjwJ4{k7F80}qPC%06fIPX>I5-;S@j>sxu`FwK1vCy#VtKE|1|07 zqh5vgvqbzDFlMwX8b^s6Ys*qp-dO8PQAOhfRqk$lfy&EANlQ^xBL=G*%TuH`&M5y? zifS6`$Vkytz)jmdO4Cy`1KnC#GGmlxrl<~|vvyb4(VS75o1%G*vYBs;K9-{S(x0EE zGt0;6tTK9)&Q4K1z!&U3K?_G|QHmDhYl*wpIVoC-&jy^bEJcmzE|-})JoE6(03M^X zB1J13XZUE0&OJf?F*+|rs{pxLAXnn4#-rn@@lDVg>~y}{U`RGujA!Wtt;Ng*wHK!8 zB79e%Nz*#?1l<{UrjFA36g7QAarHU812hxwHRQwZoH|`V)9FI0r;G60={jnp^|Xqb z@U?+jX(M&hCNk+_8lg+*MruY)uLWhNR>)>EeTUlVN3?~$j@=AL9`roPIZP)Yt8$SQ zo(h?xL7*Z?&lj}Pv>{C!(Gz~ur)d+KReJ5kX}YAg8Lu?8$VeTvI%93QW9`n^mh4!X zIs`mTTX$v2Tu4cjhL8USG#)M?N~{hL_h1Hvy*3cX77 z(rB4RK~R}ez8FR;G4=|M6bV$N=`ycdou*FooA5wWwxOvLF#xl`3F=DIb|v=9)6|X8 zm4NO++mxnWG*#oYL)2<#XC1XKA1D0sacqn}UPq!0?m>Igbj9ScE1faD8EdXW@b=C1 zjZ+`P{LYr%)r>vO^%eDVp&I+JxL!A#kAL`cVU&+VA2#7+6Xz=S!ppBB0Lr;>tVw_!Hce2HUZ#VJiH&NnDbVC`$Q#UY*|{65pP(1Pz^xAKsmYyWOjY)>uxssrFlzfPe zx(q5$Q&76Tm*54;g{3`X)So8HV_>0pio%|v%*JVeLBL>E<<6mMpm+qFycQO99j&12 zVOuv+C*1_z-V8q8ijmuL)b6Our+gA$&ye`EOJcuA!bz3Q)si6qpCS#yB&By_6w1=! zn9d!b<4$PAUC@ZTX(eR01qJ%;@b^I!-m|9DUzCP7GBjMP42qP!moS(d#B*&YP5Z|Q zcZfVD&r{a-Adq_q$UO|?9szQX!eKp@gVe^NNWJMH#m_p77H&kq^qga~aCAUu$+aAy^%Wd$s0DI2P?;WFL&gYf* z%<9Qb{HYEhhS$*J5XBP^#c_C!C*e7s!r$sVO+#>)3E25v5X=J*$Zhx;(hm3CVnxbNS%M&BwkGN=}V_A$d&R8*~v^)O)WK1Ih zE5Yy9&}%f8UZ=(M27>6DbSX&RL2tt;z60m^F5N=k%@T`b%n6_Q&fskj>x9z#;VR3p zujo3S$6B7>`!BD-?s^&K;M50U!^ zkm7rE26(xMevJ746NQp^XdLCDvsEkd#w?unq_fP?Uh z5;E&2=%^rFH9nkKW-SRwOBrXuxC#V0E=F!g-Hs`CTQtx7e wo#^}Y``BH2?|}k=u7^(D3+jZjAJHEuX8joBW%Lui9IfP#H{j2Ww$t4I1A-+jr~m)} literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/HeartbeatTest.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/HeartbeatTest.class new file mode 100644 index 0000000000000000000000000000000000000000..6a2605d81fc017de51393ed34c4dfae3382a61ad GIT binary patch literal 1929 zcmbVMZBrX%6g^LxWJ%b*45YNRTCLbNq_k@bkyl$N1uIgjkfP9PU6RMLB-zD%arzhh z;P_epg);)3I^$=j|E7OHJvUiGfP7$@nLK;%eeT|K?!9OB^}nzF1~80gS@dH(MVmi0 zn8AmbFff@#3ghB@YT!=lzxbC$7w$@QO75q{y_cMt0@8Qi{3gHOB|anZ`&rClF0PBT zc%SWg6-js?q0je1|LifiC3o|3d|_b0z(WNibn4oRLIpLikQ7fLzu@`LEqlu@Z)mG( z|4_B9vRAKrt`&t&EwCPXxF* z5B6SUmGq?JI^i9K6ZwG^(wOloq;Yz|arI(UU)O%gUav9Iv*4BO+KTNv;>W>cxZwl} z6K{uJ(m_ZDy&<2=8(vhaw1_EOSc+U3$%+#=1e|tVFSJ94%_y8{;B7`EZpB6V_KPff zVOyPZav>8dd5ii*`^|;MD>?{RR1&y~*%w=muPa25yCogyaJiYFkX`m7zpUq+#xz<8 zB_`h7w12Wq6mZ?dN4TPJ^>|A(aTV7Tibu1Lu{*(AG?B-EiAVU-#8+4{u`EXkj}5Gt z_!{4sSe0W9PfR?O{BQBt#CKRy7<(&N>6i17j;)&QKDQpNZ|X8bsr>xBD9EfkwVD%X z>aN0FQC3v<$$(mH&5Sb3H*!-!P3tYnEjYY>NmwI}aBMDmIta8+oUGk2N`6FDCi828 z+1-rIQQQpnif)dqTPl)?*-V}NF=Dd9*`Tukdi}QL$Lhdkej@t$Jvo7d?3q?VHrRM_ z4}7MfL{>1SiDV9$U3725{G(Co2G1TO zd0yoC9=dS}eRv-?88Mp$Vc}zX3bzQBXn_4&`!Vw9 mv5+qx$RdxnMKq@{MIO`SagQpP!3gfRaT)E%Wt4p9j4|JsQP)7kp9rIYwu&C48 zBMnO$OonvOvHkoq!+OK@+9vOEt0T+rOe^u#vx%7j|8r5#il-V{b;6sBOa7YZ@Qspi49w2_w<>{T2fz8A-t8A zhUXey7+A$ihFst3u^$GmV2TQUqP0c;*y1t0Zw8nkXFk}@b3{NcO%0YqoTkMV=IA=k(yh&MIdzoOOVZ!KMKD~VUIhty z0DXSpE7<%uB)-rnfpL1LX%BE6IeJeY@4^Hw(}?y*G)6n qs}#X6q!)Y8&X3@jq{k7YV%BX^NHOo=F10D#LmrcgM)VA|^!p2-5&Ied literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/RequestFutureTest$2.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/RequestFutureTest$2.class new file mode 100644 index 0000000000000000000000000000000000000000..33dda576b8016d009d4f3371582a02ca7b1ddbde GIT binary patch literal 1784 zcmb`HTT|0O6vzMDmWIYC1qJVlTD9dCM7&@T#g5E$rV3-3KD!M|8VH-33p0K!A4JCo z$7es3Chu~qBg}oiyU$I_b%YE9)AD2x9SGlaWGH;e-N5`HK1Cu3 zH=;1|#WwY-gj#i^6Rt75DG$l2v>0+VZ$~hUH5@74M+a@;Z}YZGGZPKZ;%yOyT$&s#ha^oaElksOT%cQ<6^rU9sh6d{f%9OvK)p3& z=mGSZ*{@(T-;nu2qYTFAou@sZYrRPCsopM(<06e{e@tVvgN~cl6ze0kF@0|K2-)w* z9brV#vN27M3G?K`LQI@aJn0h+vT-@KXy6L2((Y^Yt?_T>I?ddmnG9|g*U9QFT2WS) siKDDi1iz461_X z93Om1QPy(#?2mH!?VLbT@F_TJW%kUTnf>i=@0oq__dkF931AXGWN-x&2JU7M6FHed z0#h=%m&SdO52TyU;30}Kno07Uw4}@)8JNwW7msEAV*_&;^kK@tlMJ$$kR2uIo=P__ zdln3QV&GE+bJ2CRS8^=h*SPZ+O}5cE!(bM4iC1pw{C4Y)by2W zRm-VZo-Om1G7;?AwDGuH^L7hX-Ky^C!V7EXg;l6Jwsr%*P_4Osc%Z$4?FQO&Eypjc z>X)JRgM}~%J-yCRQDGpUEY%L`HD9lV)vEUWlI0UNwbR_DQ0yLq_S4nK_BNs3B=S@J zn9NG|2s7n<>y=e-EO)oC7I?P1Tbw28^Z&=16{Fwn|YXX-CAG; zHlwU?p$WGiGM|P2i0gu7JB)Y~SH;*~?eMx~cK*kzu8TQ(pINJgUR5vHlKz1+S-T^Y zOS9rZ0+l6pztKu z{mpCM94Q4x>&+Bf>>-H)Z=zYc4mikZm$5 z64mG}%(2pWj7mmIo)@fpA$KKmn{ukhmujQ8H7i5%mF37%HkUsO!5YF6R*7~HVv&_J zvN!)SWLe|!+|rVmOnR0rrR#d9qo=Pz2ah8yftO=XtH05jTvn|Plyg<&3!H4p#F&s3bc z;t^sSuMsbg9wD(oOUVXO&qn)<2GX}1=s84&;w_3SOpf0E3%w0o_~j561v1`*9D>2| zC8RNkerm1|P$2UJe25R4uH1;=TCSvVm--LsQa}4!{Kb%?jRDGNEq}5$Lft6WgxGCL zg|QPFzmrCSy6FxYeL~|Mp>c#Px%DrPjx{jQX7euDjB$GD5QA+JHwg^7oW=;98fTy< z=H!4;td5wK}FFDw%efAKY&Mn_P_D*{pPYWyWL{?X!7LSbIzReUEc5gzB8Lo z|NWQ012~Gc1P&t`!(<$%F_l0Q9*rY~$6}an$4mlkI3x4NMLZ#opG@FucuGcE0<*}; zC@(@6F(+&D2`s=!@Wi65Ep0#n%Q78%g8N*8oT5U&jbOk3eR%t$^6}8-go?6o8 zmb6r^VCbe}r*f8QmzVWY$}krE zO$yvgsJh`jlZCG@Uk0iy<=xTJCf$ zFP0fX%2%_os%ApBbw|NwM*~*GrZc!e`+CjuVXHJ|nQ7ZL=1o0c&8~tY;emSpZh9uE*|uJC(uG1` zo(K=Dn^BP%GZqmNIyE^(Qp}p0b2HIdzePbJYn4kmeb^9=?T9AH1xRw*FW9TN8}}%< z?|-V27*uCk%dMqaXG)m#i#5Th18(M zV$b=#DsHoH6m-Wj72Fv$>!CPs(DTGSnAN2tNj?8fSb5s8hn9;D>BlQi zOd2*pb8+TT&;Bq$Ne2kC9lP%Fq`ivEnI;(`uyH`Dv-eMb1dnQN7^y)}2WgD5gJ?Gv{)eFeezRdR*$x;1&aksVTUA%22UEXfc z#kaD1gE$NcCKU((_j|O@EB$YiPNOR?ga{ zlQGWPuOjZxTHIOUBOYnOmnoBIq1;WmhvP2Fy*SPl)l-WB+{+oS9GYx$P5z5}#pG(w z6&ksMcm*5eV-2dTRM4@8jlr`-1)Cz{&R|@@<~1aHIE^RPa9ahphflW1kZL}`Rt2)F zYUgun*cQ&HVOtnTKam0UQy#z_=;rT1;?;wb=*1ZNFpWcemF>q05q^mnhe59TLEg{v zZ4B*%8}I``z)IW?_;P4k0(N7+9osADqzo1+*jd3HYuHt@w%c0^H+?=FN*Lj64+B=h z=}ux9neL82OWf<>tO7+X5oqp42SYkaXpUh!$#aOIrJ2hN88Aa276`yO=J{oY{stc4 zRh;I%dNKmh<%STYs}Sw1{@KrKt zgjG1o(LE!FfM3{TXuM+(ZXNj^x15$#1gBYKn+c#IU7W{u3SyPRRNA7@NYaOKH}=Q9mG z9}GNK?qj-}sPCW2SVk=VpEF+DF=!y#BW{*)mV@6OYO@;n*Bh77XD~7Z>m)iE+$@`L zj(sqXU2L;`m}BM46RZWoXz} zK2vi9J5Tc~)O>M$n)%Iz$jJq4ktNOi+}9^g_g2u)79EVGdk14_-Sud_MvJdw8+*uI fxa3;i<0adw&}jvwKy2^0ST%4uAH literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/producer/KafkaProducerTest.class b/out/test/clients/org/apache/kafka/clients/producer/KafkaProducerTest.class new file mode 100644 index 0000000000000000000000000000000000000000..f999114a959ffc7331d7b927aba4a4736172777e GIT binary patch literal 2598 zcmb7_ZBr9h6vzJ?5;ue(5S3PoHa=8?(X0>nz=J46Xu$vq5v#P-WpfD&o87p(q0$*U zedE+`(%1R|oHEl7&=1x2e>WS5hU!R$z4x9y5BL1exp)8m=g+?Y48Tj^6mF}ylR!j{ zV{wefk(aBxaZJQ98OOIOzDr;V)9vWMy#xxFk+WG9b1Lp9(1Y6vbmM`F`2-eFjN@Sf zhp{LZkK*`Vj*sJb(vB0jCm*z2N^lI!eo*nFik}o57%^?rA63wn>0eY3%{wKbpsQfo zVzyRU7H&~rwkSyz97DGjb=Q>ZP#E=BO;5qKg5#Doy{a3lLR;5Y)^*LWOkw+;R&||H z%@D43Pm1q{tD^AyVFjmsj(NxSe79!!jytKFR?QW8%kjj#@SK`!FtWn`%svsEO2x73 zekWcSqUxKDO{X!2ta4Mmg*AOc*J{3LX^dMHu5SvD3Q5N*O)=-ZQ?q@#pHk3CQ4lqh zbuRe2u|A_$gFFLNcgR`6(RtdK6|rb~CNYfJw&UwEFavf6dt9s0wH79LNG7--$jmZ- z5pEDdLA(J)!R3AO`Y1W>zxVx0Y*ri9xbeBpMwZ5x#z*Ngu@JDHG-aiGTS}FaB$GIe zGYSsxM^(Y4Pj^70_$>H}Sys?q?~$BYqSp^PAt05+Q#@19v+OuNTSTwsJmGE#m#j~D zPDSKQdxKz0xmCyWZJmM@$H^`0?$99rSdfbFUDL?9qUyMQVAtQWO&M65aaip@*_#iQ zRWRda7)g{MRIDUX#%dBKNJwk(^6_?8FxYBQZ(n8eQ@KdR5;LX+y7- zSN*hZr$ysnaZyUU!mqiuC?!$Fa|H)2c_m2rA@DOTo7|<_oRBN~=xU4Z#~Bt`Czdqm zK1;XD+T8M*FevKqgdd*oO$9eI^5-z2L0;UVbS^PRVJk6osgg+I%_ zVt;0uO+vpsH&e4y#i#kXhqFZm*`^DJ$c>uo@(R>+->H~}HWmz1wlB)UWwidibMy&i zUd`M`a6$8`>c|)77A6`oUxw*^c+2f`6YSS-dsP|!d<*%x=$2KiUwLC5dzsrz5*;TK z&uh9xhOx}_wA4&6hZ*5{dYL%SHCHJtKx;{Y@6ig|SizBIgAl)b9W&&&0w?)9#V;$+ z27Xp>hR<)f(l{y{W9PS_yx}B*vwR*1gnc;2d9tp64EniJaGu5yX{=A(qH_xeZ@onqhtVB$KhEx;hlZ(e*v6ru=pVQ+ z6zz*XdsSC|^$tgl2PLlV! zxrfOf;Y$4>O^IZVg7M! z&?3i}7x`YJrwDyc)AACdU!mp^ME<4wsEVs91{CsDaV>-Ed?m5o;K)+)$8BP{(*3ot$4M#v85K&7a#C?7us*T(3WtR`WwU$F%L7I SY`3~(gp&8E-QxQQM*jmJUAH^{ literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/producer/MockProducerTest.class b/out/test/clients/org/apache/kafka/clients/producer/MockProducerTest.class new file mode 100644 index 0000000000000000000000000000000000000000..e5e7714b3e1b282072d7f110191595f87a4d8419 GIT binary patch literal 5963 zcmbtY3wRt=75;DbF_X2US4-784nD5nSOm*(NWmu}sP{bl zq>6i_^?Ox}sTfx=q2fLj_p6wchCij^0TmA___T_LB3OsdsQ9d$_?(J|B)WCz&N7~uxb{%Ifr(*I@DH&bw}&h98F-6hgQ}x(#e#WNo6gIJ4$ZOI$2An z=z>x}Mn`g>tewTe#b`A3Z0y;%ffg^Eg<`44dDZPRIomV~yB-1^{_Y#4JCW9dDSgB- z%nU;u-;vEY#<0H2unoq)Ig>FR&7Yg=+TWXH9wq(I&XJ+rrdf_5Wpqp6#F1RDEi+)Q z6j)lpD|2|*%p~)AQj(68s-7pHq|#a2kyaN^Yox^!q57^0GBagoQ+b(`k4S)a&%*GO zc$FI}ACacr(&*4KSuO1s2VrE5>J~S>%+r~gnXL!)6vxa=K5jImc;z5L^IEHDWw@Vx z{<1At+UFGc(3Vc?gIc=T8q5we-HT08VYREaM&N`lM@#MN&_-M$aEXSrd!vG{k*G*W z3LfW*hTUpeCOuBnR(9Dhre%8s<`+3pylQyBEjZek$wX7#hDI`;Ky{Zfn9*d33YeAXbqxcrSO%S^DOn<^2GPCLaM4z58 zhKKcjLv!@>Xd>nFF2feZckrZw??&-Ge4ph$Ma#~nQo3yqWYbZ+4A(^Q11S}|bYQ^N zog(Wiqxd0yG^>r2GENjf#!sU7DSjq#{<&=ZLZJ3CJ!gDK+pi~_AwA)xal*(X7`Ft^ zAIG$az%QeC3crft*Z7T8=JtP!c7Zcm(z<5ljOdnT>!o_Xi{khAgTRt<=qs?1MD~vY zXHF;VvyQ+#_o#JCwAr$>(RTVNmH(vR=_vk;zsSyC1vZphn#xT^hE$i)(spu_KA>gO zrM$Yf+<@n|?I)MZ(*Aj0nsCnuoR;U>97IeJo}UDn?*2^z6J&ysF2ZhRu_5~;4zJI~ zqM}sEEOioYpmLeTbNJNU>F!2vz_4tGF}y0vB8cMec#=-+V3(tXBrT*HvF_sG%oO%@ z5;hOyG{c`;Lg1{zwPrf{qT79GZAgOt4-#mJOLnRK1BNq{&@u^9t;xFlC;mkbTMv#9 z5xw6VO*p13C1=Po4`ibFx3sym5b&bgLOCZ#@gF=ZaC)bi7&a}L&Mp+Tv{~5FVhITm zvU8yDxRp!hB>GoG@n4zWi>3;e2<7t7z4GN*m3O1^q?wv^Q;ASuZUN>?`}PtJ;!<(b z9;=rqc-KvVq1rta4$X=Qo9D*H0OogtLq8q0S?xK#p?#)T3BI)ty|?1 zrRMVQ5KD^{W`uEmAz@lfUNutw74ig|ZzFsKFf)=Ud1zL!Sd62TlD8Y5{&^JTHtJHY zKOIU*^5+Gzu&QT1X!Fo@ci$#p6|d#*sjWhoy)5nI`%z=h|26IAQ*E1kcr{fxCWwAm zvppx5)b>=!#iCn~zsscWziXv{s-MVHN9C#$m9p5|UhdjlsZ7iAJa?nlt^c7$r-g9J z!ByY^>3VhkHG*lIW5Z7UAaiDZg*dt@E|YsQXBNx&uQYjHbgzTK0nJEz?9rlSWhE`l zrIuSmJ01_i;Xce^t21n^oAu-Y`GbT#fT_E!N^@T;OiE&fAi|-Gw-aF6`vC8_Ctzjb2=a-CU!a z))RbsQhV}JJ$b2K&R$NbAm=vYrEG=Sx`I!EEAv3^6G5*Z z^!go>IH!9OOS%K&I5+eFCj<3e-NEHu6IeQq+D;jV1_ojo15r1LdOnu>k8w1t=S(gj zu^NVAeJCD`hZv3Z;n<2uJY0Z2DRGPkJ#gi-61sIv6`&Dw(L@w3!o|e)3jU7p+s`PP zaT7N3t5pl`!X_8lE#4k|Wasirkw#4U9);_XVylX}I{Nt^pq*A1_aLpUad8jPX1$Ah zIFGw0H9&MdsX^wsC)G=wr^9_HhkMmil&WSZ=1JfJGbHdznZW1y37ki9l^k8|jdr?e zy*?OQ6A$`{J+Hn&VjJR6C^r;4pQ*Z5CjA9<@t~je7Z&Cxv93F|ejJSqNmG41Jcf%X zEazDjm&Gz^ph2o^ zinaO@x925pM_%F>19K;i;3$I`bc?AEr!d@GnCxvVhOG?jCH!q?NpukEP8bZb#o)f2 zJaGeA;#LOmPKNF$oWBfO; zS0;>@moLR<^QHI{_PM3lg2$0&D?m>kVL9)nua{#C8MZ?7;5wL0jWE~Rf~z>DaK&bN zxtj$ii&v)PLP)z_@k+b0cs;v4dAq%NyS=pQS?kSP>!mfD)^0=id2Ult;3&u{*zf)w zP{6-~5DG?Xo+krUSDbgH4oIo1pQo)Eg{mkduAH^xG?h>#U!1vFc_LiHa0mE|)_a*2 zIMNY3!ZC1b)nqO&1!8=H^{FsO)fi-uhTM=z4wT1+$>7VwBkG4twyq`E!UfwqNaVXD0A1TH3UNx@|WR~X`BrfvG;3~jmmB180+Qx*&d zN~SI5Ya1)VeZW^N@}x?R!L3E^nsQxtM*WKEFW0%SP+KemLCYuA7$wFX`XysH@R+cdsV;hyQaN5+Txk9eX%Os(Q&$0dZA|f=7v}_J(IMUux-cZ@**Tf zXOMTTMw;q%l9TtHnrnzDQ_9%;w+`oJUKL$PF%0fiNa`4=p@OR_ilE}!h(ZZArr?^2 zaa>n%0~6A^NzEo>3qvYy;kJrNOfkeLzA!{{AP>XHUMrBdqcXwq?806~Oan;#l zPy$KsK4R#}&E$9Cm!XtvasOM0mY^_Ut6FuBd{CY8KA|2Ho+n%?Oyz+Q_B}&?Yr_9x zB`VDqLa%j(NXyO6$|Uii(@FmBU9j@Y^c2FBQ}ocN38J~>{31@_0DV2=F9Ko~*kxh{kx}JdAF#)i3}(=%p3IVLBXM_YG;lO6Rc ziZLx6`)fBQ5vayuTuaAWcK0*Jq+3bTpXy30vZipP$|OB)ko3$F1;A0Vk0FiYIE$0? zAWq>rvM8Ys4{-(+^y3L~ct)S<7trt$1-zqA@+S=93x@Cw!}yL10mTa;jdh9-;VYqL zi0UF<;Rsm~LVQm7GL-)b)+lQRQ9>`2nT8^Iq=h4b!MyWP6Dv~6#8U~|`=|foObxT!L4n^s3Ga8pe?rbjba*T(# z#9%J_k#fXSSA%Xlnr2vcmJT;39nTM3xxOI64ZmdIzJ&*PXyFptEi|FoLJKZfXw|be zT(r=Ew1o*wT6lyh1CK3C;|W9W0acDeXki9vV#?{1J&H3k%FuEUy11bPFENaqHx6}% z2cM-= zW!G8cDs-!tbNciOoeZB)mwZXgd3GGoW3!%EQC)!MvZAChQ;Ui_0%`gLHqy5;)zYHL z29PGc#hNG=YhqojiIMbXf<^}}gQ1gVuTXo~j_3>oh7z z<$j=cCwA>3Sxc_KOI#t@9*65gH+pGBVKnV&pW!M&+fVTYsrapd+;=4QV34Yz!S+x` zDoG=?i~4VL*6Dq)A_n6BT$4vV0-X}c2+3QYxLvoKMZ|HlI>dTJ5WV) ioxBVr-W$jyDO5i#6Bxh^(neZ|J?z{D>|M=^--vOM$4`n!pu>_is zR*;F~S^}jQk8)1_j?3AJwO&;AURN-gfGW+W64;CDaomvY8wtFLx5{u6n(R);q05o1 z0z;b5$j+>snM=ULyzDM0$R$vR^x6g|6Wm}?ftA1!*a_ruOM#=n6;KB(%W!&gn(G?u z?)B`vsgG)oXL_b>2{fl{XEv$jHGR%VE@(3gT2i+c7i}w<*<#r(keJq7V`OH=H9P?^ zC{X4YxwU@moN1Zfd4baAmT`edubnjn_N7eA81@&Z4JV^b=QvW4vUM#tt~sXc7X~BV zoJnvei(tu_hUK}*yklp5-Efj)hHg9Av|(j4hU*bkxyLqRE*iE^Sc+rZ@;M-IX&25+ z%QGBH%el$X(EnJtddzVBoOB+W@jc%m?5FeAAFK+Y4jG=7)jW;f_M|;cUl`KzLD)ft zCaDHyn&kE^=|*1W#8p7}wPU{JF{*LXHAz&DW!atu-7~TREvb3!wwClgGsm=Ay6-roGugLf=p3`H^-YORl`byjY<qOSsq_O|rP?7gp`sZrDvqN=plNppk7r3H zMlSMIw4zN#JK6*;KDvB+G&{@tDP0uPv;=iqpr)Xb2GebzOdD80@;QS7RB;DO3ht_S z8}BH1SH*jHU&RObP@pb7XZyKq!<=^8XkZ%cJ640Sb_P0Be1wk$jun+};k2)-PsJzr zRK;ibT>Ab(!IvuT;VTti;~Rm~j}jH;Fc6!Y?6n=o&wF9EHu>zwt1L;6F5b@a)-*Ss zkfeO8;5!xH%QSaBtp}d(AnM?tjZDw;C{s)uL+cO)s+%{-KJ7U?xZMM?{Pyog>^sY& zdr&%xneJuVqMBLOr-)1j6BeExUMLQcmCKvO@%mXsX;uu;lyp@0*i2qy8DqH_8c3xE z(tW)n!6*OpBMA^n5 zWHsh9$=(E_d;<=*uAroK6{V9Ek-ZV7~h*v zix+T$BWmafC-EYCf?tdsgdsQilbgsjQ>#!XSFk6wioKIR|}DK;54mHvG+1A;0*e47L(`-B52@RqAWxZ#~5v2qUS2~(fSOjJj?lN zl>ANeQUzTKx)q#*Q1A+Ho^ugeU%*AaNt%1u@yTYB3xS&WH&|+aR!otvOkU?3>RP}*~*nsQXw|BmJvwtZwNk$kwEYz7!3ek<#7NK Kv^&Ct?lM5TZ(OcyYNSsQZ& z*l-J&$AXO)HeNDxp7Nj+aP2i1CaWL$DR&d?2d)=NFHuTJ?QVvN)LY@1teUV@h7OYx zg<-(3Tn*K+%OmbJgnPn|PPpp@K4ryjq{4dQ3FZ1y3nh6FyBo>Tkx(BflTZo<mPlrO)_+dbv-fHOa;D9UN*w;?Wtc=0gky5D5fXA^Qb3J2MW`Csx+F)KK zl4RL5#HjXotKERhV|T}#uu38=>f7g@h_oL{O6lI$+&kIjQQ8^Nj}Co5Oq3_y`KJB8 z8$OsabB291=C<^LF!tr~uF#FJZe!8GD=azaMV|u)-45=d$3YSI9rR<&#%l-57-M*p z^7q!39K12nD-Kq%MwH4%-PT4JiZ{h-4JCM;VI?~v&CTNr>1_!f1azvJ_M@|M^%g4K ziMSR2!{)MeYjiU(ivVzCs6?d3vlXbY7;AHZ^Y} z4^w1>$^f6q*5p+*R7_(eJKF;BK|^Yj-n(~Z?zv~~oSC1$zI_KUgGVvMkx^z`nXEDsadcoZ zj@y`u!$3~MojBStt>LbQ8HV;8T^9Cyp$HEvdgvCTKPxnbItkiKWsT&EJ4 z!Zj@E3s-X6GZur*P2oPH%zOjIvtA1ZR%A#hP}A9|*}@kUhOyF1zRQikw`{|7q#3v_ z!5T|WApIxKo-A?StP)I{U$wk+j-k*bm9p@Bg;TJkzN!rl|RrBPk`Z=5up~t)sRfXoXTyF|P^1Mj7>IzOG?*3!Q*{dJaeDigvjojOp=40EY z!}h6XFPfEWzBqm%%`wb2$Cs`+Kcm^7IV~qG$5`C=Mg6Qv7^7j6{(aFIV&>`1Cq>j; zdQ?I^m0fNM4AZOCLGM{KnN)fJ$de>OPpSw#iG^0Q(7y*)z|c!ys}#4AAIwf1gJma= z(UMIaBa$769wVCliq^w0r;ltKjtQbhSmiZstKgj56j3$2OJTrPV= zs7HvLpzS&N=n1sX7%EMiAa)ieRvMp^KMunzoocjGwvI8fy;Mi}2o#f>h>?9ua|cxQ z1I@mp@%Oam03#HSkvH)1jA6Oy N+X6L~E-88X{RYTVt{MOU literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest$StressTestThread.class b/out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest$StressTestThread.class new file mode 100644 index 0000000000000000000000000000000000000000..34897e9de2c1220da7163540dd995d91a8cdc65e GIT binary patch literal 1690 zcmbVMT~ixX7=BK&WMSFzX|Wo#tyB#`x>^-$3n+nB(JG;>U$UmEg3tGv4}-oat1bvx$S+I@2LDdG~z0pU?Ze=l4f<9{_j@*DZ|TtO>_L z2Cw&F7;k8CPK!7DsPMLh8N6fRJl@sfg5F-#t644PwAs9g_bd$JeG?@UivpIHs7NMW z7{mgiEMPRl&=)vg38R~iY)W@aIoon`TRN`qsUV4+W)wDBu8JHlplu-i*eSI(H&t|v zmKO!gxaGPkroXAmr}8uDv=YyE+%RxkQA7`pOu`+{br#Z3iAG8Ww9CzUftSn+%odIs zyga$i3KqkL5*Vy_fm&|uY^bO%H+-IqR6oW56zI$jSTOLy#swEL~uB$kyZ$(Nr z1nhDUsA$obajd9OJz`SY<(dC_XSvy^6;Kw^lwtj+DZ@Kw0{wxym2_#HHu#3X%wu7mYMMwwJ&u~LwUi;G3um-H2@~m8 z)lL}QP9t35)aj_@AR+$?Wv$dJeV=?!*!B}V`5#7E4fRjGCu(V&vazD;!DhsvPf1pf zq)XQqj#ik)7pM7fjqqv9Qj$nqjc6|MJFMBoZw7*7d!91Sod<)uK)!eYQM`xD_3Qz% z`!I^*`{*f7?IX9B8jVwqp@;HYntX#Hd`HLM)6oxjfv2ZBZtx;g(7${dR-`MPhz|P_ zCc0rCQQP3%T6Q*H{24u8W9|;jHhRCp0Qb2z`sQ{q!n4lK`m&AwIlUfl!(P^l={Ami zA;j*Vzj0?6Z4B(9w=&hn&>n+;ld{4`ISZZmIoOmx!onU=`H4xkF^vPh<-cH_=S#Rt z{rj}I$GQ5I6ZIf1Y92WjlEs;{fN5gbSli6&iw%N8ROG&fPD@U89r7b0CYtw_0l9Wn9OR56X-AQ(u%}kh` zEe#4PQWX>r?)%m%B7)GQrHCjB3W|b?B8L~?je<9#_5WselWnr0`L#V}zWL_O`~L6! z-+SNN?}`6C_%MLQN+664xJ<<-!wBHAad>YvpoX2-k;k18xlCCfpptEi!p)2)D`oc6oD$T<(;|&xY{15O#;~d71ix zT9}lVcY8cD#a0&3}CKX=`qe)DEB#agK`UHFfkILnn6YwoOCYQ(M#Y1xW zb_m}o=lzK=p2T+r$@f$|rQ-VvCbzqek;!xz8Mhze6Iyv zxJwRxyL_~|(K9QlqinfpneIvjB{j91Y5W>HNo|u_P0QGr?cHWL9s0I3XCke3LQikj z9aEljlR-CS64~-1R;NwFax?Kh$4+Jwh7&g}*KjPIj$57W>N1>7ww)Fd#7dXB0ldmW68s@Jxd41bGFPDTh{27+ni3*VXo0; zJBDvfM@qN07^mnvXiUi1*=HnNgV8s;fR~nZ+UVBP%}#f=m+l*aLR?cs@kZO)MzuYR zjy8{{iS4eQ=wUQ?gm^K24Rem>iTGM0t@j(r92AINw7?^yZZ3@r zGssVEVxQ>2&6At<bn}Y1qVC zB%aO{Yz^d5WG%G5<8QqeH>c(W{In6J*gJY4|ao<-X%v9FwXS z%cSgVIvF$0&Fblx!y5q^N$*{Qil1osDV|gDGY!w<1t#1zB&c5)mb-Y z+2_SFcCQgLGcntWxhW&oW=THSkg>d^W0ds9dr4w5C)fc9>1aX4$kIq z6gZHToASC9Gu>E9&m61axA>ic#zS-LQ@CztO3w;kLv>ukdDyAp{kVW?=tbEfSeA3f z3rdftl-}28SoBT5;m(hpr<*RV7AN|g687(fmOtQ34S$qWm^B=gV-|9Wu|ILIJ~aBV zw(8jnMaOF@{;c6Ivh0!QS|qw&XMztyZ+0BLU#gITQ;z`U$jT@Sy}{z5;jj3chQH$< zEHfJZiGOMMH{R6nAF<^mzjEf|(BomKkjEGmjDmbYjb*Zl1e-{f^mISjcnkkk@jnd* zsFIC}GOidI%gZdvo=dz=@U(5)*(T(@-(uNI!q9Qw7JoD`*wXy-7 zG*sI1^JRTmx5c9NU{~5?^FE~}*Bp58ybe?L` z821Z7Q|(s9h+l{#wJY;+=W&xyd4`_K{c5AJldfV2^P8aMJxtBZ7Zevt2e;cMgHO$s zH3jv>E!ADS*G$Blz1wQGl!k8c^V;F6Q{a|j!i2+cWoGzE+2Lf_;#C%j?5(w%aV(Dt zHt^gc$deAXa2y4sEt8`R>BYP|n=YPfoWXGbn-Suets$?SoU!;N&pdE^-5`{@{Rngp zqGS((9F}(0J%HdK%Jx95i-h)|oReX`pRfnoeV(?hydI4*UT0CyY#xHl!A#8MXddFY z1MlV=&DW3j;JtiOa2D4tk+qc@dDRzohrM?erqcrr^j@6I z98b?J#C$BmNj&ITkHutd2|c$I+i)UGp7XeTa}gSGDUW%s#&YcD0nR;GiG6q{9>*#? zjb^sHtJ#CD!RuJ-F|e2iU-WX0fgoPP**J$xgxDeJ)Lg<QcLhCp{%|?@}ml4?f3pyuw4sE0Z*0`&wBwSsg4y&PIY&~ME_ zKf#ASH&`s_TI(4r6 zh<72Qd=arq7F06#Ku-NEsN%Da+~({Q1P)L*prTL3xlmL%7-7h;`!ho}Fd02!+k6|u z%aSria|6q1|#vXWW(`{{{;hBDC0%LTpY*v08v)c!}I9f21dm) z!dkmzQmF?6IB~(f6bN`EM5q*GxtuIlQppVT7r?8^!K1nyJQ+!# z2l&NQ$_l(n5_vusmSW1EA8!L>S!Z4SF$)H9(pW$)pxO(m@Dc*K^pHR#3wZ>wy0Qjp|;4nX!!#;c>wG8;*{M7UaSm8%e(?TM zhbUc2!aqc=?i?CDiHW&6dO8<2C9J0x@e3ZL@pt}*J+{Jl0T**rLM+dC6*Wji_j}=0 zMqGCh$43}YA0>9#1O1tgaYk}}2|@=5B*dKOPoW_5Ud1Kch~LD_w`BIxH&HceFlWrP d`8ZVu@ChcRY{(_t6k2d9&05B1`Lo2o{U6WO-YozC literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.class b/out/test/clients/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.class new file mode 100644 index 0000000000000000000000000000000000000000..012a3adffcb06e43cd0b3c861747df19e03439ae GIT binary patch literal 2835 zcmb7GTXz#x6#h;!=_KhCN~9EQk*R{TQ3xPZAf?g^NK3iclwzr(ljO9Sb~;mLCSZMr zAE8gKbRA}?KG13Ozo!?DJU|d$eko+6~C+F^ECEhE{%JbS0`WS z_%e+y+*g~g+VM54B#LR2P)x3c27xtmmI zD4wnxoT+nWab$j1%g|f3O9GjxCF`M;b1Zi;R|tLEU8LJ|Gk^N(tZ7t3+sRGYK}dP3 zU@y8>SoNjAg*U`QJ4SBaal&2j^279_mz-)4DvbJev>L0aP`ho&cI|LjAl^SPE1-=L z1A)C$wkvN|SBlb~v5F2)GE-j3a%L^xR{Q#)7FKLtX1qn5H{Zf`nUZTcL2g_wSXC!% zCe9Et0^lIuI=n8|nv#j25Z0onn%D`534ue?RX4O(wj)h$nZMcch=hm|`nU&wZp`(`{EYbp?9u>LwqGzhPHzLK+UUfZ)QI+$E( z`G&O`fe}^h5?{ICRsE8@YAeusTHp1gstyBZ@DX8WmFrkCunfn5i9Uhh-7EXQ&@}Ka z-V^BE?q##eS#6Gb&x#j>Dv=cfM{&%+`-&V_q#pxBL*Zs1iw^{j?c}cspQ=FR%+*1@ ziU$Th#0mDzPL}m5R5reWQ#h?7FmMrkfTIO#MViFM3@TpLDVuAyBn{N5dZK6!=%^Za zh(`kF@5+XvD<4YVELA)&kfvoi(ke^;gd>B%R0F`YUDH>UK*~}u@D0`kF3(h?S!N&l zc9A{lxn@y@kEC?X@R4UWD`hiu&a*l@Ei0@l#Y^*jbP;-uT z)IYx^quMR=18>Yb4yIf^@yN@V=Qu=z50l@bx#d(vJ+b=>G;39j$NMdy*`C_`Xv?$h zv1$l5ChGvFRHss^4Wf}CkZQ0PRg#p>vs>i)T`ur46Ll(H`wJ?#t2Mi1rEYWwh_e zw*9UQvOlA}FsG3jg}DS-M`0a%vYEZ>=u|5E=HkyuDCpD#=^pjlK!!Grb7f_`CQ39! zk^8NpXLxHJ`Y_7LzDd*IIxcHpCVMwk!F!5?SY0~bl@-!68U~&f1bEsBl42C zFLQ?e#pyXnO{0bvKBfi%#06p*Q&_^Mrx^Jl!|0j4>ROsmwEI^a)G?Gf)bpgZ=*cs@ zy_wM-z7hC_-5j##_=YO#ftu%f)NbZ?5qU-*qO6Vl5;?ywja)<;I{(xGGx4GLC2ja{ zXZc5r)PsvHNeyb5o-XY>q`I`k574!z(BjX~+ogSv)MFu@qU~`^JdTT}ul~#&?s?MN z$TZH(z}jjjt=(VaiNRd^nByR^$rG6o(s7Aw?++j;;1 literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1.class b/out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..b5c768f375baf75819a0fd22f3da937a36117347 GIT binary patch literal 1883 zcmb_dTTc@~6#k|yEUgPIASl!eYUNUp1q2n#y`mujL!f~eo`&r}SC-v%FT@|9uf}+Z z@4oQj3r4_b;-e4#BxBSw1%pW&gDFk+oSD;ezH{bWX1@LW^aa2m?nI%YLz2!IYSE=2 z9m7?0#}Gk}TwRl-HwF!T3i@M+VL-uk1%nLHEpDc`t_Pb8% zbKEILM}5E$TeJgL7k3SLgw~RRN>{xSlukTt>89-&*7}_A3wBIV#--vWZb_426}OR5F@lVOQ59pDWN0jN5*1StKU<3b zA1!*2Jch)PgIvKCJkKy$F)%~3PZ?YEUEz6afw^e-0hfoRMAL+kFxA&bv$1><&D`}@ zwaFtDhDzPsSt^~8{qe_tzx03aMEtKz9l;Jr*wKm-NW;z@M_ANZSsJIqUMxx$gcq1T z!@F{Emus%f%GAo#F{UibO>>oIBGYvY<$qj!snTev7;%THLft?PaxT|t!saW=P#G_1 zqdUKr-hfadA!iUEP6H_=^fjOn45w+8rG1!qJl(wumhRa_D7}yHPEoOmXf-JU&(KV? zxo8~68MM%jjEazSiJ%q9L%{cBAps}CBgy)X#9IufKOpoOkq5-tM^q)NoPAUuY?K35 z!}bte-bT$_&mK-aVQl-?*B!!(qMshI7~#oCEB3b`Iw$)(}R?>SFN(6Ngc^5k?&%KM-P%6kLjYM-|yzCTR#)kRl$TC$Sx8Y1Ke7 NmAL$k(;GQ~mfr(vA-n(p literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1TestCallback.class b/out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1TestCallback.class new file mode 100644 index 0000000000000000000000000000000000000000..b2a57b1b4b52575b1657590ef63a5aa583613d81 GIT binary patch literal 1589 zcmb_cYflqF6g^Wa3+pNss0h9)qP7KDRD2d6r9o4JPY5RZX*wOs!gdyCXNmqS6Gh_( ze}F&AcxQ`%h6Id$=-k=q+;i^Z-1+|F>o)+?sP&>37wvM%g^Qw#63RUocQJv>E+*l- zn8FnYR~=k)FwHQ!!NW0?wCCOlxo(0;y_7-*8?sqfH6DfyF4h>v=U4d#_Y)I@zKE1a zw3f>F+(hj__|??Zm60u}D-7LcC5XqS7;5v8Zuxwjixugw@#Qt{i!dP1*k9LCGZ9kz z0l8_#!`LU{NH?oOB<&<5XuT+7W8r236_|SrZ;Sr{?=LbbEiq(kQByMX%?C<8P1+5q z7kMM3lm7Wg@NkLizAscRvRV@dzM`1dTvuIy%@8ucy&EyYcu zDYCAj4cv=VEo!fa()N_$@j+_W>A(|dc#|7m>69wus^xF1qhhgp+FMr`t{xznKVT_Z z`AcjJ`8`<}+=VF7LOu*^u}AjQ#-tS`^=3|qFp2}!qR3aGri1GqZeYa2QS^E6aKu9n zBMxqQxP{vcXP$3~uMorq>nfXt<%r0ZlQ4Wg>EMos3T7PK^>7dO9X#+*#Vo_Kf0py_ zjbq5AbCs6(Sz}cS!%*IzWJg-n{V(q`U{qW~YOOXRZR$#NBqL|Jp_wJI4(tTDtpRFI zOPVJ#j(LlkGgvI`>fwUX)CrR0uGBhbG|BR9LX%ESFYfZ_Bwi`KCLf#5YC_}gS7}%@ zmBY1Dqs^jDh$>rrZS_|=@M}cs2$ayWO^;4hH6KYrr1_|$Ku@F#hQr9@a#kY%SrQCA zqZ#_;aExp}?P_GZ$o7@TKY^7eJ|R>7jIIyq#Q|Eo2?lHsgS0=MI`<)u6ZAz+DI~)% z?K2pmn}>OQ~rp|CVF1&z#p~jYzQ7Zfk8H$(iGB&^4m0d ioW>a&2A!Ox+-$_>Qp!=BC!56>E?|guHs@Zl$AO$ToIrOn?|KjtI~O<4+p`26<}@_&vw2j z;J&m6XcnI#k7|Q!q@7blzvsDQcpje_;InwX!3zR>HZPQqTAk|(uNoX2l!=Svz>5R4 zgqMurrMyh%`Z2V^UezFP%XMB+M5}pafLHN3@>o5F8+nb+=Nh~=!0WgvK14FFu$*m=-Caz z+cxbdLC@;XLx6Z>eeq2MyjK2RhT>okA;WaJRuR zIZ52$9^X~YG@B$7$)rnUMPdvmd9{3|D(Kj>-cgRdB5~rq4NiH zzLLpov689f;b?SwxNRps8)Gpmu{;`1CaolsFBR{Jv@r$O?g(ERu8xLdoz=~$L?qT( z#}w!ZCsL7ABp#c?1hIKK=6XNofle&mZeiBcwedt}b+{+o)@4=i40r4dSGUExyW_Fy zCQJZLH4&=|Q$3hok#~BF@U_9+9}BU-UM9N}-x)E^lqobTVZnn5O4tX}LWG zr1WkJB5s5DEA!5Frc|%9r(#vW?rI5lV&QmOU&3Yex^NGMi<%>yv2ZG#06S;pjxHLm z+~GY-K*PT%5{smk;II{yn;@j+(3$KNiCInQ?(J5hCA>X~p5WSeTR6HYoQTM`)2XGp zBH(QK2uF%WtXL{p-IIv7r`xPVbtDE^#ljHqMyoBJXs>T;OLwQE;Z!^!Xay%m3hkYdTl~9DL*Ty?n%2bdjjKwd*>NQv$7Wzy}-y3V7sfT-dtXMmasc#Rb zfN{JPdxX6%|L zxhay209y62SUeS0!y;($Ej!YHXte{A@dA^2rIWScL?@tGq(pC9Wk5y&-cB)0BdT83 ziN>T~d5CayMIs!D!LCz1UXO-JwVtEr6Qq6YA+Bmk9fC77B%8pMTFGQbI@%mbrPVor z$W*SoZRtu_;dY2hxkl{lwe4n0tRWCfciYo2tikJ`G0-Pxz^k58BxFoTF;#nb2Zj6^ zQ%+gbRPI?}pEes090uiFA7Yv;=W4Qcr8Z)#-sN3Z+s+kMGz{bPxP*D#$Z1NGbEiBw|@v9Tv=r7X2joXJ#ZHDeNR zj#lJStGzLnjo&jpY_`Q?ZRtb;zOfo^p*zx6U9TGVL`V*R9XLN=u)LT-E+N@O(2}CE37(dPs?3(mh zdeEfL$m3)5UnYH-9)*F3@J;$SeZr*MeKWfswbe~C|mdE{c-zXdBCN3sD zK)YcjCSS+bkFv^f-L^P^(;G~_k#92TAra5Ce0(}7jm|fld<)+yoPNxtJ@h%GX8G0( zNo-^dl=Cos9(hzs+8N2RD3(}>p)_=U(B#kZL$YcQ(}b*Rbu887&+)@1f1d9*`3w9A+=R(r&->@?P#Uxu5qT5SzT82S!nv0}nU5!WquBdNZ=> zmXt;3Crm!TPn!G`e-(DeeM}QKNur!P4|XX-<019#ctWHs8DP@nukq6+f1RIUI(bx$ znEWh%gXxrvk>r&-lF^2As;oV2m8IfkNo49}+vDk2ds#dd?Zv~&>dsYJSNKv)vaA^R zLXK|1`{Ec`Ez~Cx;a&+3pnd!b7~)CG=LinJY4SmS4q+?{3p?VP{4ILaIg)j`&Vh**T(}{tru9uLnQHRN-x1c3g}5BtR?)cTcvr1ib79^kLz1exyc0cg zu$7>s3$WISSp0T*L+78t)Hd3s8FjgK?{JUgmvC;vTPH{ufs3KJn(IISfZ@^~I(d|` z5$sbHl04i(%swX+rSh}wR^`^bcLv<+uq`ZX592_e1PL)L$v;$OP3%JsYgaPrQHV{z zKbcaIO{7*{l15!jT!+UxTCT7d{m?kp&%Hv>u?zzKsxd_6Pz!lwTr9vvgHfU61Q~X+@*=lIh-OAw4)M zq}^V=b5EzDfYSvgD;$yK(#wqFUapc)*!~Fc%Q60S7P8e|8)ZpwJZm^y^UsI&JE9R4 z5S(+xBJpbRxDDx!4%8mtf+J3GPfqU@kQ?VPM~flHW18fS=N{J1QgaDo&1DIRL?O85 zd|RV#BT_&b0m4T)mMsAAqIZRplHe&sU&u7ZhVLqX7bqVoyj`eZlZ;vx_|Ym9maiQN zJ;!18sva={&&NSRIljVhJI-G+60UGIUowUJA?&%`!tW@k1*CAqMel}nyFes@(00V7 z?pPu!-I!ZRlX_La7H3OBerG63;6idq+LHIh?|xe)@o}k~A6Br_cB?ZIa|%^{WrEoM z1Q%mNqc7l>HQucML3jAFaKtWnNxXw2LYxHez{&ZsBx+QmbcF#8)(8n)@@`HtXDm|Bm^VHoo49ER(U}%KsL(}Odx*5NXPNV{i;i)vReP~=FZuV;_&zH3cIm2Xg15_1T|snB%q+xWlX)La>Z20xfbJP6 z^$r*q80)NiPOJ(}>8F$WsGur%vU=gFViR)<@W=aU+O|F_!}RH`zNh45 zLcY0G6JpK#=@kD{;7K}5?9S5WRv*45%h*0TwU5fN_B7%8#C>#nAI-pL1$M6Nqbdj# zYJUvRjWh|**`!f5e$uU>@iYfLb16jgsEW>{Iy#G*Xg-B$0j20{KxZL}Z?*I_j66?^ z=m)fzUZW-S7A@r}TE+{ho*oB9(^eMkr9ONy^+S<`qR4_BG!8ovFBi`GY38y6G^=%j zhFxbrM%9z32HVTAX2>}HpzkxAFpk%<=zOhkzOTerqV1-6v(ZzGKP4{ir?WN(=NCz5 z`Lupou$RsrpoL7=?x)&oe7tAy#S!M$VYa!S7O7Qu7k=!u`)RS;DHDQAi~4Dq%n^`E&s{P^(g92h_C?>fZ;e^8wjQ7&`a4uDeTFF0QMAEz^aLG%!e{Yh zdXk=k(&zJ7`YPsW*!cqh&eP<>-tF{tdWQVi{XBY>zCk*D6DKGIEbUnbG0Pr%4rAZK z*c{RZ!H7?%Z)-X|&+rC^X#OBwLx$U>(+fJisMAYoY#Dt=35e-s`Yzu5VC#Em8OL|f z2!}KdED~*+P(X_WiBoJG7L9zi5X#zFDUL&QF@&&{ z%ISR&=QcVYRwcGlW^>?(P=iW-0CAzB1UnQF7UVt058ZR{%Q%N9Vgd{(*P^D4aFV7G zY+xUM)hnF)LGvtS zL0QDx^ABRaJ?l9@H6L_e$*BBMs2rTotHETR{2>T)=`_fs2P0fl`4&nw{CuB2<}D*USUAw;FC z=_mBzjH){5B;}_3STT)$jFd$z@l{8C<%myWV<^lb2@y9{0P44a!qc$I=MfKWMY_(( z9(y%2CQ1|MbgDR~b;>!3Z&HrvC88sAf=)l>f6a z>|2tfTX9Mf|GlBHRy#l!A*^5QD9PAOQ$oH2w6(P=r0t{k4bZj%+N{*{hJ-3nvN9^A z` zg^HXNljTjV2-jl!ezI_wEQof{iBQya(8KkxuN&YlZ={*XhR;M&vzTr{RJ<83`EFSG zJ%H4w0H=E~`#!oAKW~1T?nktGfSyA}`y0UQPxK&<$1g@D^blg^9-c-I^C|QNuBS)1 z5gDxbi94am1vJ$GUkb2$jebr#))gxI9|ubFm3NRIP(Mh&KvXQkE*C4v1whMQA<5Dj z2d)LYn*Lh_2S2Z%Un#g6h}6HvTOrK^9|C3#|BADe{zpaHtLO!K9ix7nFt%j!QKMvtz-`35IO=ZGf@3JfURQd$%iHCzI2FAK0;$ydxy@^>9;oO zk5Gjjgb&e_LC}`yoL#IT1*aozv50eQFUJdyEwH(M&XXuk86NCZLioBHtegl| z>kD=iNd(F!j=|1p($R$+QY-?YG}4V2sJx-%af);hLqisIup-F7eO+8)vh_ft#^zz%?0X)@j)2R!y70QMu>A3!Gm1X9BR z8S8b-H5+ggb(LE715XH*|uIoh;ViA0FGuVjI0e|4V;x zfVhl|K@iSO{-z$GI-UN=hiK|ynun(B5}n=}1RI%o7^VVra$czYDFd}{0cv7wf0#r) z%O+8xA;M{b3)xJ_ z!`5E9Jaha~|MX)OkUep@fbui(elSryDic}ju*1U-VI4mLhF_sd*M}Oh_ z>971H`Wt>Y|2w})|H$Mfz*@!@e&^VN!LK9bJOG=mq*7=_{*6GM3y28FM6w@TS5;yP>?+b z)j6q4Z6tjK6F4)3w>$U=>Y(5D!85 z|AuQk4A*!B+1z2=jNidcbr815)C-4q1N&$<`{^KJ;TyOMzlHW~9s{+SbQm`Q*9BIh zcp@%P13>&L#uvePR;b7>Ir|Y;?es6#1tK-IU0^$sZ*hDYa*ZwYSE$(Mxf=_OgUI{s zv9}%fSV8`Gs8U^v4^ibibP@cXPJcs$K0;INu>3EYI$93;_Y5>7;`+hkZW|gOX)WAG zA8k^;a5Rk6+NBu*Np%QGsaaS)}piBtnqtVL$F4Q&UXOuKk0UC(873s0xJkf}Yu zfjC%^lea ziw&s!5Uf?eSB(~vNdigXqj<54a#E;}qMV>f71E&Pe@h_^iw^@k!xtAE+v0adT+GL? n7@SF5EdXP;qCExZybkRx_}hv<`S16rJiSRFz6-UVyD9YlQ=M#A literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/clients/producer/internals/SenderTest.class b/out/test/clients/org/apache/kafka/clients/producer/internals/SenderTest.class new file mode 100644 index 0000000000000000000000000000000000000000..80d43e6bedda5d1092ec07a3859b837e3f447998 GIT binary patch literal 7552 zcmbtZ33yc175;BBnKzj{1O~#c42mIyWdK=}02)H530WW$z(6ZLCNIe&lbLZA1eLm5 zm)50fTf1v%ZLO_}WD=#NcCm|gZ|%O=eX+IsrdIl&_hu$TG82h~Z{EA_-gC~q=iL9E z<>u*Eo;VI*rMke2?KrIAgC2ayixNCgjx+IL4IlBsjl=TspoWinQI3b?<74vpxQ2%{ zJW_Vm3Fn^|6Y!`E9P#2Y9F^99hO`%z4%489Aunb);vU!VL>Z?BCV^6q;S*lW#wUg8 zxQ0(@c(PPRJovO1^Y9tz|E#R#b27_Q9(>-5FW`$Fd`Z}zF2^(YvJig7gRjaezvji) z@r`nP6W{Xk@og{ppOt~{c<@~hzE^?o;|D_XLuvg;!;ihF#ZRQ?r?TL~Li=0=eukfG z_=OjB_@x)W!mp+KHy->}B>zr?{@#P-Td&9Qv4Z)5*!Ny$;3aU|ILVYxnNE(sk4kMg06+CtI zTY`J)nwvQsEERJrR5+|1?Trn=QZb~0FDt2|v8ADHd&iy@8Cft%p(GhosBVtNdjdwx z2=$tQKBK$O2!x{j{n1FEBO0?pTa9?qN?OrKt%56Q^>g0h9I2!gP6S$_p}r1MlEWPe zTV^DwaCU)N>>-D|-fokje6NuV^#-lWMZbPCX>=J$L*cA3*fT_A)lGB$Z%w&o{o!Op>2mp6j35)XUB9Um) z5M)w3rzVK^rMS+(MzcE_7w~zL6cDufW3o7Rw7Z*vcvPDB(VcR(==8iT3PmHKR6I_& z1~%Kn6fx~&l@7+TYHb^faos5>>ueNpWh1;+_TAfXAY{gDjWoPSH;|`P#ugQ#QoDfB zq}Iw3JQ@db9TS2{Bh*JE+Qz55w4LTlv=5Io&5$&^c3R2ahIm|%)mS~2614@CaE=gR zmyCv@;eahCQ7fQOH1Wd1;u;Q^A)b(EB&QyAEi;`p6hC?+XQ3-qI5g&xUz6R(rRK!U zk!v>-F#?ZD?=J2QNBGaC##2x>Nr)xHJ)}Z{eOH_lDNN3J;o*t(#z@4B*N2TnLa?x@ zs2*|i(iBmX2yAuSMOMW8Q9?PPCq{oS6l}`ej*=_JEu5kb<-viFEVMZVeOPNgg$Z#)5|(80V7gT&XZ%Hj=cymevO`I zRP+<+|JBh2Q%5&;GKdvt$_2*L>;PoPl;&upM`XOBqX)ejPUt+AN{5Ag3JXTda*RJB z!s+NkSYi5bT%()sGrKM{6206+9sP(XtRF3H93e?lCAunAE-qG;>9`d1MJvW5(b-Fd z)#GOp01(1v9a%0=%q{fXXaD%|DoHTIi+!=GSO51amsm+)tfp5Kc2yTevbZ*r-wD%CViP1n^7 zHB(cwXb9Y+tJ!Lfj(6Z*U7fDZp!w^<;Vg5)pYz>I6=t_Pr#U~H3 z=xV77Fffc*v~8@o9mTr*Mx<+41zjzpIdBK=RPYD$Oq=YDM=y(z*l%WoByG7|q%Ro8 zYF-E*9d&fILYo)R;x9#1LrDi9Bm-yU}X006x^nO=sZoW)m5!pSJ32ROBDX~ zy4s-5XUrcy&ux46nIVF8%5YaMf)WS?wRRF0<}H~xnV)PnN}OI$JX|@B%TC;dvaGtP zN#1iu8Ax1VKn~J&N%y&C2ldHlB$Hh8WEPyisAnHe%9oCsEmMrd?Zh@v&pd{4DZzB2 zgQ1d+xwWmefkrPXyD%f;)7{mLP2=TiYT8wlWrQa!ogqqsvM%Ks;m&hQkJ8eXU4 zY@D^_3K?!-JjE2jRc&v}R4cOCj(nLAP#vlGboUl7Ht2Ups*<_Opf$Rz1P^h9Q56rMqN2T)Nn`2q|;XYYj0@~N0Oh{*%+)%Z$XLzpsvsT`=xkWU-HbbijDpqT@h z#pY}o(Whe05u8z8UdoT@BALHggP1!2KWXL(O$C2G{`|hPN=hd$n8c<{>6<@@vj;#Wxj=J)c6+hq0(4<6iZs!Ta}w~>B(QdfR`N=X)L>82+O5e!$;ZNBUq8f zIfGc4#;OqZdv0Wp82?SkTJR?JDBPU2V96Ea(AagIYh3}D%qC0(dxO|Or9C!MTG%RL>BT~1 z@9fgNW}b!Di|OQ-#`%xq0L^TVDlheClGPj z5Q%dOLFZP&Q&z)MwuzS{ZFEk%ct6rjfLzOqkK5>+?xkxwOy~44ozpSIaGb8`SvsfZ zcuDX)ozu&>OijZ9H4g{XVqC5+#1$%tE7ip~q%8JDah1B9HT1Ovl0a+&+-g2yc?%t) zTR@Qoo4N+K+Rn5_b;<<?wNCIEahrOrUf35s=Vc#G0-n}*w6 z^pG#|2o#2AiImjj72k1`Gsyv8;#;#Qd$ic#$=d0=a0rbLpt5yw8e7t6UXn%&Uqfhp zlnAfDbVh)tj6u&3jB|MvoYKeBFr6yS!cF|c+s$-rx7y0rkajjE0A?#4Y_Qud@dV>fV26xC3&OT*hH=jr7HjklA97bYYVogPFP|0)Pn9Yb4ZrRy&L zAhxb8TQsw52;0`WkDg zRc^UecA%5xR@wU^j;-(o literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/config/AbstractConfigTest$TestConfig.class b/out/test/clients/org/apache/kafka/common/config/AbstractConfigTest$TestConfig.class new file mode 100644 index 0000000000000000000000000000000000000000..833ec122f30ee9a6f2b964506bed16ff69a15bdd GIT binary patch literal 1651 zcmb7FZBNrs6n^dsd#Ocm=nzn>f;xt*A}_vdBFh+NW-zlZ7(bD_(v3pbE^TMx&k_?! zH1VS!{87f!b;y92jeh94=k%QCJm;QsZ-4*!{u96oaxo0!kq8NoBS?x6FdxA}6i;GU zL^=v3hNoDH;+g+xIf502aK5~=ReZ~kD%nm$;VrHm=*khVA8|#qn@!syVbzU>lJAI3 zy`Ev1*)CKo#r$5S@V;EB7AkxBQchJ1>fR|@ymZJvamD0TLs31)uo@IKLJ+RUEpL~b zZJlBIe<+*fJTc1}mf@{4#FGJ7+cQjMo3}Fay9`5ld!HaiONOQIw3{{Esq&ghmUzk5 zxVg(6!`HjUA@9I&8N{mYdO;lwQqi(>CvS4s)#=-Mzxs1E*K@emjrWn}&fuuJ;g(~! z2x#;ITZY7WN}pS&w8qc7W>cwk0*a|d!{T1sp=uHrYVWxC^+&Us+1(!@=^I4VZabR3 zW%!#gahYt|Z-|5uT$eD8RfgoC{r3H5BUod|_6fP1wP%YM#+o{ff|hplmhE`D<3{jY z!b`kjKm-{HuR-lj=Va4xJ=w0yTF@NXv*os{%iNU%1XunCk(TfV8w`tAqDG}8`}O;5 z-p_ZbGk!}~O6Si>e~SMS&Q)ndXvDjsR8*@B%D{X(;WS6feK?-wD>nUxa7&}CVy_~U z${!e32Ty$#Oa#LFdYz6Z!>{fX_g)8sWBS{W33|{0DAx{k&C3i9&_k*Rrh6^T4~&z6 zOxLSKH?4pv{~*z75~COc!wp)85GH%PM;FP)==9tqd5b($iVTsArH;WCP7wM+ihu;Y zeOF)`QS4xXR*Aq7w=qo`h2ReE64(r_C6a;V9<5|rhj5=FA7GYrKhYXd@t>(55I&w@ zC>}mWBpy9REG~Y-wXa})X1ZSJ9b_(0Denkpjw~TOq(Uf~xD6vkUqt$U2zaqEN&gQ4 H*^SA+(l*HP literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/config/AbstractConfigTest.class b/out/test/clients/org/apache/kafka/common/config/AbstractConfigTest.class new file mode 100644 index 0000000000000000000000000000000000000000..7ed61488886e1a2d10368cd8bf67005581b4ac71 GIT binary patch literal 2464 zcmbVOT~`}b6x}z3ObnsWNTIP(Q=|eRnUs%GOaP?_iUwK%LHr()3k-yrIGI%cQJ?z) zE?LWWef9sS`_2py5-Q5exp!{P-DmG}&YZ~~fBpI!fJyAe(2vWd7G4v>;=W6|2nKfnBW7xnK zQEY11((t7~Y(v)E1>4#+_XOh0mL;8qiczb{nm}a6v`lwSplx_$Qy{!xm!&{g(X`}h z{b)xz8^%tBCyAn6GAf&fW2)!iFzoJ|#F!}B&R))_8l`=iJ2Z9=jak6a@O*b5?anmY1!XYgi@4DY;g+T=Ph7nl+Qb<}J&14cD|S{?_fqI;fL$ zE&%FMK$}LzBu2IF(&^4{@xXXtms@uo)7mR2b%H`3T}ReW)Z))2SyeH~w8NLd z5OpsTE!YC>RmZL}a<3n_?wXa{a~?^@HD!$$5g$Wf?z;G`$r@BbeOg57Tz8Gq;fhi9 zis03Fo9)6*VN(d)R@qp#UNq(-wrDLfEB9BGWQsH~~(*Hrgv zs{1w7k0cXXs_9F{pjP)+_TO(;SQCqv5d0g0+X#!UL*gT=!`&**AY!c7nmX1|9dSOD$Q2++3IIx zYjJ#quT`185y-s`JCK-bDoq^U;yVHTb=Um%%=uk*-u&#A=4O9dH!EdT>hV=OCC}cH zTFP)FAFHzD%5rL7Ix>^;dNAd^8Q#zNIKIaZ%COmXP|{G2L#h&vFP_~rQa+Ol=v28b zQ?8vd{hki+@&aSmXld#S{3eG}~5rOH(DdKw<8sfa=ZaLJ-G8ESlZ~jTqIcw8{e6D*zgK`ItfFI6^M@ zU!kMfKkC|Nsp%PR1twy*s)aw5=lNlAn7;6?VOH>fpS6B|_kb{jnlQ8je460OVT1Pc z3B)UILU@-p#uZ4?c@~4*#eF9{#Cv=a7$QK(1Nf1v0vJsvLMLcTC&Hd>_iW?@S~?N+ z>>Hkq(O!fZK$KVe5y5?icn1kGyG!W%gdQZ3H0?$B(IEUV??f1MglkAmN$!=FKhcJE zu5+j8*gipgC6l-{dW^2EQ*>@;PSCxYP25V{K1O2eCGO?Jr|6;EokZ_3`ZCA3dyM2S zclEiz6H;Jc`;j(<~E59sysq)9E9K zFoj1LBkLr88)x?0_)kP7I`lUW!WtfHn9}e>!!(43_xtF|;Tj11{l5!5({}HYwwGwn zhkb$5C^QKiC%p*@oFtto3VcEid3sG#V8IiZzKN{D6|w|n6{tIe8CG4j8^5RQ(EEt} E1F!FVkpKVy literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/config/ConfigDefTest.class b/out/test/clients/org/apache/kafka/common/config/ConfigDefTest.class new file mode 100644 index 0000000000000000000000000000000000000000..270dc7517a2fdaaa0910a5c2c98b7868bea62521 GIT binary patch literal 6953 zcmb_h3w&Eu9sW+5aDu1XNpTkHH`!VWMO94+1i}7#}kKjNMpU0yb z4hAt_hVlg+U)1oJ@2VsAKe_NrL7a^*YxqhKt=KI;zN+JK*?vvO*L8eD#}m>9{rRRG z{g$-#q=u(-d|StNbYz2g8qY{A-_`Lw9pBgStd1XO_@RJ*Sv(ofEf=V2oV7{7yCxO0 z1Zq0tNvkVA)MsUS%)SIijLuZQnb>4z;!<`7y}2#%tiasPRA#WnOq=~%td?!&z&5j` zKQ%OzO0r1~#0Oi}sAjD-&|_tDZ2}W>Y*(Avcz@5OX^R@FV%C7b?27eJhcBH;rMb1C zbE~<-Y{}>1iI%fDWMy)3E6WAm9cF^_HLm#$W||WX<)P36C!EFQOPASV!b}dfM01&V zad&<;r^4{Yd@>gwvNpxDaVBR~GMUPmxp*o`^R+6bt$8N7#fjdK(RU>2 zUOdK}nE6CbK({VQQ=b(Rm{(aE+Arz1($We&sF5nW@`*%I-y}J*A)d|B?Tyy?Y1W15jhb7ucGVn7|Yh8zjN zcu6`9KV}snV+1D4bd(>?MgoAoP8mC6qr$=SCV8G_E|p;teMd;|$3&2{3cE^8$^K9=BMDL`shSnWM1~UX-K43gIuRGZ4aGRcA1Szp2g^PR9l6#w$KF zIX@DKgz$I#g9(V3k)}vrQ>0(RKc$QRA`eTjw=tyQ-y!@5ha~zlk{P};%tW-EV$a5N z+e3H>FB7C8yn_E~cr}E>I6_qdnh+u35mlP-O4%oHT*;&LW_AnVUS`f_8TBq&Fn2ML zTXLi`l^hHSztBR6gB+b)nzDUdosTDCRwg8L5fJdqo5$7CW1sYxE_(-5!SyE1AxaC| zor1>j?Q0-n6)ILFm5dM;k+`fmk%3euLj2{eh*TX6iE74#xfLOxSOj^8P@`R0;xSE3 z2#Fd|8^TrCNzVpTDb8sQL|G}6WukNb|MN0NmvSk3N$=t@$47ytiacW(>|rSpXs&d* zs+ufQr2|xlx1i$vq*;N)RUKVD1uUrQ+L6pzgBC%psZ@9Kx@~CenB^+b&I;Zpm8D+R zd8ah`oL1|zau-`xQeab~OSAoua2F_NWLD*+Mp>Y53oFh>$51+z$(c!c$pzhlcYVjY z^#Y44H(uIJ5Bx+C?@xi3-7yv(2o<}hu1Hssoa?W0r;4ZDR{0)RATHC6@0Fhhv*7LQ z?pi1C&@sqC#VI>>-ut6H8#|=`j~@g7OLg*fpW(5K$lKWwCFA{TyU$(S+0Ht+rgK#^ zO1iJ^?(S?~)y4YAS}SmxJK%!9V$#MjIDz_dkBa&9o9VQbo`iGRmm9bMo3NR!z+1T4BR6|C zLW7q>*B*eUcR#8O?=XD&DEw9YtDVS?QE0=^jlc+kM)fd4jxfQPIE)&j))9|0jA2YF z&eaX0z9=*dV{%cLGK_FhINmKxbqSpZa01;qaYK{gHBK7Aw9QRMwU6G|UBigjow_=l zJc{YI&|u6M!6|lsiZRpf9B-UzccvPRvNNY?1hcxlRMl*FAI5ACMlq)jO{19GY*ZHp zjM`>nVnOgfiW$AkOtjZ)R7ZP##>D6-TAB@`u)ubr*{CZF7&DrUhJrv_^O}t*1(AuR zczi*$XQviKd45&u{C40(_TPlLn2ysi183qCp59Eva4NQ=5m#asc4IbwI%>imG~-UL ze1Kqh2=nk5f0}s`3-Bxf^BflOOTrR@<%9OEheMgh2JTTKT?0S(k~F}hsEwd(a2f?T7td7ZA1v6 z06fM-2kM#F5=9!0Zep-#qKizDCL(SV1xz2nd;)g?K{m!%7p9@xWh`=ZV_S;G&T))Q zcNwcG7*lL52xII;iJl&pz3E4{7cbZobD*o>UYN~7P^&$Pg@k@Bp)Vxz7e#yhHrk`T zS{;^So4scQ%Z$_cF@oh(SiZZSymK2b zpRMt#+48|jJ~ddonDnO0<&$BTK4XQ9W~KDAgu0Um9k4d8B-*be-ft(?_Yv(+5$Vqp z>m``H;L2nH={x0jP^OMmRkZQYg4DIOsjKr;?mD)au48*m0lBhh zaQzOn`w>vtyjs%E9h|(2 zZQ#l~i`L};!CM=Ut&Mk?5YE(A%XUA`qK+Ep((1JRCD=Der;QB`3Ebnh5@VfQ+9mgw zC^3=c9Jq|IxE#B1g)6G7T~QS%;E}&wI23q=CpUq6_ZHJQr@5(+vsQoDE75;&iC(oN zP#>t*??!W2s}C$_4Hn2z9j+!hf;KtA{&2MB8!%zhC#f8EZE6=^<7xXG_>Pta$+Z$ek~dQ9v=2}l&(#_x5%-*MUL&IbZIueDjMDEyL5VGpw+-=WvBwt#OGAM1q&+g65|Ne|xX5AgrGJoqIse=FPH Ma{Os{_%aFn4=N7)hyVZp literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/metrics/FakeMetricsReporter.class b/out/test/clients/org/apache/kafka/common/metrics/FakeMetricsReporter.class new file mode 100644 index 0000000000000000000000000000000000000000..d97d6bba9527984ba6f8a3126062487a5c19be64 GIT binary patch literal 1214 zcmbVL(QeZ)6g_U&hLu9Mj+Me-Fg8RP0wckmS`CRwXcB=vH;R3!CdwOXIZ`JhCB0}o zjzT%sX>7bi9;%T(vg1#B62+;Gy8_Ez6rLD=8Z&>#JylNLW$lA zJqda^75bALiVn>r{RL-te?WYvK?&C=@-ooGGR3;xfEupTjBLjZ+Qr~^^p!Zrz;X__ zo&j5SaRy}QO*AO3&~8i0nN`X(Gom#b-NO1j^7a+vE!-}Ux6nQ(zeA(D*r3SD_p*XB m<@qCTTtR*h_X#SndG6dZqF|H$acwi_8gr&G|7(LsTCWLY^MTLqtL@2a0wBoghB7&D`afUKD{Bs*(3A9N~QU-l1|GS$KmUIH2C#?^3dWF^Fs~pc+Jb@v@?xOLSP*Sd zLP^FW8H)<~@mRtW1%qPAQ#_OLT*eCtWd$noGL~e#l(8(~m4w#}$tBY=y=8`Yc6OT~ zv1;!c48s-EGB!G`y5Vf`dXo^fV(YxQ%^g#WLong(nJz=IVml3ux4FJ&Xb1f30oQc9 z)v_(EWq6LMyV}Rr(WcS19gp}7*DEumJ!*ZpVY;5fJ=3-rq%eSCHj=X%a1&S3H7wV5 z7_yN;HGd_=NI^`AL0LP}jkeg|m9WAvzS*%nvt?|Xu1Qi?EX(%%^hoK5zrlWoEYm_E zO3m&#y0LBwmqsrLE$s6{p29dX3^P$Cg@PDzQ7g~wlENFTG7R~)7aZeThkSKYc!0MI zlUET1TW#V!)m9E064q0AhxcUC z8DXj5k$262lB|vt7O-svK}w*h-sF2!k!PgEf@_Psdq^onw5Vjy`6f<}QWfW;HM!N$ zKGpXPouG6NIx7r@0lvF?TEPs9(S;Ts=4nh>T&|^TrAJ<)n_)G|(RF@SvNM;2UqVP5 zO-C6KaMGnGSqz`AmeVVga^&5r8r=#1{iyEM(|^i$)r?jM~1r z6R@BD^aTBrJ~)Qk)K3Kqa0hp3#Bh&-<9_g8^wIkV{OBUKbAos!*F}OleaA@VROuLU zHQz=5j;g2wCrEWMNRvay7!J@8^_q{Wgr*72siPFKG4F?54?|8T1N6oPw3qQTJ-5IF zWXd^Bxn?Na1TaTy&69>zhR zU>r&*f||?2P!MLX;RhXyUvc|FSY3YJ<(BPxo-Zv=M1f<6)=}R!MHsDq7u;shyi}eZ zvr+k_8tI%no$Y!x>C{dGhV{RZX*ANmrfJm~^iw|$Y;oWuBPo4(@%kn2aRWoh8^}S^ zu*9(YzmDmGMAUM*>{uu5OJPS0=Fgg+zhLcWa60;AXS>1d9GYj`nZd1%~k|QkeLJF@i$TRv#jG#mj!zjuFl^P*c%4N+vC^zINNCa{z m;VmUE(?tXrkP~kV<7qvM2}}}0!BoneMv<&ewG8G^K>h;}aJ@bN literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/metrics/MetricsTest.class b/out/test/clients/org/apache/kafka/common/metrics/MetricsTest.class new file mode 100644 index 0000000000000000000000000000000000000000..66eaf2da41be28201b2383aa5647757456cdc9d7 GIT binary patch literal 10894 zcmb_i33yx8l|Dz7^j;Ljc7z>b!XnayIJRTQ&V~~w*ogy4u$>S)iwOl@q-R@+ED1@@ z0-;crQd)M}u$9uL>5bIv{cIakVCe|i2zB3jIE1n6=08SD>`hka9M2O9>P22Tl)U*G|Qrv_*W`vM%~ zX#w(au^^QgTx#-klS3w-U~rkqjG3OqzetM z4^X`<8w_4Fg%|UZ0CZYv@-l+!JdTo4D?6}V4 z^`gTDvB9|pw*{!1+YRpUuX^&$55M%>FK!e&Y&3b3zd57{O+L@xJUh4EY;vc+dFhqv z-Q+HlxA>dqs0&Q~xXBlqe38i)n{0`|JLMxXm7{FSc|9ig zn)H2>-Zkkb;@&>_h>5_v#8$ig&H4nAx44+IUoaAmZWI-}?>0E;Kl*EpDS)`iDU&ZT zIUV2(XN6)w4&Gz(UV-d0dB4Gz`kQBJ8vo0=#Y+wslP~i(H|kxJ514$pzj^b^>OFUK zgXs7PgRhuMkBKF(WD2cGCNfzok!`oF%s|@O8Mm2&>k|n(y(VsDGIoZkXziv>CT?Y# zmc!m^#Rq`s&BprisBBB7d+V%}73s6V7NNruEj@{Pra9wsDWHPbo3(mzW=Y#FYmcR@ zP}gpyu;EEGFwGb#=KfSp)FmVFoAhhkO7zxs!fc7&6~{tWf$Ox+tQFaf6O_tG5fm1K zKZz2-RIzy=A#U0l%fw)mmP8_%wX!j|ovBo5vugmxt#i~a5w$yG{i(POrdc3y4FA?M z1^Qhf(mbvmMtM)z&dG&MA;=&!qWU8uC&#AKPGpj4v0Mh_X*LoSlC0equ~SM1gRf#b zQDj~hv(r{O(icMnXd>`8QdZhdWZ^hx1?&5P6ORI<7k~tM(^ev?z&Iy`El&zyPefJ` zS@lMk%H@>RR#Y>`adO&z=#gSF?R_ablC`4}A69a?VNxB6N0X`_Fix`rdE-7XR zY-2pyYGti;R;DAlwxGb#X3`;bnuvB^$HV6~WHV(c7L!Pt*=(nh>8zcGzbB&D(7!K-ZSd7WzJ{-b&itt$ zJx7B~v&W;?D(K?sO{f~{>rE#IkWW*ib|#XJsT_w%3mK)$APVemd%vXc9%)4Na@=(Y z=0dea-1ea!q6h~0(|o-ke1>Vcwq95c%Y-9o8<7_c#}eV}-efo$Lzqjo2=BG_XTq7j zA2=9$$`@&WtoCHZiwWM&eCk!D~r_3?y zv10Kc-^4e=Dg~X!s}eYBJkx1C&{=hhlvo?1$igDgSQ@EY>n`2i-n5jsrmj9a+>^E< zQb$EtX%eD>l}g!(sGWuxV%bc%Cz%ds(Au#VkE6W`dac;=mLT8Cx4~DEm9;uegYc0*ECF0JMgXL`DJ9&>&>x8}m_7*aCdI zYTyM>Js4=i=I0hQC)7HxxxjGa!1!*Qh8&Jg%iOFpZuiMm5>Jvksv>hY~x}6+K8UenjBEq7O#XPCINXsL}h?_9hC;Z5abv6r69k|uSf*H>g+pF z8|2scC8oKZy6E>=d+cy;GMlwg4#StoC=%WiOU8BaV_G&*a;fQMYmOp6_vI>-nl^Ni zOvyaqESD`2`@fD<7>~aW(h=Hk+__5)XVVJJ4~>uMghGRHov&M|T+8ZO(rNTLSl!CB z#l&!PD|fhNSIu|Dd<{|*f`%E$9{;|VizY3nGi8qEFz%G4Z;R@tDA&BUN|!j}-hYn8 zackj_3A>`y0F*q_&D7D+#bTt^YKV70OzYgF$03hPf4KyTET!ULeeY_)B|8nlf^yqV zXV=QG0k1TE^kL&m4(?u=Gf;KLU?GQ3z&NArehDktT@Rx!~2%Jr?SlQhD3+N{`d+zeDS)Q>M_ zE)muw`%}2k7Yy&m?4C>`NWGOxeqSbT+bHec$_=8<^h$SemePVE*J~rR&TuEeN0l_6 z8m&N9Ilq-@rkl*kl-5>tG-`gKpEz3+Tf)i8iNEU0r*CM4PENxqE@YDKROOZH9o0ux zXn&<6?o_+*%?eXJ>@}g+bjfmBmeAmc@!D|CW20z-{jqpFmVwg}QIv2P72kL)n%MCp z*t9C&;(Jo5inx$894Gw-@no=leOuf5e1Qqb-A;eQ!?dY#!vyu_xD}r}k)hN)ggFOa z+GM)=EZn)O5YiLcSCJvAeQE^z$YPyyi6)tt^$43YT|2&&HRkd5GF3e%RB(h8lGA2? zGHq)M-9ARj9UsHLeuC2Jw1XbUj|xR}5H5bO~Hz2S=A76)i8Owhp6Z|@?z=h9wt9t4etwNb{7rP6ubmF57X4DXURK6!RKh& zQy}%w^Y{#r7oT(SIiE_Xik8yPX&8{8W=~(E7x2dPB98XR(Hz9l@(W1sFco))NAn{z)Lj%hp|jf?D(f7g^0r}`fm2UxuL+$HDmy|mi)dQ`y=I6iIzlH^FE~uI3Q&lv znSg#<=;Xt6%0uK^?iHJ?C5xhTDufh~tMF40Q7xTJbu@<-;0Ek=UIy1Ww1!@xSE1G_DE}H@Mf5s2i|Tss4eY%MNCkP0f~D7> zzcuLZ4ElR~5*zdu)7P=>2f;V+WcmkKsYt9;)BwxEN*-&NPJ<=FZ7}{crR?dU*`bdO z(VXf*nk&H28G|&hy+-X-hGq}Z{9&pB3HEC=bb$k{1+*>?UFbmT0d2@b7dgSmxdsrZYj+I7G|y`zxG%wV|O+-KEhoT3JSCJxwj}@oI^t)d(;6eT_mdqt+oh z8*b0T&vD@E1U^I?2I*X-WE+(1SfEJTos&8s!bY65po}&R(&nP2-g0kg;1HG8lzXoO z;2JMKbo5sev}M#eNLxZ%fhf!donX?8j?i`weAhil=a zWs{oe6l$S)w3=2T#yhE%V#toG=xjPf=O9Gav4`4_$n88EKN`=cO}vQCN9d^c_9SMeM12Xwxtly-Q|qzgP3)5ld#EygdOHB?6bNdF|+<*}UXGCke&P5Ks6 zVV-9>eVe|6-*JPyh5i|F%Ox#BU7(EaSvyb}ZZhlqZD?yye~s z1U!;p&sdmyHRb~0xZGFaQ<(ckVIsNr*Wlj*{HyTFhYufsURrd7F3ZJpqaQdUAtoqB zLIOfR5)>3}BSFO_GmBMv9Ka$nnn`Xam|FxQGfmez7@6tUuZ7OB3oVOKYy-5D!W2PS zjw0}E+KR#(r(U{(`sg46?-AOC^z27zO`vKfQ7KahkW1(t%FqXth35wlihKAp+RF`e zDX*Z*5HJV$Lb{xH(I|MIqe_ z9?~_}swlf$qEM|q;fj%V1-Zfn=}?d>bgk4SxK&Mv2 zLST?SiJ-Zvy}G6Zf+D1@9;9nVyp++k)iwFO>vTL!j8RZPKb=-uu zeKShZE!0J~B35ow%7(RsxsrnT`Z@gvw3tbo=|AbepzO%5`-{AG?>X9CK3cn{>C=OB z{jmy)I#7Mf#A-|)v!Jk^R1QMnJ7^Z&NtJXL=9Rl?CEWv6zCh7 zzoK8mhe3>AzoGwz4@;0|zg6vH4lPpZ`9}HmcX{=a;3nhlN(eSBbSpAWRn=Ras+t>@ z)b4GgcXM+_!7s%sz@ETF`BiGd++7%WS2_ZZkwgEJlZIQ!Pz)Z*OXGB*+Z}n-iH_;f z?GMxE2I=#IbWr!IJI25&K1_EyIC>hYF2Mv8x=T&L-u5!ON6yL(KKh)|qridcGP)O3 zc--eEehS9kfx_H}$UBG$-~l}K#IOoZAlxzn_+@z^ZW&#;VeG*8dNJk+8-srzEEB`% ze={cLgP4`?#k72g-T;T!;Py9_-DZG0AprD6Apo4;tI8yE^Gxy{1rv|z_3y~U{2^T? zAEO-ozd6%-E}-eSUJ!3DjW!KaQ@1gz4{b~4PyfjI^UKPJWprrN{CU3v*K;YFuQRVc z;J_br=F;}OdmeTwpRUihE+|5yAEZa6$)o$K={09}eW+vi$CgW)8>aEH`HzqUjaha#M}Eri{AKrowlG@KagIVLE^mt?;|v9;`6(Z#f1f72aaF`Whks zO|gr(*hO0GB9t7^zTZmaSign0ucL{78_oDTbT#I*>*;Oy_Z@iqyJ(@`qxaEVe~IVs z=v_>WKj2b&k58o^@jUu5FTisV{glt5pYgf$K5s*_kzDFQME%g24oyBCS0R}bQT%zk z6DLJz@qYk}arTF3$bUi}V4x54)V*}p;GotJrgFX7Mo-F@NHT*|Jmb-0uSlXb6y{G|l4fc?X08ik5l{Vr3 JR^HoO@&8p47vnYM%FReD@cHRVxMNk#XPf^ zWB12X5>|JZRxF|oYSDf@GNAPhXpyLyOqxa{soPdOnu^e(or-LUr))E3Y015M%5J1D zHv92K`WNT_cKVc}q9l#xZi*Y7nX%Gt7`oM(&{8R#HXG+^vvOP)E$7g0TSEO@l?bOh zZ7n*mT^n>TU{=UP31xfIh8<7p`{JoM@o6y(lOe~6wS;`f%F#4&j$|S)7LjK&G47az zf}5|pH<{S+iOOy6- zL1w9+;+hp>a*A#uO$J(;(Q76J?Hyfdo{Q{p)*5+-WxOaM*lDJ%sD6i|v2dKmBEu`h zAqBg!M?%XS@5apR2}M7U5SgPQaV)9ehxm~K1H%#;=BPkKqF@9UWQ;0!31br0wy4KN z-B5e`*oGQZmB(H+UQkN{boARt0uKU)i#~ozPOdLRpv**%eW}CydvTDtQM`u zJgz%(>A}HSwD8h9N+c|571VOgY1WWx_NhJT=zwmk^|NiRnr*W|K?2P6#yR3sIgR-O z@K+UF!jE|crc-cJI-cm&EeUGcAROXmsN|Ds>Die8lVerEPw-RG{ESYkF^vdmYHz%+ zPq%c#=7Cf?VXI*w@f!K7p3qY2U_y)Py(&)&4H7(RwSu4H7c4&0`77g>3SP%$2}*Zu zeO=vhmCd1bGJd7t*Z7T$-zs|3W+t*Gx?PETF`oV&WS=cpdFWxS=}ZTwZi-|%PP|fZA={ z(rlBU`oryQZK6K-Ej?;ly}aIGXPy&oujZ@gb6D%Ha+)uqa0_w^pA(Bufg;`&%ihc) zkziTyhud6*KHFpn?nypP3&S&u%hs*JvX3q}FDNrW1$<7$OQw<62HmN>$+_wosxl;D z2`#7Jvu?M|wjHodx9~{_g=Y#<|0LKuhgh9Sf3KM)4#lq3xEa~S%4t)Lt|fV($bo(8 zGRyIx5K0iA;+q9XSSInW#j>;^fvuM6aGu|)DW%#?e`Cg7r{W4R+Fme`a`#{$KA!oK zY<$jvEF*YG?$=T~^t%`mQ*n?9G@Qv)JTf*}%*Ae-nRdQ)ctDWEvFb20yCFL?+Sw<< zoss}ofL8quAQV`C6FeqAHB+OVifbco<7eh(e!H;(UyS0d~s z_P%8!kX~>^_Oca1z^(W&Tg4T?KJ4dA!T}!kIEViQ897|*8iS`RUkSk*f^P(J#~1=# zH6zH~p8ubwioiMKEw8wD7{OCs>HLlNyiFDP!E;zrQ;~CI6!SdbW(bNILSA$j1?(=M z&=Gx0IOK(JXbE!JRwAFD&1Ib3hNb*lf?9;Jnjgk{*dM?$$Kh4*p&SM1!a<&naMvBt z(M%BeXh%2hWH4p?_&$sy3`nE?669P*xr`neQR#i;UxUOhj9%^uNOblR`o;lxjXw@N zTstNJ2u#FZSd?26LTDHzr!Qb(VQC0uoZph2R|GlpW@op$yT0svaR@5+KaiO(6T0OE zF0v!2Tt;9kCo<&%vx13@paOLSq#lh}iEaGC@8-{-0m3tgH4ZQ(uCo~s^3X$xeg-r( zy!d!{I|-x+uV@TaUHL&KX#`6;ra7>H7z67tpJofOev+Z0jG_6&uA7cbHRKU*Cf5WD zuh5X6eYbZ*dCe%6O8oh$JUx^!HiLay;$QLLv_~(mKD|vV|n>7Y719zP(6%D zVI2p>1hKZTo}*!`EI(cB6Y!t%kaf>ZP-&54Y9o2wj1XE#rA?&RMv|p{oIWA$8N4=L z?l!>WocKLH|Xj)+_Q`;C(lQ+e28#+tb;V&WaP8{S7<8Y@8!ljWjoe@EL_6SNu1RYKUS?}NdKfOOQL5d!fGJzeu&)ZpJb|A(Y zd5jm~W8~)p>>njXo+3S7Wq+B;zrq~9$$IiO4ocNHBsJhpsR@UrZR~gBh|~>DI?4VF zdZY(gOdjSX6UDTTT(Gg&sYpIN#tWJv&3v?VnCnMaPGg8Wl=4bzaEmJ^?aj*3PPZ#Z ztGX*En1wWsQ+q(VA182EhK{88f3a7_kc_*pWAS)dyOkvMzK@1~BXA9yrfn{o*d)_G eN~IoxaESdP-iBrZ^btRK&OSzxSFt~e-2Va7W_36K literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer$1.class b/out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer$1.class new file mode 100644 index 0000000000000000000000000000000000000000..3015a854885981fe02487865479d01a81e6858d3 GIT binary patch literal 1857 zcmbVNZBrXn6n<_UnDbo&pFRIdG5LAoV)wSKfnA2;5y7^oW-Cl zH{)o>kcQznMlc#j6k{63#kduRhTEduk!3=BSy}FiV^WqWS*FGNiH4kpdm8R5#5av{ z(s#`@D^O_5uNs?1#<7A-A%sR08iSJUC$DOlVeq6`atoHXX?Y5I&T%X+TQ+>(@)d^i zuD6mg)(x{{W!8-4H6vrXm5S>K-InXEWeQf=G6UBuT7Hl`W|BQdZO0DA6=qT=W_cs^ zo;ihR)?Km`5_#LP9#kufmRB?u%S6uPUDGJf8J;cn0T`8|DvX>ohh#s`(%@AcuBmW) z-`?hCDyI-zd>TlZBJ&e8Y-hd7dUw{6w%yEKBQSCT76Q*QDpC`-8iYaTUkjQI(W9tS zg}^Y^W{mal2%HSgL)hI1Y%m*EaI2naP1$nhw)zHKkxsKRla5(-ecM@?v4WDjq~juz zI?mx89dF@n9T$*PI2Vq^n`vs8)$u7FO4w%#9ra=LiRvhzC_$g=n8O!3zLez=9_yIL zS317Nf{rJmeIvUYvgg@DvfS+Cx2Clo*e>fjbiyG^_A9i8jAg@F$rMYh@{)!{g~1at z9%gvR=Wf*|D-rH;e1#NFuUJ7{Z&}V%&rwh79Bg_=>OUPaO9`^n*HnU!Y3e=X={)9n z%_BrDwx4x*bX0fNLU`ZEAi(D4MoBJFvo_ zXeuX2EJP@m%T>QbDO1InOaGth;yu3O+WF3lw6@Ch0hAm?LHJxi7ZkdwWob81CVKl` zLiMI!BGS8uh8JPryIdO?2sJ=LkVgKT!29@sQZ%9Dt&~x(5vrd^r_uNP6{3%Ne@0{% zu{|{AUqL%CoA%H=Q}e`sK&Ga({D@0^yU@qGer+0#_H{;gaVp)li_>XQ6X{*F?(F|r zi*4J%<@^pV=8wz{n(f1}BRe+S*cto%1(PrGS}foU6Dnen(!?q%2lfV0AKQ3FdxM(T z4Uj`GqmzfeiG&(Lw;IMZHG)AkicvL&2{n#sb&D_Y+jyw%ps1!WuX0#W(|Drp@ghvX zB(_SdLd+qyp(f!I+a~sm*msoQtNUS5uOo&*oW~`6$lXWTt;lx2GanG5uu)E37R-W;qPD8OuTsj literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer.class b/out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer.class new file mode 100644 index 0000000000000000000000000000000000000000..62b8b110b7fb37a080ee788e67589a96b4c0813b GIT binary patch literal 2168 zcmbVOT~`}b6x|mB6Ot(qzA7zLN1;7L?_*EPQ zEX(C<4J$EprOr4M&BtV-K8d05kMParmDmfV8n?OC3Ho^u?_n=Ko@Z}|d|UDpc) zM2;F?nG{@e+oI@fe#_W1vW^vG!@HqkN)_xBY<1mV&+M7(vCe z1cpwir^9pEa5l5c%E#1<^yjOP^mpma7Ze6%qwJffZO0B~1R9g66{axj78$H9Z#&jP zWoN_kmW_=vB^`O!G|DT6XUlU{7zs)?BVEtC-e%U=HO!Kg-8LR=8(Guc*>N42@VBnF z&7{kg8MxlE;{1KV|c4fhGdt2hG9VHN_3Fo`j*%H+B_97MaYNW!JZ6T^7r%Mp_qWQc{*H(C~mj3U0+Sty{LtHc=0FR7TN}LRz5v zjdhgDhdM4|P=|pH9dBbmpi|kZdfxOr(lYc1JYOCWUG zXcUV?QFX2IUEJ6rjH%R`z^N}Q&BQp z&!HI*DMUmX?P3JyMFuHx8IxiZm#NnomvJzPET32Sl}N_;eUE5_Lh~bKjns^V!Sy_! z@sSre!yk*6XkAT@9HMPLbAWa#Vwq6!^wsbAXUch&t`l@1;S3^5IOB+l33Q69=oj<2 zC>D?ri^z&4ToDBYYk>Aq5;V?VK7oF;;VRxIIRsIKI?2682?zTcO*pt!?mV6(+F^2B68II}5^c}ZSAR&NLS(J-l2{?4+ZYme6p9|Y>}T>4g^WF17dwuI zzYx>#!Jn-75pM~`=6j@yxna8Tt=Uy%N+pf}-8|8nCc}ccF=Uh>QD3 zigi-;0OR7}F-X}z(^Y~dPJ-;N1KCVy34F+PhWRw$BV6FKgS$T5Wv)^FNl>4E0A7Um AjQ{`u literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/network/SelectorTest.class b/out/test/clients/org/apache/kafka/common/network/SelectorTest.class new file mode 100644 index 0000000000000000000000000000000000000000..d7fa444adccd7d54ed953dc0f1036b6ce9353d95 GIT binary patch literal 9432 zcmbVS3w%`7ng4$?liZuh<&hyF1_T3G2#FBh3PF?v)F1>TJPar@%v{L8WG2i^c&N~7 z@zK`Wms+)3U%M^a?bd3mKuUG3?e?{6TerJiYrEUl*V=ZiwQH?)Rrdeg$IQ)3`dfZ* z@44rE=X<}-8NUDS*B%G3K<rmEIec4mj-?n#TYyjMFpPIti7D1&BPxR{D1W0b#EO}6Mxi+H+0vZ6w{j~{%qnc9saM0w@tib;$0J` z41p<9fsllvm@8pJBHSothLlHft{D0-O{w5p`RtOYA(c^_PYs9_#RV$M1v1K%(fS@^ z%2?fAMYJ4 zrYtaJq0TMR)pJcb&s(eQt(3*G#FQ3YSnAEr7;DJ+hAdOgFHpuVjAEZGH>5QxZL&hA zR~mAWAi6Tqo$a)9`|O;+YR_csTw6Mx&)a!HW$VU^E?T+%s?PQ;D+SUn7}b`|A$b4J`TkZ62IT6dN&O=VJrWrA?S z%uOWQmQ9lEnANF_-O<;x!_I9`KtXJEHW5#6isw@L?M+4s-6`s2_FqWSm6Jrv@^+!G zSCxGrVfPkNRH7jl3+mSQWmN7>seFo5mS-~ALflz)7@#tD_mSJ?R(oeQ=g3{KUR`6ZNaYh*S~F3gkQqnF*y`Q!eevdWJhQ90vye+=c5%z7 z&O$t~XHC4<5t05O7#qN>hVl`&rBimM5JWsd>9x@vI_*qybE?qYo>`a6?#kJDYRR+@ z^pd=t6if+FZcnG}UGX%@7VM%rNMh1pT-m+?y-g(`r<7OMJF@FJZEwh4lFd`F8W(RW z+uXi3h^xlIJvNclx=83PbWe=~cY!R84T7)B z<_b(Of^jY*xt@4>ZLghk?8WFv(YJb2c0vPZ+}41=MO7O)uBP_Ux#pbKfjZoPS?}&3 zFOek0=POT53#OkIfm(#y%G3$SQ6y*Y)mJ>3WSpN_61kmny8qCV~4%jvL0f zkQ#i=bP_IB5U7_9ta6U>$8A|UvS9qcpgGv<2hO5_XqFCDPhh4!Q?Pf@pO>{Tu`|~j za)}}BOveTuWa=!sc_z6!p4(-wcbR2G??};4#;SQ%_VgAG78lNNrVh~XG`sIp*-U%V zkX4jcSJ%+17>_z^OK0=;+O+Ds1S<%osP{2D()1c)DdIk_A;7t3>Y+0;hfdLDRKGq# zj}(1+7=)UsqKTZ%kWsq|I)?!oN@C_|y~J}f&sofP-jzVGqTw&y>$~W%B1Sv2eYu2v zQA$&4RWSR{)?8%aBe+Fy?*FrFShxWvV8~ia*2!g-tj8^ubjk)pHd?YtHd}JJbXl@RHdYW^9<&jGEg^L`%=sgIZJlS9%hDFb98f>-AhL{pr^dAC`bF*M4~U3vy-ze z=~3-xB+IO3Nw4g+B&WGx_UR<_qScEy3;&9LqwmbJq#%7H-dQwXDpOye$=EdX)X&fg zFe{-6+Rn{jer)Tu6MLNXyc3+3?31)Q!hW*gFUAWFgjccXKlys5y=3}e{SMpC)aUH| zL`X+EAO|hETDCDqTXKzTW)AgH*o1P*NWDkgl56FVuVF`P}hmAv|r#hh?vzesj^mU3+9GOg5uj zy7i3GtDpOyzVS4C7!i)At5SPdDKx0m->;RASaOTT;lkmz2DkO*?0uyOA}@Om~7yd)9J@^7q?=Jb3=ixQtQEj-P*Tv zr~2>5+O2tkO z+u&7Nue3P*`RI~eZFF1z6LuG-~mOPKW=p&9}NOy!BdDi`nVby~4m!_XbN z!>btVpR0ZQ1;UKeRYf&=>7AN1Q^i;r@9%P~?#B@K8BEO`_5m_Yg{xIF&c{G1p4BzD zdebSw4sb|MrkTSYTFTubAy_<&oB_qWr&k-j@q_7XoQ-R(FGprTCZZr_Z#JLOvo9Md z-+3>!4^l4LhW%ZTyzj+B>8sMb+X+P(&*g0vnL$C|3rzns_UT2&+MViR0}SmSLP)>2 zj|c|z`6CNE97mOUlt^W~gW>>x8AcG7xstz}H6$fz8`Z2`&PKfwVS>oTGs$d^yOAEC z^@Wn9^3EwpWgj4!N0TnxmvKR-4k6)P+o-d-cP{EWY1?Khuka#HbCF5YXgSfSZch!D8J@wgTZ%M<8jEt z4%n@H>a`~@0}+{qk8xzVK=?R5!7srli6HDCxSFtv;KIg}2z4Dt_)$a}A3@|8QJzFO zXN*T-a#+z-jp#{OoEb#`GdB7p#_(h8aa28uSc&#{=)K=uRAL@tWN;D|@S1aBfR773 zK0Zab2&vzO+c_%d=nlRGpDuFO#xLcriWFo07{{vU@K#+SH=2R~i+Lfugg1aK0Wwu1 zlDX3%b61hf!(>Ah))A{&-Pm*-<72grO|jbJIO7EBT#6H14kohK>BpHN+>MydRAW*< z&I;q`spp1lR35`nb4(|9Kr6pi!o)?WMmr{A6=va5EJX*FV=dNTox}ThmlKcoFd^>7 zXQ+g~A_iVFzm16oK3k8Wy1Rz}f_sZXJxe>NP=|e?lwUGmA5ta8DBzUinA(qN?sqlL zb|i^SuW&_Wf8LJf{8@ag{KO$Y;6QD;a7wJp$NFC0tG= zbkR4qU@Ep^Hm*P$uH@`Cq;VDYVml5Y9#F>NqB5@WlrfU@_Z6i-KE~C)fU{2?*4YOU{+j?{4L4w_5{}iJ#N4jfJdPueVE%C| zs2U6Y?Z=`rlyu=Hl|#%~hXaJCo{sS6Na*uL!r!GrBP4to?Q#<5c7>1Qyfx<>!{TMi zNt46Kl1T$u(3O4^L+`F3p~w9#rD&T1c$ELF^(O}ykdpZRk&bLToTqe-#R zWu$1k4$pQLqx2~Lmcdhn3-JYJ<**ZdqY!!*O^iMRe>`+D6{zzb7_^l ze!eSPNy#1IMUk4w#0OmK*F;WW>8ya_k3@u7V`%gJ>4-sWPt}x9JhS@&`J!&9aKn9t7w*Bxb``!- zW1TGq#v%|FT#@&!DEzsFDTB|MJ@ z@hZNIckqx{cvvR#vaSVR4LsTAGfJZqBp*A!P$G{#1E94EFpm#>FMqa_!I4Z+B z`3+7It{h*drnHQ@P>&U%9wU^N(JqWWMTW}^$|VK`1KvXYDeBbprwly7zv8Scd>e}l zeB&M5Z{SG--<^=B-93!%`6W%(cdq<~#yM?01}zc9t(C$M7; zE9W_bE2i5?Jw@;zP&YrMmVd+o^t7Y%dKXDCd5!my*JxJRZ&Rxyv+|uHD;vp*b_g{^ zRuZg;iNS2d2nIYuffbW{4+-E!s_!ML?`1X$zh;y5TlUDWu;=_8 z-Tzf&@OxInKTv(IGc^Cm@OuN_!JinYZwA!;f~Rf`nlgkJpVcy=ga4DSbX-G6QLmcs zdlfl0tC8);-mueodW9$FN>VKA>>f)l;Pzeb5Y@X(6+ksZak7LlPa*-|n_u+3IrwL% zOZFY3bfnO>IHW&f@Tn)JAJd!lfU;f5J3&|S0>A&$rQq1}02YmoSzaZa!hu%^BXvA2 zUDot@L>ucOtcVXCJ@xaWr}RI&xZVF`80^dqj1iMfOcXPu604YaJ7pBM@a^{^t9X9V z)Tq(6-IGj5ZvU^NA3A}p))*5>PdaIf-k!n%Ie)z_rJ5XfWzY0XfDLQIBy6E@kE4@d3W|%p79veLu|q E10u;kt^fc4 literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.class b/out/test/clients/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.class new file mode 100644 index 0000000000000000000000000000000000000000..b11f8d4e9f0307a3b573073406452709a35048b1 GIT binary patch literal 5359 zcmbtY33wD|8Ga|b$xN03!g8hpZYk8wxjRLwN>g}4{E(iEJ<2gq~g_j*Lv5pTJNh~t>v1Rm9}XLMi4Zp!c&V%}GjOA z;MF=_6F?BJHE@-I*9B05*Oy}x-k{@+0nEdj0(diSmfc$dcq?uR;B9z&0Pn!925t-B zop_gyBO-9S@Vq;~-FtjzcV<_Xo;`c^8Ry<7RvLJ(f%h4Be*hoA2MydI>xT?{*uY1` z+($+5s66hJ$HxrZ6~M>w2_2s_@F{WU(+2LApN`4-P=LTcBaeFw92Z@~ay%ltQ8~WX zzZY?XcQT?%Ktx-dq~os9;0ssn)Zky2>ajIUZ+DV*N4CG)PIp?} z37!PoQ!y*C(@Hx+=MMdDpOewhRxIG{Is8$ZY7)+2%XL!8PCMfgNNL7)vjZ9`cVv^U z({Jx|G7c@ZCX*>w@zD@LFf#kHL>y_ezF55*9XruOp{Y^VitXQO z4X9}9c#vhlPGP~awlqu;-0SVQl}#|S<>^#5+2f`ihA0{?)>J3!r1?U?RTmyH^-@$C z>L#U|b(8Ae$1>{9#^ZKc!)#A=(n&?y4!L%lI;OM!v~Be;%PiU0ewNJt?E;InENy30 za01a(HXXA!I#L1)FIr{wk~tGw(LpyS)ngiJi{w|yrMphjU7_Q1CQf6YnEX6_=h?C* z6JG!WCPc$>6JN{|O^qhLB!o1xlk7F|W%W}xqnI)AkRq&f+B(EiD-ukruy2ct{fB8X zE`=w=S4?cimPrIbO$(9ys*bOj_&UB};+y!Ejz@&}wvI<-{f>_B%KAMv#4;VR!+z0tl@HqjH}@CaE{JQ&K1pE;paJe1@} z2pzN%S-W1xZ%q6azthlEWMt2q4Qzv+R7?W&dkyn*wtHhm(HLT{S zydO)cYgv=GHZ1qnrbaexv~$Pijw^Z4*44Qos$uEm2At?#9-d?%JfUKJ$lVr1Xj&l< zGNd)=st>mDEjBaUHva7(KXZEf_;S{8O?X@(7m3tdj6%8nV2bo^b!PZIB(yVZF zJPLJ67&8ito0G2HD=(Zm;mzY&bLV6p=VW^-*(H?NB zHR_3f8SxjJk9>Fe=>BBh(4O4*(2m`U4iw!5o|19(^*>jW!CH2DR1hmT-J-obns5cTb_05?8 z^d5yMZ4Ww0-V0}jbIvQ1`AcKf#5g95lh?<*6C@()+QV3tS($bx!=x|aWG#%2yu;p+ zk`T6LeyncdfAFKPMD9mw$D0aQoi`L-8~NNuD#DeQ7L-|f2&N)+MNS>UG({SUoIZpKMOJv+Gfp7br4-EcjyNKEFgs^s1apLq%W=#l z&Uszc_h9Y_DuaIiC@vYsbAmcI^JP=c%>vm>35;T)Y^DaEE6a3QLb97d6PI@RgfU9` zI2KXeqTnp=IgHDKv&#w`UH(#G>?$qG@l#}1A6}QmI5iI83nDOWlI_eRD z;1WXe_U0^HPJcaOS}>g7ROL6U4N*|x8%0fS8?0qOMo}kZAtCz3T<S% z8;)WfKRwnLg2xbhWtMtEEbYv9!JBu?9y23ap)=w}LOI zs8$uULVBxR8jH12xiz>_o!16u4`bCRnpr0eg#^x_HDDLf??w%}n7BPm*xGr~C>A#yYB`OySRno6TIpa7T*YPi478anJy%}R(dKA7!&qfX(t---QnSb2v zXC_K0{X8VlPc(~}rzFq(JmqI$4(DzBMY2>|T~j-P^&PbjV#5(s)}F!!{u{-{t#xDD zj^LHGb&Q|H&a0P8n03%+opz_uawV-UrnMSc%GU>VkKwBw!&jTbr?X)W5S5P}0tU1* NSgc9`Y0@Rg{1-!z#9{yd literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/record/MemoryRecordsTest.class b/out/test/clients/org/apache/kafka/common/record/MemoryRecordsTest.class new file mode 100644 index 0000000000000000000000000000000000000000..18a5de92b1ded855e515ff741b84793eadeed231 GIT binary patch literal 3851 zcmb7H=~o-q75_bHq!D7`U>jkGx42#^5TMvjNDUYVut@=7yM{n9Y3e}?Fd$}>nUNck zrYmmJCTW{4Y29>-)5LDNC2fKsuH7~L(0`*}@~u7nl;e}e{k@R{0^3K&=V;#Cci-Lb zefN(3`O&-A0Njs%L=nfC3N%2EVjSrxYB8b0h+r~`DNILTMiIhH1hWxjBAAQ7io!;= z5|6`?iz`_?NtBD?EIucpCnRoOE>FtkDHWfWq)%7k3;3d3z9i94NAYERMa5U6*ntJP zIVYj7N$43V=2;1SUBx$|*o!k!Jcs91ydWigQ^mIgD%18{)-hbyv@L=AQnoXd)U$ee z+DOjole2n~=X17|bd0p^OeDKksfXsXMw>vm-Ly=vL*QWiPMPb61(a@k!VtJSWm?AZ z+}ya~4C&(;Vm77hw4NE(9aHWLNX45rU4aATGz}PYwljaqx7i&sT#tro9^bv5;pm?2 z2n6F&y&&}Qa? zHmx|>LD7dwBuHLLSymt+HdU|$1&-&J|JX!N9zOCX#fx1&U%QNSUVbe8P51p1} zd%oX{T9t3%OpbM!EF@c*&^=wCHeO$1Pq&@P7-<@6Ym4Ij%R zGP*UDJTX2)#n5Ugn%guq;Q=<~9!RQyH5Dg2dX>z|g?mxjOL?`*y6olG znPnXF<4VronxZM|N1-~cyT^?u7y^#y(`2>$O?8oeZ8)k{i?fq0)FE3`_EN_ia&l5} z{IqmGY)?+QGOU!@$4bkS9D6*cOSeMtzCNFdlapUL9PCtfyMW|rl7-l^Srj5sYS+!9 zLyE67ESE!LSkIW$QJJ@)hZfMz<}&KmeR#>wt@=B@&U9a&pz7EyOqQYExf;w{-gNBZ zM{b-Sy#fI_2M7fTH=&tplDi(_1AIptE<-dV-a+6k?gF@v>ju&Sd!X<`v;zCEpSzko zH4fk)cjWV>auD#d`T?F%X5e+G#6z6isIc7z8{S4>2})|sZXkRWq0!(KE}O!OP&Y-m zT;`x2ReW4j;wp={kdMz(yWw4JBgAxw_CI zHZ~T3O-tA;5!HMzBDSE2i#Og&ETV?bmXSsRe721EB10GOQs^pbM}rC4(>O?7%cv7* z4b_E~agV@xWF@p!P+4tjxQ-rdYgL0SkyxZ?PFJ1klP+PqgnZH+trfAzxw?u)>|DaV zO|gh?uiWoi!pHpgB6crCXxj1$`<{z8p7*Vtk(p8a%$&t`!VSz&0^69?c4qV>bNVRZ zGt7ZSs;8K*1?K5x?q6YcUSodVKnn-&gPfg*IR0AkKHBhaa{mv9g^CWb8ArqpJS3XY zDO%7j+R-g~a8&f-kT}6wkrj|dC1u}BNe}YhN*W#b1f?nD-A##Ytd#$v&&Ne5dxX*t zvmV~({O`aKggLLz;~~i*E2L0I8$=g6c`C3?+>b6o0s8SDbmJ&%@EEnqy30%Jp|t$j z!#w*W&sq?*)Oe+I8AKK;AHQ9_PCP zCm6w?e+)4MU z5x9YkP*n7*7!vgC1MI{jJPBZU#fj702QlVjAN8@7l=K-skEQWhjPfAL%?Q4WPw%Vv EKTcE$jQ{`u literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/record/RecordTest.class b/out/test/clients/org/apache/kafka/common/record/RecordTest.class new file mode 100644 index 0000000000000000000000000000000000000000..08ca276dbf8a11754ea9a61233e903568f746525 GIT binary patch literal 4539 zcmbVPiCq2GCLU;yPu(jRc|x%VvhoO8c(?&JDP zPhA190sjheNCYu}WDzzYCBvW$4`@gS;YUUmwout1j^Jnz$1o(z<1)NSG~X=LTZH?d zP;ZstZ8E%FhIhztLgY>gbxMYZMEJB&?+nuX!$Q4FL>>{wqq2M~fX8L=ZVm4d&Uqw!$E|Q(9jgXM}qh$J{G`cVST(9pTH-J@hN;-hR+007sO|AM#JZX z^LY(lP~h!1hZHPoIcyv;B1tP1X&kc6#%xcInQl18R<8A`i%(r1F2*rZN^jS?#Rx9^nsy4lZN~*M?vLmy89gyqO#RWTJ|;t zn?s$AosF~UO+T}$O@X>I)lKi`wOC2BH9OE{rVkih3GS4(q~b=R%}86a&Tp!ApOsNi zJ=;mHhX>4zO)ov2jSA+(Y$M*k#~5@hF|QiFs9{(^(5BQb%S?3Bus0+=mG93cZEL`6 zvoaRX-jPhEY{O<=DN*7aeK^awBYFF}SlHQT#``nb0Y(&04f4Jei+@qVlgMDEWOHwi zf_bhJzv}J_lxwY>VaOuWaf~5H0!JST)J6$ZqX)Hxfp&H?p_uFwbe&)}Y}D4PVmm zWd%hst2b%b*);FBb_Ta*GV)bk)jlI_44Af=rlWc^nKaWo6GkRua#=I!UeZ}%Wg-*e z)^W3Y<-{t($b+#|HXS#2Spv0r1-Pu1=oKuz$y$V%Xt(S>>aMtnIqj$h9ox`Mbm{mS zzOG|4w&WEn(fM2cSC-|v?jk81(O%hsGw;NQ`vA8)X;M4Il z{9KOvg^pk1R}$>=8h)+gH~6iB1qBT5NT-b<0ha=3_??d5OAvp+9|anPSkyW?@i*3~V`35#b^IOwP*63Cy9FpHSUrpEw0<`3_tX}xU~VBAH;~f8MwDPS z>{MF8+RoVue%f&?tbZy9i;r$Qnho~s2uIV#Ap0A;b-pJ|^_Ae97oYktC|JirePxND znUaEAJ6ND;UY{zOMPMPvzbD>}74U@EZ6G0Web^s*-1@?*Doyqfs%gS(!E|C3Y`2l?T%D@T*LK zJfs=NY42c`p_GTBGv08qtGvrdWT;(iCNu2EdBds{I=?k0-%ZR;Tgl!AIU`^s5`2|y z>MaRHXM#{oUVB=iXAUyz*<5F4zs%fnT3sH-eSdA(~=*=25k`s2%!A`u6E1pPB6TJ|sj+Dot#8)qN zk#2=LhB;6*+@&Gx{N4?Pt1${p-Ss#!QNGMGGkMZ(H1m}0wEY5ok&{n{&oiL&2#la8 zhaf5MRcPukRBsN&E<-1k=eZkmCM3&5VFYtexr}+IawsWs>G?vFRNZB%tD|0p=%@vq z1tM2SVi-ZXwE=UnkqO)6B)yMQkP+AA!VbNcAbeg=4vQz$mgG>b zZC&PCTY3g-!$OW?*`Asa+_tQwX#}roCrP>6IV~@($YBK&-Fi|{hR3d-qmRdNf`Pp{ zkEceI(90c|&p;~~U>$MO$e+83j{Ud`_mk>FlveaWR?o68O0rEDe7^G ztBgiLGXZ|h%||=R@Os>ij)?$ia-y&*<=c`61{ic3vEw1#K7RcyeXkR|fHN?DM?|M! z&^;^28ARJWrnrEylDz%NuqvV#9kjQd2z!Da3U;5H?|t#~p7GEQ)GqZlR@poi(~CCd761{N-HewOQNIP4^B15Gf ziYQf2%r$YZ8ADFb^U~5YILJNF)(F3A-@q$*BUe(+?#_Lrq@3@8|0Qg35O=?ZL;uaw zhsIbZMHAZ&amX1Bh0q>j5B5#t0{@2R2Jg0uW<7{cfV7uuDG}MVoZVs`ewcrCar84* NNFgxxufV*c=zr-jb4>sM literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/requests/RequestResponseTest.class b/out/test/clients/org/apache/kafka/common/requests/RequestResponseTest.class new file mode 100644 index 0000000000000000000000000000000000000000..cbee421f473244561e892c57afa80b52f0ce8490 GIT binary patch literal 11233 zcmcgy33yZ2l|ILk{lr6nk&TTLASMJbF-ELmhuExkj9Fwz6++3g{A>$k$w;yZBqV7H zl+w^7-PP^L=UlQ=0#h<_FUJw>1AF%@1|@5&d{F z{e=E^GX0c(Ce6=vs$kM-KC_G}WaI%r_WHO|<0_3coo-{F#*=i`d9ucSofgp>KAwV} zr^>@=KCYG@({&DTjgM=E(+v4hr*XZHXX@B>gT^=Mw3=_$c$SZ6>$H~VXnc#tjXs{M z(?-5krwu$$W`NxY1g>u6S3RBCOAS8BXUr!e24@oFEh@$p)n4)QvU*XwkcH)y<3 z<4r!^>|=;BxK-meA8*mAkGE>PO{XX?(s;X0M|p?FJ2h_CDb5`_4f0(Ycj|OZIPcOa z#a$Zj)_9LjBfM9qVczHCkbvsec)v~$@BxkQ*69?#P2;dm5A!`bALM&AKBUv5!sP8b zo#w+D->1`AHZ<72|*bUM$y8k_d+7efD!ai7NhIz1r-158yLqVZ^Y6O*TD(E%p! zmPD_~G_5llH@k)hd(33F(G$Z=pfeFMVh4<5RDRo&-t<5;#dJq!BH14_hK$I789ZwA z9W{cH#Nc2e9!#3YhRswu72IbH_L-@nL_B47V{$#y^fdaA37)a&Nh2Li!~v*hxUbJl zGR^2bVvHEUcr+1gJCQcq)HD|Pd(D)+M96Nk$YMskAL9CAW+WZlWu^xby+yRWOzXx6 z+S-#!CyhwD2rzhguo&bRO2HE)ou4j%puWeO5>wU4f}@U>FA(&AexH8ey#C%BCQk-^HnPV zM~2}CLEA7jk)&y)OY6z;XyMA+Znw^+OTCa zt-KDY(94{1Y%S^VXp3OGnT`zPgiY#`pIaecfSE?K2ZMlU4V!il)Z7!S2*`PmZN1Fxq_>D$q$t?k| z&eAjto=msFuX5V1AM?P$ZZGLbMB_V>iQ%E*C9Vg~5}TQX-|oJ?l$jT!C)96C4G-Kf zHH?v5LKQQu9lJ>#{15)W04r;x85XiJ;b&VSKkwl6#??_Vr>ysO;W?bsmo$fB(TITn zx#i$^#GYshiAIaWhEr)tIdjV{*kX}DK$V$HCX!(9&a4R74VFUH$q-5>qw)Uokv5b} zq!W=uEVxbaQc=}lX~c4-EjZAkE|-!6E}FJn>eLOhTnX7Foa&}hKpiu6TrW74{4C`S zB$8>R65D+(Sni~R6UGgqC==ai1r>-Zp&6_qKH zmFc)2sBFpFS(i>vV}}RXI*pNC)@FkH!l(KzPkL5}Vqu-R#AIRvBWb`L8`coPGxs3jjUeh??=OI3Z zO3KekM)`Ea&uJd^^9UdJ^Zk6n&nNi-KR?K){QM9Nh7$9fxk%*Hepri?*ziJ#xY=QV!J&lmW;$jc7R zfH9J%Y8)_90}&)~kUuWlzQ9i~Eh%TRWKpf})OQ50;U}4b6If`4h_zCdy>BAUR8_I{?d<6;=g%n93B<<`nM2m22sQt4$P=L6Dw5N|iif~tJn2~FZeVp>p+ zNnuF{c7k28`w-Lma;tJWcjw}Kh)ZG+Hr z)AF~%&DxmY_{2nTxT;n4K3ov^R&=9`Ny}S-K$ADnwAp_`L3?Q;&|Cr>cdb$mq*#2C z>CSO?=PpSV$U?JC1jQH@qM(aRt#3(qDW_qYnoozFiFm)CKgFLG_Mc%|aUJZLmX5nB zU-Bw_h_siHE6c633sR=}Fsi#SbWtNX@Y-Z=r8yYa$(O8&TO$Ge-6PT>8` zs(sX3M6P_B2+E=Y5(%^4OsjV@XTwbUn%Z4&c-;fZ#BsTsT>lGQQdt$5+E!~aX`H}o zWtEYV%!a^jI_MH;6>N}}-?xy-8*enX|2kY=<)1rx0gZ;@s>{3|TygPw502Kj`cdo9 zE0x(I03j7UY3}Y*#O)AZ#A3LPO-rrb)NV^CEHFjH@g#tBCxplXQA8pazEw%{cr=c? z(HcjxB62QUeNeko#ySKFnwo@==^)r@Mq)_u9_wJQ!ATb%XsXuWy6HD|50VJo6B z7q>I>>Avt85J%%9iKE#1TDKTmRc{|jivVz(h`<;UQhN6Q* zF|!MWG|KA&t=J4;p>)C;!*eBkDGcz3|qiP}#c~b@L<3>q@Xy z@+zA;R6ivj9I8I{3=Km+Yuw9hGrI76>{Qd#;f@_C)S#VOcuL;PE@q`>lO=an z&Unhv#<^4+=dtW7=(ft1L`*&+;ga9hpxUf!R#wz@Zfo7TZQtRp-Q9;<545&-wzhS` zX;w6qjS1(z(zR*%s36Shnrw%0>t7xe#6fw-D7xZPJYCliA-TR&W^fxv&B8kKODNoh zF=wv&QXrKx!oH;dC*_6evuqu)+#jL+AbtnYGy2NgGsKrLsGtv_tx}V7 z>BAWNEdu%oeH6c#K8D2}S?rld8vYaisCxD_st9|)!xIX7(Ro8*;aC|8S4ro|QdKxZ z+9>%#VGT2rLSY{|Jru4M0=4g(OojH4TrZ zqOV3jP5MHTYD`VXqv_}a=mXLhlGI?T29Ijd*P@>xeIZF5re@$#J^DKIGo>#iX~0xH z9^Hg~Ci|TsfN%Md`WG-N;fm@T6~peF?9Y62D2GAXcb*8qcrCV-LlTx(%>DX zMrr2WjmcXB^SsySwy-BKKQu}U)>SrCGA0&=Jq=z=G-2(!sw^#n4Q>xKkJ4gUGfHR| z2bQ22rC>`#)fHM=hGv;fGfKaQ8UK#$wv!l8a%CznrJru z_8>^N(;axWi5AmdYN0SKAp^5%T1pSnGCEDm;bbf51(uxzDaB8 z+q90pM+gx7y@3yZcu-B7cph!$X1bGCP%CezHtxWmL3Ghp9-wV}41aELoObXj+R0B+ zJHJF7{8hS(zeAn;ee^$|UHn7pQjWcesz865o}#D83+i5ahMuKLtl0xR&*A9+U7+VN z@`#gL0RTxa(+l(>0^t)@aM&|1Vdj&VnMxHu<^396)aX+`2IZ%fj+j1!byi@#g1$oT zV6j4CaIGC~)3UTOT&;TptE%-3-SN62%SYp0S2Rwuqp?D6acwp$0aHR*&(dnJ$dOE|FAlc8-{+8(Z@9f9@?bzG&pJao79 zurvQ~m+N7cc84pfdjor#GqhK5c$D7;;J*Zq^idDIH$rXj0!eyxmRuW?k9N}M=?g$* z+Z^;Va|JUMa_Md3J%f3{ySfnXJ_}MPe3`mOX@4ypsMZn28G4%qJWJs)VBM34HC0@W z2EZ9~!Df1|eLgMEQlAYCd#t<$KS6$lv-Er6EI5_ak52Ui(lRtqEa{|6 z((;@ne7LYBowA)VSkkyrI3vuOp(Bn>kA^E?)R>5|e9bcIL(sy*@U=&jILpt1wm2U$ z=dTtSRc5{n2o>`A#D;Q#@K;;lySXHkK}QeQC?578My^moG@PL!2ir06w`4K2(|~pc z3HYoF+U6Xz)noikW`12k!$%z(8j>nTM73E;h28eeP`X&-=Up1F&1oFphHPFxp()@- zG$@J0i#+1_F7ikxxNh9F*wY-6L01a~SsE5u8^ai5qZp2e^+&>ih_`uhONNeH`@er3 z+(2a#IU)lmF4IXzBM*cdybYBZdN8k)1vCl5-;0d?IMvV-R8LRREYwkrsGsga=I%xY zF|Y;67qV>twKp-94%2VZSFnv5x`)1s)2sq3_tI};38UNk$JkQfqoD6uFT~X zYv%VcQz75SY(>0zUOA=m;HnHg-2|!QXyaKY)B^*L?R?zcWPFt&PsDv=p7E8N5$RVSq$TOz<2?=c+mx;Ip=Qk z#<-iz{IP<8?}avuornZ+w>eolE$OW|?v>l0DR#VzN}^TR&dE6*K0w-B9|baruazY> zm!-3ozO7wm=v~e(&&e*|UA)Ve!Rr-NK(D%XxiYuQ#bb6UGk>ObiEp7co;OprI!YV} z=fmFVHRk~1J;g9C0mfyTgU_`2l9({xULcze-SR2M8b_PVQ9b@w>J!Yq! zTDp*-#~nmZh*F*`Mw0=WD<}`Iy3jZ|*N2j^_+*orZz?q3f>|o)uPK9m9q??Y-870Y g3Zb-=K1xSbKS;+^e+tP~=FieO)jx(n{wr$uKNlK;Gynhq literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/serialization/SerializationTest$SerDeser.class b/out/test/clients/org/apache/kafka/common/serialization/SerializationTest$SerDeser.class new file mode 100644 index 0000000000000000000000000000000000000000..17b0fc6fe02d0d879259ef980ecdd1187add7da3 GIT binary patch literal 1255 zcmbtUO>fgc6r4@c1{>3~1;Tepp@%deETKvjB#ID4AfyNmv3lRQOS-LNSGI!$7k(3m z3W)X$x3=Jq%?!j+Mav(gYpPYj?@w z9I3pAQLUQUicNh0wt^) zwcsB#i7dtxK8+o&1V3txl{Hp;<6P!kYyE&|Jvm3I_4FL&?@W~m1vW_xY;k-Di*44h zjSXfR8I1ck8Ij)+pIO6ufzTe Dt_)}v literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/serialization/SerializationTest.class b/out/test/clients/org/apache/kafka/common/serialization/SerializationTest.class new file mode 100644 index 0000000000000000000000000000000000000000..37af49e55455723a3e6cd9357e9c7e94b25d79ff GIT binary patch literal 4681 zcmcIo`*R!B89mpMd}Vp_l4B==QUs*IZzX9kiEIpUz=;#v0qi(9Zc8@OT3%aTiPb8> zX(^P_7kyCZ1JcqLw4^-RLQ4s*r%b0mb@~sqe@dq_ZF=tRN?OJCjLb}D?7e&Me)rt( zJKt+pfB)~FJ_2wUe~jTE%qSM4Sdz%c5LUD0R#+%RQB>m-t*}v27e}HT!vS1sK|e05 z%aal-G3-=(uBh?z>hgtFJcTc|;%Ph+!)&EX;47j3Jpax=CB7QN zZ7?Oi7Q^id;OkL*L*n@u?ozXFw&Gj(w#0X0xErQId?AYOO1v0DkD5P?7bISaQHSqI zd|%=R0?{eMe$XfxwtzNfS%!T)ua`}maCo-6V3LPpQe=(efm{$ki!Fo|(=ykO^2dQs0T7^x+Feo0Sd3d_p{E5#E{J#Svo z9g{OEQJYO0C1+4zx5IJDu}v#id91XIa;LBG$j3$BM?e>Y$Ha)|o~^(SZ4uG1iJ)Z_ ze^rdb4Pv+o_`8S_HLOe_OJfzIi~1!!mDjCY$|Hc!LnVg_%H*_bg)Cgb~Za+31YSc6EL5nRKPcue9I ziK_y;az-s%-JJq6odHYyNbYLz@QN8O*NWU~{LupG2EXEUSE-tM#|zfHnInNa0$!(6 zDc_b2q-a&{o_e_;UB0pb_0L>fG&0Vh;;&`O%vriqwpsla|1TzPpo%&;HdQFw8DrE` zb+=<3L;b33XlzjTP{-w!BrAqnPg+z=PJ3me%`Gi%r8s8W`id$SEbeEgM~^V#HT(!a zX6fg&@8E!jSMd`jfrg*rXBvKvU$E5NT$(g|4v%XX#{C)|z%hY6H%p9^&3sn*>g?1) zp`6bq87WC;!AKTtGiO?Q-d8lK&pU>ltT@zld{Vcv$(r-LwaccnkgSxyq=whws-o`ruC;XF3nJt7a?>cM0!KbBMHNomm^!JK7MOC3f+b_6zv4>6?ytB@ z;&&R}#d`t+jhXFbVvR2izsDa~mKA<2Fajsgl&v7w^Z7u7TQ_ZvD*fJXI2tNuRf}9* zf_WuJV0$fRy>yM&7RHz|x>H~n9oU-cD)J6Bwxyn;keZ1Y8|w{mqoG~paPv6psz3gU2`J>t%11Pc;noNZ+g8cI~CvF>%uX`1v(2H;A@{*n}8)xR(8al^<=U4i*z_wc8xLF!1mZ`&$;GsV68yvBI zi411krt7xCHXp_<1gh_zSKke0=N212->Dni5sb^ht{L3H&#Z&|LIXd@gldDl4siBq zj)G&P`#QvXylKL{yv8^JdntSij_^+Ngm4sT&IASt(4+vw+mLAC+FS9a@px!r4dL1A zh)njyn^z&5aIP;N^>0?uB9IOz!mEf0T*Vc2*DCNAXz56IB61zu(#?tH_pv=K)vzNS zUBgbI?TW`gKwBch%LmxqlZf^vWN*(Z+Cx}HM+hHcPepKVKu~ysqE7JIjc1jWVb{u0 z?Be}?g!oC=j81Gv7xthVN%XR5_hFP@gA+($n$#6>2v;~hMe?rF`qw$WK?dF=l49sl zL@9p(L%5Gv6PQ(PpWICI341@E4r2i$IF5*7%V#o#Yj}`rOl5ZIhj`aSEnX$k32Jqc zlHJtR!yT*Qo~q)WCaxzxPWcJS--@OiG$kT2DRIXAdsyNT;NN(Ls(>H@Qyc|Vt2lAO zcjD#9pOKgi#c%l#w@%H5~r0Yi`@4Z4d08$eXf&S zQGN;$ebV()*k!$ge(;{nR-c`xCltd;#qeQ7U54jehCd_mS&2CzfPeAlgdojL>K)L} z6%}{QM3`Q~U8Hf}L~s04jN|t#}Oh2Tr5bcuIYWqK&ntC&xODb}aN z`e*x8>ZIN!Zpe-5{cb)^()Kfm<6&m-Be;_;@1es7$jbfXWsE$is8w81AD%$j^xlV-)M*F86?5SZ)bAoOJ=EQFRXL(eZ+QOWMS z?b_k0z_aW{5}bit$?fob8{S(UpucFl)>gA#w)~PTJM;_`y^3^pq;D&`?To7f2O!*R}i&M+Sk#==ufNCjZVz;e;bFe45}>Ys>n` zyC2NvTH74yR?U~?J*%Sbzjr&7mD&s0Xlv7z&GFU+p65%C|J}dZA@d1hdfRLI6>HO0 zcQSTfU(Bk`)RDm@9RqlvqYwQ$(&$NGPRB#c>sY`e9gCPqU`b%D^VU0cI|8$vWf0J^ zO47d5Pl&x2*!w&cqkJ1$)uPrz*}}PEC#2&%^<3_r>#VqXIhf#!PVwn$h$&&fmZcR# zFF&dUWHL&>$j8pdW%>tc?Q_;R4;s@)5XLve4DAS-F>{3Y7y2~}aU4LL<0308FxwLU zUdAvz6Rm$Sf-AI`C8{B!)i^0)xXSe~XB8X4HO^`^9&xn@uaor#Gqrd=po=zO7>A@d zMEn?C0-rIe*2XJ3U&r4N@EF}O92j4bIN^Q&@`#^;Whvp`!WYQ8YLC dX%JIwxo**DVco_gLop^AtMyAfYgZ4Mg!D>AXr`NFic@!G7~40F1YWL zc;mG^)A-RqH>P!b zY#@bc12-{~#V7bw$E<-i%&FJBfzPm@q+3e*TuB8bePLh`UuLm{f{tY!w*}0STaAj5 z6UrzEIW79W6E1qPT6L-dZS$`0MhgPz{OG2DwixU=0vAiJ@2u9!yH2}Kv!Z{AuL6aM$IKnfx;W$ z8EVZEx9WeD?@6V>dH7#p=g)z;zzsJ4FNpuzxMMw%_R)&0#6{3?hm9dHaIfY^ZrRy% zt1b&v@ckf?ksJ7|Pge}(u*O1L_1t%>+%J_Y(MhuaeJ#uyS`x{hqi)5vwtMK<(VXJk z?i^PrJ9`3GTUMoxme$#cq%A1xi*PDKHuzp)F&#w{U*T(}!9)ow0=Hhj7hkEp3btxu4R>{XW8xmxO>E#>6PwsF@g44( z*v6KQ9TVT?4kXV`l6W;#UeOT;f~Z z-@s*Pba;!BnK)qpZ*wMS$@_QunKC&d&gR2eoFcIrC#i-6U4aq2jWS!;lxxz>(#+YHGA+8;!XFoNG?b=J*2yJ=VEYs(^{cqNHMHIZp6DWpjB7H>gUn+PxeNs@vEwJ-|Os Cg!5zo literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/utils/CrcTest.class b/out/test/clients/org/apache/kafka/common/utils/CrcTest.class new file mode 100644 index 0000000000000000000000000000000000000000..d385cd4a390df9d046c55a8f56995d69730e199a GIT binary patch literal 1544 zcmah}+fv(B6kW%bC2&M8P#R33RYL%Civp&k;gS}UkeC{11EJGq`fzMyERZE4OJsQO zm-MC2eQ}s}CLho*=~F+T4^7vR+{t9p4z{)TK3hk7ueJ7(fB*Z}*8mprERP$odvG_8 zc`O(x<)Oh=W>J~Xda$JYy*xh0{XCYjqRgtwt*PjNGG7?@(!jcazUumJ^gtk6oH`KD zHiAP(;9Ax7ot?NYyZd+io8iYr-JeBpPW1q;ziL~oM zs}=Zm9JyZG-U#daPCHr=$Vc?gP7h_olEURx5y;ifnHI=AFB696_=Hgp>+=HRmlrNA zu&mm?kh+pU-(Do^C)@HgL75O8rAz{ar?DToE$6^(yCkvh`#~flH}Ki~deZ7RrWke? zY~Ne36+a@*GwH>|t=Hm4!wCfjtH<(8+P)juWl|_7X~l9ch{L+G=_=vOF8e!buZeyP z2n?+I=hj{ny8e-M9>ms5=|=|2CN}U;VC+&gOA96*VN;pM0;B9;CD__lyBWmZp;dFN zsOea3*>X%gLB+t9iLX#Kux(-oyC%L?<{LaV@e~hD>|vjBU)oqHL#-0pp7f9G-P*BJ ze>au#66iT{A|(sO)rwPUwtC#1yit5!u2hoqR!p4QdyuNjbGXG83Q9VS^>*6{*}{}b z@aRP>J=VNgOw~_dq&-)*wp7i5|CkElNYCRDRtIm0P5)wFv4CkwtL7@+-tE1O=>=cZpl{K6oEL0UaY4&g(_X1W14F~k*5 zCm+tJhs7fI8O*)x&vcNj&b(gx4cd3FklX3{W$oV}H#d_V=|Ime>4W;nPv|dbGlP1m z1LH?s{QTz6LXP)y9(OQeFv9DQtNxz34tfh3%?0w^RW4mAxY-2mGOjWnA)Hb4;uA8p zFig&4{7&OGN*Ko~eHjy|ViMnD3QgWQ!%RZ)32Dr+(l9<^4HD{-)$xK=M@hODK0fB% z3~M*o!D8oO{J&FUw@5Dtq^l+_ z_;`-F%^TT2R*ef>*`8(8f%)xws?H@h`U=`?2UmGrGv1Tx?JOqEIhaX literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/utils/MockTime.class b/out/test/clients/org/apache/kafka/common/utils/MockTime.class new file mode 100644 index 0000000000000000000000000000000000000000..c38384e56072df158be18831cbd880d471bb57a9 GIT binary patch literal 900 zcma))NpI6o6ot>rt9BoGoRgBrQXBBc^b$9=SJJXDTTi66y+ zib*W^0sJV$c}^O&EDDQv{f2wKd+wEg{r>S2z+-IbDC2eo8t&*=!DOgnI+L%TOPmR>5bakW-;c1hR&FiK@8P*!bCURN@zng%Bm zdE+FJQEIEVx12u5vb*CrJN;MNz3$7tK-0N0pL48sUvzu3e2vgknGn=6JO9wOJ6PvG zQiec|CGnsAk($5(V2<=9m8f$(ylK literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/common/utils/UtilsTest.class b/out/test/clients/org/apache/kafka/common/utils/UtilsTest.class new file mode 100644 index 0000000000000000000000000000000000000000..fb12f9738ed5a8da45ad06685933c07af61ebb78 GIT binary patch literal 3389 zcmb7GX;&L%6n-WN$uON-m_{rvg{}q?NkX8cfoel-l^6;wloqkofedtLGJ}&zF|Jix zx4P7Q-}il|IUd^|;5mNr$N16XeP<@gfca42oSA#?dza_E+vJabe!C2y4;KaO!*L!b zc-$@E9>@Y_F~=jvL*X&cV}Zv>9%>V5(_hD4rnD$v8%{CJd{usdN5P?ee>p^34-bv0 z>da_~$Grj!6nU78FY!1Xz_Nh*Q08%e01tT2Py}}LpyL2Sl}I!3LshO^B|59bM3wL| z#fPgL1$qQ$csweg6OS?GarQXN9_IuM;|WGR$sSKJ!>6l;o&cT+;5-NakgBQX%^bX; zt|<<_eR@veupz5z%E3}$Rxu{!**r<4tUf2_r({EAyd(6PC)FZ{uB>h>MCB!U?xYeu zCC{IdqjP$ppli{RspgB(BkVb;6wN^n>r8szrkJN*4Gg(GgH4Xtrre{eQD3)fK2M(J; z*+u!Z9Mx1kI$SoD;nMuPVhpmI`)4hh!_^H{$O}a%ZpD5M&GFuWo*4bdNux%mlCfAUfM-QKhvx%$ zLBs{T$U!WWb9zBmwI0eYha1MT>5+qz`;SegqEYt|nI~gZ&{T@N3N!RNo{IM-`Z#Rw zjm6@r+-x$H?3WX%WG<14C*r$PNx4@^B?e-tfka=D!+_iK%$3+2PxbE_NQ!s~FN=5u zuQJ51QH2`73jZ&Muj367Z{jTu2;glI@8DeyTtvisc%KNmV2f4IysRxmCrm@t76SM{ z#E1BZN)nIsMiK<0t}>#T%7S8u_!ysv_!OV9$L9fjA>vEC%3)(#D|gy{I?H;gb6M6* z5nthJ5#Qij5#Ql^5kKHZ5r;7$;wQ%bj9*0jiWLqwUf~?9LOnLSsLWAsZo2|wYOjUr z?Cf-U*>&~7F1vT#B8tTv+jEpI78N!+qHGcS?=8uB>fIaNu53JmE>DhxO$iHcPv0sV zJ7cTDJDH}D=T*&|lJg~+0$MU7RF_aDV>vs`q+|0O+Cn34=ylum^Q-53w;QPw)rGWb z=yv94(O_F`z*bUppPtWC((OgltQ3~aa#m#vLJB#imKu}M1$>LvLK$*mukql{RD(R! z!rP5%8K(_HE>qt9a?uXEBQ)c>MK#~H$c@A+vwh~HY*8AUjr#Vv&cU{3v5ky$v0c^P z_|!VvXk}%!9hE4r7F5ayPxW>doG)bQ^Kw2iSXbydipKlX53D+Xe(d@8R9tHLB<}-c#hu8 zxJ&X#-U@sbG)aCb;848O>`;QV&Zg=C+Q2@(Tj>YDHUzO9o3R5S>~sa}asp;akHHzW zf*-5}Uxv>Lz9$HgMZanV8_reG5=5)C(ai=Kj{0evw5iUn1=NiV2osnHZleDo^tb@+ zX$WYn2Iv8JEkJ3>D|uQw_Eykd!L=+W8zp~h$AvoIEb*ngZjx}k#23{i*g5;~{7QpZFETWZp+{!7?4 z?WH=lPfYuotXGqty+pN9j>2ZB?M4ayL8|f?x-m}h4^cKJ=~73K#S~_66!SP{U8&QC>RgFHlim@! zNE_Kqksce$W3@b=C7xZaTN+qFG8?{J!R}FpD|Ixog2AKRk&74#SFmRVH;-CO9o<&& zWO)~*_&BYB6OeE>u7Ql}k)xSa&_$S7l5IrY4kACDO%o3KveGCqY#zFUdy4O@p*wHw zBjAuAcz>rYaSJ6~5So5pH|-5#@A$MYi1@f8CV_PzG!`?>X`kR|*DqnOnngE4(C literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/test/MetricsBench.class b/out/test/clients/org/apache/kafka/test/MetricsBench.class new file mode 100644 index 0000000000000000000000000000000000000000..9c9a086f8fa544689bdab740ef71c593a913453c GIT binary patch literal 3354 zcmbtWX?GJ<7=CV(=1zx!N}z>^(1Ha6*rY`T+X8lTDYgYotwdC~q|vLOK}B5lF}6n z%W*`(sEP$JIQFSn;&Y5CNONRVER!}&4ok%f3~*$tupc&uqreTd_mE%or(zXy3dTe2 zJ5LuD>hXZIbU?*H9Fo-GDxw}#@DRttDjvb3DjvfT6-RN5<2XZTvtb(UR)#=y?I1(2 z%No)ds(TGn@6ToS>h^%xo2Hd;uay$%L17zmURVseqlUxK&}-Qvagi0NQ9V8;hQ~zQ z)g3q9r@OY1ayoT0HQGkHGr}+#YNNY)_la>4PYZJ-o*));q)plmIlAdsHbeAMeJLxG zvCMeJ*BDQDoTQT(^xs3eg*`%^EGna#q0(@5n|9eY23}CXuy$4}1^>xRC@XB;bV*fe z)JPA}=(xhBxuU1jBJ1e1#JU;gCtQ&l>l0ZIAdW6d7)lV1#~IXwm9ta&Erx`1VKFu_ z35SLou#w{l4Nu}JhHEZ?^dblf3N&oR35GSZvHOTg?x#7P(eNyu<9J@f3wV*d)9?~r z*6<2mWmq+nD&Jzi$dLa_E{2hT$8DklJ$PlIk=3`2#Sax?yi=2ESPyQ*#XBtl8a}8hMOATM)Ylh{kV;hF+ zO{YGq+x0$R^|qd}>>(<3|lY;pcLJzNBiY z?mc}uI%~X#sz|u9Ko?vv(rcL`3{_Mgj_%Vn(K6*tmAARv`m8m0zS8wF;=AT5K6|a7 zx8noo9k|1`#X;&DA>s5I4&~qKlGf$p+qTv}t|uYcs9pH@oL`|fAv=(dU2U{jU>Z{r z;-kXp*AI{xYQ^_Z$(Y_`@if2JE(>s<_6FSoLshR$4SOm`E; zA*u=n?V@RqY3g=YS~!kQ;Z?e}_a_EA`Ue==XLZl-O+^i$#1tVIof8b*^1?e9?dk6R zzZPe&fs0-CFm0aM&FMDWWo5F|Nr%qIp{Tbc4mz%$A?!?HS_6i>5-4k_%0qrLq6Ce4 zXAD?5%IF$DaSSWIgRXm*QcM>a0;@=4x0UlA8hl)&bNcpS5+V&n3FkAE3Qgx~XxOqd zf(*5kN!sE!L21nvl>q5o&nC8_xgr=hg+ z2%o}&p2W4A zLV=A+jdHy1m3Oc(60A|4$fNO?!cLz1i*{>6`iZ&|R*P~9v2dI~Zb}B0DwAj?P&edp zos49_i{zbHO!NwH)X{ssp5E{)5v3>jAg;y{G|;8H8h>I9<-j`Di1ln6(S5jN@Ur4w%7j7uLr=Avz}ZgJq8%Odl&5S!C%VWcXHTOWH&d=u zv!l2Lw~|Lo*kRm85@(l={Bc&mZ{vgM8jve0b i4&V%{{ueyR&Y*_!^-kigKms4osFJn)bjtVJBmV(I7NS)E literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/test/Microbenchmarks$1.class b/out/test/clients/org/apache/kafka/test/Microbenchmarks$1.class new file mode 100644 index 0000000000000000000000000000000000000000..5d4024f03f4a6d047c52ba85556bc10b73a9bebd GIT binary patch literal 1745 zcmb7F-%}G;6#g#RWW%y51(AZZqJ^4-;%aN91+)d!77I!ZqF{gBY;Ix6W;e6Dv16a@ zKhhVU?MR$yXWE(e&40oF#0TwnH_Rvv?a+subMCq4p7Whwo2M^+{T;x0RJzfJ4>UQi z$%mR;u+Wbw3m@TQO{TjsgIN<7Ep%flk4u_=qRFS4%vqSnf+m+OT)|ZnpPBevAb(H# zV~JN+0;Q!e+H_JA1A85O4}KMZvDG(3(bZ%6(m7CB$jZ+N+#+hi}Yo`nlxXmb1LE^|Htb49Am$LaV^pyY#Cl}O3)J;yy`rL3S@EgqTMAw2ugS>M=Ut_d zY4TJU8YD;|rTfD;ub&a&r5VMPV*R$WH5c)|O=>jhpsx(F8B7a@dSqb=A`lkNzXT8Ql-&awTy6F!qAYWtD7& zRTE#>Si-W6qZqJp41+e_#E^|)9JeuoaTC{V+`x*BFLBdE*~Thv2@Jk!Z#jwRZocXH zRSorUe4j~0Vc`9!s?(!3)^J;sJ2uv_VdAchukf`d- zij=JGQME@+pns25_(QvG;|GeuzSpJ4MjR{LeQku=Y0l{06gvF;%Z}*A`ARkDvT#km z>X zAC3^je_ZF2u$?5Qr_gCg2ko-%1;jCJXm~y=Hn1Ncy-0;coqaLw%0YIXC5RPEU=%ir zYdFgz($!;`=m#@imn3EG*|C8fs@Rtveztd?3Z=Q06g>As)Vb_6zdAp?m!atiGNW>|GC|O27W8 z-ol|44!3ZGX2)CTrPeT7c;gYXc^B`&VSI=G2byP~vA<8m;F9LC0*iavhnvj5OzTw? zaSKyCU&L)xaR)VSVT_G5&k_t)vp~B+O3Z3EV+-3jO&vdUsPMkX$R*yNgz*9c!VoW6 m^sI>qiY5_}Og@MCoW;)I3=O*o?ku^1a|Cyal9f86fc)PZqOBPK literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/test/Microbenchmarks$2.class b/out/test/clients/org/apache/kafka/test/Microbenchmarks$2.class new file mode 100644 index 0000000000000000000000000000000000000000..356b7d5d82e1a481b47487fe4a778056763603cf GIT binary patch literal 1550 zcmb7E-EI>{6#m9GS;s6*nqMGLLdcKpBy3A*DI`!5C!q;;pmm4@RR!bqU~jy;W9^Qq zxaTQ&0j{YUBefEhK;oj0&_}5%!kJxHO=?<&i#=y%=bZ0+=V#{kx4*mua1nQU(T6!L z=CwGVK|ibvKE-DlT)^kOWb%c9FEdEPO5>8&Uukh!iz^u{V9~%;1J@YRkGMNomA+t@ zE$;9~+-fPuwQT9ztsoG-vbd6-V_Qp+T9(ok++XN8D#s(od{G89i#NGl7gmEmY;f}T zJn2VRp;dkyD&cLB?caW`v>OZqaX^>*HS1PoN7#zadlntvQN(N~`vH6o&!9VIZkr)h zkX6d0uju$a4 zJCGIO+jWlzjd1b;Wtj|GKFO?(ZP>EqD-kfTb%tYQ#qCCkH=|flo4z)1ogq_}t-uy* zj;1yAFVOS4b;N(ww_O=Jeyt=_T~-Y&nOH`_#K#ygF^C}(!x%9!ig5!gCRVX#;D(8B zux?<(#7z_#Mx!~1+EP{lfi1Tjx2g&D7GkPb#!Zy4sl_c5xACok?@W}j#W2$~3-2!i zL;pWvw(5c4)dRocfnXRsU?qQy+cI^iDAZSvJ5=bA?DzZn9w)V+YoCkI4>R)At(s6# z&+ai~yW;=H#4{WTT_Ku8CA*;`?#nv97X&F=1Uwuwbfk5gYOaW+;GVt^yx9~!6=%M4 zA@((9A=i~wr&r{7D!cxF$7vn!q@G9CCrS2@HShz* zNSe`l2&eD~?HJBryo2=y$p~vTxtRYMtc@d2F+rYC;M5(TJh$c|LS@LBO(il>IA;I=2K+?b@GcW}b(|;nBpe$x6Kmu9h SNT-k|*weJikaHhtApH)Z0f=h= literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/test/Microbenchmarks$3.class b/out/test/clients/org/apache/kafka/test/Microbenchmarks$3.class new file mode 100644 index 0000000000000000000000000000000000000000..91368f86148443880b61126d2fc2b73a5299128c GIT binary patch literal 1772 zcma)6U31$+6g_Jzk)x?EmlVGDlVFM&B7>_3V2deV1j-vxyz9!g^cu=-%1@ip^`cfQ3f)fX2Z>v~ousO@#knzCIe*v>QP7l^Mn0l8nt}S zUCoxYC<>Ge={UUNhkkld;5+B=z$=5F7zAzb&4FM_+XBXVRA&xHH~dgt>$GYr-jcO| zJL61L2HP_BwcXz|(uPmw>`EA_cs-CwqG*;5CX%WobvJ!4j%q6O8Z8+&li9OOKOc8O zN=Lsfd;!yoIw9*JFqEV+PPxnS9;v3%Yi`PR|6QvZbz)Cl@inrEf5knc%f=Kghh7jR zez?1-(neG_@s^DZY}z=E6E>c}q>U+@v@wk*ZJffKiEB3A#&sL-;D(8+jV;XCcvqin z+jtM}3!KPWB9P&(TTNreUhDWlUBeyLjgRq(7Scq`hKIVq z+<_JyOpw6X-w$pzVkPN(`cTd+WlQ?x_PBGa*8`8BUeE7{UBBin5i zl8OaqKxzlJ`Zc2CH1Yq>aUqR*Rb^8-2QM(kv~&oY)p4Az?BW0t?c1!WgGKGj^8Of1%VlAmuL@hohr}o zLzJiYkt;9kBVV30_Ff0@V)!UcTu<_L{A&JV~8LB{OumBpD}v#0qpT3T|CzJoKf2A z`&Ji6yBO=@ab7#w#W;5jql;r2V?M*UNMt~Y0y?m;hjDz)Y`-AN+i>tDT-xXH6({d& z4%#W`3l8}<5&;}d|IHW+IHu5%BE$w=w6*Igt z^fvGd`od$M%QR*NX8HsAANo%^v}a|{aIs)`&^c!>=X~F}tp0xa{4W5rxLw9BOc`<9 zh-n+cn6csEgpJQ|(#9#w8gaUeGdOGEoDCZ@C46r5c_ZeGn745O3l_exaFL;OkB1Wt zB?ZG&?FW93d#MgW&sWk<<5)=Tajl}j_ZBm?q*N%loa;I2AR@KZM{ad1A6fnH#E;O4vK z68&&$`9brzRMr^=mQ{mN-&qT!xRyp6B3|PgA?*wkT^_FUI56wp=74Sn34^mDrHGe9 zo+N^nm40$sB-&dI{8()W={F-Dw~~oj%Dxz}EnlTli5s>|i%W9E@Vj!5-{&un*%FzI1R2 zmo0qd;A^Z{xZ>a{#vRnm;i`jc_=aIm*2<8}O|PzFioBEtVZ)FtFL&vlKkVQ-ZWwXX zK^44Ti}Na;BC=JfIqmx!-kL{TkOfQ`SoEFy>7a zHDXiftQ~h5+&=twOh3c6Bov}eRNN~D(alB789DMWyYvWB*)?9PL-%(641so$+P}&t#afmFN KF76@?l>P@cMxW0B literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/test/Microbenchmarks$5.class b/out/test/clients/org/apache/kafka/test/Microbenchmarks$5.class new file mode 100644 index 0000000000000000000000000000000000000000..3012f2cf0b6bf1bce5075ce7866633ddfb63296d GIT binary patch literal 1451 zcma)6U2hvj6g}hJcpZC_rb$C-(ozFymlBFy=(1` zE0t#?Bz^`63XdR?5|H=-`~m(85+7&Qk>aKm>cieU_s;z|d*{sj@z1Zn1Gt1whcS-x zMtDYCFyhrBCa_S%MO-T3GF~&{^%CB|o5S>7aj;lK2@3^WHTo?hmW+72h--Mq!F31A z0>k%ZI1^}D44yrdce@Jfz5i5YVMZIu5Kkt zHW`q=8bpC!61Z5Y4aAy417?9H zizq9mx7ZdKO0`Th*S7>l8d~~oBH!;;Y{b39S2qKL=h&fY=grUv{6^%5aT-Kht4g=x zrh^q1H*nL%F-*F64#!%Aasszl%@(Uqm|t28a^prfvoT{N&}#78dHvEkrj7x(aq!0a;)g=c)w zZY?`jfiusPOr~VxB`6N4QaellGvtBdjR&b#9VSIGir0ufI|vSb^niedC}c$%eBR$B zQ>`sU*JPzO*C$Xmmphx=$~SBrImpdmoUJ?=_0&Ww?wPXv=pgp)a?qONRWtgQUsc0tGe3^TAN16g1q`=Q4H@w#>mypY_Td>Wv2T;NMF z!8gPn8#B}csL8%elYp5f2N{v$H;t1JIK|Zp?IG%^>humo_1q3})%hJ*)f4s(?CK*7 z{g~yvK$&L_@C{B=y4iXPWxPa7;APD0SKMQqP2a{Z$X~OTPE~(~_!_fRIewP0?L9aG z-8~d?*u!uR8(&!B;lF?WgyL@~-FpOgd}J4+EO1(VW%9p_cX4DFW4kzN;8-wlrq~Gh zmd(DyC?4V*^#Z=(P2vyloNNT3e5LCBao|t8J}STdl3FwWnCE)@rpU73uHIY?2Mx+UWO{kG%Kh|Ni&& z-o7W^d*ERJ3&cbXQ?N(F)wo8(wYW~h^|&FQs~a`kgfFW2k_Hufyx1$}o8_=i4!6kd zFKbwZugKx6Ufe3dU-RNNx$|`u-_TGYIalCz3Aw|IJLSh+8or6U<#3M|_o}#0!$j;+ zaleMi=vOhIVo<|W>{s!ChG|G^ScwPa@Q@b|OQdh9ctk^kB>J|7S@@2MM>WjBV=BI@ z;aJC-?@5!sUx3H)123L%p}1v<)bod4JgMR-4aehYFMg!r#~K#nX%){%)Mqs`<0o<; z@8>kM;HL%n8J?HJ&kOJiydZ~Ps(4ZQ>Q^d$Eugo>VrH^6Vx&@LO2D%?91B~`0+T$T3a*B}*%kK=(ZS{o7V@7*C*&Q$vMySUO z^cq`xjeupQtiYOZC>h^k#zH+&BiWm3BKxQjjtNxOZ*JdaY&QZCBi0@0u#(|ecasz^ z!u)F^o{{V(w%H|ccJ`xX6tQC&Z)`qe6gyPX?JQk2FQh|=0Da(v*Ag(m2D;^`w zF7;{)9JF{8`7FcuSz_$ck*~3rD)lBy%QpR?-UA>i7%Zmcu(b{))dzvcK!N99QVL z5?2V+ICU{6ll!th9QEsGI>i5AMQ3ngv_K*_TW{i@4AzlFs^eccD5dEP6|XKl(flA)eptw8;lhV)+wSYYOe*#A+tbf$0(>6ultY=C3L z;ofmvhXF)I9{@tKq(;+7_p#!lHPbWtKorD0{`_QOJn zUd~paiwfb>MWv`>Q_#gx!mo=7qIy*7IITk$HF#BE`WSt4&9mVZb9B!Wi zMI%_!5*{WafHoeTJd;?|PPA;=Ov`3$5=Js*@>9T9pVgX%PH3LYPL(>n-|kSP8Y*V{ z62UIzC}88PF{~bTFWEq36I(Wu=d~-|Cj~hC!|}jLJpL_;FpVhttz*+@Fxz%QB1L$r zOD61(9wU)3V_gEXb16DT_e?J%{~FT2&$ic!JJ!l`XDlKa#(~XJwosb zhvg=@-CIBYYfR>RwL_MV4xWyQn3${qPF?RObn>0F6kC9r-Qnag(G;Ud! zN4A1tn@h)Bb@=0S760VFJP3LF;<%Py0nS1~8wMcm=0d?bjvgCWgOj+{9R!?=_58|5 z2R0Ck#CdhZt0c+g4X$~EP!<=p4Z^j^?P~P+Jk|S9QeQs^cc^W*N8Eb&t$WxY`r+Y} zhT1s*Z&`jnG!j(0&5Nrqi1(wQA9_Cus|QiEkwfvu{6!v{VWmgHj78bRCBwwk=STZd z>d=m$7mr3S%ZW~9Dn0!ucQty`sGw55V1C8~&8N|Xa-Y{vH7n0-^zPPZ#T%sXdHYe7 zWyVoCS(m!;peLxg8Vh^{e(hnh@@rg?ng8V71!VQQpTRL_tW8Z{&G2RHAAjuq;V{1=LRcWjXs~Rq-!6l{S`jnAm+(o{(dZQ z;ndx}jIm`4DKxLF(O)r$MN*05obNP_@AOv;AeaF+4S^Ti;1dXzdF+;XEQ-*|buC|1 zrZ97-q6kN$imxG)I8VcL9D_QR!c1m%15V}FCccB5f!XNcJIF;Wj!XFtat-E!I^q#5 z#G`1$lURgja2#IX1tPrPj z?!t*8f|cSDtP)pawb;w+*=@W=@5gE4acmP$p;tVMh{WqS&etIc)0KIdl}}bVJ+zVZs&WxJxy##uasfTI zncbjB*~Q-jr_t-1lt!G+l?RW~k~4_Mry}tf&V+${UN^U63qq83g6KdOcMIr)gD|m` zznb>qEp*fOh4jZO2xA+H=#}TuiwKJ8o5vC5s)U}n2{FV`N`G8}1Yu?L$~icTt8)6n zKoTjwky+H1-75p>BV;=v322AufIJl+7cTznq4=`HxC>qf;1(+QfCmpHOkOichNel>Pslp~a_dHj{8JrMl^_&p?wuF5r^gQ*2{a zs@%CKjpoi-`>A{YXD1wY@>O;yb1d-(eF;9rL%h0Y`=Q|t^2V|VVUHyOkVQ4 z_cNgo%=72%XLjK_RLRweg2}ov=<&0+)!lmd<#C|RAlh)*uoyHdwwL8{Gm~W>3+NUs z;2gy54E-H+@SQa3E?mgxN10OV8RQAXl1Yk`^1M+;Q_KuQ-ueq nLYe{|`j9Fp_ySc`a1~Z_ZlHNBoWXF;$;`7d^R&!7)tUbTl*|TH literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/test/MockMetricsReporter.class b/out/test/clients/org/apache/kafka/test/MockMetricsReporter.class new file mode 100644 index 0000000000000000000000000000000000000000..b733f8f60b70b6303ebb2fac05c33f82690ee543 GIT binary patch literal 1503 zcmb7EYflqF6g|^7r3!-b5JVA_wg`jr_|gW0jgrO!2}O*b%(4t+q1|n^TN8hlnng#T9qG;Kcw=NLP=Jo|*CFf8H3{)&zZMPc-%H1;#>hug%9Mh=p8?L48jx**RTP$pn>3 z$A(=a=2G@Xlr%yjKoD5CQK~@PtUE2%?BgsPSv6nNP*~HJ(F~d$1n>q2LJibk1!gaM zih)_D8d>Fy+OuGm=WM> z&p18z>4{*1r%62Es_U%!KGwbzUwvPSTU5CxYBxE8b9g literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/test/MockSelector.class b/out/test/clients/org/apache/kafka/test/MockSelector.class new file mode 100644 index 0000000000000000000000000000000000000000..0b32da93024c9205d9a55b40c582750e383912c8 GIT binary patch literal 3363 zcmbVOe^(Pn6ukpUmcUjD*59;Z1o@?|Qm_h${P7onG3>OX6Dt~4+tGrRG)`en zns20eB+X-Ka?-4K;0ZRQc`D7f9e9RK4c}?_Uc(Op3C}*T1p2a$yK9(-W`55y_RXDr z)5tpq2TsYTcy_UDtaI0-z$v?Ad$#FWg`8C?lm$M{zA#^!YLk(*%O3ZnbM|h@^eQgH z-mQB*6`Q7Hd9NII-;i-9r)4VUKP*~ds;=6?V=HgjFD<5@{owekex4wN%A^I(f6WVY;@Q2a5@B&t|zzy5w19&scHt`#Gz~0vwmrX#q5W&j%a9`zz$z zSfa_%)$(Qv1=lK`QJ#2=+nI`Zhz|<>q->Mu6v<}K?GGBDc%)zH+y5v?S7Z{3f z#<9~TLtKmP+p(>TedrX6d<-s2H%+R?sd%h!nUoC<`K-uZSLHC#u=oKY(5q)&%0l4Q z$rY{o=Ud50BD`^8VNnl(`Tqeoh!E9OXBBv`sE%UX6;q|SE{-BNI5MpLzZO!3!KOB; zMQOWU%ZY=E$Mch_l>C7vA2YJgX@5zL+jCCE&0CAMlxbH~b2nvF9iL)U;BN4`spTzHc1}eDw-|?qE#Z8 zqgn`v@>ZJ~%L!nGwE z-;?Ru{1WFR{n%WvYoBrDG9&(qRz?M8hTlO9|B04o@6h@diEx+xHk*U^C+D&m;Gw-!++iUG!>KXVSL-8zOb!lLEcb&eAB1+BnM z{tTk@iJ9ka8C_=9Vlv-m=FvzkdP8P`+uY$t9APwxMT7;U)WRJVLib+=A$WP5>^RZX zk%|g3Zt*EI_OpiM5jv=mH%L_nT6NH-1|h1mipKj$W16Su0WPs+eQb|xnvb%-hH@Nt zf@D#`5;24=kg&xV!p8B1f_oSDNW99DXj<{UGr5$sv9^A;eNm bcCkXQM9Qe55In3^y~aAeZtD0@Rh#`6((07= literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/test/MockSerializer.class b/out/test/clients/org/apache/kafka/test/MockSerializer.class new file mode 100644 index 0000000000000000000000000000000000000000..d3fc40e230bb94381d701bfb5a1ec5f38f0c73e8 GIT binary patch literal 1400 zcma)6ZBG+H5PtSrxc01opzwHhEOUXFqhD*zJjO!y~}HWQg@DujRH> zd_Axr(C!Orh7M7GV;?cFEr#3ebdG8*N|Wtw`9Q32r)-v_C!BIFy+}ofmqgj&(r1#q zz~tXPKD~Iol@7x|(=Cj>y`0)!CZ^J@2}>2tZ%o*NO1#S@XV82@tK!zF7%d86%aHH- z&Vg;ooR1yLTa{+ZUSBr3+zHFAR$a$z5$95nbl#Qx_OhynQF0reWxo`v*M04cn^8$5 z(2br161bkg0-hw$rB36RU`Sp0^(m0243k%RE(|^A#S03+Fnh`KKk3ms5{~8BRT_vT zXZMvY{g+;#HY5wFO-#^y8K9wnhC!7bP2-@-jrLBur=)4ULDC}ajP^wO6zmw<2_o;m zlB}VZ)-FV79Yz!*gf)sjWf|DxCT@{LZx;Pzt+3wGr|g*mi1+|17;x8j91oODN98&| zq?Ddv2zLl%n9c#b(v8slDBWlnqZ9RnRu$$s?JA6x{o-pC(IC!rD2{d(N2B=n$VVYf zVv1Ip?Xvsm&@U<*N<0?e_AC*j`{bz@%@ZHR$Vg|hztC}l&Wp?j yLI_5aSq#ZD7IVHo(EkJahsY^n-=LizqpF}ZcPfNWYLL1yM5mxUDWplGA@dhY7$*t< literal 0 HcmV?d00001 diff --git a/out/test/clients/org/apache/kafka/test/TestUtils.class b/out/test/clients/org/apache/kafka/test/TestUtils.class new file mode 100644 index 0000000000000000000000000000000000000000..89a43f8124bf7ea8503f079d3a0b38c4fdceef0e GIT binary patch literal 2885 zcmbVOSyL2O6#jaqXL@KFR1ge^YfuLe+7%a!37a@NEHa27Nk}s^&@yz7J&lUl6SFU7 zPo90rYm8N3LgI5OKOj$eOI7|uQe~3wb`LlPD65>Bx_$52&v(x`-M{|x<2?X}@QsRE z45=u?u!4&*jHn3XSq0Cjh+;^=^D1H(EyfEl6pX1zV_Zcs%pz(`sMv~01yd^43t20M zODd+35%d|cvjxWyOIE?<7+e)Ocq*zeD-2vwa5cg(_WttQU$=zPHL(l{)|`Ut64XTR z=sUDEE@InS`Y zK_fG3#_*zqXh)(eF_4PkC7}~&kGA%7(1D;+C)L^6(b+NDPhxNPvf)|J%IN)uJ?_lV zaCC_qJz?3FcTz$)UNL(o?2BZA?rXx@YD*eSph(y3Nty`W!RLn4V*|XJ&Jr=}KshuS_{nx6)SH znVE5HJ+DVw5zon5X+oPdT+i|>VkALINGMJRwl7%Tl!PPk#A7;;EIP4?1nq`|@ogh( zhSftc5tE38lIsJPKH5nu=aE(UFm!X-CHRj0Pxi!`PWqM-P1qAo%Sjq7NqI(ky4%S5 zSD~Oy!OH}|^)I+>j`%aCc%p8kjhC9UuJZ~bV*xoNXj&i{RGNeeimn8URgp1mVml_G zY6ZJ(*2;{VZVS6dLhM93ldmK-<;=QiQ&c!Bez^#mMCohTj{_Q>0xPa&Mb+eQT0knH zC?hIv%E@^`=@kj(e((6mwz{q{C*YN^Yb`4eVJ1wxN{BSPhSzzYYboW6Ou-u(-oy=| z_Lhbw=o;R}I}$eeF%&du9)#6QKHLKmPPVqScXXcWI-NLkHrd_N+jp)%HE@3L!qD(V zV=O&xPE1Z&m!>l_wv)Z==DgV}SFg=o-?#t3!9$0ej~qRw;9U*x;e8Ds;6n``;bRRq z@ri~{@fq*4aIg?|acr`jCT}J?Zn+9R*YE|t6#0CmVHb9@QXjw>j0M5`TEjl*OH4n{ z-myz&nm`|VobcJC4>HJyr7t%J@m#$}If>lWro);I;&f?5yK zZ>hVu1j>T~WQ#?OT)qtBBdaI{q~(0sw|d^TPiuEF8Dh%;)>0r@Kov0++2%EGxuZwg z1TDC`f-+%(f=pzOwa~dYg^y zIZH!gV|=9`Qz{H*j-!2sb7<1^`dlY#y55|G-GzAO-)*Z}dIL8dFm&(sZK2j8i+ zeAPzz?qvmYSA{)@LqZ+*A=LBSP;lPMb7R4|oM(2jug4i-^(6oY+OLm4^Tt6a3eyM-xpCl+;|sSmArrw z5vS~@xtmddsYG}xWwhJ)+{HwjnCcNG+Rt@}=?WN5Gx8X=;AuuL6Tv+nWwTfmMxC>4hV?7V{~MGbRqa}Cg_Wk3z6!Cq1aNo7g=2c2!T AMF0Q* literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..8fbed45e02a02fb14646766d67aeccb3673c654f GIT binary patch literal 1455 zcmb_cT~8B16g|_Hez8LN5X3J~3zQaF5Z;O;1~F+;G^T_HeHzM8mM*)^?$qdC@mJu3 znD}6P_D31d>@G=-eXHGc=FZ(a=iYnnz5DC;_a6Xuuv0(|7RJ%M+X|A5!LkRPPo*N$u7_^GcQ3uN~FQzbCjbUgLCe|e&UBYEOd zGu`yt(mj@eW6p^(V^S2b8lI?w{tcZs935EEG_*Fb>La>A*$=&kopoCO>oW92zY`?u1m?q5~g`F5C+b?AIA)433HpW zfsHkD)MQM(s4+Q(lQY8U)(&i9MFEfV)Jy z7u^sOrI&=ipTKV?@P%saC&sRkHMpq=cQA`MzCwbT!Gp-^Ar_;336HpE@R&JIQ8M}E Hxbyc1;Y)yk literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$2.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$2.class new file mode 100644 index 0000000000000000000000000000000000000000..418a039957dc831cbe54e9ad98974a55fcc810e9 GIT binary patch literal 1457 zcmb_cT~8B16g|_Hez8jV5X4W^3Z+FBfd^xe#2_Y3ipG@ipie^?%F<=G*_|5sEB*?6 z5ECDa&;BUmncXF+v2V4T&fK|s=iGbGy?1~8{{92NCN>Mmp6xC6K ztJHq5uR>iX+?L}xdRyRq7N~{U|$}( z)J!$~wsa3<;Fxow%$O7ftcK^QV8@kVsCe8RJ-+ffGfsUXc{9Tz6`1`yob%pQNXtX) zw_=?u$#;W1cT`Z{Hk2MmR0}LshFBy>(XEyaSS(%(~&OjM{3DXbW?=$r%`V1j_3IFGfNUcsbI2 zGl(QabK5=Z>GPOEfpo=8&RkOOe`hMs**=hNpG?!VA&}R8TphM|Ww0lbnqh(8kvzXN zqEs?8Q%dtc4h!rvN*|DlHPzZzi0aBWq`pK(*SThC1U$n8*M;O_2~#{72m@!{k7F7$ zgtSB;2|Di Ll9I_U$DO}FG$etv literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$3.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$3.class new file mode 100644 index 0000000000000000000000000000000000000000..900b85ad183a08f63936221b7b6e449b23677880 GIT binary patch literal 1454 zcmb_cOK;Oa5dJnv<2(#W+R{>94G`KqN?k~ADV0ztQlz9sq{;zK8)uWaaqY<7Aowf% zigKVx91v%I6k^tKq#$w&mb^PVp800x+Zq4*{rv}kLmVcdp^!oXPf}RHj#_p#6jP)w zX((%`FvR81@(TwH^``3$3_j-OP#7cLA92H%p5T|3GcZO|(XbpTJcrx9@!GOU6ukzm zT)N)5@MVqQjx5KL#|-a^{{gRg!9Q?gA}!mP#i^A#46(Y~6AYQA<%rYCWmkCTylazY zwdtDN?r_gi=TI3_B{Jv@#}QuL=DsiJc(Qzcg(_7X`AFogFUpi*v*r3k_3VkgF;^)pjSIDX7;d>eix`+gBYzop9oqw(C>% zEg^?)Ps5&$r`Xq#L`p{*hK>U~)A1ZthFs8A^F-^|#BG{^rAMGZ@Gq7^VR*TezZyh_ zw509SZEl-ZXSjkCF5qk*TI!`3hFv`-aGu!00BeaWVp0$k3V(9kW=alY&ri>U$b%SSOfU zBrDjdW`i^_bz7uSf!@+NLVK=qjc8UIe@23&*cH1XWfUr0Dlj;saBqT;3W6Lk3z@?f z?hxGFz=LJ7uL%EMh+hryljX`!tXw0ma3cZkd>PXOlObjd_XDd3*beqQ9?~AeBRr@7{;FsNlpj{i7{$JYb_RP03j8Xs!$Xu)YJr8gJ?k79>M_*91?nR;>ZQJ z9dCQpsWbJq7rnAKNFDnD`c3*JYM(uO@`uVyJB4N@`|h9TeRub_?{4z@pTGVF;4;3B z;xw*k7>*)yLH6VH~5hCOFEmTdc-L3hn!jEQX7Sxp;TMsdwdZx|~ZM!MoUrm<<2R?{1*nzl-= z>68q+lAf|`R?X!MOT#TYvu4E|5omX5)foM_6x*t}0lM3_kmRWSoWp245O-T8(e*ct6nvokue5MHR)<| zHEZ*(!-ad&?k4HqLpI6U^R7|c$QfH6Pw#|s8m0x3cdI4W+BD~^ibdSxrBc~7T&r9n z?U+Y2gV)8*p^HQ zO2>UX&{4p=js-l_u&Cope03x%96FLH+{5^xY3;)^v$W347C6>GB5Cl3HacqADWE+v z?5a7l!t_sNdJjnKT5Rurd2E?tVWXmA_M{e?eALt=JoTqubwh~^S`P30S z6HWK39$LOz+L}?BHXjoM-?4c<9wo1ls4d3JW`F(E7|VPaPWl(Rc})7o~)^O z8P;_rz`HohmcV;-4oGKFX0Q)%6IU>90Wdxc1$CEm)3;?aB|6mQEH!faAH{Sta% z7wy-B7xr*W;0Gj(U08U{2BdgXu|NOM>o&4;vti>n<(n6C05GClw@i#M2^$obv>ozE~S zU1b8@U=G|w3b#1<1j{Mh=1;uL62&Lu^Wk3tpU-LXQ+&?W`J~?9qx6!G(n~%{FZn3_ zK0c{`!Dpz>XTj(5?0-(wL<9QC!$ZHrw8=5SX5lj3cK8$XEgs-|{=oc*hfUC*)swOY z-L6A->(F=JfWGJ*l1fT>ND{izOXy0!5BknOLJ!~)b0NTNc#F1^VG^|R6IaGdQ&#^1 DsDWYk literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..15e196920cbd0662d0f4cbaea60f8e7a9da72df0 GIT binary patch literal 1298 zcmbtU-%ry}7(I9EDAdjkQ2Z^ZQ^tmrB_=*V7KCU*9BMMM7~d}Cx|Nk~X*&}C75@QW zeDOg{d@wxwM;TAM!b*r{32WBgd-{Fn-0$4)_V=HkzW_W#Z3;Rb8JI-LfQH99p6FOK z2$pp$>8R+a3QU9@Cn&87Y&3d)+md~0ca+tYAG^{DLSM;4$7@^N;m~rtQ2Cy81M77& ze!1yO&+hE1AY3EYy5l)vU0|zxS+b>;fVS1!R|18GKp9IR~#(_MNmMdw=HmztYe=P;k5}LKP z^g$roRZN&oy*!O4-H~($aTz1FBvl~a45i)Ok^Q8s#kA(zy#r;3X~_hMB&V8A+mqqI zCv`g|e3~@I_P+DgzGKT!t*7!&7oLKW)z;~a(d-R;TfK6kjhVa96)RDR+{2fi?e+q? zv!lXJZ(ql{UTfGY1h%ojqdN(1G(@o&8OLLu<=;e}=v9vD z$nXM;O7%NLN8|;ePy_r#;|j?g1WRh9iT~I-mUg1=DOy$N&HU literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$2.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$2.class new file mode 100644 index 0000000000000000000000000000000000000000..21878afe89d0fe3f315e62b0f08a6fa3ad48cf09 GIT binary patch literal 1298 zcmbtU-%ry}7(I6@>!_U@p!i!+r;H7$qR|J)f)GiFLrq2&Vv zFFuHg4~A#|DC22YpoD0aux9POr{8zZ{m%VvfB*UU3&1lx8HbMh2J$Ey(C|RVLmjgQ z!HSMK9aSAQfn3;eg7Ra5l}68RTe2_hj;1=qtJBcx|gY99oVSD&LcCV7+d} zFIRo(*_|yFgiGXFc04Dn3#?VnOE%XM(AIi8N}$+qJoRRMZv*|g@{?OkPuX~_hMB*&Xh+mqqI zCv`g|e3Uf8_P+Dgj$_MEEvNF27M_BV)z;~a(d-R;Tdh0M#!R2-iiM~|?%^xXc6$Nc z*;HYtx1(d;#3L-2FmSh{{G(@Qs8OLLu<=;es=tYj| z$nXM;YVA8j^#?NFILn~OdyX-{QU*y*guaq{j@PkzqoL(^q4GWH2G;9V z{Bqrwp55J2LAXM$RmXF}y1+)|ykra80@_Butptiq$5U^HdppYCk~=Pu>1N-S?zZ%u zXr6#tREU7t@I2*jxH1S7r|aijNO_rc<>X-CeiEoun!9peTCSue4O-Dr{z?j>B{XYg z;e$Z7rrX>?3lALTg9Z!Zs zpVXa{@KMqj+xyN}ZO4|OT2192Ej$G!tF6-;qtze!wtD468#8;RE0&@XxrZ-3+wBK* zXH$jUep|<)iAPv6Vc>!ZGa6Z77fzj6lIRMQPOy=%iA%W5jvb>S=id!mUOq)VvPocZ z4Ax_x1nOgi|7Q`bIOO3uN(Tg0_ zk>Ldx)!KK6>JMbTah5@m_X)-T&r#xib`*sv%y7o|m`yN(#wJ|v856M=YX`{Y2A`p8 zU!Y~b63V6sS7XAtQRp#YkrG#8!6L5W8mX=`FNaKFO_W|xU-S=5?*IS* literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.class new file mode 100644 index 0000000000000000000000000000000000000000..6866e0c2854a2f0c6ff05a12aa7d5505096c73cb GIT binary patch literal 4070 zcmbtX>r)d~6#w14ZnCTj2-cu&rM>{g6tq^sQU$a{13`mminT6bfrYS%n+=M6*;f00 z*xK6n%YJbBu`>!<)af^8oavW-=uH2Les`v|=k6vkxG9Y_1A8BP&+B*2J?HGrzkj|C zU=x0j(1;^`C;_a;AV*w+2S>zWh(ndY?Zq(1hy)2oI7TJN&^X2LnS2;zu1GRbjF!b3PNVG1WWo|jN7Gnv&2?s0Kr8IBAT8l#~)5RYT*-@VF9+CnqM7dMI~FF*RCs>83iO z8imAy-=vZ#++t`ck)N$nAuX}CLDlF}1{0QT z$+(i}SI7)nXE#0Os74rUcPfxJ4ON*S1VZEXG^CL=hOQ*ip*^Gsy=*rWJwDo}rcF|# z!lZG|3kIgqe-{HE%xH-rf!xMY1m(EcM8ZTZy>n9488(zc-fO5sT3j(zp_P{I&Lm7N zl~5zYeWQj-KzOxuWN3)uKs}yPaN1Y-JTmz)3&$g9eeJPT%ijo}vpoN8qr^bPM zBb8D{GOni6NrQ;%ix3N7L*2>vcyI1tr#dW988Wm<0cyQdphc5aPdKR4I!Z)K!gHcv zDwT_b{Z1y%5XeKjt>B_$qrA0Xi{eAvoi-8}EroAZ)6%*26~W1=?me^ZEyA`a{H1HU zAZjY{@opt$-8d@;jZn_x*v0WW$8Jh$48eUF-P9)3el4xh<=mm`Nz+Op6cXhYfMXfD zYTGe%t9rHOmJBx~iJ+qgj_F&U3xGX=Yq-DHIDD8ttOowj5&p_ySzX$)-l z7v?D$8?lL@$wB|GPHWGljO``R%#)f{H3=l%pd5Ffe8~L!WwDXGd3S-4@h0A)d{csU z`P)a4Bjatn!||?+_wYW$8V7mNu;utb#uYF?Jx{w zilIzbk9Fy3lbQ@lb8G4f%(t4i)~@=3Ejtp(NS>{y3`!B#+)+ATc0+UO>ODfbd0rX0 z)+q~2_MTYbQ*q}KWeX^&PMu+f8Wl?Fb#si%o)9inQYlrZ?xnF1kEmU53-PeGauOMq z&*gacYrmPaw-~DGoP0>~)*=F(_&axmlLJ|>RZ_g#K9VuDgzf3DW)-!?79exCshv$z z%Wd_xUDVsJr*9AXM1V`wGn%hHWIoC4{)n(HBn{ zZ8gtAB6elrS;#RGxS)F$3&mtn&n(Ktq@u-B?z<|*+@46xhL^Ibz#%)LGu^E5_d-k>LX1=TLJ@G2|`^6RHB zV`*iOCYMq33zpGei{HIDP#vJH^H?49R|jShB=wd@gSEH;|2;6WD-oC>fDxBr;M zL$P%;c(^6t-Yiv1K`)Ut1p`8ZptoAua2YFRQA-RSsa!dYRo31#R!?KiG}fMyh{dl$ z6TiFpBI>45FBFo3eow)&3JpQOw_rn{?twtPpKu1&!B6h3piZHNFkgl3=)f)4`yh74SR6|`|um0xP^V>>wZ>&18fBjvL?hV&z&IOokatlKpO(M zNq!SENq8*lO+RbElO&D47FaEIpq)Hh!=s>uLqz8zo1Uf7g~QhKIc%V@hNPF% ND1Hm=fiSXu{{V#fgc6r4?*xHS$*TPWYOK%n`erV=B$8C)&P^8-O+B<%4X5Z|yfBpXP6To9wv(RzZzzpsg&`{QKU&jN3 z;EIllj;fBDKt9}cg7PDQjqcFzTXH1rU1bgAhk>+$&{uNb@%q+a@@P3;sC-Ylf%UQ% zzkK1iMBb_(Y?5o;@tm+F(5{@9thyzjwTC-OpxAXh^=iDoO&z)I5}EG~ZRu`F--+f4 zs6~Ycm>thke%qBnpg4VY&V`iMSXWMtM(#&}N~OCe52WQvTGF8vedRY(5G|ou&FXuB z>_9PLA@%Y&o^%J&9mi#yvL&ekgPf4uM%l&rQzZ;ak>?AwZcK-tAJU9lRK$US`S+3ql)JDV!p z9q#C;n=o*}go%cUhgcPuJ9Q(G6eyiy6HODBaGA|IMKR95d$qjw7xly;fyNZ9Cl(2` zrU?JfBG@FAsLf{>vo+}m0v#;#3!MS?6QWd#Oybv^q=#Y*jd zzdZERTD`b>snuG78s#6*Q-4nn*>9gS=-^DOq19?NEcWF)`}=*@{q66!KYzXr;7NR~ zA%wANJYR!dh)cYn!H+SuIU{jagY-5pO1z{&!#VZkWeGz=Eyg6qHPop&q4pE&U}91Y zQxek}8ewUe!K_45!)7HVrRKC6GHS@GGv@r%@#ITVv1sWnV#+(sO8tJKU%(1QXRF^>AtU2Fj=1ki%lGa7j z7U+s(?5VJkH4@Wic-EMlHNuHZDwRoxmrogv#a(~eF{ezsm|5Y?8OdUjKxc(`o=U~M zgj;XrhVn_r$|lX|khsBT`!nhD387&Paprtb_fgPxxS+SUN6Vv)5o2{V_=P@7OGt5QEalu68< zTt3!kPAd0K*w&n~pyh9P3Q8+C^tngL)D&6a>-p@mzhk8!o;k%CR6wje1)`;>WzDG^ zu{P!yiP<3|>*k4j#SMubiHj^PtRVu;r}AmXN|~cp&SF0Gq|+J4y`GFyy{pemo_QHo zexXNdl|o*6MDq^QC6>wC36m72W>ZN*b*O@&vkdMR2)LC*M=N#<9Qwy4LPs0g1rAmQ z;9aodnxLZtVP2II{?Y)ha#MEfRaU*KV-NNU?5zN7<)sjKyo9H0tQm#-i>{8>@H#`N z;|*NW@h099*i{~_B&n6wZ%bU(aRpZeHr<;X(m6);v6bhkqD z5iQheRfN|l?ZB!smY?aB*UF9H`g=)TcyjrIv8%|l17MK~^|usxeECtP7maMzOiu{3 zhl;+(ygsLx#UtH&-L|Ymy#I5xlkrFb8$+cyjNhR$0*h4eE<+2V32p-0i|a zQgsg)htSC#dr}JZDWT#Gw%n4mAD2)|_y;2O@)s>$<&VbwfoMDsjK+g98kg14cy&ED z^}%`>9a=&i&(?SPceh@rX4g^U{}t=wRa{lY7SS-=&Xs-(8+>T?-@-;8#+J|+-+cp3 zoq?(Y!Ny?Q?e7q74k(7U#$d-yY+Ax*Vm3ECxQK@oX%Sl%@$e!Zxe^rD?)<(HTbupY zv279CuinIC%4|QB*&SHNaSy8L**2QApVT25eU7GDG&G09_=fYhcm_Yb+K-Fq!!gRWaGZb7(%~n_-E{X<|J_pq zJT=H~Tj9GyQBoozF(ff85tTS8@f;)l2NU5G4+SYO##vzGzlqC^e={ySSBpy@<8qAN t9H3!?^ksx`8K(cDjLS*JCCa$O7?+X%b6k8l&FJ_riWbfqxU}vrPpz8)}xopo(ylv_FdHp%M7;Q`fgYexR(3i zXwxeK#!_od2^_4szPjCgu&RQ)a@C{dP_5OF-ii!dJ&%osPDQ}3`o0R5JlW|ePM1Cu zNw&z$hGe_#Z3&cfwN3d@I-VpeRkE?Jf^xz{R>)U*I$^4kGL4c^gv+ZckZvl*I-GQh zLvg2v((6Vk?rs!ECBcE}biGhuvbVMyx}LKb1ahn9Mkg?}mzb?~pPbFNEm#NXzw^Yg%<~d#uWuvKep%bSl1xD-cx-Y|Sz|?+ye-b|y;!8st&uM+K zIIsH+uhpUIZ>eyjwPvDX<1((;IEajm30$GlBPs6{tBplmwXuY2Ca&99#`iXU0E=T{ zSNAa!0-0S~PkJ^!!{=;T0}%bLo|P@`iQPLsf#LvR_nx1?;PhAUh*rtH%I)K(3k;!TWkh>UMG=Y^L5cJ8$a9?I2UcU4=eseS#!-AhZ|?I3O$oh(o=Iao-aJd p=*)9i+ZfY8_miW;(=mo)#NqNNMBX!)Xq6EsQ^!H@t3LZS`^M?~#RI-!%yv+d`3-=} z@lhBpIH2LyIyB=piQB^n;DA~f5+h;A;vy+A8b%lgB+M{$j7f}#u~@ASD!R1@6PQ#( zN+KP`a%95DA}4W27%PBz5boA+kA{0S+^6Aw z4G(B|P`!Idy?a>Vu*4$*dVeNk+Px_wpEvUYbxt;CB?pbPDG(W$Fs6)H%E*kz;*M=) z#=8WTOq$buX5O?dBV`>jZGo?kc+GGJxQ!9se(wD%e#Dc81a z(nt*(G=u9BG2k4u=!2e01@ew<8fkhUHd%a(S+vH^7^!@0Hx=Q_9Vx@vW90Uld53x| zayXYr0>a{dzknPmSgBFvv`w|hC?RhvGAZlQMlM%7yE#2)oDEZbl0P_7s#i{^B0E~4L6N~sJbz2Wgww?{zf$aZ;147N%H52c?6 zK^>y*g~;+sse~+1wsfgKfp|^GWa2UO0-s%dg*sS?S~Y53#CcQ|I4ABH$w`$1${lWj z8I^ccqF3TERyvk6iQNJXHxx3Cl{Sa1yv2<@nM~GkO9anky{ptjfk_-wXMlQ!-3)8#H?S5l+HZAa36||}y+Ixg+?doMbRf=}QHqe3x{ zygjr;QO3?)EKDjMEix4xD;4XpZDV@Ca$CYO*CFL(UiR|5kh(WN-fQYrz7_jO53ja1 z(^gk^T-(`xM}gNi7LCSTyOcoxxXG8%a+^yEvG93KnMZM~;$^>umvBFPs{QcEr@Dwc zvm3axP+bZF9bE0>PQiIq({YHVi0>HuO|u9jjw5&sAwEhzYR6FbxyyPb#|S?vfYlt= z@MB>eqF9fsNYRTx*o;oDcq69}pAzDe{PL(H?T=Z63Aboq7J7mL&)Flt3Mc&jp+q1s zln4fg5~0x0;4Bt%(~{0$bJJ0(a1?ccuTh`y^T{8d!P3q^L!jjgM0{unet~5^?3=~% zMDuY(J45~s8I`Tye1KSk;`Fyh<;D|OF^iRiJU7xXgYy(=2ItS}Qq=hk zGOD$^08woQ8&06B{8|U9K=(>1}4j*L{yKyyL`U5U;pETv#lG`=>O6;w4|5E%+^d5#_ zElyHKFC)DZpHj+pYPAvHVh7hj%07Z?DKA8cYp{#bB&CAb{u0I>ViY+CPje2D^E&Qo zLEvwyq)A*aaf3u$Vz0!F62lVvc(6{P{ZDjA+$2unVkF2BqHZ^H_F=y}-a?;%b}=(J G^Y;%#dssUF literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..885f8652eccc5380d304440bb75c40236d04c62b GIT binary patch literal 1585 zcmb_cT~8BH5IwgoEL~S9pNfj2RxB;Bl=_uGNC*Lg7JYzV^zG7KVPV;AcDEQ_{7w2G zB$^nX{ZYnqw}rF_vOcZX@^mudI(c*0!WuSK+1#qj^FuIGz(N3cSi)5$5EUKx)~qDuKSTBX0IxPWf&GS^S?8-1y+^$_sJlP_%?2@&bdnAy{mG|VKv|UM_NDAqi zpKMXBkue>ZzH+E((h;A#LDm8l*4;>8xN%yK9M@h70(n$+Vp>Pq^E|4Z=GyeueqrIf zk41)%-f(t28Px-ZvCv`Umq1OlUVpp2rz%lf;!Tyn=ax{K%1-JK_Y`}prh*G1Xu&*X z_yViNd7ZSm;n#zTT6J{0j9u2bGrA~juobW3`XS4>uA*JPYM^Lg4kZf~u3P9uNnkLp zUpwIzhwrI{XLv4<`M16nMsR~Ob|oX0Ew*`RP%1Ff0eB0i0xKO{`2WBQ^hccgwhw_o zGN)(uTI84gPaHrQlKG)r>s&NPg{HDvW!GQSO&>7M&y@)xL1Z$z0{qA&`R_#^wHZo_ zNbmv7eBm2J{sf7y+$GS@wVO7;GzPeiG^{X)VeV)jqX}BDG(6K-b-p$F=no{{pCGk9 zRX9f1F}mMrf$J9@CS&LUq$tm#2Sx7Z2vLgNj5h$z+>GKTM(L!7R=04Q4({-tb*34k z<}Nh}+{-L6lyRQuP)gM3w9@&)FBr$@(XP*DrGaSFbeC#E-N%F2`XRD>r|<|97^0+^ IO!Lj(UvXroR{#J2 literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.class new file mode 100644 index 0000000000000000000000000000000000000000..ae152c5ce0beb4bf7c11e8936a557aa6b883b8a7 GIT binary patch literal 3764 zcmbVP>vI!T6#w14?lvsYrVSVY1)NmgYCmh9dHz{mLgg|C;&)&QD{Lbs1bMF4|@3-dw z+={Q^NvaHFOh&MGRKsqnMd}d; z@yhCfj2bo6iEZjM8Eo54`nU>)F|<~LS5$|vYBG8{s%EokQl+V!Rd~QZnvTY|&u<%0 zV?zB2UCRsgoBxNWC_#8~z}!kq(!Aab^}8#b&LtxyVV%y*hc;0zr97q zW02!zDp6F6I7S(KJ8~&QOR8ZltC8LPsZ`o9vzks%t*O;ij`GegqD6|jL+P9zRkvs& z)m9YkEXf10v0aoG89lh2!D-eV86D_k*l^tiMMf988SbbKpx|0-1(B(u-^S2c#$SF| zidGd9lBz}(JXbFoYdVEPUSU{U23IlEv%)P5xmK;t}IX;o`DL&&kF5`22K~=aTknb=_ zSdh*@o`OsyEJ#-%-(`}pAl-pH#g9n1TgI0-ArMY743_C%9+0a$x5Qb&@hK{S4DD4w zTM7e1OBD=7$1)68D2dmjdav2dVtP8s@O1^uiXGQ_va9XVB|H^sY=v@NZcovCsssyT z*0#v7r-JU~{1$)rUZX%>Y)1^r6-8a_TK-U3t3&=UTBU00?@G_z&Yn+_>mrb3XVJEnP=qZ>-C zNW4Oy>8IPOPRT2aYpZSy?$6OpjJinE@UReQpH%6kx76mMLR1W`rd&aBtfE_dJKe7B zu!+V*Pd3qCOiF#iq)hZLP|Km&F51b6u4*|A*5a|9g1u!9j>u^^Pr*fxoF1i9sQJcZ z-9V#CAHe#nIduSGQ0f*1gdKMlV ze9p6|w_(>D8Y1gY!y9nfd$^akojVS{PjK4XyuAGkmds%(LEhl;&EiI}G>c`kSU!st zhd4WV;im@N9>MDjF+BDF40 zGmYHS5(;p8kK~nnoWw5kaj(>6f_bG`tUrU+LlTMjUAWHy;l53%rLhGPT2V(XY9x2H zle4y>kD_rm6!arbyN(b&ib0&9H@{#AzhM}E5j~F))7>$0{scv~4|kD!f5l4kNfK@@ zhCsrfgx61@sKXzGxS7Jyh@UY)sCGh~!4?`FB>hL+L&eT<}RgNr1tAvvRjJVuBs zX@3DDNEijq?H4)kCC(kRs}+t5q>{uD;@HV?A4iyD7sve^5jte&(e)2DaXi4z<7Vt8 Y4j1Y6AW<70GRHme6Lr(7Oq72A0XbAmYybcN literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..21a392e4a29bf51185f86459510493e8f2afe601 GIT binary patch literal 1381 zcmbtUZEq4m5PlXY2iL10R*UbbsJtJoO*DZJYhx1?^#cuw9~W5R%Jq)i9ZmFC=?7_| ziSe^P$~eni%>`^i(!1>J?(Ez%GtbQ4??2yv0@%fF5-J|3SVvyLLlsfvrFfLUV>uTT z6jib)DJUzbD5x^Ty^-PO>kO^7?F@B3;r)@&$Nb}%>#pYrer#Anee5rF!}5erePUgi(#*DF{omfA+~Q11VgHASmNFExF?)L-ZM#)Y1@5ncDZB7 zeV~lVBp9?;mL;5hle?~94A*7#&%eqwl5Xdm$tuA7|5pv;Mr#L3nTS;G#VFr*jlG;ASDUA#iF=QeSkJcNLK{upX2V1@`}Xs+QV(~^7ocSLEVkS6VATD1_N z7f?&(Z(yZUM8484f(*$x*#K2sA$iTW!X~Zcqt4N&OtKgOR+nrfdiDpRA5IY~&#;z> zlc>;`LHR;wVdVN8Ip=@#4>^nLxH0#f#Z6j8(tG6NR^a0)SxX-qrSdN%Dl}$DE&}@@ hQ_|8f0FL4I-2ENgozFSkqdA8AB(Ks?$y%kEp1`^7M4R5n+&l%w<8!*b*aSR@U$hoW8QK|lBv6G?lie4^*R*B zv=a>0Tcw1z=WyQ_w0yag8Tl>pX@w64&L@WTLj8oFaogb(4Dy^}vQwNF(S|eb498~9 z>CaG(S|_3%T%d#^hGau_6%U3Uk$O|dgbIuJ)zo$zkom%^P7y-e=Os1r)@-;#uPxq4 z9q8;#hAY}3%E)Wgc3htVa3F%7+cB_V;Sw%eNTI^8HtY7Qp>#UJV_-Q8Ha0Ch!xlq& zR?os3Zcr62Q-t}8;7PO?Dhn{D!o^TsV8efJ0z$xWuR07H3)M~-C(1%1a3@)! z92EXhWOzVVcokIcVCl4uJ>C5%wG`4My+X4ZBJ=`gsr(hJbdJauT1AkdK2AFNUvZWC z>k}We9A_AfPyo47@`?h^HO yNZk&p$x`_z5+fvMQ4<5nMLHrGqGGr+KJYH?jmP^G3Lu7u)L)}!lD0`BJ%0dNKv7-* literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$3.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$3.class new file mode 100644 index 0000000000000000000000000000000000000000..d2efc76bb7a7dd2d073d176d9264c914869dcb8e GIT binary patch literal 1336 zcmbtU+iuf95Ivi?F>wtIrQ9zqZ78`wjTGJzS*Vet`z-SmU5#fRMK39HZF_PG^Cf#3t_b*=vR(UM*y0*||)^`aHOeC|u{ zK!j19m>SZPk1QLIr-3w$_q-!YU*&0~JTEtiuRR5@u`Ya3dOGPUo>9%iepuh2e_JZ8`SJ7HcK|C`o`i-6IwtT?M+uKKJl60;r?9M{ zqM@pxCQt~vmS1`%&}h0|$CQ29?kck<-}R*F2cD8gmeVnN@zk`OKzWX|eRI1NecW;_ zXIJ?_oupOEv4V!cTKQtA%AP=e-91nO(@o1!Zw5#E%G;IuHYKx7w=L~G=~>}A5$3}~ z2pF4=qr7!n`o7}w<%QfRt?*3avfsB)1a`~KLwPJsTQV4%3_?eF^^}OgV3_KaEEc!b z$)2lZXox&t!4P4b0;Td-Wo%T3Y!7 z9H&EIIRkT8x^d_QRx|9e{&i)r{L5jh*D^qchIujcrR52^c^BH%@@v1yL-9 zDdIhv<6~fo(#xDRFvdTiS8Jajs$Venk*hIGb1YB?SVxiL)mRHNnB$825lv8o`Qx{c z^@{J1UZ^d8MeYMm0DPjv5hcDlw2xaCdMw!ro&@fIRMVoaLiaTjQI*Uk$O(X8-^I literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$5.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$5.class new file mode 100644 index 0000000000000000000000000000000000000000..e88bb928911fa308d150a3619c4ae9f0ab369b88 GIT binary patch literal 1795 zcmcIkTTc@~6#j-5*49Oqi+I;6ReM2J6vU*Ykr0hY!H3o+#;2i-?a=LRv%4kXXZTO_ zK_w;{p8Wy-9OIeoR#xhzgxF?hcg~#o&h0zr>-SGz04!iBfm4`CB7wAqn@QZl?IbeD zYRDz&B(Gss!yOHC3?qI+dg;3i#ggOJP2T3UhA^A_b(5Q(?+V_McHL}tKTT=-!nL{O znd{}?@R=j+itzjbAuUT=`bCD7%*mj#RfhPgvn?3LO41gaoz|9cD}2i$$wbMiajVK* zsrI2Tu7Y6D*KAw3s}}b>LEA?sawEM+F%9!}+uCEOWJ)`Hmzx%+WUNsV>cTBVM3f9l zQz1Ku#dWb)) zVu4D3R4w$aSSV6p52HRk-`Wv1Kbo#kl8r!yB8G4M!sBFhgDNK6sAIqJKlJrDxc@+0 z`+G)IPLgG(XO!Z0ika7X@W%uvf@)OeWb z1F1Q!mSdY8U(#@Tp;l_4R6){AAAeqa*2g`lL&^y`mSN!UU#hNBRL8BMl-3ohhtM3c8m$BLMKH~creoJRj#LWOP)z$rpXy6>q z2Nn%nz(u-oDY$ppSRor%LVuU&RuDlhe}IwM1H?W+J8GcovL1kqh6dsx+*Mo)U{knG R?>KJII!O{ijegIf50QUd@ literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.class new file mode 100644 index 0000000000000000000000000000000000000000..bc9607fe21a00ef492f34c73ecf8c94f4cd496c9 GIT binary patch literal 6152 zcmdT|eRx|{8ULL$$xUuA>y|Cs;)XEYN7rsko0j3yt-4QW*R70fU1L?eP4Bw9rb(BN zj)H(FLlGE)vQ0#gi6DyTbZs{Z!w>vIQ6?gOgYXys`wVv_?!x_-MCx9 zJt{QZD@P6~xKBl$-T1tM`&Ibt#sdnzpu%rAzNp|q74z-JVFgE2EWkkpM^#)d>o3Xn zLkhm^!NV#R;VZRx1dn>~)ml7;W3urz1&^z!$Jb@?go-Ee4FylBSRpz8rmVjui>GDr zZ8_%|FQLDq;=4Gm;(PeM2Pa$CS@iD+w!9!fx01MY?F~TQl=g^Z!%H> zZC%OKU{D{?V|$I@kiKU~55|(i!^uQ2KTFS;Y;`0u#-NcZUQFP+Q9WMVB@nI<&*rUo zt>klG(xg-ZO{FO5Ov=!QDNk_7CWAeCJZp4XE4bDhF$WWRCYv$@_Ex!kZK={K>1pS- zwbVe?*rtz+RL8fyf&e`g^J*`th*!y(R>T>PKK@_brz2ye^!~U}hMV>_x%&)Snsq>V z)wEQy|JG@X2=h$!tBP65S#4XH(=0LjQUl9#QyHQyxAd?gxvBUU{|`j?563AF)VBE0?v)e0eV_p7aewccDUb2?UeQ)15h%i$9r^!7`Ahr&dgquQL4QXRhW!4 zse~R+2e&dv(C4L#?=;dGGU3UvmTduH^1nks>Cc++0qJ~|S(uHv!Cpj2&A4&Ih-Hic zo;F!@^hr$H78p)fRKjtNPlDwZ^3=Tt%)~(QfH6Rj)+#`D=avzy03qQuMyKO~s5Yru4Eo=lRAe zdFyNVF#0rX#ug3NV2i+-k|0;<-LBzU&=EdR830d|7jyAtuAviK>F_1%OYPx;{=T?b z3iJaFUD$T1P$3Q1VaKJSv}lN2I!ddCU0?{kH04~Sp;w|bSD@mYvE+iDz|xX6Wlf%D zcz!ae;fMGULr{5uU5U6kY%-SGZpawKkZ1=&>nwjvAJFh3ep2WT(Iqte6hG7ObNoVJ zS$W`+C?qZ8B@MsCuN1tj;n(<$z`XOBEyyc+HN1i|viPlrSMfW!+iMzrkJlCaLBk*M zCk21j@E81*mr9h}HHCYd8lz35n!~-#jZw1yH@qQdys6>u=;J^r+#70)GBztM;og?U zXp6ww67H)o_i5Vh?%&5tNuZ$u#GG;jLX}U>$4Pt!|$(xc(IRx~JvRBAVg42?ZDtB9b(LM5?d^r{5vp0EQ zO4`Wm8&R*0)eM!_nOOj^#xspZ`E$CL}-M5(5G&%UMRzl8ANL zc% zeXPs05{KEALZ)|9TWLN3SlK-9`I$0u*xAN%I1(B(XRWk#zYGx8v%`^Z4+bz1A z^LU{b+SjwqtGUqL$u=+9LVGt?@wzOuqZ}79U1IqWjva@S6cB5eglOvX63RF{B)ufnanzF3qMUl!IA)Npll07S%p&cR^z3oWA?=s++;Pk!Jzvs+alGfa zCBR2nHb7;$gj+8oRganMS%5|?#Y%+u0@jW;rcoQnb|*f@nVQWaZotRcqF<7Ihh*P* z4;0i=cf(UyKo%EvDbK0VnwsvY)7c$$xw@lnclWj_T+XrghLy&K6O`rzYMsyHif9d~ zn#csM++N!6lKo-l$|)?0`rbE*_n*RI2g0s^)9P7L=y3;J0XKnWoWjx?1e8-)=D_YL z)JGd9u{`Vzc)b&7s9CN0Ro5{D175l53ctGQBpRo%f_$y?t(rhkc1@sZ0?iW$-Jyyn z&zpXS7F+CADu&B#V6OII|iFUiqKxUzz;ezr9%gneHM5Pzx?k}>hZ0w zOexZkosfq71eUYh1rK&(A&qkhO>-UkX}B>AVgO^B0h*)#SYfHal5zy2Zf0{#eJ+F#9iWX+$~PvUhx7Bi8HKU z$Nl0B92W23h@%dN9ezCISd51q9eBjC6OTIfu#V#~%ZZ-GEK(tI7o*d?ou|arxYF(u z{bDC;2Ia-E}z5}T+&k7FM8k#b_t;l>c+ za8b^GQ2sbyvX?lnr8bgKM5|bX5o*MPqvAEBkY-fOP_M0=Z=+=kXxReVXaQ}sfOdf4 z;6|>@V#YZNTdQD9!A%NoR`3by?-m98yOq;!W7PX6n$Mz6usX+pe8JfYKAE3?#aXDF xA3BRgtln*{mMz@Qt=&|{r&v4iX{)>gn^-rH+4-#H&u>?Jn74m*xmvU9e*?C6B#Zz6 literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..880daa257d5bd62df476d4d4f63bec95480ce384 GIT binary patch literal 1648 zcmcJP+iuf95QhJ4)3`M*Od5Y8Lr->q8<%*`{U|=5zvs-RqwD))0 zyQ=M{6BA2{DeG7yFgh+gN&Z-O1LdY8P2x&%NsMmtk>&^bqZL2p{ zP??KI8)7*yVgjIj zaGuS2ieAlEk8q)&TCPBI z%o02SyM`qmlG9+PLF975R`EcclMENJzFV%!{kSf(i}77EJYfBI9ZY}hJLcRgv+$J653tO zy(IZs9OV;Q1?1UMd4x=5gsCsc{_P+N)Cgg-v4c#EyN(+n_9jyTWN-_&vA~f|ZH-@E FzX6)m&+q^M literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.class new file mode 100644 index 0000000000000000000000000000000000000000..704c4e155f23a62c619c12c48c198e69d8fe7b8f GIT binary patch literal 3706 zcmbVP>vI!T6#v~O>D`7!Xdqw(6|g{CprNfIq=1yCg+h^FG1k{M-Ik?!vDqNSw}9^# zzVL-Qe$erYGtNK(tLTj1^ouio)lc|WFe?7;CV>qJ5bR9v-93BH@0|0y_ndR{%fH{9 z18^rk526i&8XjsuD;}13BnUqS)ytz2j|CwMFNVZW5J3z|B!bW}EMW$5lX@Rf^3f^S z3rh|AB*uc6iDVEdq$M&z%vJ`nYTmB~TMaq2#tF~|_2+LORz8RWcsz)M7}sz}!xI{w z)bNy&KCPtBNIWa?oPge&PMh|Iq>;;+Ie`WzleH2%jFc%5>f2`=Fd|7KJsOEQwv`_3 z7MM0>j&C$`rfnHX>yT*+boXWK(TI^X5_`?am@zVDL=u@)DwB?sRvC`Px8Ag4j+(Yt zS>YWpl3tNOv_`ywOI}^V)vvbFma|sC*U~y5;NOrLW+c=4thBi!pBggl-NsPTjV+Tf zk^=_A;CoT%clKK3U`?e1Imb4Q6d8z&6&@oNqp{OQG8frKNBD8Ok=@>F1I%rO#U>a450af`^1#Pbq85{FsG zSg<5`yqb39(~gxg2dtdMg+1wX#&OF4&);;{t9^MUYDDcf9Sz1Zc{^clwp4D;D9@;N z#hH#ybO{99@}y%amI~fmZL+Uqe4GdQ76iCnZ>0BovYQB z<_<-}5rIV&AXT@OR9`-Cg(!3!#S7dj9WUY~9WUb*fyLF)Dl$?Te^uf&9k1gJf!SAL z7wZ*E-hhrb@s=8n>3AFOC`0e+cn|MO9M|yyK9u-K$H(}DC3;|abh(;3qNLP6&>8Ji zQ&+T0#|eC@R(-}MQ{iJJ)mNjfjE4T9eJq7F?p;}d5NNBxi6UzP>#9`8+I{YJyVr=F zNeO&i1*vL>^&ZhGmFoIBRf=qtdR-YsF=!^TkDW@usN!)ItulJMZ_tt~HlpiUyVS2D zWi9Cjt*xjnqL`cirN*uHu<>%16l*oF1HAL(&HfRASuMS-p3+^a=IocMcc+I1mbG}k z6J9kTuS!8wvM4aO9P!G}nv*Gr1ZK8W;&2C7RWaysp^!h`&RCSFU`v++la7ZAN_odh zMtW@981J*(Qa{5bq+HB|&+#(p-rCqarmH+F^oJf^OKqmBuI{I{vuS^xS1$H}maSWr zLEos!kKyu|iw?1{c|*B|5gj-kpW&KoGp<1ADkqaGu|G z8lo*!cM86?N%-TZ5jcf*^a2p>#!9}C zej2G$M#SIzz0g3}Uy}$DZc5)I^f(Pp-7dcm#(lp2xZmF&4+Q$-_4WNbCUFxNO^XJW zw4J02C(+>l0n_6?KKWu3XpH)s{7cUwREMU(SXQQ{W`#O@OWMvDYqBu(cTIAvVmT;V*(qA9H8yPw6X>DwAf5L zy+qzh#Ccp_z}QBNf@a?(&F$3O!BtD)zetxfiGGQl68A~$l88y{mbjm%=>j_c!4io9 caRIk8R?v>X1Dxv+cgF|G2^bgCfiwU90|S#jQ2+n{ literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..90ec776f0be9a9504f9c213a45a2d710e78d1732 GIT binary patch literal 1285 zcmb`GTW`}q5QWd?=ENA97ATZ!fzajxaSOaPid0376p_-0RC(W|+r*9QM7C3je+3T| z!~^1)AB8w}OmRzvB9%n(?u=)@%$}L~@$>690B`Ur2Mwz_vUs8+ji)-Ep`hWpPWPII zbqz%gC4r1Tw7kNGK%+f&2d13J{!p1Cc{q}$=etTCTh727&9o;mch) z=}LR5ynW^QTg25Y$MWj}&EkLHmAV3{#`r)9EVM00y_+8Ql)Eo`Ha$!2abMb9=~{su zIa9%P1dNvBD7Rrt&r>vauAWO_lebID$;5syuwHB*$rEYXlG;!RO0-pqxmuKIpxofZ zNuW-U>|X~2*5a7G-jVA2=P;2=AlI=5j`XK4Nmh&$16 z+3TJ0)a|RL73jTkS;?!xwOE}Ur*Drv>aeT);rKv9*+2y^4CuIKzzB9PQPp4?*ubVh z{-PWPZs9h&@XAeCI8UERBv73L9Mec(dyWJD9tVL%pHtDarGGXi0?A^~)frnk1AiEr zEe8IPBhp_$3tMOJP++;(x~#jf%302W?E#U`2TF7Dll&RDPVWuw>PYYl=;g{Mi1KG7 zKGI5HfzJ$Mz#`@OT%JW?5jSZuKBNgouqqL&#x-DNE2l_)LFxndaZ)u#%AS$3WZz+a z+a%qMNUwP=Amz%H??|5_a}hO5%3wHyB2)@1;UC_^{cwGNhg?&5gvVIoPG_{nm7ibN CwNFd{ literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.class new file mode 100644 index 0000000000000000000000000000000000000000..2e442a671b0d322f3a9d299a65892507ddb1f30b GIT binary patch literal 3500 zcmbVPNmN@^82&C9Zb)BSpiMQW4qzvy66kvvxg3{Aw;oWmXdcmMAn?mt|9`0MK%01n_& zKXzf%hp}qx#FG+HKin8qo2Mkk{gAoM(-LR=@MBcstREVlk$BdRO=_gTQ|TilNXl76I+_TiilmJb;pY(>o~>b%5r66Xc9{$$dy zdg6K}V`K!X?Nr)~4eJR*pl)zhpVvciJvkkU*p```?h@EMXDsv@8N)L5xOu^_1iA)O z)^tcu>#-RlG^bC^>7iIEkw_&&g;Tn1a@U`$Ffdc2q<5Uvm}r&2L5;k-x8 zq-h@&a5XlK3b=bxleA>>pqVs=vxy1AI;~H{9otedJwB?_4DRzpw>@Lh2R)?b29zkr;`n(;~H zy#1BP%1bzwV^UTo^mJOWw^zdMs6{QvL0-)`pEhE)F*#r?(AcVR>f_BBCD2+4US1ty zRi~{~%*bR?RHd<$Re2yZl#0!rEF9}Krj+`VmN~D~Z~7OWyaeURUgs#8d#;e;fD`FX?w^rZkT@c7ft7|OMWRo$O_# z)HrI*W|?0hbu)zZM^afUX7rgVrGj}wSL0nBYEvad!(Oxrc%0gzVGkMw4*&0>p`i)Q z0*5LC$oaLnN@!?7t3XEye`!$UEy^bzD?lcQd9m*Nc3+yg|DtpIt`|U+o!wYzk z*QVhmysY6ByehD#JX%Q}DdDe4ysqIQE@^lJZz_PdG`x*>B;M8V9^RL@tl?RB)bhxapiIk$%oX!r;pt8;eQS!3qN#4HPxKwAZ{MQJP0 zTmeJgB?5hA3f7&f-R1O{DJzu__@WG6**;hti>5X-wB{OX^e4aV)AU60Kl^MCx-OY!d#q845i_fx;pUd(Q!}M-EaY6}n zO&k1pn<;q}S!aq(Sx0d+@O`_Bk5Lz@RK4e_N|kMAX8Cnyq1pljLfnn-dfa?&sLR_wfz{D6)eTEeC{s*w3Bd6TFSW zDMb9i$96SozpTJdxDA6V(4rK$aY%mSkGk9=QFo1hB8>T*eNl=lw|?D89Ofei1nLtjvLB3$56wu5kC67mab^RL5`gmz=Iga zVHgPWzkn`wfG}(RQGA9+ut=`&(1V{i|BT~~GpFd<89MeLc_o%n@2q^viFpV6@FOt~ z(b2W|p2%V6roSZi5#H$lF5@BMyLtKw9;OrzW!vbc3@;__q`Y1tA17ix&*w1uh>_!b zbd~cla`y97E8MrJl277<#DK)0#E`_W#E8U6hR{vy_!BJ>kBgfOj#K3DQnv`_Dx7x4 OC$O8dpHz)Ae}4nMz52xf literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.class new file mode 100644 index 0000000000000000000000000000000000000000..f9ab049b011061f0559f9c4e2c3644576f149a67 GIT binary patch literal 4011 zcmbVP*>h9Z9sZ8CBgvN~1{<8XNg#_Q+gODpq}av@U{Y+n5G?E}q@>r<#lE&SuC4%Q zBMVu&ByF?r3vGtUOkUb4Hq>pVZ<)zc^4@=;51r{#nhE{Rz1Oy6sU4-B@jd7K&Tn6j zzWnabp98oXpF|PIsW2XiAcmiXp{sRTA{|8tr_>=MVMGye52qz&qR?H zdAre{B%YPHC=js=MKe97=L~`7kvaXWp2+I?nZ$%`nfaMMf#^9ipDCPk zF*eN`3xh_wHxTS^?IBD|b(oMdE@M#-ik%{JE+ zn2RAGO#TlE$mz0~%_!&H)rc%l!aAE3Asjc{G{MdN&rH_o?lrW+<~Jv~3F7oALYDtks^ zK;k)x=h;TsJOtW~m-DunGbYWF$)rD!&lhZ`G?_x19j)fdES`jy!I5-<_(Qgl6WA7W zom95XY=R(5w+mKZoM{&@c@o%K@17#-fcuk4$H9>ab0)9bWsBvwr$5`~23!*1*m2s&L(Lnpcf!lMgr z9)TY<#=uRl%9f=efgcL&uA#3jgz7FJ5UZhaD(*x+Yvve~%xZ2b54^w-uW#)dUc}Ft zA{u^yUkY^8=jVxCt6;_MR~lZz%PjjEnp(YAJpF6@M&h>`UcswOsP)Qt&5iwWQp0O_ zU47or@Fw2U@HXDj@GjodZ~~JO?`!w~9}+Vq@sWm)aY@7P@Oy##8_?!|_U$Ssxk)I{ z*#KA1No>9iAg=~SpvBAZ%7N?5tE$V+n8VnW8XMjF9Nk* zUo)X-71BnjRIvC!T46(lWXZ6TY_4=fY*-1pVJ*YGK|>@ycFM*$Cv9tNC|=?6l~Kgu zs27VyJ|nQJu3~FU-6*ZS3EHbUuD=f31sB9#9jg@WLX`!cWo(CaL{}*5b>~`$o4I@k zu`yJ#@Cb%IvDHI_HK%IE9X$t1d^6g7;ONfg@R>4SSZv<0VaKPye)O%JhjTz@j z#W&wW^ncztR9oq(fE|1k$N3Hpz^6umdJ(G;;H>QA&dQF=)ep91?BVw=o^!bl@f`aSx#y znUf{<;$E(tA{6IS;(V7O5hm!VWoTr%VMP8Vn)3U{Q-Q#EDi|D3g+k+_%h;Ie{1bvp z*wh>7;x5eprodmZIpychKe31##&)#@uA$k7sby?QbuD3QZ?G+>s20)kH?;PK+Jccq zZ1eAxEi!ls3Fp$^-6DIgV*4`MDE-Ffn-EeQGv{5S7Xrq8N+Nd}N z__Gc;g>C$HGb%l3XN=;69$*xQ8M`DSH;sLm#eST@0EgCoywBS|;vl}jA^aVK_$Q9w z-xv}*a71)qL=0e5oWPiv!??JBV@^Ii%%6qx%z=IAW&Hm|DejJW;N?hIoJSw}nJeug zLn#BymjmKH+=u&_pm8xpsRyZXQgl=9Aaie8+>FC`fWJO}O#N>qmWzL66@Q5MF4i!y zjuI=5z;!ATmKc#3l^By4mpCTzpu};e>^Io`Ehfb`RCR*rA*z$)>Qm#{O<=r0n4hp4 z{rkJF;ZDIBzWJh0aNi*?b@}=~+CSs1>sq%&k`8?nIc&rfwj<@ZL@7)zp80Awg`beO kj~L21KF2J#kknm@cgh3_-?#9I7Cgj*4<`wJ*xCK`KfY&R^#A|> literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.class b/out/test/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.class new file mode 100644 index 0000000000000000000000000000000000000000..ea0b062309de5a94a14c3aa0c7fd44fb3766c370 GIT binary patch literal 2134 zcmbtV&302&7~NlPk{i?17D_7$6%mp2$3_7QnA%dXr46ZZnNq2&qAth!0KS0_-~%Z8oFq~rWf{%j-hF<)efIZ#`=0#r=l4GWIE|$w zTuiz+lfc;|A~@&by(D60DjVYt^D8Iu3Z~j4-3Hg&V66>GJ$MQ4o5yq#GdOS1tcwe= zzgj;R7X|St&-cPI0>q$(>a@XWY184YiE zb-_Zu7OZE~mMY!QnN4+lQ)TL5pjFlL*E3r|t)%PqT9EPlPzS!M)H6$-zwA|YJyg}L z~kVtTlp?p7)H)4)5WN2~+tL4&kt1vP-kId?A1#_&{)>OM+);VkS?#&}>Hv zuj5z>bC^%z5*A!6rjWzs6!MrC%yfBXrxgVAT{1jFwW+vZDE&_><;=IARORvswv*ojOV^>;KSul6A!P|f)Pv@+FUh7=l(txA9=v;f} zIS|u(K8HEZ@Xc-_Ho60OKoG&Je8%nMK^!4YH8{ z=kW*sMTOT&FD}V27GwmMqrjIsYIzJ*$i<==e_Oa2h6 literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$1.class b/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..cb4398453048de6bc9e41b7bd9d500aba9494242 GIT binary patch literal 1081 zcmbtT+iuf95IviiCNYMlDNybdYC>Wp>>wTxNrRvQQ7I`8iQ4ClyG^!nZOgmnzmT|z z2R?w0Ld-fKKzTy3WY3=C)y&R0JKukN{RZF(yc(*wZli`9Rot|34YzD!ZdY-~fb9zI zR&bADP0yrq1BT~=B%As?<>5^DbAB@CzS5cCXEL7pX_kaSsU-7dtVI^{Ncp4U@0hFk zSSa14R6Q9>-Dh~&S}|AqkiptZjs?T|K*r)AKbweb%qI~sjX@If=#Xd9;9sO=+F)?@ z<5*;S5m!nOe6iX&+zxeB8_8+Rb)He_$>5Z~<9@DXM=D#XGGWAK6S8*66ii!<|o+ZZ>>^QJ--hl(xc$%a#>^QiO zrh^A){iYr}XroD9uVl5`Vc1v_f?;>1tPJIr@!jU3xoU0@(50NNe3O>V&~#hd7n}88fEVT*cVuzaPG)!tSwN*p!c-1gmwBhVA1y(lbz-h8vS9<8 hgeY%Om1YQ+2$gV|c9^(@s}wDof0GNei_uAo=O-`pF;xHn literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$2.class b/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$2.class new file mode 100644 index 0000000000000000000000000000000000000000..bdef3ba95ae9a10a4c8e6a5e3a0beb2da33c67f3 GIT binary patch literal 1275 zcmbtU>rN9v6#k}Ex|9lKxp>E0#Zq9Go1j)DP(jh6CN%hax}DOY+nr{1O04hVL-<2Y z{NV%mP{y+@AcO>B*ktEi_RM$Ane)y5{`2z}fJH3#VF-7VuyHSe`xY`59wf;%Vd0^L ztc6JnISW%3rY+1^a4fhMW)ql8V4k7Z)MS{MWmqXHz3cKO_iDne^J<;Dq0xdjWU%Wt zwemz5D(%X^2pw=gbhkTSOFXQXL}>B^^-KoRykJ<%UJz??hap~26~QoAl!18HYLta8 z@v=`IyQn1r*1Wxav^RxESYk*=h!t@hGyk8MFHh`F?{mK;j`BxE z+FjSlEW<`Nh#9=mbi^m4xn~rGPM^wcD9G7TUcGb}zy2?LsH~JBBJyUGa9?N|>wq6I zpr%6ebbfAmj$ypqs#b*-6-NtC=}PD{g?5@;8yOuA16v@1pD~8>`JJJ=G7^p3Z5ipi zAGD$rMn)`qej#9no@~VT{!o|$)HcK8Y_XHf=XCpsB*c*XY0u;_*ZcCG@Jt?e=w7Fpk?K3&NXj^Z)<= literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest.class b/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest.class new file mode 100644 index 0000000000000000000000000000000000000000..19fe162e12c70106951c3b1d84e37aa7ce39aa13 GIT binary patch literal 7533 zcmcIp33wFc8GirCG08AU3=xa>fw}=?Au1k0F^QpVL|HAyY=XOY7cvBt39>0K7IQ1{pZ-7VJ9Kbr{UR|`M>$@|GWO- z`Tsoi41nYDZY|EiHS%(;h7PP(@o^26ST7+PR9vS)^%mEwxIu&FEpAk?u?C;eFc~*V z%$sX)i@bkQ#jP4{!>2S%#U=^4UBw+5_Q#zn?o#n-6?bbm5cg=f7oX8E9rsD3`&B%k z;y9M za?t1>G?J<8&`>s$%=uV`V{uiYX%?vMv<5PUlebNQv-X(5f{sjAIU% zcJpcEI)$p8VFqHQa>q}QcbJNd+V_HNOr?5995Yw4USR$NxtC}btZ0??X~Tx?SkjyG z1QKI%>B~xXN%l4Ti66#muWQViq?7)aXYdLk!u}x#h;>Hi#*_4^fW$h$)-Lx}C zI+t7;z~};(2RT=#vYCEsKw$QG+&TlkZ7w$HYDkyM56LK+TTYouTV}?|G1NUI*CNuo z6r_&Zd((5}9GYByp7Bv&l0b}7ae8%t6YTj?0NL4oFSOUL)P=eI5rOII(j zGM00iKt-alTcEOyd&{isurlU(`JrCZ?lOAQgw%IrQ%1VmU=iWk2Ua>OEk@+Rz1N_N zkwq=4a;B3X7MQ*)pK+`qv)js96w{K)WF5B@Q3^F+(OQ+Ky2)0vKWobvtdpThXUecz zV)w}{m}Hx&tligbr1R8US&6=pId}eNS zG*WWsX3U*CS70EfJWe!{-Cb=g-!-|*<&{ghgyJ+`_B<(D^D5h!mW=hOr3{gL=g7{b z6t<*ee_X|4ShYJEwla3y_+ShIE;fAE*MIA5Uo08^RI+kOFz>G3^F`FX^ zzAbQi8SeOj;8%dG9W)Vv1$)avb~o0vJtU8Y%na9C-%;^h9pA(ErBy%B@gW!jGsj`< zF=3}-a5xr{ni{ zP4a$S$EEnNjz8d!I^Mu*D&ExbC;VB*U+`BQf5YEZyrtu9iTw|}qvN0W7w4@qp%b-7 zmK3(6(m1@Nca@nUXj&rFLTEqZpt)T(We?ZQFE^FhPpXYvhb8$O#cIAQtDqOT6cUYn zXu8kbg|%hVX^P}f+AR4G#uVPz^FX)^#f%-hr;*ZHW2Ehg?_8DlE0vs*G4xA~IW+8y zoNElTDkc1O<(foq#{3CO0wHtr)cyiALXMY|+V~R{n0g+R4mD~3P8iGV} zb)^?%VItS!$<{qQwDTkq!@GTPIrnmij26x4>+|X}^LSxgh{w7z#H>9SyCpl7K_h2N zmQR_h-npTL-$QIxOVR=O0B3ezn zGxy|D5h8%IevY44yi=%SSQ#?jFYFPo&NnwMm$tg{yKH$xsZ1XLl6V9pCg>!^;?;J$jJLdxT z&gNdk?@y>dkUx;MKvE)KGuw)mX8`A zCyheqGKxBHwXe6T=W1$C)zd_%n7I{oojsMj)pu@3Ljn7ZVjAD?-$PKX z{1xz?Q7GjgLC|!`DYoce$89LVR0$vlJhZ44!Y-G zTm+jdfgF9WNZ;dZRLOTZfZ`M7p4*0GNgItcV8l#EcZn-?^0+#DS501d-UU_%5|lTo zIq8%|lu#||wC$L?1M@1-!Sx*e>Sk6yji#Q8oxI59HXP^Oj@Njv_CY1OSp}THEh$HD z!2B>lLm1Ug0t zB03%$At>p%4GY#b;MDC{NNts^IDIYsET4sZuJxYhcvvlCVEGh2)=42OY8~Ph#q-~| zG8^Dixl+S;{&Nggk+&b@Nocj_0CBx!%fR^a9Tr z@A9NE3;kjl4=oqK68%^uYz&IadFtMPjJOS1aUU)cTVRXlkP|N=FJ8uK@ft1`Z()Q{ zyGGdumnsM0G9`h_mHD_rIRjU^!9UFlYCrhvloN0fwHm*+%6Py@{>z-;qStAnK z!UPOruZ+0aT{3GziICWaMLTe&K#(O8K<@QBaF)XVaXFjM~$d)+sJV&*MvfR(naPF4a`!8dd`}LwN#HpxD zJTIB#r9@tjBbX{{naJyy$k#BDuf=72z8V{_3D+@^ugAY}gE$s9iY{yr%W;!fiJOIk zTf~*PRcypGMGEj{15d#0 mP$#a!M`eE}@!D8oF#%#S|ClRc9nR&wp6kPTmwyL(xBmie6o}{m literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/test/KStreamTestDriver.class b/out/test/stream/org/apache/kafka/test/KStreamTestDriver.class new file mode 100644 index 0000000000000000000000000000000000000000..b4e2ddd5a4741806d524d89d64d2d83b0853ecf8 GIT binary patch literal 2796 zcmcguU31$+6g_J@QEU_vQV7&gT1ZPB$EoTzDQ%O6e6>L9Bn6w4J}QdpsQ!>gl2gi0 z;5YEZ3o|^?8S2c?8F=SMF`QjFk&E2m2?Gya?cLpb?zwwE)_?r@>u&(&@GyhdaU+8< z+{|DBi)o6tG%RH>gioZfoWTlKrMM=CbvfLY!yP$%s^K#Yp9_oxZp*E^)u#fhCD*GO zX3MnpY@=pAs+oo#c(&PanpLCaxt8twu4g#S!1kJE-8b$Zu2s4i3j&(ua{rq^pb&=( zY(FqIT&s4ZxfY(L;_#x=bb=*;Yq_{$#G@8YjeBK9ZME&xcWrNBdRrj1=I+`8VRg#9xt3YqHa$nm-NjU}=lBBCabt>Q7r#Qr>z=bu6ZCB-v?g#huJw-; z1-?ItXK_oG+i18=!?)>A-TBrG97^S`M4|&~tlNhO-TTW#vcQ!AQF`nO=(|0b0&@dm zyu?Ic^5t+nZ5@Veen7m4(_(&!ar<~cyL#m~#lj+9=vmp?d2CxjpB0IcW>aviO|zkjCk7*stl4ZB6LCC&p?$O7 zrru~dFs<5#*;4hU;V#Mfc2IF7XexI%v_r`c%tlK<@W5!*4yYE`<-nC!e!GtkeF`=z zy{Jn?Fie0{Q@N5UyKT?1?>Mq4OvWnwl5|wZn>Ztfdm2hQHn6E<9B&BZW6klE-At-) z>6pMN4fl0?fiHEGQPJ@gwskzf*E%kvsI!aD3ta4v8)GLh9t9$dp1`SSsj}y}Pr`ne z%*o3w#(<-H>fc2fJk{X@)^h`1^YM>?)2C$qtM0XYjJr`<9K)ASkgxJXVC&?07a0hX zZIf>jGjOEU_V?smV5QHcM3Fg`ssdAG{>b&7nBFear)xaUqn|?zz6(; zF~l?)(lxF#F%)I7w+4DV_mF@-splKs@6-fwwtk)cQGS#CnJFWOxpz zk)=cpyo>jUBGJxpPojNC^c26@`~edAnGTZAk;=co(8I#)0fwJLllV!6KcP-eaXb$l z7l@lvcJd*@k)3SU4pH$w&MFpJoFl6wJ|tSmOYTh_?Vab|1@0yA5&2yta_H?(Tp!|k zG5-rR{!*kk(m`5^h1m`=a?-(T4}Qe>&D5cAI`#WALXDB$6qTn4Ex(}~SqyVN!`)f# z8zgp_yN2u^J?<2b+$mxjdD0;XY84daB6SPIyTnxu2}^^a^N559d9QFY$xTJ{YzO+^ z!!*Zm%qx50b)=8%rEvvU3B$*pCqlSev?_zH;CKb2KT`;!KFlM)^;bY$Q`GrhtMk|Y E0vV+UhyVZp literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/test/MockProcessorContext.class b/out/test/stream/org/apache/kafka/test/MockProcessorContext.class new file mode 100644 index 0000000000000000000000000000000000000000..bbb3e073720134940a4fe12794380e4d7942cbab GIT binary patch literal 5842 zcmbtY`+F4C8GdK8$!s>01R}S9O0X6-o0wGu1QJt$fCd7niJ@pO4x33bB-vSJXSuc2 zTClCX*o(c`i?nJhw#8Oj5*n$}+tVNVJpDiVH~91^eZMoayOZn$Rth{bXU_S)_q&|$ zd(T<^`QP_G0I&srPa=);I$lg7hVzZ+#Fuq^C5d|f5hA2;DA_^CWz({Vx8ekN-_ zm(lAY@J0e}Ch&^{ewn~q;?PC0^|p+DC68b0_>GQBI^NNcELg5>PTOZKSHtEZ$DPQS zc{4j{Wv0yWDKnFGW@emRrX*#0Hou0;I=vcXa<=1DVC^z7-(F4r~B z3|J?MN@&=g$eCWzrNz-mp}zmmkA8ZRD0q%*9W?VAT0=HC!Nrk-C0kf*HwTV|LP>9+R_$4aTxvXfP0DvQ;TjWaI!{s~Z{Z)$!ZXDSJW3 z?=(EvcVK6=jnRm?{uEINA@)->c{(VfN#*KbZp=DO@+TE+`N$R(PJ{5J?I}W$I`W*n z&0<(lYKQP4#RTO|*R$nTz|+PVA15^_O&&gpe6cX;r*2n`Rln4^F7fGkC-0*@64iWB z1+x?!lQrxV-K-^jt)V^Ykm-@$H_!`ab4AoW!oX%cs9||!?J=`3$-+0V4G$UEft?1r zai4)6WLR@bP9^WDT%nlHJFaJq9nM>>x>M~vowf3Q8rutUsFQnIENh&%6U zSQ9Wz)w+hYAsVPsHgQ(RWdpy*9}HZ-{*1p!fDah>E8aKoHwh?R70{A57M>=R?e|$j!`3SOsN^rc{E&ev6*tH; z#U$-(xTAtUGU+;}{42o9FntXB7fb1-<_d(mpw{J!Y}@T&>7{A0n#I`hWO)r6BLIR! zR26=iFB6KZT#Ka9R^k{awUqkHZRn*NQ&S5==4YSbG zE0CB)GTk+crgZl#jLVd-LqDH)L+7&xaiq{n57*MS4fJXwtGJ6^eS)hUK0Y47E`HIH z@?TRUi6DmE9IxWrU)w|fWYmKJ&iT>o<=j3z%BT3;ANbsmUQfToo0#%Oes#*vJbF`*$MHmjC&pq=hL!|*lAa9L2J*yB zK}PTtp(H0;xTlhn6MuqQBqI_JFhG)6J%=^(XxDJ)zIm+G_#Z*{bFfaMw|DN0r`zLm zxT`(h)jfy1({osV7C{&ks{o7mjjh_?;f)jO`*_Z}oTcY*}sQo^Im zNfO#AN}Ob*r?CfTl*eh(%?74>f&>#6ugZBCz$9#XE_25@3R8=$O_kTCxhC7N3Gx4+ zS;x!`GFZKioDN6*o6(VnrsKqiX#9}Eeha!PEd_o{&Hg(`%jO_0cg>?icJ0=AkR+^O z2%D4(o$Bo;!DrOx$mY*e;swf_!@W39iE|A1MU? z+!KNDawNj%%Iv+6H9i)B@Jb}Y=kbL|AMTAncr_AXM;Sp*uo9am-(iyd!^l5GUJEIg z*j$raW?+6lrDN2Qu8()k;p1}3Zmb+>KEb@s&Ul0Wd;6wBtL}ce07%F3&Klqz32@g5 z?i@C;D2LMiIK?q-bXBOuH}V#ma1rfzJ7i>CP-9J4CihZ;$K(PY7zv+a8!K>rr&G|6#kZ$hQ@%En|Be5P_7X~Pzop%L9hZ^21ouaVSzwNI!VgNGx#*lC^|ah z1NcymXOq&4=`eLVv)Oa@?03HNos;zUpC7*fOd=OYD;}$8!(<#!Fr{ExMHG{AdD?V10zo89yil|ZugE>} zM^kIkr3S3G%oFmlHNrFUwkz>c@^~$r@;?i|=m*hKb9T`&#Ih-etE;9zG{9rBI1LG0 z)6foG!IFlTSZ26hsXFI5)TPCuncowRh84Ub(lc_is$mVU6=XHM!McVGEGyX5@D}eF zCL4Q&YKM^_QAr$hhhZ#)bZOTa`a>7$zP++H!RU!S(k4*7=$Uza!EyLm)^t4*Fvbf7 zVNt+noyi%jYE_L+bg?ErI80v_-{RRp6hmh!97u>AzBaWCX=wPKq+gHS4)wo0eiYPJ z5jSgF2lQU}gS@d%*fg3|#mM6{dg+_m2Kr2aB@%K<=u_KFb~`#~?IddxP4olQq2aG! zL*LN!nXCx9XpK<}kVH4F-9Z$t;}%&I@1rr?roHCJsdRRSEQY(dS3+MQTcWdnK~q(r zG^{01h92Cfs})dr7J&BR0YOR1pNL+Bp2?vvU|$gViRg!KXdXUCtX(-r%Lr}G5kH6e zzLHk;d5jRqsGs_N08vhzq&oZjI+A!uyC$DRB}$TzS|rh5jvAn-6nUF4NWLKq(<%w> zln5pQf>(OJL#xMTocc0RB}AgkCV>&+SuKP-3xpVhUdl5^xnzez>Q4KhL0c}=4Qm=H Wk$;47zlaIkq^C-dt@P)51MDvWhR7ZO literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/test/MockProcessorDef.class b/out/test/stream/org/apache/kafka/test/MockProcessorDef.class new file mode 100644 index 0000000000000000000000000000000000000000..0a7f80b8c5a95ddfd7f2526590f4d3bfecefd7e5 GIT binary patch literal 1161 zcmb7E+iuf95IviOI1cG8ZOWw-8d};Um1;nJB2gq%KteSaDUqc1?Z1c9#@3U576sDnNa0ueBngZN4uGnDKze!;C!%f9teDSm0o zK-VdzCA%Kip%M($X)%o$>~pVcwX~96x89_@{2=sP9daF!@BAy#_LH=1LwZs-8Pb)i z!;m@fkE!s2Ej{r*JnsnA=AAwvMca3I-{DF|=LF2?Q%Pf0Y+rRPKH%=DuzLKY$1N=a zZHKe~8pHN%xeSZLCWea^i^K42BKNN<)7T8VGw(;$L785b30h}*YfI6< zvLQT%#R@O4tqyhiqjiW;9NAg0d_X*NbE%?Zi~i73;_ Wd=K%6PAP0q6QKDS{abRl4fYcYa4cv5 literal 0 HcmV?d00001 diff --git a/out/test/stream/org/apache/kafka/test/MockSourceNode.class b/out/test/stream/org/apache/kafka/test/MockSourceNode.class new file mode 100644 index 0000000000000000000000000000000000000000..a36245fddcb54739c6b9026d40b797296932cc7f GIT binary patch literal 2292 zcmbVOYf~Fl7=8{31j4r1v{0?37mFnUl2uzTT#TlHY9xRHO{JeaY>w%cWS7}&n86u; zf`7#swGA_N#vkC1a(vEi2#Jg0knTf@MRJP@|q~=<10bo`2}x0JweX>)b$TY>pdxaHe*({dfF<$22SP3gN0+cFEm zRdRe)Qy#61JFUjHvXp(Kssf@!p9ku=DKOb%rr>$?--kl!Cj!f!NO$80LkmUa6F>D#1M z)hW#Y2GPk>7-Z3{GJuJ)?WoOGV_$i@a=%Vas_a^_UXk=jy2Fh7&us>BKkCg_P2b#b zt%Du6fs!4x;VRG<`<(17eYF% z?O?j0-_{_74jm7KSbNnaPaJx#rJAhcY`SNjw)S5r%U_Oiv`&nL>>t05IL9q$Z;CLZ z`mE^LNQddws-r7bz0=yUv}QC`)|aAQD(CR>(KwsFr(}bJZ+4<^J=5k8dycF(&9mm~ zsc$mmKDKpRU+wSFoZcq_*Z;Xz9<}UxRe1*PVo_iwGLYwVlz_3ZRb0>RY(3d7u4N5u z;cEle@uq=qu&v1*1G{*lZB`7tg&RZ@_zgBxceNOJ8#fKy*2hzDk7dq;E?8sQ^q#(E-tWZl6K!kj({p<-F&_U;Lw$+M z^KHEGl#hu8{Wyb}V#uJoBo>&`ER&P#fjHfTIE@eRAaFj-e;p6{AZn=)qL%uwEA>$q s;>Q@pCk$T%X`TrH0>*1x*^8eb^&3VsB2GTfkwE`;S9b}^MJqJt#|%MMl;3i~Ql_JAR~S#2`p4)u*> zm}sa_o_E_VX|6>pAY`hcT@f^eQE}b_bM{U}3|kFtJT5xIy_4J*x4z(3MwXvx*S}V6 z3Prnf^k|K6tu}iMj!YAV-Oa|mcn~}gp~tVRQK7f@3%O|BOV!dv~>Fycw)H>M_$~bvO zK3{tSdr6*4WD5y6hf#{lX#x2$${5BmK{h5|l30#>vHpq7E3(fNXVB*_CHz`i)8kV$ XDws?(iow;bYz0 z)6FNkxu3>37E&xbAjZcbn*>#BrsWhCJ&|M zNO#}*YWJC{g=K-OdN~NCA8y-CB_P%Xu4tKZ>!*@Lcwh(l`6QMF#=mwQyJ?52{*CR{ zz3&CgHP==CvLk~)1p*JNp1*I&maH8pt0DIq(h5}&S{L+I)L#CAh*%O8+qJ_b0W~|i znbSkIL))>|LgmZQ^UJ*lUtcfm2pG#=oo61e+OB%qZtg07TkblPOjf;`batd~>v>0L z=s*ZmM#h$(=OJT1+!eUq52Zl%yxXHz*Y%Hm_V>XNguaqZoB3_|UQGpo=UcYRD7exI ztgTQsTdF=#uW@R^+E7P3(rK&8_C|Sor@R#Xn4vw}p*ewSp9~GpVjUYMs<|DDSuHhJ-SObqFtS9gyufk3ej^;l&4)GhLKslA z*4u0o)B^eT4%?fNr5xVnNPjs^&}cn}TX+vKyiaMCy0OJZ zVEEaF4V66hJc^J@jJ6{#18SU`z`80Io7#Cw|y8t*D48gHstdI|Fdu4@1B$p7mB rGNWLmI)1gc^Ae+D{0=Ox#c&t%>6x}zErX2>Tn8vpREt;m#6y=>#h4PTL0+LcFia1Sf+M($rYcfIjq~G+% z_(9eZin5kJz#paVGm{sYq2`0>TDgxocb|RE-RDgD*WZ8q3E()KF-&05Kn6=GEE|Yp zNp`;6gKM~+!j0sQG?S0P-LH~A{1bdqxGAZU@XW;Z>B%<=>4p;c&kcCD~%J6pQ4 z%>AOmSjpaSEU)Eig`Xmg=9cU3M!{-Wb+cJ_3%28F z*Rg8NLdmnX8@jUVxvc=Au%sWaTD6v*TUjcutQP0}KZOx*({4`AD&)h?J>B#Q17Tm( zRmL(hXFIkxuW&Rs{IRK3CTXEwVf_1<8GWs_y{_FAYrV#u1IzWYRa>=OTh{H(gpgA> zJG76<0k))?c^UX|qXT|nXw**1X2R$R$paZiq#^3+Zgehjq0UQue<`8dx}dz zEz*>lQ8<|65Q#SD3ySB2#rZ0IC>S!opsRJ!dp?R+BozIO(X;EVkOFg~ok}pv`raeY zblk@qE#5^g!dE!RMag{fp;Uy#w@SZ;c;L)`lyEHZpU$^=7Sqw4a4XeYZdqTprB>b;&>?GbMsCr> z`#5MKiv<#o>Ov;o#yct8H}M@Tg?+uNU#>eFCXV71wH#_fujm7V#Ws1&4CfKoge zf{BuHsF$UnO;kZLS!|lH@gRjQ6E$oroDM+_$xLCq7iv%ztd$U2kKoJqpm1Ur+g@WH ztZs!-S8s?QSP#`9TLgQ$r%NnRUEED6J+C)eskFY_a>{-?8Ra|5uW|Oij(YvKQD5!r z2r@(FvKZA(!YW_%t7v+cymfNic`>nxH7uK%Dh*Y@Q@dqn5Sta0i9q!^#;pqFIuY?$ z3OE;v;1zjdjap@SLr>1m@^v~te1?k=Ee419TO|Y5I%51|*>e6af5XJ^-uSrGIp=%0 zm!+k`2VC9fua+3ciTw0asQeCM`4@<WI( zW=Kb(sbB;Njz)eujnV85j9@X>^bHgkn9w$G44)89`~*RYmH3fo z1$;`YAfv}QI}znMo5r39&-4GQ=aXH})ppFLA9J+tg+AYAH)^vRwHb<9tb(Y;sz2(O zH2ikHA5jn^%uM_~r&<+kf!|S&PwN|1;SNb?*x|}jy zP8pn|E~Gv(eum;cr$9YoA8-xCAG{9YfF3PH(E0Cn-dp{yaHF5z;U)jUUuGZz1a p-~F|x`>5J|gsk);N4CV`^gu6T7o+mtpgyKWNu+%u@@3}qTs%JNSSE^o>Aytz~`gMjC zEBd(9Ch_j5OX6HZD)DkW>a`>RlU8%>D$%KC(X~lA5BXjT4|xNB>~=&rGSc_NuLRwNA8Us}wDSAtRONbLEin9eN{T&+}cu zhm?uk54(04CK0nu*;CvfyQF=19_^GXgFBA!ykezNVW?Zt_2n#TBta^t0@$nABV7MDm|EElbxTR10&Uv<{jdl7qQ(#EbwWKSjPc}h)6Bp^eK%=*a z(H}6%-w`Wse*^nMqd0POU!WG)AjKxKC?ZdzLijcoiKl-?KxMc@eUq4aW)aKOGU}~N zR(V3LewHkM1^Y~$Z~<)%*b2a*I5`0y + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1.6 + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 89395b39873d0..fcba969141f40 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -83,7 +83,12 @@ public interface KStreamContext { /** * Ensures that the context is in the initialization phase where KStream topology can be constructed +<<<<<<< HEAD +======= + * Flush the local state of this context +>>>>>>> new api model ======= + * * Flush the local state of this context >>>>>>> new api model */ @@ -94,7 +99,11 @@ public interface KStreamContext { void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); +<<<<<<< HEAD void schedule(Processor processor, long interval); +======= + PunctuationScheduler getPunctuationScheduler(Processor processor); +>>>>>>> new api model void commit(); diff --git a/src/main/java/io/confluent/streaming/KStreamInitializer.java b/src/main/java/io/confluent/streaming/KStreamInitializer.java new file mode 100644 index 0000000000000..43464dc884adc --- /dev/null +++ b/src/main/java/io/confluent/streaming/KStreamInitializer.java @@ -0,0 +1,62 @@ +package io.confluent.streaming; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * KStreamContext is the interface that allows an implementation of {@link KStreamJob#init(KStreamInitializer)} to create KStream instances. + * It also provides access to the system resources for a stream processing job. + * An instance of KStreamContext is created for each partition. + */ +public interface KStreamInitializer { + + AtomicInteger STREAM_GROUP_INDEX = new AtomicInteger(1); + + /** + * 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(); + + // TODO: support regex topic matching in from() calls, for example: + // context.from("Topic*PageView") + + /** + * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + KStream from(String... topics); + + /** + * 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 + */ + KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); + +} diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java new file mode 100644 index 0000000000000..3e97475b8e9ad --- /dev/null +++ b/src/main/java/io/confluent/streaming/KStreamJob.java @@ -0,0 +1,35 @@ +package io.confluent.streaming; + +/** + * An interface to implement an application logic of a stream processing. + * An instance is created and initialized by the framework for each partition. + */ +public interface KStreamJob { + + /** + * Creates a job instance + */ + + /** + * Initializes a stream processing job for a partition. This method is called for each partition. + * An application constructs a processing logic using KStream API. + *

+ * For example, + *

+ *
+   *   public bind(KStreamContext context) {
+   *     KStream<Integer, PageView> pageViewStream = context.from("pageView").mapValues(...);
+   *     KStream<Integer, AdClick> adClickStream = context.from("adClick").join(pageViewStream, ...).process(...);
+   *   }
+   * 
+ * @param context KStreamContext for this partition + */ + void init(KStreamInitializer context); + + /** + * Closes this partition of the stream processing job. + * An application can perform its special clean up here. + */ + void close(); + +} diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index bde4a5a9714cb..e2d86f61e37f6 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -22,6 +22,7 @@ <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.internal.IngestorImpl; <<<<<<< HEAD import io.confluent.streaming.internal.StreamSynchronizer; @@ -47,6 +48,9 @@ >>>>>>> new api model ======= >>>>>>> wip +======= +import io.confluent.streaming.internal.TopologyAnalyzer; +>>>>>>> new api model import org.apache.kafka.common.metrics.Metrics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -188,6 +192,7 @@ public KafkaStreaming(KStreamTopology topology, StreamingConfig streamingConfig) if (streamingConfig.timestampExtractor() == null) throw new NullPointerException("timestamp extractor is missing"); this.config = new ProcessorConfig(streamingConfig.config()); +<<<<<<< HEAD this.topics = topology.topics(); <<<<<<< HEAD @@ -219,6 +224,12 @@ public KafkaStreaming(KStreamTopology topology, StreamingConfig streamingConfig) public void commit() { throw new UnsupportedOperationException(); } +======= + + TopologyAnalyzer topologyAnalyzer = new TopologyAnalyzer(jobClass, streamingConfig); + + this.topics = topologyAnalyzer.topics; +>>>>>>> new api model @Override public void shutdown() { diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index dc30cf4ae30be..0f56ed2a657d3 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -5,7 +5,33 @@ */ public interface Processor { +<<<<<<< HEAD void init(KStreamContext context); +======= + public interface ProcessorContext { + + void send(String topic, Object key, Object value); + + void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); + + void schedule(long timestamp); + + void commit(); + + String topic(); + + int partition(); + + long offset(); + + long timestamp(); + + KStreamContext kstreamContext(); + + } + + void init(ProcessorContext context); +>>>>>>> new api model void process(K key, V value); diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java index cdcfa0cfab084..8231c8ed790c9 100644 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -5,6 +5,7 @@ */ public abstract class ProcessorKStreamJob extends KStreamTopology implements Processor { +<<<<<<< HEAD private final String[] topics; public ProcessorKStreamJob(String... topics) { @@ -32,6 +33,12 @@ public void topology() { ======= ((KStream)from(topics)).process(this); >>>>>>> fix examples +======= + @SuppressWarnings("unchecked") + @Override + public void init(KStreamInitializer initializer) { + initializer.from().process((Processor) this); +>>>>>>> new api model } } diff --git a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java index cef0b7689b293..b617badc3e7aa 100644 --- a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java @@ -1,7 +1,12 @@ package io.confluent.streaming.examples; import io.confluent.streaming.KStream; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.KStreamJob; +>>>>>>> new api model import io.confluent.streaming.KafkaStreaming; import io.confluent.streaming.KeyValue; import io.confluent.streaming.KeyValueMapper; @@ -18,7 +23,11 @@ public class MapKStreamJob extends KStreamTopology { @SuppressWarnings("unchecked") @Override +<<<<<<< HEAD public void topology() { +======= + public void init(KStreamInitializer context) { +>>>>>>> new api model // With overriden de-serializer KStream stream1 = from(new StringDeserializer(), new StringDeserializer(), "topic1"); diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index 2ae4c076bd1e9..a1e0e716ac3eb 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -26,8 +26,13 @@ public void init(KStreamContext context) { this.context = context; this.context.schedule(this, 1000); +<<<<<<< HEAD this.kvStore = new InMemoryKeyValueStore<>("local-state", context); this.kvStore.restore(); // call restore inside processor.init +======= + this.kvStore = new InMemoryKeyValueStore<>("local-state", context.kstreamContext()); + this.kvStore.restore(); // call restore inside processor.bind +>>>>>>> new api model } @Override diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 880676e936e51..2074f57b5da28 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -1,7 +1,11 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import io.confluent.streaming.Predicate; import java.lang.reflect.Array; @@ -16,23 +20,37 @@ class KStreamBranch implements Receiver { final KStreamSource[] branches; @SuppressWarnings("unchecked") +<<<<<<< HEAD KStreamBranch(Predicate[] predicates, KStreamTopology topology) { this.predicates = Arrays.copyOf(predicates, predicates.length); this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { branches[i] = new KStreamSource<>(null, topology); +======= + KStreamBranch(Predicate[] predicates, KStreamInitializer initializer) { + this.predicates = Arrays.copyOf(predicates, predicates.length); + this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); + for (int i = 0; i < branches.length; i++) { + branches[i] = new KStreamSource<>(null, initializer); +>>>>>>> new api model } } @Override public void bind(KStreamContext context, KStreamMetadata metadata) { +<<<<<<< HEAD for (KStreamSource branch : branches) { branch.bind(context, metadata); +======= + for (KStreamSource stream : branches) { + stream.bind(context, metadata); +>>>>>>> new api model } } @SuppressWarnings("unchecked") @Override +<<<<<<< HEAD public void receive(Object key, Object value, long timestamp) { for (int i = 0; i < predicates.length; i++) { Predicate predicate = predicates[i]; @@ -47,7 +65,17 @@ public void receive(Object key, Object value, long timestamp) { public void close() { for (KStreamSource branch : branches) { branch.close(); +======= + public void receive(Object key, Object value, long timestamp, long streamTime) { + for (int i = 0; i < predicates.length; i++) { + Predicate predicate = predicates[i]; + if (predicate.apply((K)key, (V)value)) { + branches[i].receive(key, value, timestamp, streamTime); + return; + } +>>>>>>> new api model } + return; } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 31e887550a92d..2cceef6bcdf98 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -2,6 +2,10 @@ import io.confluent.streaming.KStreamException; import io.confluent.streaming.Processor; +<<<<<<< HEAD +======= +import io.confluent.streaming.PunctuationScheduler; +>>>>>>> new api model import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StateStore; import io.confluent.streaming.StreamingConfig; @@ -17,11 +21,15 @@ >>>>>>> wip import org.apache.kafka.clients.consumer.KafkaConsumer; <<<<<<< HEAD +<<<<<<< HEAD import org.apache.kafka.clients.producer.Producer; >>>>>>> wip ======= import org.apache.kafka.clients.producer.ProducerRecord; >>>>>>> new api model +======= +import org.apache.kafka.clients.producer.ProducerRecord; +>>>>>>> new api model import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.ByteArrayDeserializer; @@ -113,6 +121,7 @@ public Deserializer valueDeserializer() { } @Override +<<<<<<< HEAD <<<<<<< HEAD public KStream from(String... topics) { return from(streamGroup(getNextGroupName()), null, null, topics); @@ -193,6 +202,8 @@ private KStream from(StreamGroup streamGroup, Deserializer keyDe @Override ======= +>>>>>>> new api model +======= >>>>>>> new api model public RecordCollector recordCollector() { return collector; @@ -214,10 +225,17 @@ public Metrics metrics() { } @Override +<<<<<<< HEAD public void restore(StateStore store, RestoreFunc restoreFunc) { ensureInitialization(); stateMgr.registerAndRestore(store, restoreFunc); +======= + public void register(StateStore store, RestoreFunc restoreFunc) { + ensureInitialization(); + + stateMgr.register(store, restoreFunc); +>>>>>>> new api model } public void ensureInitialization() { @@ -232,26 +250,50 @@ public void flush() { @Override public String topic() { +<<<<<<< HEAD if (streamGroup.record() == null) throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); return streamGroup.record().topic(); +======= + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); + + return this.streamGroup.record().topic(); +>>>>>>> new api model } @Override public int partition() { +<<<<<<< HEAD if (streamGroup.record() == null) throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); <<<<<<< HEAD <<<<<<< HEAD stateMgr.restore(store, restoreFunc); +======= + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); + + return this.streamGroup.record().partition(); + } + + @Override + public long offset() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); + + return this.streamGroup.record().offset(); +>>>>>>> new api model } @Override - public void register(StateStore store) { - ensureInitialization(); + public long timestamp() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); +<<<<<<< HEAD stateMgr.register(store); ======= return this.streamGroup.record().partition(); @@ -283,11 +325,23 @@ public void send(String topic, Object key, Object value) { } @Override +======= + return this.streamGroup.record().timestamp; + } + + @Override + public void send(String topic, Object key, Object value) { + collector.send(new ProducerRecord<>(topic, key, value)); + } + + @Override +>>>>>>> new api model public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { if (keySerializer == null || valSerializer == null) throw new IllegalStateException("key and value serializers must be specified"); collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); +<<<<<<< HEAD } @Override @@ -301,6 +355,21 @@ public void schedule(Processor processor, long interval) { } void init(Collection> streams) throws IOException { +======= + } + + @Override + public void commit() { + this.streamGroup.commitOffset(); + } + + @Override + public PunctuationScheduler getPunctuationScheduler(Processor processor) { + return streamGroup.getPunctuationScheduler(processor); + } + + public void init(Collection> streams) throws IOException { +>>>>>>> new api model stateMgr.init(); for (KStreamSource stream: streams) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 55edcf5c133c0..3fef6205ae454 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -1,6 +1,10 @@ package io.confluent.streaming.internal; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import io.confluent.streaming.Predicate; /** @@ -10,8 +14,13 @@ class KStreamFilter extends KStreamImpl { private final Predicate predicate; +<<<<<<< HEAD KStreamFilter(Predicate predicate, KStreamTopology topology) { super(topology); +======= + KStreamFilter(Predicate predicate, KStreamInitializer initializer) { + super(initializer); +>>>>>>> new api model this.predicate = predicate; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 1748f7b136a1e..ba8a4637dd4ff 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -1,7 +1,11 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; @@ -12,8 +16,13 @@ class KStreamFlatMap extends KStreamImpl { private final KeyValueMapper, K1, V1> mapper; +<<<<<<< HEAD KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology topology) { super(topology); +======= + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamInitializer initializer) { + super(initializer); +>>>>>>> new api model this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 7e0c7e32acdc0..5ca0ccc8fa335 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -1,6 +1,10 @@ package io.confluent.streaming.internal; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import io.confluent.streaming.ValueMapper; /** @@ -10,8 +14,13 @@ class KStreamFlatMapValues extends KStreamImpl { private final ValueMapper, V1> mapper; +<<<<<<< HEAD KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology topology) { super(topology); +======= + KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamInitializer initializer) { + super(initializer); +>>>>>>> new api model this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index af0d4ba670265..1fe491f9eef30 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -2,7 +2,11 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import io.confluent.streaming.KStreamWindowed; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.Predicate; @@ -21,6 +25,7 @@ abstract class KStreamImpl implements KStream, Receiver { private final ArrayList nextReceivers = new ArrayList<>(1); +<<<<<<< HEAD protected KStreamTopology topology; protected KStreamContext context; protected KStreamMetadata metadata; @@ -45,12 +50,31 @@ public void close() { int numReceivers = nextReceivers.size(); for (int i = 0; i < numReceivers; i++) { nextReceivers.get(i).close(); +======= + protected KStreamMetadata metadata; + protected KStreamInitializer initializer; + + protected KStreamImpl(KStreamInitializer initializer) { + this.initializer = initializer; + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + this.metadata = metadata; + int numReceivers = nextReceivers.size(); + for (int i = 0; i < numReceivers; i++) { + nextReceivers.get(i).bind(context, metadata); +>>>>>>> new api model } } @Override public KStream filter(Predicate predicate) { +<<<<<<< HEAD return chain(new KStreamFilter(predicate, topology)); +======= + return chain(new KStreamFilter(predicate, initializer)); +>>>>>>> new api model } @Override @@ -64,32 +88,56 @@ public boolean apply(K key, V value) { @Override public KStream map(KeyValueMapper mapper) { +<<<<<<< HEAD return chain(new KStreamMap(mapper, topology)); +======= + return chain(new KStreamMap(mapper, initializer)); +>>>>>>> new api model } @Override public KStream mapValues(ValueMapper mapper) { +<<<<<<< HEAD return chain(new KStreamMapValues(mapper, topology)); +======= + return chain(new KStreamMapValues(mapper, initializer)); +>>>>>>> new api model } @Override public KStream flatMap(KeyValueMapper, K, V> mapper) { +<<<<<<< HEAD return chain(new KStreamFlatMap(mapper, topology)); +======= + return chain(new KStreamFlatMap(mapper, initializer)); +>>>>>>> new api model } @Override public KStream flatMapValues(ValueMapper, V> mapper) { +<<<<<<< HEAD return chain(new KStreamFlatMapValues(mapper, topology)); +======= + return chain(new KStreamFlatMapValues(mapper, initializer)); +>>>>>>> new api model } @Override public KStreamWindowed with(Window window) { +<<<<<<< HEAD return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, topology)); +======= + return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, initializer)); +>>>>>>> new api model } @Override public KStream[] branch(Predicate... predicates) { +<<<<<<< HEAD KStreamBranch branch = new KStreamBranch<>(predicates, topology); +======= + KStreamBranch branch = new KStreamBranch<>(predicates, initializer); +>>>>>>> new api model registerReceiver(branch); return branch.branches; } @@ -104,7 +152,11 @@ public KStream through(String topic) { @Override public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); +<<<<<<< HEAD return topology.from(keyDeserializer, valDeserializer, topic); +======= + return initializer.from(keyDeserializer, valDeserializer, topic); +>>>>>>> new api model } @Override @@ -139,6 +191,7 @@ public void close() {} @SuppressWarnings("unchecked") @Override +<<<<<<< HEAD public void process(Processor processor) { registerReceiver(new ProcessorNode<>(processor)); } @@ -147,6 +200,18 @@ public void process(Processor processor) { @Override public KStream transform(Transformer transformer) { return chain(new KStreamTransform<>(transformer, topology)); +======= + public void process(final Processor processor) { + Receiver receiver = new Receiver() { + public void bind(KStreamContext context, KStreamMetadata metadata) { + processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); + } + public void receive(Object key, Object value, long timestamp, long streamTime) { + processor.process((K) key, (V) value); + } + }; + registerReceiver(receiver); +>>>>>>> new api model } void registerReceiver(Receiver receiver) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java new file mode 100644 index 0000000000000..f4b58e674c1e9 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java @@ -0,0 +1,78 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KStream; +import io.confluent.streaming.KStreamInitializer; +import io.confluent.streaming.StreamingConfig; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; + +/** + * Created by yasuhiro on 6/19/15. + */ +public class KStreamInitializerImpl implements KStreamInitializer { + + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + private final ArrayList> streams = new ArrayList<>(); + + KStreamInitializerImpl(StreamingConfig streamingConfig) { + this( + streamingConfig.keySerializer(), + streamingConfig.valueSerializer(), + streamingConfig.keyDeserializer(), + streamingConfig.valueDeserializer() + ); + } + + KStreamInitializerImpl(Serializer keySerializer,Serializer valueSerializer, Deserializer keyDeserializer, Deserializer valueDeserializer) { + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.keyDeserializer = keyDeserializer; + this.valueDeserializer = valueDeserializer; + } + + @Override + public Serializer keySerializer() { + return keySerializer; + } + + @Override + public Serializer valueSerializer() { + return valueSerializer; + } + + @Override + public Deserializer keyDeserializer() { + return keyDeserializer; + } + + @Override + public Deserializer valueDeserializer() { + return valueDeserializer; + } + + @Override + public KStream from(String... topics) { + return from(this.keyDeserializer(), this.valueDeserializer(), topics); + } + + @SuppressWarnings("unchecked") + @Override + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + + KStreamSource stream = new KStreamSource<>(topics, keyDeserializer, valDeserializer, this); + streams.add(stream); + return stream; + } + + Collection> sourceStreams() { + return Collections.unmodifiableCollection(streams); + } + +} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index 776815f211594..ef1031773c275 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -1,7 +1,11 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import io.confluent.streaming.NotCopartitionedException; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; @@ -24,8 +28,13 @@ private static abstract class Finder { private KStreamMetadata thisMetadata; private KStreamMetadata otherMetadata; +<<<<<<< HEAD KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology topology) { super(topology); +======= + KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamInitializer initializer) { + super(initializer); +>>>>>>> new api model final Window window1 = stream1.window; final Window window2 = stream2.window; @@ -86,6 +95,10 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { otherMetadata = metadata; if (thisMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) throw new NotCopartitionedException(); } +<<<<<<< HEAD +======= + +>>>>>>> new api model @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value2, long timestamp) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index bcb6ca4599bda..6db4b6d21076a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -1,7 +1,11 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; @@ -12,8 +16,13 @@ class KStreamMap extends KStreamImpl { private final KeyValueMapper mapper; +<<<<<<< HEAD KStreamMap(KeyValueMapper mapper, KStreamTopology topology) { super(topology); +======= + KStreamMap(KeyValueMapper mapper, KStreamInitializer initializer) { + super(initializer); +>>>>>>> new api model this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index 96f4c2e78c070..d332dfeee9edd 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -1,6 +1,10 @@ package io.confluent.streaming.internal; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import io.confluent.streaming.ValueMapper; /** @@ -10,8 +14,13 @@ class KStreamMapValues extends KStreamImpl { private final ValueMapper mapper; +<<<<<<< HEAD KStreamMapValues(ValueMapper mapper, KStreamTopology topology) { super(topology); +======= + KStreamMapValues(ValueMapper mapper, KStreamInitializer initializer) { + super(initializer); +>>>>>>> new api model this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index 22e8f4989975b..b50d35cd2410d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -1,7 +1,11 @@ package io.confluent.streaming.internal; +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import org.apache.kafka.common.serialization.Deserializer; import java.util.Arrays; @@ -17,6 +21,7 @@ public class KStreamSource extends KStreamImpl { private Deserializer keyDeserializer; private Deserializer valueDeserializer; +<<<<<<< HEAD String[] topics; public KStreamSource(String[] topics, KStreamTopology topology) { @@ -25,6 +30,17 @@ public KStreamSource(String[] topics, KStreamTopology topology) { public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology topology) { super(topology); +======= + final String[] topics; + + @SuppressWarnings("unchecked") + KStreamSource(String[] topics, KStreamInitializer initializer) { + this(topics, (Deserializer) initializer.keyDeserializer(), (Deserializer) initializer.valueDeserializer(), initializer); + } + + KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamInitializer initializer) { + super(initializer); +>>>>>>> new api model this.topics = topics; this.keyDeserializer = keyDeserializer; this.valueDeserializer = valueDeserializer; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index b91dd763c63e7..26517d98be0d9 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -17,7 +17,12 @@ package io.confluent.streaming.internal; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.KStreamJob; +>>>>>>> new api model import io.confluent.streaming.StreamingConfig; import io.confluent.streaming.util.ParallelExecutor; import io.confluent.streaming.util.Util; @@ -225,6 +230,7 @@ private void maybeCleanState() { private void addPartitions(Collection assignment) { HashSet partitions = new HashSet<>(assignment); +<<<<<<< HEAD <<<<<<< HEAD ingestor.init(); ======= @@ -240,6 +246,26 @@ private void addPartitions(Collection assignment) { context = new KStreamContextImpl(id, ingestor, collector, streamingConfig, config, metrics); context.init(topology.sourceStreams()); +======= + for (TopicPartition partition : partitions) { + final Integer id = partition.partition(); // TODO: switch this to the group id + KStreamContextImpl context = kstreamContexts.get(id); + if (context == null) { + try { + KStreamInitializerImpl initializer = new KStreamInitializerImpl( + streamingConfig.keySerializer(), + streamingConfig.valueSerializer(), + streamingConfig.keyDeserializer(), + streamingConfig.valueDeserializer() + ); + KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); + + job.init(initializer); + + context = new KStreamContextImpl(id, ingestor, collector, streamingConfig, config, metrics); + context.init(initializer.sourceStreams()); + +>>>>>>> new api model kstreamContexts.put(id, context); } catch (Exception e) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 7e6295f2d667d..79be9e4615aa5 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -3,7 +3,11 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import io.confluent.streaming.KStreamWindowed; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; @@ -15,7 +19,11 @@ public class KStreamWindowedImpl extends KStreamImpl implements KStr final Window window; +<<<<<<< HEAD KStreamWindowedImpl(Window window, KStreamTopology initializer) { +======= + KStreamWindowedImpl(Window window, KStreamInitializer initializer) { +>>>>>>> new api model super(initializer); this.window = window; } @@ -56,7 +64,11 @@ private KStream join(KStreamWindowed other, boolean prior KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; +<<<<<<< HEAD KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, topology); +======= + KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, initializer); +>>>>>>> new api model otherImpl.registerReceiver(stream.receiverForOtherStream); return chain(stream); diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java new file mode 100644 index 0000000000000..2874f8abbfe23 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java @@ -0,0 +1,68 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.Processor; +import io.confluent.streaming.PunctuationScheduler; +import io.confluent.streaming.KStreamContext; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serializer; + +/** + * Created by guozhang on 7/21/15. + */ +public class ProcessorContextImpl implements Processor.ProcessorContext { + + private final KStreamContext context; + private final PunctuationScheduler scheduler; + + public ProcessorContextImpl(KStreamContext context, + PunctuationScheduler scheduler) { + + this.context = context; + this.scheduler = scheduler; + } + + @Override + public String topic() { + return context.topic(); + } + + @Override + public int partition() { + return context.partition(); + } + + @Override + public long offset() { + return context.offset(); + } + + @Override + public long timestamp() { + return context.timestamp(); + } + + @Override + public void send(String topic, Object key, Object value) { + context.send(topic, key, value); + } + + @Override + public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { + context.send(topic, key, value, keySerializer, valSerializer); + } + + @Override + public void commit() { + context.commit(); + } + + @Override + public void schedule(long timestamp) { + scheduler.schedule(timestamp); + } + + @Override + public KStreamContext kstreamContext() { + return context; + } +} diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 28ac89c71411f..77ff2291d5510 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -9,8 +9,12 @@ public interface Receiver { void bind(KStreamContext context, KStreamMetadata metadata); +<<<<<<< HEAD void receive(Object key, Object value, long timestamp); void close(); +======= + void receive(Object key, Object value, long timestamp, long streamTime); +>>>>>>> new api model } diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/src/main/java/io/confluent/streaming/internal/StreamGroup.java index a08c4c06efeee..cb6f875b579c5 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/src/main/java/io/confluent/streaming/internal/StreamGroup.java @@ -74,12 +74,15 @@ public class StreamGroup implements ParallelExecutor.Task { ======= } +<<<<<<< HEAD <<<<<<< HEAD public String name() { return name; >>>>>>> removed some generics } +======= +>>>>>>> new api model ======= >>>>>>> new api model public StampedRecord record() { return currRecord; } diff --git a/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java b/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java new file mode 100644 index 0000000000000..8dee74ed57a57 --- /dev/null +++ b/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java @@ -0,0 +1,36 @@ +package io.confluent.streaming.internal; + +import io.confluent.streaming.KStreamJob; +import io.confluent.streaming.StreamingConfig; +import org.apache.kafka.common.utils.Utils; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * Created by yasuhiro on 7/30/15. + */ +public class TopologyAnalyzer { + + public final Set topics; + public final Collection> streams; + + public TopologyAnalyzer(Class jobClass, StreamingConfig streamingConfig) { + KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); + KStreamInitializerImpl context = new KStreamInitializerImpl(streamingConfig); + + job.init(context); + + this.streams = context.sourceStreams(); + Set topics = new HashSet<>(); + for (KStreamSource stream : this.streams) { + for (String topic : stream.topics) { + topics.add(topic); + } + } + this.topics = Collections.unmodifiableSet(topics); + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index b9f1ffb3c38fe..4c53b72b406c1 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -42,7 +42,11 @@ public boolean apply(Integer key, String value) { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +>>>>>>> new api model KStreamSource stream; KStream[] branches; TestProcessor[] processors; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 784c9cefe793f..ebad9b908356d 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -27,7 +27,11 @@ public boolean apply(Integer key, String value) { public void testFilter() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +>>>>>>> new api model KStreamSource stream; TestProcessor processor; @@ -48,7 +52,11 @@ public void testFilter() { public void testFilterOut() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +>>>>>>> new api model KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index c6d06a2ca8e44..4d079e7889bf1 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -34,13 +34,21 @@ public KeyValue> apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; +<<<<<<< HEAD KStreamTopology topology = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +>>>>>>> new api model KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); +<<<<<<< HEAD stream = new KStreamSource<>(null, topology); +======= + stream = new KStreamSource<>(null, initializer); +>>>>>>> new api model stream.flatMap(mapper).process(processor); KStreamContext context = new MockKStreamContext(null, null); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index d6903108df4ef..64febf0bc9ad7 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -33,7 +33,11 @@ public Iterable apply(String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +>>>>>>> new api model KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 77a91c1d282df..a7187f01ebad8 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -69,7 +69,11 @@ public void testJoin() { TestProcessor processor; String[] expected; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +>>>>>>> new api model processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -159,7 +163,11 @@ public void testJoinPrior() { TestProcessor processor; String[] expected; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +>>>>>>> new api model processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -243,8 +251,17 @@ public void testMap() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + processor = new TestProcessor<>(); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.map(keyValueMapper); + mapped2 = stream2.map(keyValueMapper); +>>>>>>> new api model boolean exceptionRaised; @@ -325,8 +342,17 @@ public void testFlatMap() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + processor = new TestProcessor<>(); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMap(keyValueMapper2); + mapped2 = stream2.flatMap(keyValueMapper2); +>>>>>>> new api model boolean exceptionRaised; @@ -407,8 +433,17 @@ public void testMapValues() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + processor = new TestProcessor<>(); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.mapValues(valueMapper); + mapped2 = stream2.mapValues(valueMapper); +>>>>>>> new api model boolean exceptionRaised; @@ -489,8 +524,17 @@ public void testFlatMapValues() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + processor = new TestProcessor<>(); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMapValues(valueMapper2); + mapped2 = stream2.flatMapValues(valueMapper2); +>>>>>>> new api model boolean exceptionRaised; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 93d3b47c352ec..605326a9a76fa 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -29,7 +29,11 @@ public KeyValue apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +>>>>>>> new api model KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 19c90e8c02970..9bd438c774e56 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -1,7 +1,11 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamContext; +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; +======= +import io.confluent.streaming.KStreamInitializer; +>>>>>>> new api model import io.confluent.streaming.ValueMapper; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; @@ -31,7 +35,11 @@ public Integer apply(String value) { final int[] expectedKeys = new int[] { 1, 10, 100, 1000 }; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +>>>>>>> new api model KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 293c5c9f9dc35..7f8deff0bf336 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -20,7 +20,11 @@ public class KStreamSourceTest { @Test public void testKStreamSource() { +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +>>>>>>> new api model TestProcessor processor = new TestProcessor<>(); KStreamSource stream = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 07a61466728ec..303db56da16cf 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -24,7 +24,12 @@ public void testWindowedStream() { KStreamSource stream; Window window; +<<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null) { + }; +>>>>>>> new api model window = new UnlimitedWindow<>(); stream = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java new file mode 100644 index 0000000000000..014e140685268 --- /dev/null +++ b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java @@ -0,0 +1,125 @@ +package io.confluent.streaming.internal; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import io.confluent.streaming.PunctuationScheduler; +import io.confluent.streaming.testutil.TestProcessor; +import org.junit.Test; + +public class PunctuationSchedulerImplTest { + + @Test + public void testScheduling() { + PunctuationQueue queue = new PunctuationQueue(); + + TestProcessor proc1 = new TestProcessor<>(); + PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); + + TestProcessor proc2 = new TestProcessor<>(); + PunctuationScheduler sched2 = new PunctuationSchedulerImpl(queue, proc2); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + + proc1.init(new ProcessorContextImpl(null, sched1)); + proc2.init(new ProcessorContextImpl(null, sched2)); + + sched1.schedule(500); + sched2.schedule(1000); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + + queue.mayPunctuate(999); + + assertEquals(1, proc1.punctuated.size()); + assertEquals(999, (long) proc1.punctuated.get(0)); + assertEquals(0, proc2.punctuated.size()); + + proc1.punctuated.clear(); + queue.mayPunctuate(1000); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(1, proc2.punctuated.size()); + assertEquals(1000, (long) proc2.punctuated.get(0)); + + proc2.punctuated.clear(); + queue.mayPunctuate(2000); + assertEquals(0, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + + sched1.schedule(3000); + queue.mayPunctuate(4000); + + assertEquals(1, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + assertEquals(4000, (long) proc1.punctuated.get(0)); + } + + @Test + public void testCanceling() { + PunctuationQueue queue = new PunctuationQueue(); + + TestProcessor proc1 = new TestProcessor(); + PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); + + TestProcessor proc2 = new TestProcessor(); + PunctuationScheduler sched2 = new PunctuationSchedulerImpl(queue, proc2); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + + proc1.init(new ProcessorContextImpl(null, sched1)); + proc2.init(new ProcessorContextImpl(null, sched2)); + + sched1.schedule(500); + sched2.schedule(1000); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(0, proc2.punctuated.size()); + + sched1.cancel(); + + queue.mayPunctuate(1000); + + assertEquals(0, proc1.punctuated.size()); + assertEquals(1, proc2.punctuated.size()); + assertEquals(1000, (long) proc2.punctuated.get(0)); + + sched1.schedule(2000); + sched1.cancel(); + sched1.schedule(3000); + + queue.mayPunctuate(2000); + assertEquals(0, proc1.punctuated.size()); + + queue.mayPunctuate(3000); + assertEquals(1, proc1.punctuated.size()); + } + + @Test + public void testDuplicateScheduling() { + PunctuationQueue queue = new PunctuationQueue(); + + TestProcessor proc1 = new TestProcessor(); + PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); + + assertEquals(0, proc1.punctuated.size()); + + proc1.init(new ProcessorContextImpl(null, sched1)); + + sched1.schedule(500); + + boolean exceptionRaised = false; + try { + sched1.schedule(1000); + } + catch (IllegalStateException e) { + exceptionRaised = true; + } + + assertTrue(exceptionRaised); + } + +} diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index db54814c6ee72..193f0025b8505 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -32,7 +32,11 @@ private static class MockKStreamSource extends KStreamSource { public ArrayList timestamps = new ArrayList<>(); public MockKStreamSource() { +<<<<<<< HEAD super(null, deserializer, deserializer, new MockKStreamTopology()); +======= + super(null, deserializer, deserializer, new KStreamInitializerImpl(serializer, serializer, deserializer, deserializer)); +>>>>>>> new api model } @Override diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java index 7e478764da471..67f5e0e8c4045 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java @@ -3,17 +3,26 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; +<<<<<<< HEAD +======= +import io.confluent.streaming.PunctuationScheduler; +>>>>>>> new api model import io.confluent.streaming.RecordCollector; <<<<<<< HEAD import io.confluent.streaming.StorageEngine; ======= import io.confluent.streaming.StateStore; <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.Coordinator; >>>>>>> wip import io.confluent.streaming.internal.StreamGroup; ======= import io.confluent.streaming.internal.PunctuationQueue; +======= +import io.confluent.streaming.internal.PunctuationQueue; +import io.confluent.streaming.internal.PunctuationSchedulerImpl; +>>>>>>> new api model import io.confluent.streaming.kv.internals.RestoreFunc; >>>>>>> new api model import org.apache.kafka.common.metrics.Metrics; @@ -66,15 +75,22 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer @Override public void restore(StateStore store, RestoreFunc func) { throw new UnsupportedOperationException("restore() not supported."); } +<<<<<<< HEAD public void register(StateStore store) { throw new UnsupportedOperationException("restore() not supported."); } +======= +>>>>>>> new api model @Override public void flush() { throw new UnsupportedOperationException("flush() not supported."); } @Override +<<<<<<< HEAD <<<<<<< HEAD public void restore(StateStore engine) throws Exception { throw new UnsupportedOperationException("restore() not supported."); } +======= + public void send(String topic, Object key, Object value) { throw new UnsupportedOperationException("send() not supported."); } +>>>>>>> new api model ======= public void send(String topic, Object key, Object value) { throw new UnsupportedOperationException("send() not supported."); } >>>>>>> new api model @@ -83,8 +99,13 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { throw new UnsupportedOperationException("send() not supported."); } @Override +<<<<<<< HEAD public void schedule(Processor processor, long interval) { throw new UnsupportedOperationException("schedule() not supported"); +======= + public PunctuationScheduler getPunctuationScheduler(Processor processor) { + return new PunctuationSchedulerImpl(punctuationQueue, processor); +>>>>>>> new api model } @Override diff --git a/temp b/temp new file mode 100644 index 0000000000000..df0f0846d9b9e --- /dev/null +++ b/temp @@ -0,0 +1,34 @@ +src/main/java/io/confluent/streaming/KStreamContext.java: needs merge +src/main/java/io/confluent/streaming/KStreamJob.java: needs merge +src/main/java/io/confluent/streaming/KafkaStreaming.java: needs merge +src/main/java/io/confluent/streaming/Processor.java: needs merge +src/main/java/io/confluent/streaming/ProcessorKStreamJob.java: needs merge +src/main/java/io/confluent/streaming/examples/MapKStreamJob.java: needs merge +src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamBranch.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamFilter.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamImpl.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamJoin.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamMap.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamMapValues.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamSource.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamThread.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java: needs merge +src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java: needs merge +src/main/java/io/confluent/streaming/internal/Receiver.java: needs merge +src/main/java/io/confluent/streaming/internal/StreamGroup.java: needs merge +src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java: needs merge +src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java: needs merge +src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java: needs merge +src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java: needs merge +src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java: needs merge +src/test/java/io/confluent/streaming/internal/KStreamMapTest.java: needs merge +src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java: needs merge +src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java: needs merge +src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java: needs merge +src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java: needs merge +src/test/java/io/confluent/streaming/internal/StreamGroupTest.java: needs merge +src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java: needs merge diff --git a/temp2 b/temp2 new file mode 100644 index 0000000000000..63985256bd581 --- /dev/null +++ b/temp2 @@ -0,0 +1,34 @@ +src/main/java/io/confluent/streaming/KStreamContext.java +src/main/java/io/confluent/streaming/KStreamJob.java +src/main/java/io/confluent/streaming/KafkaStreaming.java +src/main/java/io/confluent/streaming/Processor.java +src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +src/main/java/io/confluent/streaming/examples/MapKStreamJob.java +src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +src/main/java/io/confluent/streaming/internal/KStreamBranch.java +src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +src/main/java/io/confluent/streaming/internal/KStreamFilter.java +src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +src/main/java/io/confluent/streaming/internal/KStreamImpl.java +src/main/java/io/confluent/streaming/internal/KStreamJoin.java +src/main/java/io/confluent/streaming/internal/KStreamMap.java +src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +src/main/java/io/confluent/streaming/internal/KStreamSource.java +src/main/java/io/confluent/streaming/internal/KStreamThread.java +src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java +src/main/java/io/confluent/streaming/internal/Receiver.java +src/main/java/io/confluent/streaming/internal/StreamGroup.java +src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java +src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java From f934266e288df391d52e3b87193612ff8919b265 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 10:46:18 -0700 Subject: [PATCH 118/275] wip --- .../streaming/KStreamInitializer.java | 26 ----------- .../confluent/streaming/KafkaStreaming.java | 2 +- .../internal/KStreamInitializerImpl.java | 46 +------------------ .../streaming/internal/KStreamSource.java | 26 ++++++++++- .../streaming/internal/KStreamThread.java | 7 +-- .../streaming/internal/TopologyAnalyzer.java | 5 +- .../streaming/internal/KStreamBranchTest.java | 4 ++ .../streaming/internal/KStreamFilterTest.java | 8 ++++ .../internal/KStreamFlatMapTest.java | 4 ++ .../internal/KStreamFlatMapValuesTest.java | 4 ++ .../streaming/internal/KStreamJoinTest.java | 24 ++++++++++ .../streaming/internal/KStreamMapTest.java | 4 ++ .../internal/KStreamMapValuesTest.java | 4 ++ .../streaming/internal/KStreamSourceTest.java | 4 ++ .../internal/KStreamWindowedTest.java | 4 ++ .../streaming/internal/StreamGroupTest.java | 4 ++ temp | 23 ---------- temp2 | 23 ---------- 18 files changed, 93 insertions(+), 129 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStreamInitializer.java b/src/main/java/io/confluent/streaming/KStreamInitializer.java index 43464dc884adc..d707306b0eab9 100644 --- a/src/main/java/io/confluent/streaming/KStreamInitializer.java +++ b/src/main/java/io/confluent/streaming/KStreamInitializer.java @@ -12,32 +12,6 @@ */ public interface KStreamInitializer { - AtomicInteger STREAM_GROUP_INDEX = new AtomicInteger(1); - - /** - * 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(); - // TODO: support regex topic matching in from() calls, for example: // context.from("Topic*PageView") diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index e2d86f61e37f6..b41da442d9148 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -226,7 +226,7 @@ public void commit() { } ======= - TopologyAnalyzer topologyAnalyzer = new TopologyAnalyzer(jobClass, streamingConfig); + TopologyAnalyzer topologyAnalyzer = new TopologyAnalyzer(jobClass); this.topics = topologyAnalyzer.topics; >>>>>>> new api model diff --git a/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java index f4b58e674c1e9..120b8038e057e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java @@ -2,9 +2,7 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamInitializer; -import io.confluent.streaming.StreamingConfig; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import java.util.ArrayList; import java.util.Collection; @@ -15,57 +13,15 @@ */ public class KStreamInitializerImpl implements KStreamInitializer { - private final Serializer keySerializer; - private final Serializer valueSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; private final ArrayList> streams = new ArrayList<>(); - KStreamInitializerImpl(StreamingConfig streamingConfig) { - this( - streamingConfig.keySerializer(), - streamingConfig.valueSerializer(), - streamingConfig.keyDeserializer(), - streamingConfig.valueDeserializer() - ); - } - - KStreamInitializerImpl(Serializer keySerializer,Serializer valueSerializer, Deserializer keyDeserializer, Deserializer valueDeserializer) { - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - this.keyDeserializer = keyDeserializer; - this.valueDeserializer = valueDeserializer; - } - - @Override - public Serializer keySerializer() { - return keySerializer; - } - - @Override - public Serializer valueSerializer() { - return valueSerializer; - } - - @Override - public Deserializer keyDeserializer() { - return keyDeserializer; - } - - @Override - public Deserializer valueDeserializer() { - return valueDeserializer; - } - @Override public KStream from(String... topics) { - return from(this.keyDeserializer(), this.valueDeserializer(), topics); + return from(null, null, topics); } - @SuppressWarnings("unchecked") @Override public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KStreamSource stream = new KStreamSource<>(topics, keyDeserializer, valDeserializer, this); streams.add(stream); return stream; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index b50d35cd2410d..de7454699710e 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -1,9 +1,13 @@ package io.confluent.streaming.internal; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamTopology; ======= +======= +import io.confluent.streaming.KStreamContext; +>>>>>>> wip import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model import org.apache.kafka.common.serialization.Deserializer; @@ -21,21 +25,25 @@ public class KStreamSource extends KStreamImpl { private Deserializer keyDeserializer; private Deserializer valueDeserializer; +<<<<<<< HEAD <<<<<<< HEAD String[] topics; public KStreamSource(String[] topics, KStreamTopology topology) { this(topics, null, null, topology); } +======= + private Deserializer keyDeserializer; + private Deserializer valueDeserializer; +>>>>>>> wip public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology topology) { super(topology); ======= final String[] topics; - @SuppressWarnings("unchecked") KStreamSource(String[] topics, KStreamInitializer initializer) { - this(topics, (Deserializer) initializer.keyDeserializer(), (Deserializer) initializer.valueDeserializer(), initializer); + this(topics, null, null, initializer); } KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamInitializer initializer) { @@ -47,6 +55,17 @@ public KStreamSource(String[] topics, Deserializer keyDeserializer, Deseriali } @SuppressWarnings("unchecked") +<<<<<<< HEAD +======= + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + if (keyDeserializer == null) keyDeserializer = (Deserializer) context.keyDeserializer(); + if (valueDeserializer == null) valueDeserializer = (Deserializer) context.valueDeserializer(); + + super.bind(context, metadata); + } + +>>>>>>> wip @Override public void bind(KStreamContext context, KStreamMetadata metadata) { if (keyDeserializer == null) keyDeserializer = (Deserializer) context.keyDeserializer(); @@ -71,8 +90,11 @@ public Deserializer valueDeserializer() { return valueDeserializer; } +<<<<<<< HEAD public Set topics() { return new HashSet<>(Arrays.asList(topics)); } +======= +>>>>>>> wip } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index 26517d98be0d9..96000ffa09225 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -252,12 +252,7 @@ private void addPartitions(Collection assignment) { KStreamContextImpl context = kstreamContexts.get(id); if (context == null) { try { - KStreamInitializerImpl initializer = new KStreamInitializerImpl( - streamingConfig.keySerializer(), - streamingConfig.valueSerializer(), - streamingConfig.keyDeserializer(), - streamingConfig.valueDeserializer() - ); + KStreamInitializerImpl initializer = new KStreamInitializerImpl(); KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); job.init(initializer); diff --git a/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java b/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java index 8dee74ed57a57..75c583bbb4d0e 100644 --- a/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java +++ b/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java @@ -1,7 +1,6 @@ package io.confluent.streaming.internal; import io.confluent.streaming.KStreamJob; -import io.confluent.streaming.StreamingConfig; import org.apache.kafka.common.utils.Utils; import java.util.Collection; @@ -17,9 +16,9 @@ public class TopologyAnalyzer { public final Set topics; public final Collection> streams; - public TopologyAnalyzer(Class jobClass, StreamingConfig streamingConfig) { + public TopologyAnalyzer(Class jobClass) { KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); - KStreamInitializerImpl context = new KStreamInitializerImpl(streamingConfig); + KStreamInitializerImpl context = new KStreamInitializerImpl(); job.init(context); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index 4c53b72b406c1..ec39e956eed1e 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -42,11 +42,15 @@ public boolean apply(Integer key, String value) { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip KStreamSource stream; KStream[] branches; TestProcessor[] processors; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index ebad9b908356d..5d059efd72729 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -27,11 +27,15 @@ public boolean apply(Integer key, String value) { public void testFilter() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip KStreamSource stream; TestProcessor processor; @@ -52,11 +56,15 @@ public void testFilter() { public void testFilterOut() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index 4d079e7889bf1..ba99c693df0dd 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -34,12 +34,16 @@ public KeyValue> apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology topology = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 64febf0bc9ad7..18d7d545cc1f7 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -33,11 +33,15 @@ public Iterable apply(String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index a7187f01ebad8..3963f64eaeca4 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -69,11 +69,15 @@ public void testJoin() { TestProcessor processor; String[] expected; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -163,11 +167,15 @@ public void testJoinPrior() { TestProcessor processor; String[] expected; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -251,11 +259,15 @@ public void testMap() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -342,11 +354,15 @@ public void testFlatMap() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -433,11 +449,15 @@ public void testMapValues() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); @@ -524,11 +544,15 @@ public void testFlatMapValues() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 605326a9a76fa..8090ff2802973 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -29,11 +29,15 @@ public KeyValue apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 9bd438c774e56..488dbc7c9fca8 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -35,11 +35,15 @@ public Integer apply(String value) { final int[] expectedKeys = new int[] { 1, 10, 100, 1000 }; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 7f8deff0bf336..8f51ede716228 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -20,11 +20,15 @@ public class KStreamSourceTest { @Test public void testKStreamSource() { +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip TestProcessor processor = new TestProcessor<>(); KStreamSource stream = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 303db56da16cf..559d77a1e517d 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -24,12 +24,16 @@ public void testWindowedStream() { KStreamSource stream; Window window; +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null) { }; >>>>>>> new api model +======= + KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip window = new UnlimitedWindow<>(); stream = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index 193f0025b8505..669fb487ca3de 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -32,11 +32,15 @@ private static class MockKStreamSource extends KStreamSource { public ArrayList timestamps = new ArrayList<>(); public MockKStreamSource() { +<<<<<<< HEAD <<<<<<< HEAD super(null, deserializer, deserializer, new MockKStreamTopology()); ======= super(null, deserializer, deserializer, new KStreamInitializerImpl(serializer, serializer, deserializer, deserializer)); >>>>>>> new api model +======= + super(null, deserializer, deserializer, new KStreamInitializerImpl()); +>>>>>>> wip } @Override diff --git a/temp b/temp index df0f0846d9b9e..a7984a981b472 100644 --- a/temp +++ b/temp @@ -1,25 +1,4 @@ -src/main/java/io/confluent/streaming/KStreamContext.java: needs merge -src/main/java/io/confluent/streaming/KStreamJob.java: needs merge -src/main/java/io/confluent/streaming/KafkaStreaming.java: needs merge -src/main/java/io/confluent/streaming/Processor.java: needs merge -src/main/java/io/confluent/streaming/ProcessorKStreamJob.java: needs merge -src/main/java/io/confluent/streaming/examples/MapKStreamJob.java: needs merge -src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamBranch.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamFilter.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamImpl.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamJoin.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamMap.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamMapValues.java: needs merge src/main/java/io/confluent/streaming/internal/KStreamSource.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamThread.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java: needs merge -src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java: needs merge -src/main/java/io/confluent/streaming/internal/Receiver.java: needs merge -src/main/java/io/confluent/streaming/internal/StreamGroup.java: needs merge src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java: needs merge src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java: needs merge src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java: needs merge @@ -29,6 +8,4 @@ src/test/java/io/confluent/streaming/internal/KStreamMapTest.java: needs merge src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java: needs merge src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java: needs merge src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java: needs merge -src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java: needs merge src/test/java/io/confluent/streaming/internal/StreamGroupTest.java: needs merge -src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java: needs merge diff --git a/temp2 b/temp2 index 63985256bd581..34493906c5413 100644 --- a/temp2 +++ b/temp2 @@ -1,25 +1,4 @@ -src/main/java/io/confluent/streaming/KStreamContext.java -src/main/java/io/confluent/streaming/KStreamJob.java -src/main/java/io/confluent/streaming/KafkaStreaming.java -src/main/java/io/confluent/streaming/Processor.java -src/main/java/io/confluent/streaming/ProcessorKStreamJob.java -src/main/java/io/confluent/streaming/examples/MapKStreamJob.java -src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java -src/main/java/io/confluent/streaming/internal/KStreamBranch.java -src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java -src/main/java/io/confluent/streaming/internal/KStreamFilter.java -src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java -src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java -src/main/java/io/confluent/streaming/internal/KStreamImpl.java -src/main/java/io/confluent/streaming/internal/KStreamJoin.java -src/main/java/io/confluent/streaming/internal/KStreamMap.java -src/main/java/io/confluent/streaming/internal/KStreamMapValues.java src/main/java/io/confluent/streaming/internal/KStreamSource.java -src/main/java/io/confluent/streaming/internal/KStreamThread.java -src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java -src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java -src/main/java/io/confluent/streaming/internal/Receiver.java -src/main/java/io/confluent/streaming/internal/StreamGroup.java src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -29,6 +8,4 @@ src/test/java/io/confluent/streaming/internal/KStreamMapTest.java src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java -src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java src/test/java/io/confluent/streaming/internal/StreamGroupTest.java -src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java From 3309e20db6703529bfaf1cb2f1e081e81bc88e3a Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 11:09:55 -0700 Subject: [PATCH 119/275] wip --- .../io/confluent/streaming/testutil/MockKStreamContext.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java index 67f5e0e8c4045..a29926201899d 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java @@ -81,6 +81,8 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer ======= >>>>>>> new api model + public void register(StateStore store) { throw new UnsupportedOperationException("restore() not supported."); } + @Override public void flush() { throw new UnsupportedOperationException("flush() not supported."); } From a715bda162901933e28f5529051f14377c6e0022 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 13:40:29 -0700 Subject: [PATCH 120/275] wip --- .../streaming/KStreamInitializer.java | 36 ------------------- .../io/confluent/streaming/KStreamJob.java | 35 ------------------ .../confluent/streaming/KafkaStreaming.java | 15 ++++++++ .../streaming/ProcessorKStreamJob.java | 5 +++ .../streaming/examples/MapKStreamJob.java | 8 +++++ .../streaming/examples/PrintKStreamJob.java | 5 +++ .../examples/StatefulKStreamJob.java | 5 +++ .../streaming/internal/KStreamBranch.java | 21 ++++++++++- .../streaming/internal/KStreamFilter.java | 8 +++++ .../streaming/internal/KStreamFlatMap.java | 8 +++++ .../internal/KStreamFlatMapValues.java | 8 +++++ .../streaming/internal/KStreamImpl.java | 30 ++++++++++++++-- .../internal/KStreamInitializerImpl.java | 34 ------------------ .../streaming/internal/KStreamJoin.java | 11 ++++++ .../streaming/internal/KStreamMap.java | 8 +++++ .../streaming/internal/KStreamMapValues.java | 8 +++++ .../streaming/internal/KStreamSource.java | 17 +++++++-- .../streaming/internal/KStreamThread.java | 11 +++--- .../internal/KStreamWindowedImpl.java | 8 +++++ .../streaming/internal/ProcessorNode.java | 8 +++++ .../streaming/internal/Receiver.java | 1 + .../streaming/internal/TopologyAnalyzer.java | 35 ------------------ .../streaming/internal/KStreamBranchTest.java | 4 +++ .../streaming/internal/KStreamFilterTest.java | 8 +++++ .../internal/KStreamFlatMapTest.java | 9 +++++ .../internal/KStreamFlatMapValuesTest.java | 4 +++ .../streaming/internal/KStreamJoinTest.java | 28 +++++++++++++++ .../streaming/internal/KStreamMapTest.java | 4 +++ .../internal/KStreamMapValuesTest.java | 8 +++++ .../streaming/internal/KStreamSourceTest.java | 4 +++ .../internal/KStreamWindowedTest.java | 4 +++ .../streaming/internal/StreamGroupTest.java | 4 +++ temp | 16 +++++++++ temp2 | 16 +++++++++ 34 files changed, 283 insertions(+), 151 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/KStreamInitializer.java delete mode 100644 src/main/java/io/confluent/streaming/KStreamJob.java delete mode 100644 src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java delete mode 100644 src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java diff --git a/src/main/java/io/confluent/streaming/KStreamInitializer.java b/src/main/java/io/confluent/streaming/KStreamInitializer.java deleted file mode 100644 index d707306b0eab9..0000000000000 --- a/src/main/java/io/confluent/streaming/KStreamInitializer.java +++ /dev/null @@ -1,36 +0,0 @@ -package io.confluent.streaming; - -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; - -import java.util.concurrent.atomic.AtomicInteger; - -/** - * KStreamContext is the interface that allows an implementation of {@link KStreamJob#init(KStreamInitializer)} to create KStream instances. - * It also provides access to the system resources for a stream processing job. - * An instance of KStreamContext is created for each partition. - */ -public interface KStreamInitializer { - - // TODO: support regex topic matching in from() calls, for example: - // context.from("Topic*PageView") - - /** - * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream - */ - KStream from(String... topics); - - /** - * 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 - */ - KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics); - -} diff --git a/src/main/java/io/confluent/streaming/KStreamJob.java b/src/main/java/io/confluent/streaming/KStreamJob.java deleted file mode 100644 index 3e97475b8e9ad..0000000000000 --- a/src/main/java/io/confluent/streaming/KStreamJob.java +++ /dev/null @@ -1,35 +0,0 @@ -package io.confluent.streaming; - -/** - * An interface to implement an application logic of a stream processing. - * An instance is created and initialized by the framework for each partition. - */ -public interface KStreamJob { - - /** - * Creates a job instance - */ - - /** - * Initializes a stream processing job for a partition. This method is called for each partition. - * An application constructs a processing logic using KStream API. - *

- * For example, - *

- *
-   *   public bind(KStreamContext context) {
-   *     KStream<Integer, PageView> pageViewStream = context.from("pageView").mapValues(...);
-   *     KStream<Integer, AdClick> adClickStream = context.from("adClick").join(pageViewStream, ...).process(...);
-   *   }
-   * 
- * @param context KStreamContext for this partition - */ - void init(KStreamInitializer context); - - /** - * Closes this partition of the stream processing job. - * An application can perform its special clean up here. - */ - void close(); - -} diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/src/main/java/io/confluent/streaming/KafkaStreaming.java index b41da442d9148..bb7c49c4c40cd 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/src/main/java/io/confluent/streaming/KafkaStreaming.java @@ -23,6 +23,7 @@ <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.internal.IngestorImpl; <<<<<<< HEAD import io.confluent.streaming.internal.StreamSynchronizer; @@ -51,6 +52,8 @@ ======= import io.confluent.streaming.internal.TopologyAnalyzer; >>>>>>> new api model +======= +>>>>>>> wip import org.apache.kafka.common.metrics.Metrics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -142,6 +145,7 @@ public class KafkaStreaming implements Runnable { private final Set topics; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD @SuppressWarnings("unchecked") @@ -185,6 +189,9 @@ >>>>>>> use poll(0) for non-blocking poll ======= public KafkaStreaming(Class jobClass, StreamingConfig streamingConfig) { >>>>>>> added KStreamThread +======= + public KafkaStreaming(KStreamTopology topology, StreamingConfig streamingConfig) { +>>>>>>> wip ======= public KafkaStreaming(KStreamTopology topology, StreamingConfig streamingConfig) { >>>>>>> wip @@ -192,6 +199,7 @@ public KafkaStreaming(KStreamTopology topology, StreamingConfig streamingConfig) if (streamingConfig.timestampExtractor() == null) throw new NullPointerException("timestamp extractor is missing"); this.config = new ProcessorConfig(streamingConfig.config()); +<<<<<<< HEAD <<<<<<< HEAD this.topics = topology.topics(); @@ -237,6 +245,9 @@ public void shutdown() { >>>>>>> added KStreamThread } }; +======= + this.topics = topology.topics(); +>>>>>>> wip <<<<<<< HEAD producer.close(); @@ -262,12 +273,16 @@ public void shutdown() { // TODO: Fix this after the threading model is decided (also fix KStreamThread) this.threads = new KStreamThread[1]; <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD threads[0] = new KStreamThread(jobClass, topics, streamingConfig, coordinator, metrics); >>>>>>> added KStreamThread ======= threads[0] = new KStreamThread(jobClass, topics, streamingConfig, metrics); >>>>>>> removed Coordinator +======= + threads[0] = new KStreamThread(topology, topics, streamingConfig, metrics); +>>>>>>> wip ======= threads[0] = new KStreamThread(topology, topics, streamingConfig, metrics); >>>>>>> wip diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java index 8231c8ed790c9..4cc3be63b0818 100644 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -14,6 +14,7 @@ public ProcessorKStreamJob(String... topics) { @SuppressWarnings("unchecked") @Override <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD public void init(KStreamContext context) { <<<<<<< HEAD @@ -39,6 +40,10 @@ public void topology() { public void init(KStreamInitializer initializer) { initializer.from().process((Processor) this); >>>>>>> new api model +======= + public void topology() { + ((KStream)from()).process(this); +>>>>>>> wip } } diff --git a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java index b617badc3e7aa..80b2f4a7a3280 100644 --- a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java @@ -2,11 +2,15 @@ import io.confluent.streaming.KStream; <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; import io.confluent.streaming.KStreamJob; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.KafkaStreaming; import io.confluent.streaming.KeyValue; import io.confluent.streaming.KeyValueMapper; @@ -23,11 +27,15 @@ public class MapKStreamJob extends KStreamTopology { @SuppressWarnings("unchecked") @Override +<<<<<<< HEAD <<<<<<< HEAD public void topology() { ======= public void init(KStreamInitializer context) { >>>>>>> new api model +======= + public void topology() { +>>>>>>> wip // With overriden de-serializer KStream stream1 = from(new StringDeserializer(), new StringDeserializer(), "topic1"); diff --git a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java index 002c71ec70ebd..72f87852c348d 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java @@ -40,10 +40,15 @@ public void close() { } public static void main(String[] args) { +<<<<<<< HEAD KafkaStreaming streaming = new KafkaStreaming( new SingleProcessorTopology(PrintKStreamJob.class, args), new StreamingConfig(new Properties()) ); streaming.run(); +======= + KafkaStreaming kstream = new KafkaStreaming(new PrintKStreamJob(), new StreamingConfig(new Properties())); + kstream.run(); +>>>>>>> wip } } diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index a1e0e716ac3eb..9140fd54a40a9 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -63,10 +63,15 @@ public void close() { } public static void main(String[] args) { +<<<<<<< HEAD KafkaStreaming streaming = new KafkaStreaming( new SingleProcessorTopology(StatefulKStreamJob.class, args), new StreamingConfig(new Properties()) ); streaming.run(); +======= + KafkaStreaming kstream = new KafkaStreaming(new StatefulKStreamJob(), new StreamingConfig(new Properties())); + kstream.run(); +>>>>>>> wip } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index 2074f57b5da28..e672e1158a5c5 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -2,10 +2,14 @@ import io.confluent.streaming.KStreamContext; <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.Predicate; import java.lang.reflect.Array; @@ -20,6 +24,7 @@ class KStreamBranch implements Receiver { final KStreamSource[] branches; @SuppressWarnings("unchecked") +<<<<<<< HEAD <<<<<<< HEAD KStreamBranch(Predicate[] predicates, KStreamTopology topology) { this.predicates = Arrays.copyOf(predicates, predicates.length); @@ -28,6 +33,9 @@ class KStreamBranch implements Receiver { branches[i] = new KStreamSource<>(null, topology); ======= KStreamBranch(Predicate[] predicates, KStreamInitializer initializer) { +======= + KStreamBranch(Predicate[] predicates, KStreamTopology initializer) { +>>>>>>> wip this.predicates = Arrays.copyOf(predicates, predicates.length); this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { @@ -38,6 +46,7 @@ class KStreamBranch implements Receiver { @Override public void bind(KStreamContext context, KStreamMetadata metadata) { +<<<<<<< HEAD <<<<<<< HEAD for (KStreamSource branch : branches) { branch.bind(context, metadata); @@ -45,6 +54,10 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { for (KStreamSource stream : branches) { stream.bind(context, metadata); >>>>>>> new api model +======= + for (KStreamSource branch : branches) { + branch.bind(context, metadata); +>>>>>>> wip } } @@ -75,7 +88,13 @@ public void receive(Object key, Object value, long timestamp, long streamTime) { } >>>>>>> new api model } - return; + } + + @Override + public void close() { + for (KStreamSource branch : branches) { + branch.close(); + } } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 3fef6205ae454..1c88c87ec1d48 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -1,10 +1,14 @@ package io.confluent.streaming.internal; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.Predicate; /** @@ -14,11 +18,15 @@ class KStreamFilter extends KStreamImpl { private final Predicate predicate; +<<<<<<< HEAD <<<<<<< HEAD KStreamFilter(Predicate predicate, KStreamTopology topology) { super(topology); ======= KStreamFilter(Predicate predicate, KStreamInitializer initializer) { +======= + KStreamFilter(Predicate predicate, KStreamTopology initializer) { +>>>>>>> wip super(initializer); >>>>>>> new api model this.predicate = predicate; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index ba8a4637dd4ff..1d8709ddbefd4 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -2,10 +2,14 @@ import io.confluent.streaming.KStreamContext; <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; @@ -16,11 +20,15 @@ class KStreamFlatMap extends KStreamImpl { private final KeyValueMapper, K1, V1> mapper; +<<<<<<< HEAD <<<<<<< HEAD KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology topology) { super(topology); ======= KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamInitializer initializer) { +======= + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology initializer) { +>>>>>>> wip super(initializer); >>>>>>> new api model this.mapper = mapper; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 5ca0ccc8fa335..0b6140c3d9eb9 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -1,10 +1,14 @@ package io.confluent.streaming.internal; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.ValueMapper; /** @@ -14,11 +18,15 @@ class KStreamFlatMapValues extends KStreamImpl { private final ValueMapper, V1> mapper; +<<<<<<< HEAD <<<<<<< HEAD KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology topology) { super(topology); ======= KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamInitializer initializer) { +======= + KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology initializer) { +>>>>>>> wip super(initializer); >>>>>>> new api model this.mapper = mapper; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 1fe491f9eef30..268e3371c8a8a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -3,10 +3,14 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.KStreamWindowed; import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.Predicate; @@ -25,6 +29,7 @@ abstract class KStreamImpl implements KStream, Receiver { private final ArrayList nextReceivers = new ArrayList<>(1); +<<<<<<< HEAD <<<<<<< HEAD protected KStreamTopology topology; protected KStreamContext context; @@ -51,15 +56,20 @@ public void close() { for (int i = 0; i < numReceivers; i++) { nextReceivers.get(i).close(); ======= +======= + protected KStreamTopology initializer; + protected KStreamContext context; +>>>>>>> wip protected KStreamMetadata metadata; - protected KStreamInitializer initializer; - protected KStreamImpl(KStreamInitializer initializer) { + protected KStreamImpl(KStreamTopology initializer) { this.initializer = initializer; } @Override public void bind(KStreamContext context, KStreamMetadata metadata) { + if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); + this.context = context; this.metadata = metadata; int numReceivers = nextReceivers.size(); for (int i = 0; i < numReceivers; i++) { @@ -68,6 +78,14 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { } } + @Override + public void close() { + int numReceivers = nextReceivers.size(); + for (int i = 0; i < numReceivers; i++) { + nextReceivers.get(i).close(); + } + } + @Override public KStream filter(Predicate predicate) { <<<<<<< HEAD @@ -180,7 +198,11 @@ public void init(KStreamContext context) { } @Override public void process(K key, V value) { +<<<<<<< HEAD this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); +======= + this.processorContext.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); +>>>>>>> wip } @Override public void punctuate(long streamTime) {} @@ -202,6 +224,7 @@ public KStream transform(Transformer transformer) return chain(new KStreamTransform<>(transformer, topology)); ======= public void process(final Processor processor) { +<<<<<<< HEAD Receiver receiver = new Receiver() { public void bind(KStreamContext context, KStreamMetadata metadata) { processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); @@ -212,6 +235,9 @@ public void receive(Object key, Object value, long timestamp, long streamTime) { }; registerReceiver(receiver); >>>>>>> new api model +======= + registerReceiver(new ProcessorNode<>(processor)); +>>>>>>> wip } void registerReceiver(Receiver receiver) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java deleted file mode 100644 index 120b8038e057e..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/KStreamInitializerImpl.java +++ /dev/null @@ -1,34 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.KStream; -import io.confluent.streaming.KStreamInitializer; -import org.apache.kafka.common.serialization.Deserializer; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; - -/** - * Created by yasuhiro on 6/19/15. - */ -public class KStreamInitializerImpl implements KStreamInitializer { - - private final ArrayList> streams = new ArrayList<>(); - - @Override - public KStream from(String... topics) { - return from(null, null, topics); - } - - @Override - public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KStreamSource stream = new KStreamSource<>(topics, keyDeserializer, valDeserializer, this); - streams.add(stream); - return stream; - } - - Collection> sourceStreams() { - return Collections.unmodifiableCollection(streams); - } - -} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index ef1031773c275..31d4609448fcb 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -2,10 +2,14 @@ import io.confluent.streaming.KStreamContext; <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.NotCopartitionedException; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; @@ -28,11 +32,15 @@ private static abstract class Finder { private KStreamMetadata thisMetadata; private KStreamMetadata otherMetadata; +<<<<<<< HEAD <<<<<<< HEAD KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology topology) { super(topology); ======= KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamInitializer initializer) { +======= + KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology initializer) { +>>>>>>> wip super(initializer); >>>>>>> new api model @@ -96,9 +104,12 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { if (thisMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) throw new NotCopartitionedException(); } <<<<<<< HEAD +<<<<<<< HEAD ======= >>>>>>> new api model +======= +>>>>>>> wip @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value2, long timestamp) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index 6db4b6d21076a..4c3118a5b0e55 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -2,10 +2,14 @@ import io.confluent.streaming.KStreamContext; <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; @@ -16,11 +20,15 @@ class KStreamMap extends KStreamImpl { private final KeyValueMapper mapper; +<<<<<<< HEAD <<<<<<< HEAD KStreamMap(KeyValueMapper mapper, KStreamTopology topology) { super(topology); ======= KStreamMap(KeyValueMapper mapper, KStreamInitializer initializer) { +======= + KStreamMap(KeyValueMapper mapper, KStreamTopology initializer) { +>>>>>>> wip super(initializer); >>>>>>> new api model this.mapper = mapper; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index d332dfeee9edd..b996c3e533710 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -1,10 +1,14 @@ package io.confluent.streaming.internal; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.ValueMapper; /** @@ -14,11 +18,15 @@ class KStreamMapValues extends KStreamImpl { private final ValueMapper mapper; +<<<<<<< HEAD <<<<<<< HEAD KStreamMapValues(ValueMapper mapper, KStreamTopology topology) { super(topology); ======= KStreamMapValues(ValueMapper mapper, KStreamInitializer initializer) { +======= + KStreamMapValues(ValueMapper mapper, KStreamTopology initializer) { +>>>>>>> wip super(initializer); >>>>>>> new api model this.mapper = mapper; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index de7454699710e..c01b77dd5fdda 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -4,12 +4,15 @@ <<<<<<< HEAD import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamTopology; +<<<<<<< HEAD ======= ======= import io.confluent.streaming.KStreamContext; >>>>>>> wip import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +>>>>>>> wip import org.apache.kafka.common.serialization.Deserializer; import java.util.Arrays; @@ -25,6 +28,7 @@ public class KStreamSource extends KStreamImpl { private Deserializer keyDeserializer; private Deserializer valueDeserializer; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD String[] topics; @@ -41,12 +45,15 @@ public KStreamSource(String[] topics, Deserializer keyDeserializer, Deseriali super(topology); ======= final String[] topics; +======= + String[] topics; +>>>>>>> wip - KStreamSource(String[] topics, KStreamInitializer initializer) { + public KStreamSource(String[] topics, KStreamTopology initializer) { this(topics, null, null, initializer); } - KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamInitializer initializer) { + public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology initializer) { super(initializer); >>>>>>> new api model this.topics = topics; @@ -91,10 +98,16 @@ public Deserializer valueDeserializer() { } <<<<<<< HEAD +<<<<<<< HEAD +======= +>>>>>>> wip public Set topics() { return new HashSet<>(Arrays.asList(topics)); } +<<<<<<< HEAD +======= +>>>>>>> wip ======= >>>>>>> wip } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/src/main/java/io/confluent/streaming/internal/KStreamThread.java index 96000ffa09225..f3ba83fddedc0 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamThread.java @@ -17,12 +17,16 @@ package io.confluent.streaming.internal; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamJob; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.StreamingConfig; import io.confluent.streaming.util.ParallelExecutor; import io.confluent.streaming.util.Util; @@ -252,13 +256,8 @@ private void addPartitions(Collection assignment) { KStreamContextImpl context = kstreamContexts.get(id); if (context == null) { try { - KStreamInitializerImpl initializer = new KStreamInitializerImpl(); - KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); - - job.init(initializer); - context = new KStreamContextImpl(id, ingestor, collector, streamingConfig, config, metrics); - context.init(initializer.sourceStreams()); + context.init(topology.sourceStreams()); >>>>>>> new api model kstreamContexts.put(id, context); diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index 79be9e4615aa5..c0e646f651e9c 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -4,10 +4,14 @@ import io.confluent.streaming.KStream; import io.confluent.streaming.KStreamContext; <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.KStreamWindowed; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; @@ -19,11 +23,15 @@ public class KStreamWindowedImpl extends KStreamImpl implements KStr final Window window; +<<<<<<< HEAD <<<<<<< HEAD KStreamWindowedImpl(Window window, KStreamTopology initializer) { ======= KStreamWindowedImpl(Window window, KStreamInitializer initializer) { >>>>>>> new api model +======= + KStreamWindowedImpl(Window window, KStreamTopology initializer) { +>>>>>>> wip super(initializer); this.window = window; } diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java index ebd7622e1b07d..ccaac8cb26970 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java @@ -20,11 +20,19 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); this.context = context; +<<<<<<< HEAD processor.init(context); } @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp) { +======= + processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); + } + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp, long streamTime) { +>>>>>>> wip processor.process((K) key, (V) value); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 77ff2291d5510..86068f218cf7b 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -17,4 +17,5 @@ public interface Receiver { void receive(Object key, Object value, long timestamp, long streamTime); >>>>>>> new api model + void close(); } diff --git a/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java b/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java deleted file mode 100644 index 75c583bbb4d0e..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/TopologyAnalyzer.java +++ /dev/null @@ -1,35 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.KStreamJob; -import org.apache.kafka.common.utils.Utils; - -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - -/** - * Created by yasuhiro on 7/30/15. - */ -public class TopologyAnalyzer { - - public final Set topics; - public final Collection> streams; - - public TopologyAnalyzer(Class jobClass) { - KStreamJob job = (KStreamJob) Utils.newInstance(jobClass); - KStreamInitializerImpl context = new KStreamInitializerImpl(); - - job.init(context); - - this.streams = context.sourceStreams(); - Set topics = new HashSet<>(); - for (KStreamSource stream : this.streams) { - for (String topic : stream.topics) { - topics.add(topic); - } - } - this.topics = Collections.unmodifiableSet(topics); - } - -} diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java index ec39e956eed1e..a29241804aebb 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java @@ -42,6 +42,7 @@ public boolean apply(Integer key, String value) { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -50,6 +51,9 @@ public boolean apply(Integer key, String value) { >>>>>>> new api model ======= KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip +======= + KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; KStream[] branches; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java index 5d059efd72729..637c912b8074a 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java @@ -27,6 +27,7 @@ public boolean apply(Integer key, String value) { public void testFilter() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -35,6 +36,9 @@ public void testFilter() { >>>>>>> new api model ======= KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip +======= + KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; TestProcessor processor; @@ -56,6 +60,7 @@ public void testFilter() { public void testFilterOut() { final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -64,6 +69,9 @@ public void testFilterOut() { >>>>>>> new api model ======= KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip +======= + KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java index ba99c693df0dd..022cf36060c5e 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java @@ -34,6 +34,7 @@ public KeyValue> apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology topology = new MockKStreamTopology(); @@ -43,16 +44,24 @@ public KeyValue> apply(Integer key, String value) { >>>>>>> new api model ======= KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip +======= + KStreamTopology topology = new MockKStreamTopology(); + >>>>>>> wip KStreamSource stream; TestProcessor processor; processor = new TestProcessor<>(); +<<<<<<< HEAD <<<<<<< HEAD stream = new KStreamSource<>(null, topology); ======= stream = new KStreamSource<>(null, initializer); >>>>>>> new api model +======= + stream = new KStreamSource<>(null, topology); +>>>>>>> wip stream.flatMap(mapper).process(processor); KStreamContext context = new MockKStreamContext(null, null); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java index 18d7d545cc1f7..31bac0e5986cc 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java @@ -33,6 +33,7 @@ public Iterable apply(String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -41,6 +42,9 @@ public Iterable apply(String value) { >>>>>>> new api model ======= KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip +======= + KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java index 3963f64eaeca4..a99ccad4633c8 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java @@ -69,6 +69,7 @@ public void testJoin() { TestProcessor processor; String[] expected; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -77,6 +78,9 @@ public void testJoin() { >>>>>>> new api model ======= KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip +======= + KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); @@ -167,6 +171,7 @@ public void testJoinPrior() { TestProcessor processor; String[] expected; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -175,6 +180,9 @@ public void testJoinPrior() { >>>>>>> new api model ======= KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip +======= + KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip processor = new TestProcessor<>(); stream1 = new KStreamSource<>(null, initializer); @@ -259,6 +267,7 @@ public void testMap() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -274,6 +283,10 @@ public void testMap() { mapped1 = stream1.map(keyValueMapper); mapped2 = stream2.map(keyValueMapper); >>>>>>> new api model +======= + KStreamTopology initializer = new MockKStreamTopology(); + processor = new TestProcessor<>(); +>>>>>>> wip boolean exceptionRaised; @@ -354,6 +367,7 @@ public void testFlatMap() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -369,6 +383,10 @@ public void testFlatMap() { mapped1 = stream1.flatMap(keyValueMapper2); mapped2 = stream2.flatMap(keyValueMapper2); >>>>>>> new api model +======= + KStreamTopology initializer = new MockKStreamTopology(); + processor = new TestProcessor<>(); +>>>>>>> wip boolean exceptionRaised; @@ -449,6 +467,7 @@ public void testMapValues() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -464,6 +483,10 @@ public void testMapValues() { mapped1 = stream1.mapValues(valueMapper); mapped2 = stream2.mapValues(valueMapper); >>>>>>> new api model +======= + KStreamTopology initializer = new MockKStreamTopology(); + processor = new TestProcessor<>(); +>>>>>>> wip boolean exceptionRaised; @@ -544,6 +567,7 @@ public void testFlatMapValues() { KStreamWindowed windowed2; TestProcessor processor; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -559,6 +583,10 @@ public void testFlatMapValues() { mapped1 = stream1.flatMapValues(valueMapper2); mapped2 = stream2.flatMapValues(valueMapper2); >>>>>>> new api model +======= + KStreamTopology initializer = new MockKStreamTopology(); + processor = new TestProcessor<>(); +>>>>>>> wip boolean exceptionRaised; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java index 8090ff2802973..f931f9fe04c81 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java @@ -29,6 +29,7 @@ public KeyValue apply(Integer key, String value) { final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -37,6 +38,9 @@ public KeyValue apply(Integer key, String value) { >>>>>>> new api model ======= KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip +======= + KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java index 488dbc7c9fca8..4588eda90065e 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java @@ -2,10 +2,14 @@ import io.confluent.streaming.KStreamContext; <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; >>>>>>> new api model +======= +import io.confluent.streaming.KStreamTopology; +>>>>>>> wip import io.confluent.streaming.ValueMapper; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; @@ -35,6 +39,7 @@ public Integer apply(String value) { final int[] expectedKeys = new int[] { 1, 10, 100, 1000 }; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -43,6 +48,9 @@ public Integer apply(String value) { >>>>>>> new api model ======= KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip +======= + KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; TestProcessor processor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java index 8f51ede716228..f1b06ef8424b7 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java @@ -20,6 +20,7 @@ public class KStreamSourceTest { @Test public void testKStreamSource() { +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); @@ -28,6 +29,9 @@ public void testKStreamSource() { >>>>>>> new api model ======= KStreamInitializer initializer = new KStreamInitializerImpl(); +>>>>>>> wip +======= + KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip TestProcessor processor = new TestProcessor<>(); diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java index 559d77a1e517d..55d0e63f4737f 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java @@ -25,6 +25,7 @@ public void testWindowedStream() { KStreamSource stream; Window window; <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -34,6 +35,9 @@ public void testWindowedStream() { ======= KStreamInitializer initializer = new KStreamInitializerImpl(); >>>>>>> wip +======= + KStreamTopology initializer = new MockKStreamTopology(); +>>>>>>> wip window = new UnlimitedWindow<>(); stream = new KStreamSource<>(null, initializer); diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java index 669fb487ca3de..873f8f0f18879 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java @@ -33,6 +33,7 @@ private static class MockKStreamSource extends KStreamSource { public MockKStreamSource() { <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD super(null, deserializer, deserializer, new MockKStreamTopology()); ======= @@ -40,6 +41,9 @@ public MockKStreamSource() { >>>>>>> new api model ======= super(null, deserializer, deserializer, new KStreamInitializerImpl()); +>>>>>>> wip +======= + super(null, deserializer, deserializer, new MockKStreamTopology()); >>>>>>> wip } diff --git a/temp b/temp index a7984a981b472..1d9daf2d1f97d 100644 --- a/temp +++ b/temp @@ -1,4 +1,20 @@ +src/main/java/io/confluent/streaming/KafkaStreaming.java: needs merge +src/main/java/io/confluent/streaming/ProcessorKStreamJob.java: needs merge +src/main/java/io/confluent/streaming/examples/MapKStreamJob.java: needs merge +src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java: needs merge +src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamBranch.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamFilter.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamImpl.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamJoin.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamMap.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamMapValues.java: needs merge src/main/java/io/confluent/streaming/internal/KStreamSource.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamThread.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java: needs merge +src/main/java/io/confluent/streaming/internal/ProcessorNode.java: needs merge src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java: needs merge src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java: needs merge src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java: needs merge diff --git a/temp2 b/temp2 index 34493906c5413..eae39d824ea89 100644 --- a/temp2 +++ b/temp2 @@ -1,4 +1,20 @@ +src/main/java/io/confluent/streaming/KafkaStreaming.java +src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +src/main/java/io/confluent/streaming/examples/MapKStreamJob.java +src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +src/main/java/io/confluent/streaming/internal/KStreamBranch.java +src/main/java/io/confluent/streaming/internal/KStreamFilter.java +src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +src/main/java/io/confluent/streaming/internal/KStreamImpl.java +src/main/java/io/confluent/streaming/internal/KStreamJoin.java +src/main/java/io/confluent/streaming/internal/KStreamMap.java +src/main/java/io/confluent/streaming/internal/KStreamMapValues.java src/main/java/io/confluent/streaming/internal/KStreamSource.java +src/main/java/io/confluent/streaming/internal/KStreamThread.java +src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +src/main/java/io/confluent/streaming/internal/ProcessorNode.java src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java From e964bf3121f3e0cf72e49d26e7eee51eef28b733 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 14:37:45 -0700 Subject: [PATCH 121/275] wip --- src/main/java/io/confluent/streaming/internal/Receiver.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 86068f218cf7b..cde0055383fe7 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -18,4 +18,5 @@ public interface Receiver { >>>>>>> new api model void close(); + } From 37f83a6826ab106bf7e905142a599f10531aa97b Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 16:27:45 -0700 Subject: [PATCH 122/275] fix examples --- .../io/confluent/streaming/ProcessorKStreamJob.java | 10 ++++++++++ .../confluent/streaming/examples/PrintKStreamJob.java | 8 ++++++++ .../streaming/examples/StatefulKStreamJob.java | 8 ++++++++ .../java/io/confluent/streaming/internal/Receiver.java | 2 +- 4 files changed, 27 insertions(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java index 4cc3be63b0818..16492b29ec2c2 100644 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java @@ -6,11 +6,15 @@ public abstract class ProcessorKStreamJob extends KStreamTopology implements Processor { <<<<<<< HEAD +<<<<<<< HEAD +======= +>>>>>>> fix examples private final String[] topics; public ProcessorKStreamJob(String... topics) { this.topics = topics; } +<<<<<<< HEAD @SuppressWarnings("unchecked") @Override <<<<<<< HEAD @@ -35,6 +39,8 @@ public void topology() { ((KStream)from(topics)).process(this); >>>>>>> fix examples ======= +======= +>>>>>>> fix examples @SuppressWarnings("unchecked") @Override public void init(KStreamInitializer initializer) { @@ -42,8 +48,12 @@ public void init(KStreamInitializer initializer) { >>>>>>> new api model ======= public void topology() { +<<<<<<< HEAD ((KStream)from()).process(this); >>>>>>> wip +======= + ((KStream)from(topics)).process(this); +>>>>>>> fix examples } } diff --git a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java index 72f87852c348d..ead2f2142c6d1 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java @@ -15,6 +15,10 @@ public class PrintKStreamJob implements Processor { private KStreamContext context; + public PrintKStreamJob(String... topics) { + super(topics); + } + @Override public void init(KStreamContext context) { this.context = context; @@ -40,6 +44,7 @@ public void close() { } public static void main(String[] args) { +<<<<<<< HEAD <<<<<<< HEAD KafkaStreaming streaming = new KafkaStreaming( new SingleProcessorTopology(PrintKStreamJob.class, args), @@ -48,6 +53,9 @@ public static void main(String[] args) { streaming.run(); ======= KafkaStreaming kstream = new KafkaStreaming(new PrintKStreamJob(), new StreamingConfig(new Properties())); +======= + KafkaStreaming kstream = new KafkaStreaming(new PrintKStreamJob(args), new StreamingConfig(new Properties())); +>>>>>>> fix examples kstream.run(); >>>>>>> wip } diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index 9140fd54a40a9..131a8d7074104 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -21,6 +21,10 @@ public class StatefulKStreamJob implements Processor { private KStreamContext context; private KeyValueStore kvStore; + public StatefulKStreamJob(String... topics) { + super(topics); + } + @Override public void init(KStreamContext context) { this.context = context; @@ -63,6 +67,7 @@ public void close() { } public static void main(String[] args) { +<<<<<<< HEAD <<<<<<< HEAD KafkaStreaming streaming = new KafkaStreaming( new SingleProcessorTopology(StatefulKStreamJob.class, args), @@ -71,6 +76,9 @@ public static void main(String[] args) { streaming.run(); ======= KafkaStreaming kstream = new KafkaStreaming(new StatefulKStreamJob(), new StreamingConfig(new Properties())); +======= + KafkaStreaming kstream = new KafkaStreaming(new StatefulKStreamJob(args), new StreamingConfig(new Properties())); +>>>>>>> fix examples kstream.run(); >>>>>>> wip } diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index cde0055383fe7..5720266bcef1f 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -18,5 +18,5 @@ public interface Receiver { >>>>>>> new api model void close(); - + } From 37e6e6df535f0d893b22a61a0d74ae094dd63f33 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 31 Jul 2015 16:56:57 -0700 Subject: [PATCH 123/275] fix examples --- .../streaming/examples/PrintKStreamJob.java | 10 ++++++---- .../streaming/examples/StatefulKStreamJob.java | 14 ++++++++++---- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java index ead2f2142c6d1..a00b48435e600 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java @@ -15,10 +15,6 @@ public class PrintKStreamJob implements Processor { private KStreamContext context; - public PrintKStreamJob(String... topics) { - super(topics); - } - @Override public void init(KStreamContext context) { this.context = context; @@ -46,11 +42,15 @@ public void close() { public static void main(String[] args) { <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD +======= +>>>>>>> fix examples KafkaStreaming streaming = new KafkaStreaming( new SingleProcessorTopology(PrintKStreamJob.class, args), new StreamingConfig(new Properties()) ); streaming.run(); +<<<<<<< HEAD ======= KafkaStreaming kstream = new KafkaStreaming(new PrintKStreamJob(), new StreamingConfig(new Properties())); ======= @@ -58,5 +58,7 @@ public static void main(String[] args) { >>>>>>> fix examples kstream.run(); >>>>>>> wip +======= +>>>>>>> fix examples } } diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index 131a8d7074104..4fa5f1317b6aa 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -21,10 +21,6 @@ public class StatefulKStreamJob implements Processor { private KStreamContext context; private KeyValueStore kvStore; - public StatefulKStreamJob(String... topics) { - super(topics); - } - @Override public void init(KStreamContext context) { this.context = context; @@ -35,8 +31,12 @@ public void init(KStreamContext context) { this.kvStore.restore(); // call restore inside processor.init ======= this.kvStore = new InMemoryKeyValueStore<>("local-state", context.kstreamContext()); +<<<<<<< HEAD this.kvStore.restore(); // call restore inside processor.bind >>>>>>> new api model +======= + this.kvStore.restore(); // call restore inside processor.init +>>>>>>> fix examples } @Override @@ -69,11 +69,15 @@ public void close() { public static void main(String[] args) { <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD +======= +>>>>>>> fix examples KafkaStreaming streaming = new KafkaStreaming( new SingleProcessorTopology(StatefulKStreamJob.class, args), new StreamingConfig(new Properties()) ); streaming.run(); +<<<<<<< HEAD ======= KafkaStreaming kstream = new KafkaStreaming(new StatefulKStreamJob(), new StreamingConfig(new Properties())); ======= @@ -81,5 +85,7 @@ public static void main(String[] args) { >>>>>>> fix examples kstream.run(); >>>>>>> wip +======= +>>>>>>> fix examples } } From 392e2559e3e54089d98e5d4983214eed693b0926 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 3 Aug 2015 11:43:42 -0700 Subject: [PATCH 124/275] removed ProcessorContext --- .../confluent/streaming/KStreamContext.java | 4 + .../io/confluent/streaming/Processor.java | 4 + .../examples/StatefulKStreamJob.java | 4 + .../internal/KStreamContextImpl.java | 32 ++++- .../streaming/internal/KStreamImpl.java | 44 +++++- .../internal/KStreamWindowedImpl.java | 4 + .../internal/ProcessorContextImpl.java | 68 ---------- .../streaming/internal/ProcessorNode.java | 4 + .../PunctuationSchedulerImplTest.java | 125 ------------------ .../testutil/MockKStreamContext.java | 11 ++ temp | 27 +--- temp2 | 27 +--- 12 files changed, 108 insertions(+), 246 deletions(-) delete mode 100644 src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java delete mode 100644 src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index fcba969141f40..77a867dd645f4 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -99,11 +99,15 @@ public interface KStreamContext { void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); +<<<<<<< HEAD <<<<<<< HEAD void schedule(Processor processor, long interval); ======= PunctuationScheduler getPunctuationScheduler(Processor processor); >>>>>>> new api model +======= + void schedule(Processor processor, long interval); +>>>>>>> removed ProcessorContext void commit(); diff --git a/src/main/java/io/confluent/streaming/Processor.java b/src/main/java/io/confluent/streaming/Processor.java index 0f56ed2a657d3..e2e7401ea2a66 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/src/main/java/io/confluent/streaming/Processor.java @@ -5,6 +5,7 @@ */ public interface Processor { +<<<<<<< HEAD <<<<<<< HEAD void init(KStreamContext context); ======= @@ -32,6 +33,9 @@ public interface ProcessorContext { void init(ProcessorContext context); >>>>>>> new api model +======= + void init(KStreamContext context); +>>>>>>> removed ProcessorContext void process(K key, V value); diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index 4fa5f1317b6aa..a1d1a6d0a7a7f 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -26,6 +26,7 @@ public void init(KStreamContext context) { this.context = context; this.context.schedule(this, 1000); +<<<<<<< HEAD <<<<<<< HEAD this.kvStore = new InMemoryKeyValueStore<>("local-state", context); this.kvStore.restore(); // call restore inside processor.init @@ -35,6 +36,9 @@ public void init(KStreamContext context) { this.kvStore.restore(); // call restore inside processor.bind >>>>>>> new api model ======= +======= + this.kvStore = new InMemoryKeyValueStore<>("local-state", context); +>>>>>>> removed ProcessorContext this.kvStore.restore(); // call restore inside processor.init >>>>>>> fix examples } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java index 2cceef6bcdf98..514065a0e5797 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java @@ -3,9 +3,12 @@ import io.confluent.streaming.KStreamException; import io.confluent.streaming.Processor; <<<<<<< HEAD +<<<<<<< HEAD ======= import io.confluent.streaming.PunctuationScheduler; >>>>>>> new api model +======= +>>>>>>> removed ProcessorContext import io.confluent.streaming.RecordCollector; import io.confluent.streaming.StateStore; import io.confluent.streaming.StreamingConfig; @@ -250,6 +253,7 @@ public void flush() { @Override public String topic() { +<<<<<<< HEAD <<<<<<< HEAD if (streamGroup.record() == null) throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); @@ -261,10 +265,17 @@ public String topic() { return this.streamGroup.record().topic(); >>>>>>> new api model +======= + if (streamGroup.record() == null) + throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); + + return streamGroup.record().topic(); +>>>>>>> removed ProcessorContext } @Override public int partition() { +<<<<<<< HEAD <<<<<<< HEAD if (streamGroup.record() == null) throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); @@ -274,9 +285,12 @@ public int partition() { stateMgr.restore(store, restoreFunc); ======= if (this.streamGroup.record() == null) +======= + if (streamGroup.record() == null) +>>>>>>> removed ProcessorContext throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); - return this.streamGroup.record().partition(); + return streamGroup.record().partition(); } @Override @@ -290,9 +304,10 @@ public long offset() { @Override public long timestamp() { - if (this.streamGroup.record() == null) + if (streamGroup.record() == null) throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); +<<<<<<< HEAD <<<<<<< HEAD stateMgr.register(store); ======= @@ -327,6 +342,9 @@ public void send(String topic, Object key, Object value) { @Override ======= return this.streamGroup.record().timestamp; +======= + return streamGroup.record().timestamp; +>>>>>>> removed ProcessorContext } @Override @@ -360,16 +378,20 @@ void init(Collection> streams) throws IOException { @Override public void commit() { - this.streamGroup.commitOffset(); + streamGroup.commitOffset(); } @Override - public PunctuationScheduler getPunctuationScheduler(Processor processor) { - return streamGroup.getPunctuationScheduler(processor); + public void schedule(Processor processor, long interval) { + streamGroup.schedule(processor, interval); } +<<<<<<< HEAD public void init(Collection> streams) throws IOException { >>>>>>> new api model +======= + void init(Collection> streams) throws IOException { +>>>>>>> removed ProcessorContext stateMgr.init(); for (KStreamSource stream: streams) { diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index 268e3371c8a8a..d8ffc03e56a82 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -30,6 +30,7 @@ abstract class KStreamImpl implements KStream, Receiver { private final ArrayList nextReceivers = new ArrayList<>(1); <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD protected KStreamTopology topology; protected KStreamContext context; @@ -58,12 +59,15 @@ public void close() { ======= ======= protected KStreamTopology initializer; +======= + protected KStreamTopology topology; +>>>>>>> removed ProcessorContext protected KStreamContext context; >>>>>>> wip protected KStreamMetadata metadata; - protected KStreamImpl(KStreamTopology initializer) { - this.initializer = initializer; + protected KStreamImpl(KStreamTopology topology) { + this.topology = topology; } @Override @@ -88,11 +92,15 @@ public void close() { @Override public KStream filter(Predicate predicate) { +<<<<<<< HEAD <<<<<<< HEAD return chain(new KStreamFilter(predicate, topology)); ======= return chain(new KStreamFilter(predicate, initializer)); >>>>>>> new api model +======= + return chain(new KStreamFilter(predicate, topology)); +>>>>>>> removed ProcessorContext } @Override @@ -106,56 +114,80 @@ public boolean apply(K key, V value) { @Override public KStream map(KeyValueMapper mapper) { +<<<<<<< HEAD <<<<<<< HEAD return chain(new KStreamMap(mapper, topology)); ======= return chain(new KStreamMap(mapper, initializer)); >>>>>>> new api model +======= + return chain(new KStreamMap(mapper, topology)); +>>>>>>> removed ProcessorContext } @Override public KStream mapValues(ValueMapper mapper) { +<<<<<<< HEAD <<<<<<< HEAD return chain(new KStreamMapValues(mapper, topology)); ======= return chain(new KStreamMapValues(mapper, initializer)); >>>>>>> new api model +======= + return chain(new KStreamMapValues(mapper, topology)); +>>>>>>> removed ProcessorContext } @Override public KStream flatMap(KeyValueMapper, K, V> mapper) { +<<<<<<< HEAD <<<<<<< HEAD return chain(new KStreamFlatMap(mapper, topology)); ======= return chain(new KStreamFlatMap(mapper, initializer)); >>>>>>> new api model +======= + return chain(new KStreamFlatMap(mapper, topology)); +>>>>>>> removed ProcessorContext } @Override public KStream flatMapValues(ValueMapper, V> mapper) { +<<<<<<< HEAD <<<<<<< HEAD return chain(new KStreamFlatMapValues(mapper, topology)); ======= return chain(new KStreamFlatMapValues(mapper, initializer)); >>>>>>> new api model +======= + return chain(new KStreamFlatMapValues(mapper, topology)); +>>>>>>> removed ProcessorContext } @Override public KStreamWindowed with(Window window) { +<<<<<<< HEAD <<<<<<< HEAD return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, topology)); ======= return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, initializer)); >>>>>>> new api model +======= + return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, topology)); +>>>>>>> removed ProcessorContext } @Override public KStream[] branch(Predicate... predicates) { +<<<<<<< HEAD <<<<<<< HEAD KStreamBranch branch = new KStreamBranch<>(predicates, topology); ======= KStreamBranch branch = new KStreamBranch<>(predicates, initializer); >>>>>>> new api model +======= + KStreamBranch branch = new KStreamBranch<>(predicates, topology); +>>>>>>> removed ProcessorContext registerReceiver(branch); return branch.branches; } @@ -170,11 +202,15 @@ public KStream through(String topic) { @Override public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); +<<<<<<< HEAD <<<<<<< HEAD return topology.from(keyDeserializer, valDeserializer, topic); ======= return initializer.from(keyDeserializer, valDeserializer, topic); >>>>>>> new api model +======= + return topology.from(keyDeserializer, valDeserializer, topic); +>>>>>>> removed ProcessorContext } @Override @@ -198,11 +234,15 @@ public void init(KStreamContext context) { } @Override public void process(K key, V value) { +<<<<<<< HEAD <<<<<<< HEAD this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); ======= this.processorContext.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); >>>>>>> wip +======= + this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); +>>>>>>> removed ProcessorContext } @Override public void punctuate(long streamTime) {} diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java index c0e646f651e9c..6df4c936e9530 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java @@ -72,11 +72,15 @@ private KStream join(KStreamWindowed other, boolean prior KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; +<<<<<<< HEAD <<<<<<< HEAD KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, topology); ======= KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, initializer); >>>>>>> new api model +======= + KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, topology); +>>>>>>> removed ProcessorContext otherImpl.registerReceiver(stream.receiverForOtherStream); return chain(stream); diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java b/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java deleted file mode 100644 index 2874f8abbfe23..0000000000000 --- a/src/main/java/io/confluent/streaming/internal/ProcessorContextImpl.java +++ /dev/null @@ -1,68 +0,0 @@ -package io.confluent.streaming.internal; - -import io.confluent.streaming.Processor; -import io.confluent.streaming.PunctuationScheduler; -import io.confluent.streaming.KStreamContext; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.Serializer; - -/** - * Created by guozhang on 7/21/15. - */ -public class ProcessorContextImpl implements Processor.ProcessorContext { - - private final KStreamContext context; - private final PunctuationScheduler scheduler; - - public ProcessorContextImpl(KStreamContext context, - PunctuationScheduler scheduler) { - - this.context = context; - this.scheduler = scheduler; - } - - @Override - public String topic() { - return context.topic(); - } - - @Override - public int partition() { - return context.partition(); - } - - @Override - public long offset() { - return context.offset(); - } - - @Override - public long timestamp() { - return context.timestamp(); - } - - @Override - public void send(String topic, Object key, Object value) { - context.send(topic, key, value); - } - - @Override - public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { - context.send(topic, key, value, keySerializer, valSerializer); - } - - @Override - public void commit() { - context.commit(); - } - - @Override - public void schedule(long timestamp) { - scheduler.schedule(timestamp); - } - - @Override - public KStreamContext kstreamContext() { - return context; - } -} diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java index ccaac8cb26970..7050d9a5312d7 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java @@ -20,6 +20,7 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); this.context = context; +<<<<<<< HEAD <<<<<<< HEAD processor.init(context); } @@ -28,6 +29,9 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { public void receive(Object key, Object value, long timestamp) { ======= processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); +======= + processor.init(context); +>>>>>>> removed ProcessorContext } @SuppressWarnings("unchecked") @Override diff --git a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java b/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java deleted file mode 100644 index 014e140685268..0000000000000 --- a/src/test/java/io/confluent/streaming/internal/PunctuationSchedulerImplTest.java +++ /dev/null @@ -1,125 +0,0 @@ -package io.confluent.streaming.internal; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import io.confluent.streaming.PunctuationScheduler; -import io.confluent.streaming.testutil.TestProcessor; -import org.junit.Test; - -public class PunctuationSchedulerImplTest { - - @Test - public void testScheduling() { - PunctuationQueue queue = new PunctuationQueue(); - - TestProcessor proc1 = new TestProcessor<>(); - PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); - - TestProcessor proc2 = new TestProcessor<>(); - PunctuationScheduler sched2 = new PunctuationSchedulerImpl(queue, proc2); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - - proc1.init(new ProcessorContextImpl(null, sched1)); - proc2.init(new ProcessorContextImpl(null, sched2)); - - sched1.schedule(500); - sched2.schedule(1000); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - - queue.mayPunctuate(999); - - assertEquals(1, proc1.punctuated.size()); - assertEquals(999, (long) proc1.punctuated.get(0)); - assertEquals(0, proc2.punctuated.size()); - - proc1.punctuated.clear(); - queue.mayPunctuate(1000); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(1, proc2.punctuated.size()); - assertEquals(1000, (long) proc2.punctuated.get(0)); - - proc2.punctuated.clear(); - queue.mayPunctuate(2000); - assertEquals(0, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - - sched1.schedule(3000); - queue.mayPunctuate(4000); - - assertEquals(1, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - assertEquals(4000, (long) proc1.punctuated.get(0)); - } - - @Test - public void testCanceling() { - PunctuationQueue queue = new PunctuationQueue(); - - TestProcessor proc1 = new TestProcessor(); - PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); - - TestProcessor proc2 = new TestProcessor(); - PunctuationScheduler sched2 = new PunctuationSchedulerImpl(queue, proc2); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - - proc1.init(new ProcessorContextImpl(null, sched1)); - proc2.init(new ProcessorContextImpl(null, sched2)); - - sched1.schedule(500); - sched2.schedule(1000); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(0, proc2.punctuated.size()); - - sched1.cancel(); - - queue.mayPunctuate(1000); - - assertEquals(0, proc1.punctuated.size()); - assertEquals(1, proc2.punctuated.size()); - assertEquals(1000, (long) proc2.punctuated.get(0)); - - sched1.schedule(2000); - sched1.cancel(); - sched1.schedule(3000); - - queue.mayPunctuate(2000); - assertEquals(0, proc1.punctuated.size()); - - queue.mayPunctuate(3000); - assertEquals(1, proc1.punctuated.size()); - } - - @Test - public void testDuplicateScheduling() { - PunctuationQueue queue = new PunctuationQueue(); - - TestProcessor proc1 = new TestProcessor(); - PunctuationScheduler sched1 = new PunctuationSchedulerImpl(queue, proc1); - - assertEquals(0, proc1.punctuated.size()); - - proc1.init(new ProcessorContextImpl(null, sched1)); - - sched1.schedule(500); - - boolean exceptionRaised = false; - try { - sched1.schedule(1000); - } - catch (IllegalStateException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - } - -} diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java index a29926201899d..c43eff05dcbdc 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java +++ b/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java @@ -4,9 +4,12 @@ import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; <<<<<<< HEAD +<<<<<<< HEAD ======= import io.confluent.streaming.PunctuationScheduler; >>>>>>> new api model +======= +>>>>>>> removed ProcessorContext import io.confluent.streaming.RecordCollector; <<<<<<< HEAD import io.confluent.streaming.StorageEngine; @@ -21,8 +24,11 @@ import io.confluent.streaming.internal.PunctuationQueue; ======= import io.confluent.streaming.internal.PunctuationQueue; +<<<<<<< HEAD import io.confluent.streaming.internal.PunctuationSchedulerImpl; >>>>>>> new api model +======= +>>>>>>> removed ProcessorContext import io.confluent.streaming.kv.internals.RestoreFunc; >>>>>>> new api model import org.apache.kafka.common.metrics.Metrics; @@ -101,6 +107,7 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { throw new UnsupportedOperationException("send() not supported."); } @Override +<<<<<<< HEAD <<<<<<< HEAD public void schedule(Processor processor, long interval) { throw new UnsupportedOperationException("schedule() not supported"); @@ -108,6 +115,10 @@ public void schedule(Processor processor, long interval) { public PunctuationScheduler getPunctuationScheduler(Processor processor) { return new PunctuationSchedulerImpl(punctuationQueue, processor); >>>>>>> new api model +======= + public void schedule(Processor processor, long interval) { + throw new UnsupportedOperationException("schedule() not supported"); +>>>>>>> removed ProcessorContext } @Override diff --git a/temp b/temp index 1d9daf2d1f97d..5a79b2890e456 100644 --- a/temp +++ b/temp @@ -1,27 +1,8 @@ -src/main/java/io/confluent/streaming/KafkaStreaming.java: needs merge -src/main/java/io/confluent/streaming/ProcessorKStreamJob.java: needs merge -src/main/java/io/confluent/streaming/examples/MapKStreamJob.java: needs merge -src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java: needs merge +src/main/java/io/confluent/streaming/KStreamContext.java: needs merge +src/main/java/io/confluent/streaming/Processor.java: needs merge src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamBranch.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamFilter.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java: needs merge src/main/java/io/confluent/streaming/internal/KStreamImpl.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamJoin.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamMap.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamMapValues.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamSource.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamThread.java: needs merge src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java: needs merge src/main/java/io/confluent/streaming/internal/ProcessorNode.java: needs merge -src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java: needs merge -src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java: needs merge -src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java: needs merge -src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java: needs merge -src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java: needs merge -src/test/java/io/confluent/streaming/internal/KStreamMapTest.java: needs merge -src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java: needs merge -src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java: needs merge -src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java: needs merge -src/test/java/io/confluent/streaming/internal/StreamGroupTest.java: needs merge +src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java: needs merge diff --git a/temp2 b/temp2 index eae39d824ea89..2e2d85ca10d95 100644 --- a/temp2 +++ b/temp2 @@ -1,27 +1,8 @@ -src/main/java/io/confluent/streaming/KafkaStreaming.java -src/main/java/io/confluent/streaming/ProcessorKStreamJob.java -src/main/java/io/confluent/streaming/examples/MapKStreamJob.java -src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +src/main/java/io/confluent/streaming/KStreamContext.java +src/main/java/io/confluent/streaming/Processor.java src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java -src/main/java/io/confluent/streaming/internal/KStreamBranch.java -src/main/java/io/confluent/streaming/internal/KStreamFilter.java -src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java -src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java src/main/java/io/confluent/streaming/internal/KStreamImpl.java -src/main/java/io/confluent/streaming/internal/KStreamJoin.java -src/main/java/io/confluent/streaming/internal/KStreamMap.java -src/main/java/io/confluent/streaming/internal/KStreamMapValues.java -src/main/java/io/confluent/streaming/internal/KStreamSource.java -src/main/java/io/confluent/streaming/internal/KStreamThread.java src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java src/main/java/io/confluent/streaming/internal/ProcessorNode.java -src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java -src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java -src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java -src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java -src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java -src/test/java/io/confluent/streaming/internal/KStreamMapTest.java -src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java -src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java -src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java -src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java From 914a1497e25584788549dfd947e9fa717ef33367 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 3 Aug 2015 14:09:21 -0700 Subject: [PATCH 125/275] remove streamTime from Receiver --- .../java/io/confluent/streaming/internal/KStreamBranch.java | 6 +++++- .../java/io/confluent/streaming/internal/ProcessorNode.java | 3 +++ src/main/java/io/confluent/streaming/internal/Receiver.java | 4 ++++ 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index e672e1158a5c5..ea18d3be3fc50 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -63,6 +63,7 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { @SuppressWarnings("unchecked") @Override +<<<<<<< HEAD <<<<<<< HEAD public void receive(Object key, Object value, long timestamp) { for (int i = 0; i < predicates.length; i++) { @@ -80,10 +81,13 @@ public void close() { branch.close(); ======= public void receive(Object key, Object value, long timestamp, long streamTime) { +======= + public void receive(Object key, Object value, long timestamp) { +>>>>>>> remove streamTime from Receiver for (int i = 0; i < predicates.length; i++) { Predicate predicate = predicates[i]; if (predicate.apply((K)key, (V)value)) { - branches[i].receive(key, value, timestamp, streamTime); + branches[i].receive(key, value, timestamp); return; } >>>>>>> new api model diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java index 7050d9a5312d7..cd222a2e374dc 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorNode.java @@ -27,6 +27,7 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { @SuppressWarnings("unchecked") @Override public void receive(Object key, Object value, long timestamp) { +<<<<<<< HEAD ======= processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); ======= @@ -37,6 +38,8 @@ public void receive(Object key, Object value, long timestamp) { @Override public void receive(Object key, Object value, long timestamp, long streamTime) { >>>>>>> wip +======= +>>>>>>> remove streamTime from Receiver processor.process((K) key, (V) value); } @Override diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/src/main/java/io/confluent/streaming/internal/Receiver.java index 5720266bcef1f..16e80ce490b68 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/src/main/java/io/confluent/streaming/internal/Receiver.java @@ -9,6 +9,7 @@ public interface Receiver { void bind(KStreamContext context, KStreamMetadata metadata); +<<<<<<< HEAD <<<<<<< HEAD void receive(Object key, Object value, long timestamp); @@ -16,6 +17,9 @@ public interface Receiver { ======= void receive(Object key, Object value, long timestamp, long streamTime); >>>>>>> new api model +======= + void receive(Object key, Object value, long timestamp); +>>>>>>> remove streamTime from Receiver void close(); From 6a93f334f70515caa72408691895772fe93e1480 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 3 Aug 2015 15:29:49 -0700 Subject: [PATCH 126/275] fix parameter name --- .../streaming/internal/KStreamBranch.java | 8 ++++++++ .../streaming/internal/KStreamFilter.java | 5 +++++ .../streaming/internal/KStreamFlatMap.java | 5 +++++ .../streaming/internal/KStreamFlatMapValues.java | 5 +++++ .../streaming/internal/KStreamJoin.java | 5 +++++ .../confluent/streaming/internal/KStreamMap.java | 5 +++++ .../streaming/internal/KStreamMapValues.java | 5 +++++ .../streaming/internal/KStreamSource.java | 9 +++++++-- temp | 16 ++++++++-------- temp2 | 16 ++++++++-------- 10 files changed, 61 insertions(+), 18 deletions(-) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java index ea18d3be3fc50..fd345acc63656 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamBranch.java @@ -25,6 +25,7 @@ class KStreamBranch implements Receiver { @SuppressWarnings("unchecked") <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamBranch(Predicate[] predicates, KStreamTopology topology) { this.predicates = Arrays.copyOf(predicates, predicates.length); @@ -41,6 +42,13 @@ class KStreamBranch implements Receiver { for (int i = 0; i < branches.length; i++) { branches[i] = new KStreamSource<>(null, initializer); >>>>>>> new api model +======= + KStreamBranch(Predicate[] predicates, KStreamTopology topology) { + this.predicates = Arrays.copyOf(predicates, predicates.length); + this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); + for (int i = 0; i < branches.length; i++) { + branches[i] = new KStreamSource<>(null, topology); +>>>>>>> fix parameter name } } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java index 1c88c87ec1d48..dd06292146592 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFilter.java @@ -18,6 +18,7 @@ class KStreamFilter extends KStreamImpl { private final Predicate predicate; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamFilter(Predicate predicate, KStreamTopology topology) { @@ -29,6 +30,10 @@ class KStreamFilter extends KStreamImpl { >>>>>>> wip super(initializer); >>>>>>> new api model +======= + KStreamFilter(Predicate predicate, KStreamTopology topology) { + super(topology); +>>>>>>> fix parameter name this.predicate = predicate; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java index 1d8709ddbefd4..3ef1a1744a3f9 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java @@ -20,6 +20,7 @@ class KStreamFlatMap extends KStreamImpl { private final KeyValueMapper, K1, V1> mapper; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology topology) { @@ -31,6 +32,10 @@ class KStreamFlatMap extends KStreamImpl { >>>>>>> wip super(initializer); >>>>>>> new api model +======= + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology topology) { + super(topology); +>>>>>>> fix parameter name this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java index 0b6140c3d9eb9..386e73106d092 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java @@ -18,6 +18,7 @@ class KStreamFlatMapValues extends KStreamImpl { private final ValueMapper, V1> mapper; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology topology) { @@ -29,6 +30,10 @@ class KStreamFlatMapValues extends KStreamImpl { >>>>>>> wip super(initializer); >>>>>>> new api model +======= + KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology topology) { + super(topology); +>>>>>>> fix parameter name this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java index 31d4609448fcb..a7a64a1683e55 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamJoin.java @@ -32,6 +32,7 @@ private static abstract class Finder { private KStreamMetadata thisMetadata; private KStreamMetadata otherMetadata; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology topology) { @@ -43,6 +44,10 @@ private static abstract class Finder { >>>>>>> wip super(initializer); >>>>>>> new api model +======= + KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology topology) { + super(topology); +>>>>>>> fix parameter name final Window window1 = stream1.window; final Window window2 = stream2.window; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/src/main/java/io/confluent/streaming/internal/KStreamMap.java index 4c3118a5b0e55..11ba6ce2d2c36 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMap.java @@ -20,6 +20,7 @@ class KStreamMap extends KStreamImpl { private final KeyValueMapper mapper; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamMap(KeyValueMapper mapper, KStreamTopology topology) { @@ -31,6 +32,10 @@ class KStreamMap extends KStreamImpl { >>>>>>> wip super(initializer); >>>>>>> new api model +======= + KStreamMap(KeyValueMapper mapper, KStreamTopology topology) { + super(topology); +>>>>>>> fix parameter name this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java index b996c3e533710..a0d42e491613d 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java @@ -18,6 +18,7 @@ class KStreamMapValues extends KStreamImpl { private final ValueMapper mapper; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamMapValues(ValueMapper mapper, KStreamTopology topology) { @@ -29,6 +30,10 @@ class KStreamMapValues extends KStreamImpl { >>>>>>> wip super(initializer); >>>>>>> new api model +======= + KStreamMapValues(ValueMapper mapper, KStreamTopology topology) { + super(topology); +>>>>>>> fix parameter name this.mapper = mapper; } diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/src/main/java/io/confluent/streaming/internal/KStreamSource.java index c01b77dd5fdda..2fcdf47ecc2b6 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamSource.java @@ -49,13 +49,18 @@ public KStreamSource(String[] topics, Deserializer keyDeserializer, Deseriali String[] topics; >>>>>>> wip - public KStreamSource(String[] topics, KStreamTopology initializer) { - this(topics, null, null, initializer); + public KStreamSource(String[] topics, KStreamTopology topology) { + this(topics, null, null, topology); } +<<<<<<< HEAD public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology initializer) { super(initializer); >>>>>>> new api model +======= + public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology topology) { + super(topology); +>>>>>>> fix parameter name this.topics = topics; this.keyDeserializer = keyDeserializer; this.valueDeserializer = valueDeserializer; diff --git a/temp b/temp index 5a79b2890e456..e30005a96ec13 100644 --- a/temp +++ b/temp @@ -1,8 +1,8 @@ -src/main/java/io/confluent/streaming/KStreamContext.java: needs merge -src/main/java/io/confluent/streaming/Processor.java: needs merge -src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamImpl.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java: needs merge -src/main/java/io/confluent/streaming/internal/ProcessorNode.java: needs merge -src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamBranch.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamFilter.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamJoin.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamMap.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamMapValues.java: needs merge +src/main/java/io/confluent/streaming/internal/KStreamSource.java: needs merge diff --git a/temp2 b/temp2 index 2e2d85ca10d95..a5478493edd05 100644 --- a/temp2 +++ b/temp2 @@ -1,8 +1,8 @@ -src/main/java/io/confluent/streaming/KStreamContext.java -src/main/java/io/confluent/streaming/Processor.java -src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java -src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java -src/main/java/io/confluent/streaming/internal/KStreamImpl.java -src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java -src/main/java/io/confluent/streaming/internal/ProcessorNode.java -src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java +src/main/java/io/confluent/streaming/internal/KStreamBranch.java +src/main/java/io/confluent/streaming/internal/KStreamFilter.java +src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +src/main/java/io/confluent/streaming/internal/KStreamJoin.java +src/main/java/io/confluent/streaming/internal/KStreamMap.java +src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +src/main/java/io/confluent/streaming/internal/KStreamSource.java From 1d72ea533180312659153302b0e305991bf7d51d Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 3 Aug 2015 16:14:25 -0700 Subject: [PATCH 127/275] KStream.tranform method for generalized transformation --- .../io/confluent/streaming/internal/KStreamImpl.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java index d8ffc03e56a82..b08eb93d410ca 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/src/main/java/io/confluent/streaming/internal/KStreamImpl.java @@ -253,6 +253,7 @@ public void close() {} @SuppressWarnings("unchecked") @Override +<<<<<<< HEAD <<<<<<< HEAD public void process(Processor processor) { registerReceiver(new ProcessorNode<>(processor)); @@ -276,10 +277,19 @@ public void receive(Object key, Object value, long timestamp, long streamTime) { registerReceiver(receiver); >>>>>>> new api model ======= +======= + public void process(Processor processor) { +>>>>>>> KStream.tranform method for generalized transformation registerReceiver(new ProcessorNode<>(processor)); >>>>>>> wip } + @SuppressWarnings("unchecked") + @Override + public KStream transform(Transformer transformer) { + return chain(new KStreamTransform<>(transformer, topology)); + } + void registerReceiver(Receiver receiver) { nextReceivers.add(receiver); } From e64ecebb691c46739187b9eaf1238e5c76cfc347 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 4 Aug 2015 16:39:03 -0700 Subject: [PATCH 128/275] remove restore function --- .../confluent/streaming/KStreamContext.java | 6 +- .../io/confluent/streaming/SlidingWindow.java | 5 -- .../io/confluent/streaming/StateStore.java | 8 -- .../examples/StatefulKStreamJob.java | 3 + .../internal/ProcessorStateManager.java | 16 ++-- .../streaming/kv/InMemoryKeyValueStore.java | 14 +-- .../confluent/streaming/kv/KeyValueStore.java | 12 +-- .../streaming/kv/RocksDBKeyValueStore.java | 38 +------- .../kv/internals/LoggedKeyValueStore.java | 43 ++++----- .../kv/internals/MeteredKeyValueStore.java | 87 ++++++++++++++++--- .../streaming/kv/internals/RestoreFunc.java | 2 - 11 files changed, 114 insertions(+), 120 deletions(-) diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/src/main/java/io/confluent/streaming/KStreamContext.java index 77a867dd645f4..7eb622df5cf31 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/src/main/java/io/confluent/streaming/KStreamContext.java @@ -69,9 +69,10 @@ public interface KStreamContext { Metrics metrics(); /** - * Restores the specified storage engine. + * Registers and possibly restores the specified storage engine. * @param store the storage engine */ +<<<<<<< HEAD void restore(StateStore store, RestoreFunc restoreFunc); /** @@ -80,6 +81,9 @@ public interface KStreamContext { * @param store the storage engine */ void register(StateStore store); +======= + void register(StateStore store, RestoreFunc restoreFunc); +>>>>>>> remove restore function /** * Ensures that the context is in the initialization phase where KStream topology can be constructed diff --git a/src/main/java/io/confluent/streaming/SlidingWindow.java b/src/main/java/io/confluent/streaming/SlidingWindow.java index 48869d5fab809..ddb25c769240c 100644 --- a/src/main/java/io/confluent/streaming/SlidingWindow.java +++ b/src/main/java/io/confluent/streaming/SlidingWindow.java @@ -126,11 +126,6 @@ public void close() { // TODO } - @Override - public void restore() { - // TODO - } - @Override public boolean persistent() { // TODO: should not be persistent, right? diff --git a/src/main/java/io/confluent/streaming/StateStore.java b/src/main/java/io/confluent/streaming/StateStore.java index 4beb3d68453c6..ec5e6a347ab38 100644 --- a/src/main/java/io/confluent/streaming/StateStore.java +++ b/src/main/java/io/confluent/streaming/StateStore.java @@ -17,9 +17,6 @@ package io.confluent.streaming; - -import org.apache.kafka.clients.consumer.ConsumerRecord; - /** * A storage engine for managing state maintained by a stream processor. * @@ -48,11 +45,6 @@ public interface StateStore { */ void close(); - /** - * Restore the state of the storage - */ - void restore(); - /** * If the storage is persistent */ diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java index a1d1a6d0a7a7f..7c7112a6dc014 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java @@ -29,6 +29,7 @@ public void init(KStreamContext context) { <<<<<<< HEAD <<<<<<< HEAD this.kvStore = new InMemoryKeyValueStore<>("local-state", context); +<<<<<<< HEAD this.kvStore.restore(); // call restore inside processor.init ======= this.kvStore = new InMemoryKeyValueStore<>("local-state", context.kstreamContext()); @@ -41,6 +42,8 @@ public void init(KStreamContext context) { >>>>>>> removed ProcessorContext this.kvStore.restore(); // call restore inside processor.init >>>>>>> fix examples +======= +>>>>>>> remove restore function } @Override diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java index a699592d79107..d1b07088e6eed 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java @@ -69,17 +69,17 @@ public void init() throws IOException { checkpoint.delete(); } - public void register(StateStore store) { + 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().keySet().contains(store.name())) { - // TODO: if we know the total number of context ids then we can just check if the #.partitions match that number. - // my id must be in the partition list boolean partitionNotFound = true; for (PartitionInfo partitionInfo : restoreConsumer.partitionsFor(store.name())) { if (partitionInfo.partition() == id) { @@ -92,16 +92,13 @@ public void register(StateStore store) { throw new IllegalStateException("Store " + store.name() + "'s change log does not contain the partition for group " + id); } else { - // try to create the topic with the number of partitions equal to the total number of store instances. - // TODO: this is not possible yet since we do not know the total number of ids. - throw new UnsupportedOperationException("Cannot create change log topic on-the-fly"); + throw new IllegalStateException("Change log topic for store " + store.name() + " does not exist yet"); } - // register store this.stores.put(store.name(), store); - } - public void restore(StateStore store, RestoreFunc restoreFunc) { + // ---- try to restore the state from change-log ---- // + // subscribe to the store's partition TopicPartition storePartition = new TopicPartition(store.name(), id); if (!restoreConsumer.subscriptions().isEmpty()) { @@ -117,7 +114,6 @@ public void restore(StateStore store, RestoreFunc restoreFunc) { // 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)) { - restoreFunc.load(); restoreConsumer.seek(storePartition, checkpointedOffsets.get(storePartition)); } else { restoreConsumer.seekToBeginning(storePartition); diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java index bfb27a3953571..275c67f689651 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java @@ -25,15 +25,7 @@ public InMemoryKeyValueStore(String name, KStreamContext context) { } public InMemoryKeyValueStore(String name, KStreamContext context, Time time) { - // always wrap the logged store with the metered store - // TODO: this may need to be relaxed in the future - super(name, - "kafka-streams", - new LoggedKeyValueStore<>(name, /* store name as topic name */ - new MemoryStore(name, context), - context), - context.metrics(), - time); + super(name, new MemoryStore(name, context), context, "kafka-streams", time); } private static class MemoryStore implements KeyValueStore { @@ -44,11 +36,10 @@ private static class MemoryStore implements KeyValueStore { @SuppressWarnings("unchecked") public MemoryStore(String name, KStreamContext context) { + super(); this.name = name; this.map = new TreeMap<>(); this.context = context; - - this.context.register(this); } @Override @@ -95,7 +86,6 @@ public void flush() { // do-nothing since it is in-memory } - @Override 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"); diff --git a/src/main/java/io/confluent/streaming/kv/KeyValueStore.java b/src/main/java/io/confluent/streaming/kv/KeyValueStore.java index d13f468477291..85b076cf2fff6 100644 --- a/src/main/java/io/confluent/streaming/kv/KeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/KeyValueStore.java @@ -38,7 +38,7 @@ public interface KeyValueStore extends StateStore { * @return The value or null if no value is found. * @throws NullPointerException If null is used for key. */ - public V get(K key); + abstract public V get(K key); /** * Update the value associated with this key @@ -47,7 +47,7 @@ public interface KeyValueStore extends StateStore { * @param value The value * @throws NullPointerException If null is used for key or value. */ - public void put(K key, V value); + abstract public void put(K key, V value); /** * Update all the given key/value pairs @@ -55,7 +55,7 @@ public interface KeyValueStore extends StateStore { * @param entries A list of entries to put into the store. * @throws NullPointerException If null is used for any key or value. */ - public void putAll(List> entries); + abstract public void putAll(List> entries); /** * Delete the value from the store (if there is one) @@ -63,7 +63,7 @@ public interface KeyValueStore extends StateStore { * @param key The key * @throws NullPointerException If null is used for key. */ - public void delete(K key); + abstract public void delete(K key); /** * Get an iterator over a given range of keys. This iterator MUST be closed after use. @@ -73,13 +73,13 @@ public interface KeyValueStore extends StateStore { * @return The iterator for this range. * @throws NullPointerException If null is used for from or to. */ - public KeyValueIterator range(K from, K to); + abstract public KeyValueIterator range(K from, K to); /** * Return an iterator over all keys in the database. This iterator MUST be closed after use. * * @return An iterator of all key/value pairs in the store. */ - public KeyValueIterator all(); + abstract public KeyValueIterator all(); } diff --git a/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java index 82f34622b0329..b3842c92b7067 100644 --- a/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java @@ -32,15 +32,7 @@ public RocksDBKeyValueStore(String name, KStreamContext context) { } public RocksDBKeyValueStore(String name, KStreamContext context, Time time) { - // always wrap the logged store with the metered store - // TODO: this may need to be relaxed in the future - super(name, - "kafka-streams", - new LoggedKeyValueStore<>(name, /* store name as topic name */ - new RocksDBStore(name, context), - context), - context.metrics(), - time); + super(name, new RocksDBStore(name, context), context, "kafka-streams", time); } private static class RocksDBStore implements KeyValueStore { @@ -56,7 +48,6 @@ private static class RocksDBStore implements KeyValueStore { 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 static final String TMP_FILE_SUFFIX = ".tmp"; private final String topic; private final int partition; @@ -68,7 +59,6 @@ private static class RocksDBStore implements KeyValueStore { private final String dbName; private final String dirName; - private final File tmpFile; private RocksDB db; @@ -101,15 +91,7 @@ public RocksDBStore(String name, KStreamContext context) { dbName = this.topic + "." + this.partition; dirName = this.context.stateDir() + File.separator + DB_FILE_DIR; - // rename the file with a tmp suffix to make sure the db instance is created fresh at first - tmpFile = new File(dirName, dbName); - if (tmpFile.exists()) { - if (!tmpFile.renameTo(new File(dirName, dbName + TMP_FILE_SUFFIX))) - throw new KafkaException("Failed to add the tmp suffix to the existing file " + tmpFile.getName()); - } db = openDB(new File(dirName, dbName), this.options, TTL_SECONDS); - - this.context.register(this); } private RocksDB openDB(File dir, Options options, int ttl) { @@ -194,24 +176,6 @@ public void flush() { } } - @Override - public void restore() { - if (tmpFile.exists()) { - // close the db and delete its file - // TODO: this db should not take any writes yet - db.close(); - File file = new File(dirName, dbName); - if (!file.delete()) - throw new KafkaException("Failed to delete the existing file " + file.getName()); - - // rename the tmp file by removing its tmp suffix and reopen the database - if (!tmpFile.renameTo(file)) - throw new KafkaException("Failed to remove the tmp suffix to the existing file " + tmpFile.getName()); - - db = openDB(tmpFile, this.options, TTL_SECONDS); - } - } - @Override public void close() { flush(); diff --git a/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java index 0c96a47b64e67..4b4420baeb409 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java @@ -22,50 +22,41 @@ public class LoggedKeyValueStore implements KeyValueStore { private final String topic; private final int partition; - private final KStreamContext context; - private final Set dirty; private final int maxDirty; + private final KStreamContext context; - - public LoggedKeyValueStore(String topic, KeyValueStore inner, KStreamContext context) { - this.inner = inner; - this.context = context; - + public LoggedKeyValueStore(final String topic, final KeyValueStore inner, KStreamContext context) { this.topic = topic; this.partition = context.id(); + this.context = context; + this.inner = inner; this.dirty = new HashSet(); this.maxDirty = 100; // TODO: this needs to be configurable - } - @Override - public String name() { - return inner.name(); - } - - @Override - public boolean persistent() { - return inner.persistent(); - } - - @Override - public void restore() { + // try to restore the state from the logs final Deserializer keyDeserializer = (Deserializer) context.keySerializer(); final Deserializer valDeserializer = (Deserializer) context.valueSerializer(); - context.restore(this, new RestoreFunc () { + context.register(this, new RestoreFunc () { @Override public void apply(byte[] key, byte[] value) { inner.put(keyDeserializer.deserialize(topic, key), valDeserializer.deserialize(topic, value)); } - - @Override - public void load() { - inner.restore(); - } }); + + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public boolean persistent() { + return inner.persistent(); } @Override diff --git a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java index 539917e55a6ed..915e12241bbf6 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java +++ b/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java @@ -1,8 +1,11 @@ package io.confluent.streaming.kv.internals; +import io.confluent.streaming.KStreamContext; +import io.confluent.streaming.RecordCollector; import io.confluent.streaming.kv.Entry; import io.confluent.streaming.kv.KeyValueIterator; import io.confluent.streaming.kv.KeyValueStore; +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; @@ -11,15 +14,19 @@ 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 java.util.HashSet; import java.util.List; +import java.util.Set; public class MeteredKeyValueStore implements KeyValueStore { protected final KeyValueStore inner; + private final Time time; - private final Metrics metrics; private final String group; private final Sensor putTime; private final Sensor getTime; @@ -29,11 +36,19 @@ public class MeteredKeyValueStore implements KeyValueStore { private final Sensor rangeTime; private final Sensor flushTime; private final Sensor restoreTime; + private final Metrics metrics; + + private final String topic; + private final int partition; + private final Set dirty; + private final int maxDirty; + private final KStreamContext context; - public MeteredKeyValueStore(String name, String group, KeyValueStore inner, Metrics metrics, Time time) { + // always wrap the logged store with the metered store + public MeteredKeyValueStore(final String name, final KeyValueStore inner, KStreamContext context, String group, Time time) { this.inner = inner; + this.time = time; - this.metrics = metrics; this.group = group; this.putTime = createSensor(name, "put"); this.getTime = createSensor(name, "get"); @@ -43,6 +58,32 @@ public MeteredKeyValueStore(String name, String group, KeyValueStore inner, this.rangeTime = createSensor(name, "range"); this.flushTime = createSensor(name, "flush"); this.restoreTime = createSensor(name, "restore"); + this.metrics = context.metrics(); + + this.topic = name; + this.partition = context.id(); + + this.context = context; + + this.dirty = new HashSet(); + this.maxDirty = 100; // TODO: this needs to be configurable + + // register and possibly restore the state from the logs + long startNs = time.nanoseconds(); + try { + final Deserializer keyDeserializer = (Deserializer) context.keySerializer(); + final Deserializer valDeserializer = (Deserializer) context.valueSerializer(); + + 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) { @@ -74,16 +115,6 @@ public boolean persistent() { return inner.persistent(); } - @Override - public void restore() { - long startNs = time.nanoseconds(); - try { - inner.restore(); - } finally { - recordLatency(this.restoreTime, startNs, time.nanoseconds()); - } - } - @Override public V get(K key) { long startNs = time.nanoseconds(); @@ -99,6 +130,10 @@ 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()); } @@ -109,6 +144,13 @@ public void putAll(List> entries) { long startNs = time.nanoseconds(); try { this.inner.putAll(entries); + + for (Entry entry : entries) { + this.dirty.add(entry.key()); + } + + if (this.dirty.size() > this.maxDirty) + logChange(); } finally { recordLatency(this.putAllTime, startNs, time.nanoseconds()); } @@ -119,6 +161,10 @@ public void delete(K key) { long startNs = time.nanoseconds(); try { this.inner.delete(key); + + this.dirty.add(key); + if (this.dirty.size() > this.maxDirty) + logChange(); } finally { recordLatency(this.deleteTime, startNs, time.nanoseconds()); } @@ -142,11 +188,26 @@ public void flush() { long startNs = time.nanoseconds(); try { this.inner.flush(); + logChange(); } finally { recordLatency(this.flushTime, startNs, time.nanoseconds()); } } + private void logChange() { + RecordCollector collector = context.recordCollector(); + Serializer keySerializer = (Serializer) context.keySerializer(); + Serializer valueSerializer = (Serializer) context.valueSerializer(); + + if(collector != null) { + for (K k : this.dirty) { + V v = this.inner.get(k); + collector.send(new ProducerRecord<>(this.topic, this.partition, k, v), keySerializer, valueSerializer); + } + this.dirty.clear(); + } + } + private void recordLatency(Sensor sensor, long startNs, long endNs) { sensor.record((endNs - startNs) / 1000000, endNs); } diff --git a/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java b/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java index 20afad2c50cb7..096bbd2c86c1f 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java +++ b/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java @@ -8,6 +8,4 @@ public interface RestoreFunc { void apply(byte[] key, byte[] value); - - void load(); } From c07d61cbe0264add5df8707fb989d3f7ac60ec80 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 4 Aug 2015 17:57:30 -0700 Subject: [PATCH 129/275] first re-org --- .../kafka/clients/processor}/Processor.java | 2 +- .../kv/internals/LoggedKeyValueStore.java | 131 ------------------ .../org/apache/kafka/stream}/Chooser.java | 2 +- .../org/apache/kafka/stream}/KStream.java | 10 +- .../apache/kafka/stream}/KStreamContext.java | 2 +- .../kafka/stream}/KStreamException.java | 2 +- .../apache/kafka/stream}/KafkaStreaming.java | 2 +- .../stream}/NotCopartitionedException.java | 2 +- .../apache/kafka/stream}/RecordCollector.java | 2 +- .../org/apache/kafka/stream}/StateStore.java | 2 +- .../apache/kafka/stream}/StreamingConfig.java | 2 +- .../kafka/stream}/TimestampExtractor.java | 2 +- .../kafka/stream}/examples/MapKStreamJob.java | 6 +- .../stream}/examples/PrintKStreamJob.java | 2 +- .../stream}/examples/StatefulKStreamJob.java | 2 +- .../kafka/stream}/internal/Ingestor.java | 2 +- .../kafka/stream}/internal/IngestorImpl.java | 2 +- .../stream}/internal/KStreamContextImpl.java | 2 +- .../stream}/internal/PartitioningInfo.java | 2 +- .../stream}/internal/ProcessorConfig.java | 2 +- .../kafka/stream}/internal/ProcessorNode.java | 2 +- .../internal/ProcessorStateManager.java | 2 +- .../stream}/internal/PunctuationQueue.java | 3 +- .../stream}/internal/PunctuationSchedule.java | 2 +- .../kafka/stream}/internal/Receiver.java | 2 +- .../stream}/internal/RecordCollectorImpl.java | 2 +- .../kafka/stream}/internal/RecordQueue.java | 3 +- .../stream}/internal/RoundRobinChooser.java | 2 +- .../kafka/stream}/internal/StampedRecord.java | 2 +- .../kafka/stream}/internal/StreamGroup.java | 3 +- .../stream}/internal/TimeBasedChooser.java | 2 +- .../org/apache/kafka/stream}/kv/Entry.java | 2 +- .../stream}/kv/InMemoryKeyValueStore.java | 3 +- .../kafka/stream}/kv/KeyValueIterator.java | 4 +- .../kafka/stream}/kv/KeyValueStore.java | 2 +- .../apache/kafka/stream/kv}/RestoreFunc.java | 2 +- .../stream}/kv/RocksDBKeyValueStore.java | 3 +- .../kv/internal}/MeteredKeyValueStore.java | 2 +- .../stream/topology}/KStreamTopology.java | 3 +- .../stream/topology}/KStreamWindowed.java | 4 +- .../kafka/stream/topology}/KeyValue.java | 2 +- .../stream/topology}/KeyValueMapper.java | 2 +- .../kafka/stream/topology}/Predicate.java | 2 +- .../topology}/SingleProcessorTopology.java | 2 +- .../kafka/stream/topology}/SlidingWindow.java | 2 +- .../kafka/stream/topology}/Transformer.java | 2 +- .../kafka/stream/topology}/ValueJoiner.java | 2 +- .../kafka/stream/topology}/ValueMapper.java | 2 +- .../apache/kafka/stream/topology}/Window.java | 2 +- .../topology}/internal/KStreamBranch.java | 3 +- .../topology}/internal/KStreamFilter.java | 3 +- .../topology}/internal/KStreamFlatMap.java | 3 +- .../internal/KStreamFlatMapValues.java | 3 +- .../topology}/internal/KStreamImpl.java | 9 +- .../topology}/internal/KStreamJoin.java | 4 +- .../stream/topology}/internal/KStreamMap.java | 3 +- .../topology}/internal/KStreamMapValues.java | 3 +- .../topology}/internal/KStreamMetadata.java | 5 +- .../topology}/internal/KStreamSource.java | 3 +- .../topology}/internal/KStreamThread.java | 5 +- .../topology}/internal/KStreamTransform.java | 5 +- .../internal/KStreamWindowedImpl.java | 6 +- .../kafka/stream}/util/FilteredIterator.java | 2 +- .../stream}/util/MinTimestampTracker.java | 2 +- .../kafka/stream}/util/OffsetCheckpoint.java | 2 +- .../kafka/stream}/util/ParallelExecutor.java | 2 +- .../apache/kafka/stream}/util/Stamped.java | 2 +- .../kafka/stream}/util/TimestampTracker.java | 2 +- .../org/apache/kafka/stream}/util/Util.java | 2 +- .../kafka/stream}/FilteredIteratorTest.java | 2 +- .../kafka/stream}/KStreamBranchTest.java | 4 +- .../kafka/stream}/KStreamFilterTest.java | 3 +- .../kafka/stream}/KStreamFlatMapTest.java | 3 +- .../stream}/KStreamFlatMapValuesTest.java | 3 +- .../apache/kafka/stream}/KStreamJoinTest.java | 4 +- .../apache/kafka/stream}/KStreamMapTest.java | 3 +- .../kafka/stream}/KStreamMapValuesTest.java | 2 +- .../kafka/stream}/KStreamSourceTest.java | 4 +- .../kafka/stream}/KStreamTransformTest.java | 3 +- .../kafka/stream}/KStreamWindowedTest.java | 3 +- .../stream}/MinTimestampTrackerTest.java | 2 +- .../kafka/stream}/ParallelExecutorTest.java | 2 +- .../apache/kafka/stream}/StreamGroupTest.java | 4 +- .../org/apache/kafka/test}/MockIngestor.java | 2 +- .../kafka/test}/MockKStreamContext.java | 2 +- .../kafka/test}/MockKStreamTopology.java | 2 +- .../org/apache/kafka/test/MockProcessor.java | 4 +- .../apache/kafka/test}/UnlimitedWindow.java | 2 +- 88 files changed, 133 insertions(+), 239 deletions(-) rename {src/main/java/io/confluent/streaming => clients/src/main/java/org/apache/kafka/clients/processor}/Processor.java (94%) delete mode 100644 src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java rename {src/main/java/io/confluent/streaming/internal => stream/src/main/java/org/apache/kafka/stream}/Chooser.java (78%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/KStream.java (94%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/KStreamContext.java (98%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/KStreamException.java (91%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/KafkaStreaming.java (99%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/NotCopartitionedException.java (83%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/RecordCollector.java (91%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/StateStore.java (97%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/StreamingConfig.java (99%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/TimestampExtractor.java (92%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/examples/MapKStreamJob.java (89%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/examples/PrintKStreamJob.java (97%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/examples/StatefulKStreamJob.java (98%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/Ingestor.java (92%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/IngestorImpl.java (98%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/KStreamContextImpl.java (99%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/PartitioningInfo.java (81%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/ProcessorConfig.java (99%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/ProcessorNode.java (96%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/ProcessorStateManager.java (99%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/PunctuationQueue.java (90%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/PunctuationSchedule.java (92%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/Receiver.java (92%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/RecordCollectorImpl.java (98%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/RecordQueue.java (96%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/RoundRobinChooser.java (91%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/StampedRecord.java (93%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/StreamGroup.java (98%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/internal/TimeBasedChooser.java (95%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/kv/Entry.java (92%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/kv/InMemoryKeyValueStore.java (97%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/kv/KeyValueIterator.java (92%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/kv/KeyValueStore.java (98%) rename {src/main/java/io/confluent/streaming/kv/internals => stream/src/main/java/org/apache/kafka/stream/kv}/RestoreFunc.java (79%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/kv/RocksDBKeyValueStore.java (98%) rename {src/main/java/io/confluent/streaming/kv/internals => stream/src/main/java/org/apache/kafka/stream/kv/internal}/MeteredKeyValueStore.java (99%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/KStreamTopology.java (97%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/KStreamWindowed.java (94%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/KeyValue.java (88%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/KeyValueMapper.java (76%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/Predicate.java (73%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/SingleProcessorTopology.java (94%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/SlidingWindow.java (98%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/Transformer.java (85%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/ValueJoiner.java (74%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/ValueMapper.java (71%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/Window.java (89%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamBranch.java (97%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamFilter.java (91%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamFlatMap.java (93%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamFlatMapValues.java (92%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamImpl.java (96%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamJoin.java (96%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamMap.java (93%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamMapValues.java (91%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamMetadata.java (90%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamSource.java (97%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamThread.java (98%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamTransform.java (88%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream/topology}/internal/KStreamWindowedImpl.java (92%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/util/FilteredIterator.java (95%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/util/MinTimestampTracker.java (96%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/util/OffsetCheckpoint.java (99%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/util/ParallelExecutor.java (98%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/util/Stamped.java (92%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/util/TimestampTracker.java (95%) rename {src/main/java/io/confluent/streaming => stream/src/main/java/org/apache/kafka/stream}/util/Util.java (97%) rename {src/test/java/io/confluent/streaming/util => stream/src/test/java/org/apache/kafka/stream}/FilteredIteratorTest.java (98%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/KStreamBranchTest.java (97%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/KStreamFilterTest.java (97%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/KStreamFlatMapTest.java (96%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/KStreamFlatMapValuesTest.java (96%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/KStreamJoinTest.java (99%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/KStreamMapTest.java (96%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/KStreamMapValuesTest.java (98%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/KStreamSourceTest.java (95%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/KStreamTransformTest.java (97%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/KStreamWindowedTest.java (96%) rename {src/test/java/io/confluent/streaming/util => stream/src/test/java/org/apache/kafka/stream}/MinTimestampTrackerTest.java (98%) rename {src/test/java/io/confluent/streaming/util => stream/src/test/java/org/apache/kafka/stream}/ParallelExecutorTest.java (98%) rename {src/test/java/io/confluent/streaming/internal => stream/src/test/java/org/apache/kafka/stream}/StreamGroupTest.java (98%) rename {src/test/java/io/confluent/streaming/testutil => stream/src/test/java/org/apache/kafka/test}/MockIngestor.java (97%) rename {src/test/java/io/confluent/streaming/testutil => stream/src/test/java/org/apache/kafka/test}/MockKStreamContext.java (99%) rename {src/test/java/io/confluent/streaming/testutil => stream/src/test/java/org/apache/kafka/test}/MockKStreamTopology.java (82%) rename src/test/java/io/confluent/streaming/testutil/TestProcessor.java => stream/src/test/java/org/apache/kafka/test/MockProcessor.java (83%) rename {src/test/java/io/confluent/streaming/testutil => stream/src/test/java/org/apache/kafka/test}/UnlimitedWindow.java (97%) diff --git a/src/main/java/io/confluent/streaming/Processor.java b/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java similarity index 94% rename from src/main/java/io/confluent/streaming/Processor.java rename to clients/src/main/java/org/apache/kafka/clients/processor/Processor.java index e2e7401ea2a66..ac98c38b8291c 100644 --- a/src/main/java/io/confluent/streaming/Processor.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.clients.processor; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java b/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java deleted file mode 100644 index 4b4420baeb409..0000000000000 --- a/src/main/java/io/confluent/streaming/kv/internals/LoggedKeyValueStore.java +++ /dev/null @@ -1,131 +0,0 @@ -package io.confluent.streaming.kv.internals; - -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.RecordCollector; -import io.confluent.streaming.kv.Entry; -import io.confluent.streaming.kv.KeyValueIterator; -import io.confluent.streaming.kv.KeyValueStore; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; - -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -/** - * Created by guozhang on 7/30/15. - */ -public class LoggedKeyValueStore implements KeyValueStore { - - protected final KeyValueStore inner; - - private final String topic; - private final int partition; - private final Set dirty; - private final int maxDirty; - private final KStreamContext context; - - public LoggedKeyValueStore(final String topic, final KeyValueStore inner, KStreamContext context) { - this.topic = topic; - this.partition = context.id(); - this.context = context; - this.inner = inner; - - this.dirty = new HashSet(); - this.maxDirty = 100; // TODO: this needs to be configurable - - // try to restore the state from the logs - final Deserializer keyDeserializer = (Deserializer) context.keySerializer(); - final Deserializer valDeserializer = (Deserializer) context.valueSerializer(); - - context.register(this, new RestoreFunc () { - @Override - public void apply(byte[] key, byte[] value) { - inner.put(keyDeserializer.deserialize(topic, key), - valDeserializer.deserialize(topic, value)); - } - }); - - } - - @Override - public String name() { - return inner.name(); - } - - @Override - public boolean persistent() { - return inner.persistent(); - } - - @Override - public V get(K key) { - return inner.get(key); - } - - @Override - public void put(K key, V value) { - inner.put(key, value); - - this.dirty.add(key); - if (this.dirty.size() > this.maxDirty) - log(); - } - - @Override - public void putAll(List> entries) { - inner.putAll(entries); - - for (Entry entry : entries) { - this.dirty.add(entry.key()); - } - - if (this.dirty.size() > this.maxDirty) - log(); - } - - @Override - public void delete(K key) { - inner.delete(key); - - this.dirty.add(key); - if (this.dirty.size() > this.maxDirty) - log(); - - } - - @Override - public KeyValueIterator range(K from, K to) { - return inner.range(from, to); - } - - @Override - public KeyValueIterator all() { - return inner.all(); - } - - @Override - public void close() {} - - @SuppressWarnings("unchecked") - @Override - public void flush() { - inner.flush(); - log(); - } - - private void log() { - RecordCollector collector = context.recordCollector(); - Serializer keySerializer = (Serializer) context.keySerializer(); - Serializer valueSerializer = (Serializer) context.valueSerializer(); - - if(collector != null) { - for (K k : this.dirty) { - V v = this.inner.get(k); - collector.send(new ProducerRecord<>(this.topic, this.partition, k, v), keySerializer, valueSerializer); - } - this.dirty.clear(); - } - } -} diff --git a/src/main/java/io/confluent/streaming/internal/Chooser.java b/stream/src/main/java/org/apache/kafka/stream/Chooser.java similarity index 78% rename from src/main/java/io/confluent/streaming/internal/Chooser.java rename to stream/src/main/java/org/apache/kafka/stream/Chooser.java index aafe92cddeabb..4d5d0575d2f46 100644 --- a/src/main/java/io/confluent/streaming/internal/Chooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/Chooser.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; /** * Created by yasuhiro on 6/25/15. diff --git a/src/main/java/io/confluent/streaming/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java similarity index 94% rename from src/main/java/io/confluent/streaming/KStream.java rename to stream/src/main/java/org/apache/kafka/stream/KStream.java index 1f90c7f41b93e..2118d38b84855 100644 --- a/src/main/java/io/confluent/streaming/KStream.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java @@ -1,5 +1,11 @@ -package io.confluent.streaming; - +package org.apache.kafka.stream; + +import io.confluent.streaming.KStreamWindowed; +import io.confluent.streaming.KeyValueMapper; +import io.confluent.streaming.Predicate; +import io.confluent.streaming.Transformer; +import io.confluent.streaming.ValueMapper; +import io.confluent.streaming.Window; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; diff --git a/src/main/java/io/confluent/streaming/KStreamContext.java b/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java similarity index 98% rename from src/main/java/io/confluent/streaming/KStreamContext.java rename to stream/src/main/java/org/apache/kafka/stream/KStreamContext.java index 7eb622df5cf31..d10db1a4bcf6f 100644 --- a/src/main/java/io/confluent/streaming/KStreamContext.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream; import io.confluent.streaming.kv.internals.RestoreFunc; import org.apache.kafka.common.metrics.Metrics; diff --git a/src/main/java/io/confluent/streaming/KStreamException.java b/stream/src/main/java/org/apache/kafka/stream/KStreamException.java similarity index 91% rename from src/main/java/io/confluent/streaming/KStreamException.java rename to stream/src/main/java/org/apache/kafka/stream/KStreamException.java index 2273a510edbe8..2556e66c0ce66 100644 --- a/src/main/java/io/confluent/streaming/KStreamException.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamException.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream; /** * Created by yasuhiro on 7/2/15. diff --git a/src/main/java/io/confluent/streaming/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java similarity index 99% rename from src/main/java/io/confluent/streaming/KafkaStreaming.java rename to stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java index bb7c49c4c40cd..0832d7ba977bd 100644 --- a/src/main/java/io/confluent/streaming/KafkaStreaming.java +++ b/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package io.confluent.streaming; +package org.apache.kafka.stream; import io.confluent.streaming.internal.KStreamThread; import io.confluent.streaming.internal.ProcessorConfig; diff --git a/src/main/java/io/confluent/streaming/NotCopartitionedException.java b/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java similarity index 83% rename from src/main/java/io/confluent/streaming/NotCopartitionedException.java rename to stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java index 0a8a2fabdfe9f..1786535e1b717 100644 --- a/src/main/java/io/confluent/streaming/NotCopartitionedException.java +++ b/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream; /** * Created by yasuhiro on 6/19/15. diff --git a/src/main/java/io/confluent/streaming/RecordCollector.java b/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java similarity index 91% rename from src/main/java/io/confluent/streaming/RecordCollector.java rename to stream/src/main/java/org/apache/kafka/stream/RecordCollector.java index 5e827772c9331..8d6d79b72754f 100644 --- a/src/main/java/io/confluent/streaming/RecordCollector.java +++ b/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; diff --git a/src/main/java/io/confluent/streaming/StateStore.java b/stream/src/main/java/org/apache/kafka/stream/StateStore.java similarity index 97% rename from src/main/java/io/confluent/streaming/StateStore.java rename to stream/src/main/java/org/apache/kafka/stream/StateStore.java index ec5e6a347ab38..98abf4f7f6657 100644 --- a/src/main/java/io/confluent/streaming/StateStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/StateStore.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package io.confluent.streaming; +package org.apache.kafka.stream; /** * A storage engine for managing state maintained by a stream processor. diff --git a/src/main/java/io/confluent/streaming/StreamingConfig.java b/stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java similarity index 99% rename from src/main/java/io/confluent/streaming/StreamingConfig.java rename to stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java index 513adce295b99..08355c4042283 100644 --- a/src/main/java/io/confluent/streaming/StreamingConfig.java +++ b/stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package io.confluent.streaming; +package org.apache.kafka.stream; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; diff --git a/src/main/java/io/confluent/streaming/TimestampExtractor.java b/stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java similarity index 92% rename from src/main/java/io/confluent/streaming/TimestampExtractor.java rename to stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java index ab700a1893460..2f43758aa6c2f 100644 --- a/src/main/java/io/confluent/streaming/TimestampExtractor.java +++ b/stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream; /** * An interface that allows the KStream framework to extract a timestamp from a key-value pair diff --git a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java similarity index 89% rename from src/main/java/io/confluent/streaming/examples/MapKStreamJob.java rename to stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java index 80b2f4a7a3280..1d37c141cb789 100644 --- a/src/main/java/io/confluent/streaming/examples/MapKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java @@ -1,8 +1,12 @@ -package io.confluent.streaming.examples; +package org.apache.kafka.stream.examples; +<<<<<<< HEAD:src/main/java/io/confluent/streaming/examples/MapKStreamJob.java import io.confluent.streaming.KStream; <<<<<<< HEAD <<<<<<< HEAD +======= +import org.apache.kafka.stream.KStream; +>>>>>>> first re-org:stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java import io.confluent.streaming.KStreamTopology; ======= import io.confluent.streaming.KStreamInitializer; diff --git a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java similarity index 97% rename from src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java rename to stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java index a00b48435e600..74d410a5c4a9d 100644 --- a/src/main/java/io/confluent/streaming/examples/PrintKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.examples; +package org.apache.kafka.stream.examples; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KafkaStreaming; diff --git a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java similarity index 98% rename from src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java rename to stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java index 7c7112a6dc014..503b25e88e61e 100644 --- a/src/main/java/io/confluent/streaming/examples/StatefulKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.examples; +package org.apache.kafka.stream.examples; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KafkaStreaming; diff --git a/src/main/java/io/confluent/streaming/internal/Ingestor.java b/stream/src/main/java/org/apache/kafka/stream/internal/Ingestor.java similarity index 92% rename from src/main/java/io/confluent/streaming/internal/Ingestor.java rename to stream/src/main/java/org/apache/kafka/stream/internal/Ingestor.java index 617bf85bdc205..eeb793f6ddf22 100644 --- a/src/main/java/io/confluent/streaming/internal/Ingestor.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/Ingestor.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import org.apache.kafka.common.TopicPartition; diff --git a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java similarity index 98% rename from src/main/java/io/confluent/streaming/internal/IngestorImpl.java rename to stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java index e292710e567b1..4c28753ef254a 100644 --- a/src/main/java/io/confluent/streaming/internal/IngestorImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import org.apache.kafka.clients.consumer.CommitType; import org.apache.kafka.clients.consumer.Consumer; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java b/stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java similarity index 99% rename from src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java rename to stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java index 514065a0e5797..7da86701b0e55 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.KStreamException; import io.confluent.streaming.Processor; diff --git a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java b/stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java similarity index 81% rename from src/main/java/io/confluent/streaming/internal/PartitioningInfo.java rename to stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java index 5088cf1109b7a..82025f0353a66 100644 --- a/src/main/java/io/confluent/streaming/internal/PartitioningInfo.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; /** * Created by yasuhiro on 6/19/15. diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java similarity index 99% rename from src/main/java/io/confluent/streaming/internal/ProcessorConfig.java rename to stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java index 3364094a91b10..6cb659fecbfe1 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorConfig.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.StreamingConfig; import org.apache.kafka.common.config.AbstractConfig; diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java similarity index 96% rename from src/main/java/io/confluent/streaming/internal/ProcessorNode.java rename to stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java index cd222a2e374dc..9441ef8edc244 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorNode.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; diff --git a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java similarity index 99% rename from src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java rename to stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java index d1b07088e6eed..408bbd807d737 100644 --- a/src/main/java/io/confluent/streaming/internal/ProcessorStateManager.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.StateStore; import io.confluent.streaming.kv.internals.RestoreFunc; diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java similarity index 90% rename from src/main/java/io/confluent/streaming/internal/PunctuationQueue.java rename to stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java index d975c6f7bf2da..f081ace464969 100644 --- a/src/main/java/io/confluent/streaming/internal/PunctuationQueue.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java @@ -1,7 +1,6 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.Processor; -import io.confluent.streaming.util.Stamped; import java.util.PriorityQueue; diff --git a/src/main/java/io/confluent/streaming/internal/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java similarity index 92% rename from src/main/java/io/confluent/streaming/internal/PunctuationSchedule.java rename to stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java index ae8317633cd22..ae79fe88dd75e 100644 --- a/src/main/java/io/confluent/streaming/internal/PunctuationSchedule.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.Processor; import io.confluent.streaming.util.Stamped; diff --git a/src/main/java/io/confluent/streaming/internal/Receiver.java b/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java similarity index 92% rename from src/main/java/io/confluent/streaming/internal/Receiver.java rename to stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java index 16e80ce490b68..95b23b320206f 100644 --- a/src/main/java/io/confluent/streaming/internal/Receiver.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.KStreamContext; diff --git a/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java similarity index 98% rename from src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java rename to stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java index e5967bfb69ab0..b39ba0adbc4b6 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordCollectorImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.RecordCollector; import org.apache.kafka.clients.producer.Callback; diff --git a/src/main/java/io/confluent/streaming/internal/RecordQueue.java b/stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java similarity index 96% rename from src/main/java/io/confluent/streaming/internal/RecordQueue.java rename to stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java index f05deb7dcb1ca..72f9b7487b80a 100644 --- a/src/main/java/io/confluent/streaming/internal/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java @@ -1,8 +1,9 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.util.TimestampTracker; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.stream.topology.internal.KStreamSource; import java.util.ArrayDeque; diff --git a/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java similarity index 91% rename from src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java rename to stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java index b754bf490e02d..ec56e254204f1 100644 --- a/src/main/java/io/confluent/streaming/internal/RoundRobinChooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import java.util.ArrayDeque; diff --git a/src/main/java/io/confluent/streaming/internal/StampedRecord.java b/stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java similarity index 93% rename from src/main/java/io/confluent/streaming/internal/StampedRecord.java rename to stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java index 8f4de9c0b6668..55ce51e721995 100644 --- a/src/main/java/io/confluent/streaming/internal/StampedRecord.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.util.Stamped; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/src/main/java/io/confluent/streaming/internal/StreamGroup.java b/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java similarity index 98% rename from src/main/java/io/confluent/streaming/internal/StreamGroup.java rename to stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java index cb6f875b579c5..bbaee41a7be53 100644 --- a/src/main/java/io/confluent/streaming/internal/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; @@ -8,6 +8,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.topology.internal.KStreamSource; import java.util.ArrayDeque; import java.util.Collections; diff --git a/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java similarity index 95% rename from src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java rename to stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java index 924d548abb408..d0973be5498e4 100644 --- a/src/main/java/io/confluent/streaming/internal/TimeBasedChooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.internal; import java.util.Comparator; import java.util.PriorityQueue; diff --git a/src/main/java/io/confluent/streaming/kv/Entry.java b/stream/src/main/java/org/apache/kafka/stream/kv/Entry.java similarity index 92% rename from src/main/java/io/confluent/streaming/kv/Entry.java rename to stream/src/main/java/org/apache/kafka/stream/kv/Entry.java index 53cd874d86ca3..efafa0c7bfb9a 100644 --- a/src/main/java/io/confluent/streaming/kv/Entry.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/Entry.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.kv; +package org.apache.kafka.stream.kv; /** * Created by yasuhiro on 6/26/15. diff --git a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java similarity index 97% rename from src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java rename to stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java index 275c67f689651..81be2848535fe 100644 --- a/src/main/java/io/confluent/streaming/kv/InMemoryKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java @@ -1,7 +1,6 @@ -package io.confluent.streaming.kv; +package org.apache.kafka.stream.kv; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.kv.internals.LoggedKeyValueStore; import io.confluent.streaming.kv.internals.MeteredKeyValueStore; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; diff --git a/src/main/java/io/confluent/streaming/kv/KeyValueIterator.java b/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueIterator.java similarity index 92% rename from src/main/java/io/confluent/streaming/kv/KeyValueIterator.java rename to stream/src/main/java/org/apache/kafka/stream/kv/KeyValueIterator.java index 9b2e1795e2f75..c8e78329a2e69 100644 --- a/src/main/java/io/confluent/streaming/kv/KeyValueIterator.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueIterator.java @@ -17,9 +17,11 @@ * under the License. */ -package io.confluent.streaming.kv; +package org.apache.kafka.stream.kv; +import org.apache.kafka.stream.kv.Entry; + import java.io.Closeable; import java.util.Iterator; diff --git a/src/main/java/io/confluent/streaming/kv/KeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java similarity index 98% rename from src/main/java/io/confluent/streaming/kv/KeyValueStore.java rename to stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java index 85b076cf2fff6..ecab79779202a 100644 --- a/src/main/java/io/confluent/streaming/kv/KeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java @@ -17,7 +17,7 @@ * under the License. */ -package io.confluent.streaming.kv; +package org.apache.kafka.stream.kv; import io.confluent.streaming.StateStore; diff --git a/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java b/stream/src/main/java/org/apache/kafka/stream/kv/RestoreFunc.java similarity index 79% rename from src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java rename to stream/src/main/java/org/apache/kafka/stream/kv/RestoreFunc.java index 096bbd2c86c1f..1c1cbe852d4dc 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/RestoreFunc.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/RestoreFunc.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.kv.internals; +package org.apache.kafka.stream.kv; /** * Created by guozhang on 7/27/15. diff --git a/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java similarity index 98% rename from src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java rename to stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java index b3842c92b7067..c7d36aa2a4a8e 100644 --- a/src/main/java/io/confluent/streaming/kv/RocksDBKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java @@ -1,7 +1,6 @@ -package io.confluent.streaming.kv; +package org.apache.kafka.stream.kv; import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.kv.internals.LoggedKeyValueStore; import io.confluent.streaming.kv.internals.MeteredKeyValueStore; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.SystemTime; diff --git a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java similarity index 99% rename from src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java rename to stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java index 915e12241bbf6..180eaaaa9f07f 100644 --- a/src/main/java/io/confluent/streaming/kv/internals/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.kv.internals; +package org.apache.kafka.stream.kv.internal; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.RecordCollector; diff --git a/src/main/java/io/confluent/streaming/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java similarity index 97% rename from src/main/java/io/confluent/streaming/KStreamTopology.java rename to stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java index 0eb368667a4f4..26e391d4603c9 100644 --- a/src/main/java/io/confluent/streaming/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java @@ -1,7 +1,8 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; import io.confluent.streaming.internal.KStreamSource; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.KStream; import java.util.ArrayList; import java.util.Collection; diff --git a/src/main/java/io/confluent/streaming/KStreamWindowed.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java similarity index 94% rename from src/main/java/io/confluent/streaming/KStreamWindowed.java rename to stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java index c058d082c947b..89967b75b826d 100644 --- a/src/main/java/io/confluent/streaming/KStreamWindowed.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java @@ -1,4 +1,6 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; + +import org.apache.kafka.stream.KStream; /** * KStreamWindowed is an abstraction of a stream of key-value pairs with a window. diff --git a/src/main/java/io/confluent/streaming/KeyValue.java b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java similarity index 88% rename from src/main/java/io/confluent/streaming/KeyValue.java rename to stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java index 1ef0d3dea4189..43809d1068e38 100644 --- a/src/main/java/io/confluent/streaming/KeyValue.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/KeyValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java similarity index 76% rename from src/main/java/io/confluent/streaming/KeyValueMapper.java rename to stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java index f501ae15ea60f..bc0825e31b70b 100644 --- a/src/main/java/io/confluent/streaming/KeyValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/Predicate.java b/stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java similarity index 73% rename from src/main/java/io/confluent/streaming/Predicate.java rename to stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java index 469d1e30636d3..7a2574545edbc 100644 --- a/src/main/java/io/confluent/streaming/Predicate.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/SingleProcessorTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java similarity index 94% rename from src/main/java/io/confluent/streaming/SingleProcessorTopology.java rename to stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java index 59686c3f8db98..e7365dcbc8387 100644 --- a/src/main/java/io/confluent/streaming/SingleProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; import org.apache.kafka.common.utils.Utils; diff --git a/src/main/java/io/confluent/streaming/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java similarity index 98% rename from src/main/java/io/confluent/streaming/SlidingWindow.java rename to stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java index ddb25c769240c..df17fbcdb5fe0 100644 --- a/src/main/java/io/confluent/streaming/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; import io.confluent.streaming.util.FilteredIterator; import io.confluent.streaming.util.Stamped; diff --git a/src/main/java/io/confluent/streaming/Transformer.java b/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java similarity index 85% rename from src/main/java/io/confluent/streaming/Transformer.java rename to stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java index f3bf196853cb2..540dd87742dec 100644 --- a/src/main/java/io/confluent/streaming/Transformer.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/ValueJoiner.java b/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java similarity index 74% rename from src/main/java/io/confluent/streaming/ValueJoiner.java rename to stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java index aeae1139a94f6..6f43b9c68bc28 100644 --- a/src/main/java/io/confluent/streaming/ValueJoiner.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/ValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java similarity index 71% rename from src/main/java/io/confluent/streaming/ValueMapper.java rename to stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java index ad71546c5a74c..cf7ecd866ed98 100644 --- a/src/main/java/io/confluent/streaming/ValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/Window.java b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java similarity index 89% rename from src/main/java/io/confluent/streaming/Window.java rename to stream/src/main/java/org/apache/kafka/stream/topology/Window.java index 32eeb7536bcb6..6e329b5c89608 100644 --- a/src/main/java/io/confluent/streaming/Window.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java @@ -1,4 +1,4 @@ -package io.confluent.streaming; +package org.apache.kafka.stream.topology; import java.util.Iterator; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java similarity index 97% rename from src/main/java/io/confluent/streaming/internal/KStreamBranch.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java index fd345acc63656..3f2499bb37643 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; import io.confluent.streaming.KStreamContext; <<<<<<< HEAD @@ -11,6 +11,7 @@ import io.confluent.streaming.KStreamTopology; >>>>>>> wip import io.confluent.streaming.Predicate; +import org.apache.kafka.stream.topology.Predicate; import java.lang.reflect.Array; import java.util.Arrays; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java similarity index 91% rename from src/main/java/io/confluent/streaming/internal/KStreamFilter.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java index dd06292146592..aca026479a283 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; <<<<<<< HEAD <<<<<<< HEAD @@ -10,6 +10,7 @@ import io.confluent.streaming.KStreamTopology; >>>>>>> wip import io.confluent.streaming.Predicate; +import org.apache.kafka.stream.topology.Predicate; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java similarity index 93% rename from src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java index 3ef1a1744a3f9..96f55a4f48a08 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; import io.confluent.streaming.KStreamContext; <<<<<<< HEAD @@ -12,6 +12,7 @@ >>>>>>> wip import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; +import org.apache.kafka.stream.topology.KeyValueMapper; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java similarity index 92% rename from src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java index 386e73106d092..d35f62bd22938 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; <<<<<<< HEAD <<<<<<< HEAD @@ -10,6 +10,7 @@ import io.confluent.streaming.KStreamTopology; >>>>>>> wip import io.confluent.streaming.ValueMapper; +import org.apache.kafka.stream.topology.ValueMapper; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java similarity index 96% rename from src/main/java/io/confluent/streaming/internal/KStreamImpl.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java index b08eb93d410ca..4123ebcedea86 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java @@ -1,6 +1,6 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; -import io.confluent.streaming.KStream; +import org.apache.kafka.stream.KStream; import io.confluent.streaming.KStreamContext; <<<<<<< HEAD <<<<<<< HEAD @@ -20,6 +20,11 @@ import io.confluent.streaming.Window; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.stream.topology.KeyValueMapper; +import org.apache.kafka.stream.topology.Predicate; +import org.apache.kafka.stream.topology.Transformer; +import org.apache.kafka.stream.topology.ValueMapper; +import org.apache.kafka.stream.topology.Window; import java.util.ArrayList; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java similarity index 96% rename from src/main/java/io/confluent/streaming/internal/KStreamJoin.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java index a7a64a1683e55..27036d532f298 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; import io.confluent.streaming.KStreamContext; <<<<<<< HEAD @@ -13,6 +13,8 @@ import io.confluent.streaming.NotCopartitionedException; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; +import org.apache.kafka.stream.topology.ValueJoiner; +import org.apache.kafka.stream.topology.Window; import java.util.Iterator; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java similarity index 93% rename from src/main/java/io/confluent/streaming/internal/KStreamMap.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java index 11ba6ce2d2c36..b8aca99af0b9c 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; import io.confluent.streaming.KStreamContext; <<<<<<< HEAD @@ -12,6 +12,7 @@ >>>>>>> wip import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; +import org.apache.kafka.stream.topology.KeyValueMapper; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java similarity index 91% rename from src/main/java/io/confluent/streaming/internal/KStreamMapValues.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java index a0d42e491613d..0c8db36601c0a 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; <<<<<<< HEAD <<<<<<< HEAD @@ -10,6 +10,7 @@ import io.confluent.streaming.KStreamTopology; >>>>>>> wip import io.confluent.streaming.ValueMapper; +import org.apache.kafka.stream.topology.ValueMapper; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java similarity index 90% rename from src/main/java/io/confluent/streaming/internal/KStreamMetadata.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java index e7493ba910a70..97fc7b0b18840 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamMetadata.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java @@ -1,4 +1,7 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; + +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.internal.StreamGroup; import java.util.Collections; import java.util.Map; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java similarity index 97% rename from src/main/java/io/confluent/streaming/internal/KStreamSource.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java index 2fcdf47ecc2b6..5a38969635adb 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; <<<<<<< HEAD <<<<<<< HEAD @@ -16,7 +16,6 @@ import org.apache.kafka.common.serialization.Deserializer; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.Set; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamThread.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java similarity index 98% rename from src/main/java/io/confluent/streaming/internal/KStreamThread.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java index f3ba83fddedc0..1ae2cb6724715 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; <<<<<<< HEAD <<<<<<< HEAD @@ -49,6 +49,9 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.stream.internal.ProcessorConfig; +import org.apache.kafka.stream.internal.RecordCollectorImpl; +import org.apache.kafka.stream.internal.StreamGroup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/src/main/java/io/confluent/streaming/internal/KStreamTransform.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java similarity index 88% rename from src/main/java/io/confluent/streaming/internal/KStreamTransform.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java index 5cdb44068c748..c4ec1f3ecdbc7 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamTransform.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java @@ -1,10 +1,9 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamTopology; -import io.confluent.streaming.KeyValue; -import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.Transformer; +import org.apache.kafka.stream.topology.Transformer; /** * Created by yasuhiro on 6/17/15. diff --git a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java similarity index 92% rename from src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java index 6df4c936e9530..342edba1ae2e2 100644 --- a/src/main/java/io/confluent/streaming/internal/KStreamWindowedImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java @@ -1,7 +1,7 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream.topology.internal; -import io.confluent.streaming.KStream; +import org.apache.kafka.stream.KStream; import io.confluent.streaming.KStreamContext; <<<<<<< HEAD <<<<<<< HEAD @@ -15,6 +15,8 @@ import io.confluent.streaming.KStreamWindowed; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; +import org.apache.kafka.stream.topology.ValueJoiner; +import org.apache.kafka.stream.topology.Window; /** * Created by yasuhiro on 6/18/15. diff --git a/src/main/java/io/confluent/streaming/util/FilteredIterator.java b/stream/src/main/java/org/apache/kafka/stream/util/FilteredIterator.java similarity index 95% rename from src/main/java/io/confluent/streaming/util/FilteredIterator.java rename to stream/src/main/java/org/apache/kafka/stream/util/FilteredIterator.java index 76fe098780415..c9e67112c8382 100644 --- a/src/main/java/io/confluent/streaming/util/FilteredIterator.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/FilteredIterator.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.util; +package org.apache.kafka.stream.util; import java.util.Iterator; diff --git a/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java similarity index 96% rename from src/main/java/io/confluent/streaming/util/MinTimestampTracker.java rename to stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java index 2ec701995b8ed..f9daab2a8fc8a 100644 --- a/src/main/java/io/confluent/streaming/util/MinTimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.util; +package org.apache.kafka.stream.util; import java.util.LinkedList; diff --git a/src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java b/stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java similarity index 99% rename from src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java rename to stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java index 946c259eaf4ea..6138237ecad57 100644 --- a/src/main/java/io/confluent/streaming/util/OffsetCheckpoint.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.confluent.streaming.util; +package org.apache.kafka.stream.util; import org.apache.kafka.common.TopicPartition; diff --git a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java b/stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java similarity index 98% rename from src/main/java/io/confluent/streaming/util/ParallelExecutor.java rename to stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java index a1102e0dc754c..1cace23052360 100644 --- a/src/main/java/io/confluent/streaming/util/ParallelExecutor.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.util; +package org.apache.kafka.stream.util; import java.util.ArrayList; import java.util.concurrent.CountDownLatch; diff --git a/src/main/java/io/confluent/streaming/util/Stamped.java b/stream/src/main/java/org/apache/kafka/stream/util/Stamped.java similarity index 92% rename from src/main/java/io/confluent/streaming/util/Stamped.java rename to stream/src/main/java/org/apache/kafka/stream/util/Stamped.java index 2926f271a211f..f0ceaeda9b27b 100644 --- a/src/main/java/io/confluent/streaming/util/Stamped.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/Stamped.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.util; +package org.apache.kafka.stream.util; /** * Created by yasuhiro on 6/23/15. diff --git a/src/main/java/io/confluent/streaming/util/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java similarity index 95% rename from src/main/java/io/confluent/streaming/util/TimestampTracker.java rename to stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java index 97a34221b8431..9e57cd8568aff 100644 --- a/src/main/java/io/confluent/streaming/util/TimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.util; +package org.apache.kafka.stream.util; /** * TimestampTracker is a helper class for a sliding window implementation. diff --git a/src/main/java/io/confluent/streaming/util/Util.java b/stream/src/main/java/org/apache/kafka/stream/util/Util.java similarity index 97% rename from src/main/java/io/confluent/streaming/util/Util.java rename to stream/src/main/java/org/apache/kafka/stream/util/Util.java index 071edc8de2d9a..e4a5200a8f14f 100644 --- a/src/main/java/io/confluent/streaming/util/Util.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/Util.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.util; +package org.apache.kafka.stream.util; import java.io.File; import java.lang.reflect.Field; diff --git a/src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java b/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java similarity index 98% rename from src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java rename to stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java index ab54b0728e7fc..65777c31e0375 100644 --- a/src/test/java/io/confluent/streaming/util/FilteredIteratorTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.util; +package org.apache.kafka.stream; import static org.junit.Assert.assertEquals; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java similarity index 97% rename from src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java rename to stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java index a29241804aebb..f5a553bd38847 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java @@ -1,9 +1,9 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; -import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; +import org.apache.kafka.stream.KStream; import org.junit.Test; import java.lang.reflect.Array; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java similarity index 97% rename from src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java rename to stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java index 637c912b8074a..710431da951d3 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java @@ -1,6 +1,5 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; -import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java similarity index 96% rename from src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java rename to stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java index 022cf36060c5e..7749ad5509897 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java @@ -1,6 +1,5 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; -import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java similarity index 96% rename from src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java rename to stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java index 31bac0e5986cc..c4e4ccc4e961c 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java @@ -1,6 +1,5 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; -import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java similarity index 99% rename from src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java rename to stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java index a99ccad4633c8..c753d24071813 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java @@ -1,11 +1,11 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; -import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; import io.confluent.streaming.testutil.UnlimitedWindow; import io.confluent.streaming.util.Util; +import org.apache.kafka.stream.KStream; import org.junit.Test; import java.util.Collections; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java similarity index 96% rename from src/test/java/io/confluent/streaming/internal/KStreamMapTest.java rename to stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java index f931f9fe04c81..e6c5f514bad92 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java @@ -1,6 +1,5 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; -import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java similarity index 98% rename from src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java rename to stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java index 4588eda90065e..40ece49742ef7 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; import io.confluent.streaming.KStreamContext; <<<<<<< HEAD diff --git a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java similarity index 95% rename from src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java rename to stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java index f1b06ef8424b7..8aea430b88a07 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamSourceTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java @@ -1,9 +1,7 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; import static org.junit.Assert.assertEquals; -import io.confluent.streaming.*; - import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamTransformTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java similarity index 97% rename from src/test/java/io/confluent/streaming/internal/KStreamTransformTest.java rename to stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java index ff6ed8123627e..4b48beed820f7 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamTransformTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java @@ -1,8 +1,7 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamTopology; -import io.confluent.streaming.Predicate; import io.confluent.streaming.Transformer; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; diff --git a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java similarity index 96% rename from src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java rename to stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java index 55d0e63f4737f..9062f7d05fc60 100644 --- a/src/test/java/io/confluent/streaming/internal/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java @@ -1,6 +1,5 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; -import io.confluent.streaming.*; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.UnlimitedWindow; diff --git a/src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java similarity index 98% rename from src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java rename to stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java index 8476092a5a49c..996c7bdd7fe7d 100644 --- a/src/test/java/io/confluent/streaming/util/MinTimestampTrackerTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.util; +package org.apache.kafka.stream; import static org.junit.Assert.assertEquals; diff --git a/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java b/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java similarity index 98% rename from src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java rename to stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java index a2ce6c467c69a..3730a4700e335 100644 --- a/src/test/java/io/confluent/streaming/util/ParallelExecutorTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.util; +package org.apache.kafka.stream; import org.junit.Test; diff --git a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java similarity index 98% rename from src/test/java/io/confluent/streaming/internal/StreamGroupTest.java rename to stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java index 873f8f0f18879..5fc848a75e556 100644 --- a/src/test/java/io/confluent/streaming/internal/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java @@ -1,6 +1,6 @@ -package io.confluent.streaming.internal; +package org.apache.kafka.stream; -import io.confluent.streaming.TimestampExtractor; +import org.apache.kafka.stream.TimestampExtractor; import io.confluent.streaming.testutil.MockIngestor; import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; diff --git a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java similarity index 97% rename from src/test/java/io/confluent/streaming/testutil/MockIngestor.java rename to stream/src/test/java/org/apache/kafka/test/MockIngestor.java index ee053c167c74a..1ea8a355fd49c 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockIngestor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.testutil; +package org.apache.kafka.test; import io.confluent.streaming.internal.Ingestor; import io.confluent.streaming.internal.StreamGroup; diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java similarity index 99% rename from src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java rename to stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java index c43eff05dcbdc..1835f1bb8f859 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockKStreamContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.testutil; +package org.apache.kafka.test; import io.confluent.streaming.KStreamContext; diff --git a/src/test/java/io/confluent/streaming/testutil/MockKStreamTopology.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java similarity index 82% rename from src/test/java/io/confluent/streaming/testutil/MockKStreamTopology.java rename to stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java index e0bd3f5e951a9..8272fdbfb6cef 100644 --- a/src/test/java/io/confluent/streaming/testutil/MockKStreamTopology.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.testutil; +package org.apache.kafka.test; import io.confluent.streaming.KStreamTopology; diff --git a/src/test/java/io/confluent/streaming/testutil/TestProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java similarity index 83% rename from src/test/java/io/confluent/streaming/testutil/TestProcessor.java rename to stream/src/test/java/org/apache/kafka/test/MockProcessor.java index 324407d0123ab..51703266c23ea 100644 --- a/src/test/java/io/confluent/streaming/testutil/TestProcessor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -1,11 +1,11 @@ -package io.confluent.streaming.testutil; +package org.apache.kafka.test; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; import java.util.ArrayList; -public class TestProcessor implements Processor { +public class MockProcessor implements Processor { public final ArrayList processed = new ArrayList<>(); public final ArrayList punctuated = new ArrayList<>(); diff --git a/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java similarity index 97% rename from src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java rename to stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java index 80a538cbc2b2a..f2da677ba651d 100644 --- a/src/test/java/io/confluent/streaming/testutil/UnlimitedWindow.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java @@ -1,4 +1,4 @@ -package io.confluent.streaming.testutil; +package org.apache.kafka.test; import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KeyValue; From dd15e13e73f8e66520f75dec11aed918b1704d6f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 4 Aug 2015 18:31:32 -0700 Subject: [PATCH 130/275] add rocksdb dependency --- build.gradle | 49 ++++++++ kafka-work.iws | 115 ++++++++++-------- pom.xml | 70 ----------- settings.gradle | 2 +- .../apache/kafka/stream/RecordCollector.java | 16 --- .../org/apache/kafka/stream/StateStore.java | 52 -------- .../stream/kv/InMemoryKeyValueStore.java | 12 +- .../kafka/stream/kv/KeyValueIterator.java | 3 - .../apache/kafka/stream/kv/KeyValueStore.java | 2 +- .../apache/kafka/stream/kv/RestoreFunc.java | 11 -- .../kafka/stream/kv/RocksDBKeyValueStore.java | 12 +- .../kv/internal/MeteredKeyValueStore.java | 15 +-- .../kafka/stream/KStreamBranchTest.java | 2 +- .../kafka/stream/KStreamFilterTest.java | 1 + .../kafka/stream/KStreamFlatMapTest.java | 1 + .../stream/KStreamFlatMapValuesTest.java | 1 + .../apache/kafka/stream/KStreamJoinTest.java | 2 +- .../apache/kafka/stream/KStreamMapTest.java | 1 + .../kafka/stream/KStreamMapValuesTest.java | 1 + .../kafka/stream/KStreamSourceTest.java | 1 + .../kafka/stream/KStreamTransformTest.java | 1 + .../kafka/stream/KStreamWindowedTest.java | 1 + 22 files changed, 145 insertions(+), 226 deletions(-) delete mode 100644 pom.xml delete mode 100644 stream/src/main/java/org/apache/kafka/stream/RecordCollector.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/StateStore.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/kv/RestoreFunc.java diff --git a/build.gradle b/build.gradle index fecc3eb3b6918..cf75ffe161c27 100644 --- a/build.gradle +++ b/build.gradle @@ -524,6 +524,55 @@ project(':tools') { test.dependsOn('checkstyleMain', 'checkstyleTest') } +project(':stream') { + apply plugin: 'checkstyle' + archivesBaseName = "kafka-stream" + + dependencies { + compile project(':clients') + compile "$slf4jlog4j" + compile 'org.rocksdb:rocksdbjni:3.10.1' + + testCompile 'junit:junit:4.6' + 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/stream/*" + } + + 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' + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + project(':log4j-appender') { apply plugin: 'checkstyle' archivesBaseName = "kafka-log4j-appender" diff --git a/kafka-work.iws b/kafka-work.iws index e5fce9c4d023d..c09380d8c3020 100644 --- a/kafka-work.iws +++ b/kafka-work.iws @@ -2,43 +2,28 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + @@ -150,11 +135,31 @@ - + - - + + + + + + + + + + + + + + + + + + + + + + @@ -231,7 +236,6 @@ @@ -907,14 +912,6 @@ - - - - - - - - @@ -998,8 +995,24 @@ - - + + + + + + + + + + + + + + + + + + diff --git a/pom.xml b/pom.xml deleted file mode 100644 index d4fb05b23e229..0000000000000 --- a/pom.xml +++ /dev/null @@ -1,70 +0,0 @@ - - 4.0.0 - iio.confluent.streaming - streaming - jar - 1.0-SNAPSHOT - streaming - - - Confluent, Inc. - http://confluent.io - - http://confluent.io - - - 0.8.3-SNAPSHOT - 2.10 - UTF-8 - - - - - org.apache.kafka - kafka_${kafka.scala.version} - ${kafka.version} - - - org.apache.kafka - kafka-clients - ${kafka.version} - - - junit - junit - 4.8.1 - test - - - org.slf4j - slf4j-log4j12 - 1.7.6 - - - org.rocksdb - rocksdbjni - 3.10.1 - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.3 - - 1.7 - 1.7 - -Xlint:all - - - - org.apache.maven.plugins - maven-javadoc-plugin - 2.10.3 - - - - diff --git a/settings.gradle b/settings.gradle index 9c7fea5dce257..3ff9144eca2fe 100644 --- a/settings.gradle +++ b/settings.gradle @@ -14,5 +14,5 @@ // limitations under the License. apply from: file('scala.gradle') -include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'log4j-appender', +include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'stream', 'log4j-appender', 'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file' diff --git a/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java b/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java deleted file mode 100644 index 8d6d79b72754f..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java +++ /dev/null @@ -1,16 +0,0 @@ -package org.apache.kafka.stream; - -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.Serializer; - -/** - * Created by yasuhiro on 6/19/15. - */ -public interface RecordCollector { - - void send(ProducerRecord record); - - void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer); - - void flush(); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/StateStore.java b/stream/src/main/java/org/apache/kafka/stream/StateStore.java deleted file mode 100644 index 98abf4f7f6657..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/StateStore.java +++ /dev/null @@ -1,52 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -/** - * A storage engine for managing state maintained by a stream processor. - * - *

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

- */ -public interface StateStore { - - /** - * The name of this store. - * @return the storage name - */ - String name(); - - /** - * Flush any cached data - */ - void flush(); - - /** - * Close the storage engine - */ - void close(); - - /** - * If the storage is persistent - */ - boolean persistent(); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java index 81be2848535fe..c395d1d871ba6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java @@ -1,9 +1,9 @@ package org.apache.kafka.stream.kv; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.kv.internals.MeteredKeyValueStore; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.stream.kv.internal.MeteredKeyValueStore; import java.util.Iterator; import java.util.List; @@ -19,11 +19,11 @@ */ public class InMemoryKeyValueStore extends MeteredKeyValueStore { - public InMemoryKeyValueStore(String name, KStreamContext context) { + public InMemoryKeyValueStore(String name, ProcessorContext context) { this(name, context, new SystemTime()); } - public InMemoryKeyValueStore(String name, KStreamContext context, Time time) { + public InMemoryKeyValueStore(String name, ProcessorContext context, Time time) { super(name, new MemoryStore(name, context), context, "kafka-streams", time); } @@ -31,10 +31,10 @@ private static class MemoryStore implements KeyValueStore { private final String name; private final NavigableMap map; - private final KStreamContext context; + private final ProcessorContext context; @SuppressWarnings("unchecked") - public MemoryStore(String name, KStreamContext context) { + public MemoryStore(String name, ProcessorContext context) { super(); this.name = name; this.map = new TreeMap<>(); diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueIterator.java b/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueIterator.java index c8e78329a2e69..8473096777556 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueIterator.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueIterator.java @@ -19,9 +19,6 @@ package org.apache.kafka.stream.kv; - -import org.apache.kafka.stream.kv.Entry; - import java.io.Closeable; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java index ecab79779202a..61ca072606016 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java @@ -19,7 +19,7 @@ package org.apache.kafka.stream.kv; -import io.confluent.streaming.StateStore; +import org.apache.kafka.clients.processor.StateStore; import java.util.List; diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/RestoreFunc.java b/stream/src/main/java/org/apache/kafka/stream/kv/RestoreFunc.java deleted file mode 100644 index 1c1cbe852d4dc..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/kv/RestoreFunc.java +++ /dev/null @@ -1,11 +0,0 @@ -package org.apache.kafka.stream.kv; - -/** - * Created by guozhang on 7/27/15. - */ - -// TODO: this should be removed once we move to Java 8 -public interface RestoreFunc { - - void apply(byte[] key, byte[] value); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java index c7d36aa2a4a8e..127e143e0109a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java @@ -1,11 +1,11 @@ package org.apache.kafka.stream.kv; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.kv.internals.MeteredKeyValueStore; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.stream.kv.internal.MeteredKeyValueStore; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.CompactionStyle; import org.rocksdb.CompressionType; @@ -26,11 +26,11 @@ */ public class RocksDBKeyValueStore extends MeteredKeyValueStore { - public RocksDBKeyValueStore(String name, KStreamContext context) { + public RocksDBKeyValueStore(String name, ProcessorContext context) { this(name, context, new SystemTime()); } - public RocksDBKeyValueStore(String name, KStreamContext context, Time time) { + public RocksDBKeyValueStore(String name, ProcessorContext context, Time time) { super(name, new RocksDBStore(name, context), context, "kafka-streams", time); } @@ -50,7 +50,7 @@ private static class RocksDBStore implements KeyValueStore { private final String topic; private final int partition; - private final KStreamContext context; + private final ProcessorContext context; private final Options options; private final WriteOptions wOptions; @@ -62,7 +62,7 @@ private static class RocksDBStore implements KeyValueStore { private RocksDB db; @SuppressWarnings("unchecked") - public RocksDBStore(String name, KStreamContext context) { + public RocksDBStore(String name, ProcessorContext context) { this.topic = name; this.partition = context.id(); this.context = context; diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java index 180eaaaa9f07f..b60e4b997a8c2 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java @@ -1,10 +1,8 @@ package org.apache.kafka.stream.kv.internal; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.RecordCollector; -import io.confluent.streaming.kv.Entry; -import io.confluent.streaming.kv.KeyValueIterator; -import io.confluent.streaming.kv.KeyValueStore; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.RecordCollector; +import org.apache.kafka.clients.processor.RestoreFunc; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.MeasurableStat; @@ -17,6 +15,9 @@ 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.stream.kv.Entry; +import org.apache.kafka.stream.kv.KeyValueIterator; +import org.apache.kafka.stream.kv.KeyValueStore; import java.util.HashSet; import java.util.List; @@ -42,10 +43,10 @@ public class MeteredKeyValueStore implements KeyValueStore { private final int partition; private final Set dirty; private final int maxDirty; - private final KStreamContext context; + private final ProcessorContext context; // always wrap the logged store with the metered store - public MeteredKeyValueStore(final String name, final KeyValueStore inner, KStreamContext context, String group, Time time) { + public MeteredKeyValueStore(final String name, final KeyValueStore inner, ProcessorContext context, String group, Time time) { this.inner = inner; this.time = time; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java index f5a553bd38847..dd62600cc5a08 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java @@ -3,7 +3,7 @@ import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; -import org.apache.kafka.stream.KStream; +import org.apache.kafka.clients.processor.KStreamContext; import org.junit.Test; import java.lang.reflect.Array; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java index 710431da951d3..a240baf39e521 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java @@ -3,6 +3,7 @@ import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; +import org.apache.kafka.clients.processor.KStreamContext; import org.junit.Test; import java.util.Collections; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java index 7749ad5509897..b1a09267beb9d 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java @@ -3,6 +3,7 @@ import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; +import org.apache.kafka.clients.processor.KStreamContext; import org.junit.Test; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java index c4e4ccc4e961c..5ef445d3fac68 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java @@ -3,6 +3,7 @@ import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; +import org.apache.kafka.clients.processor.KStreamContext; import org.junit.Test; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java index c753d24071813..5f307967f3a6c 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java @@ -5,7 +5,7 @@ import io.confluent.streaming.testutil.TestProcessor; import io.confluent.streaming.testutil.UnlimitedWindow; import io.confluent.streaming.util.Util; -import org.apache.kafka.stream.KStream; +import org.apache.kafka.clients.processor.KStreamContext; import org.junit.Test; import java.util.Collections; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java index e6c5f514bad92..1797a9eb93f92 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java @@ -3,6 +3,7 @@ import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; +import org.apache.kafka.clients.processor.KStreamContext; import org.junit.Test; import java.util.Collections; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java index 40ece49742ef7..9ac9593b17b3a 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java @@ -14,6 +14,7 @@ import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; +import org.apache.kafka.clients.processor.KStreamContext; import org.junit.Test; import java.util.Collections; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java index 8aea430b88a07..4fa357e3336a3 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java @@ -5,6 +5,7 @@ import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; +import org.apache.kafka.clients.processor.KStreamContext; import org.junit.Test; import java.util.Collections; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java index 4b48beed820f7..f9406f0496be8 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java @@ -6,6 +6,7 @@ import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; +import org.apache.kafka.clients.processor.KStreamContext; import org.junit.Test; import java.util.Collections; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java index 9062f7d05fc60..d03701a7301aa 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java @@ -3,6 +3,7 @@ import io.confluent.streaming.testutil.MockKStreamContext; import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.UnlimitedWindow; +import org.apache.kafka.clients.processor.KStreamContext; import org.junit.Test; import java.util.Collections; From 91caf0da637b504677d6327379e45866474d1c7c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 4 Aug 2015 18:55:20 -0700 Subject: [PATCH 131/275] removing io.confluent imports: wip --- .../kafka/clients/processor/Processor.java | 4 ++ .../org/apache/kafka/common/utils/Utils.java | 51 ++++++++++++++++ .../java/org/apache/kafka/stream/Chooser.java | 2 + .../java/org/apache/kafka/stream/KStream.java | 13 ++-- .../apache/kafka/stream/KafkaStreaming.java | 6 ++ .../kafka/stream/examples/MapKStreamJob.java | 9 +++ .../kafka/stream/internal/IngestorImpl.java | 11 +++- .../stream/internal/PartitioningInfo.java | 4 +- .../kafka/stream/internal/ProcessorNode.java | 11 ++-- .../internal/ProcessorStateManager.java | 7 +-- .../stream/internal/PunctuationSchedule.java | 7 ++- .../kafka/stream/internal/Receiver.java | 5 +- .../stream/internal/RecordCollectorImpl.java | 4 +- .../kafka/stream/internal/RecordQueue.java | 2 +- .../kafka/stream/internal/StampedRecord.java | 2 +- .../kafka/stream/internal/StreamGroup.java | 17 +++--- .../stream/internal/TimeBasedChooser.java | 2 + .../stream/topology/KStreamTopology.java | 2 +- .../stream/topology/internal/KStreamImpl.java | 17 ++++-- .../topology/internal/KStreamMetadata.java | 2 +- .../topology/internal/KStreamSource.java | 11 +++- .../topology/internal/KStreamThread.java | 23 ++++--- .../org/apache/kafka/stream/util/Util.java | 61 ------------------- temp | 16 ++--- temp2 | 16 ++--- 25 files changed, 177 insertions(+), 128 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/util/Util.java diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java b/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java index ac98c38b8291c..2dc0bb76d98a8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java @@ -5,6 +5,7 @@ */ public interface Processor { +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD void init(KStreamContext context); @@ -36,6 +37,9 @@ public interface ProcessorContext { ======= void init(KStreamContext context); >>>>>>> removed ProcessorContext +======= + void init(ProcessorContext context); +>>>>>>> removing io.confluent imports: wip void process(K key, V value); diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index c58b74144ba01..c02458fd84e84 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -25,6 +25,7 @@ import java.nio.MappedByteBuffer; import java.util.Arrays; import java.util.Collection; +import java.util.HashSet; import java.util.Iterator; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -509,6 +510,7 @@ public static String readFileAsString(String path) throws IOException { } /** +<<<<<<< HEAD * Check if the given ByteBuffer capacity * @param existingBuffer ByteBuffer capacity to check * @param newLength new length for the ByteBuffer. @@ -524,4 +526,53 @@ public static ByteBuffer ensureCapacity(ByteBuffer existingBuffer, int newLength return existingBuffer; } +======= + * Creates a set + * @param elems the elements + * @param the type of element + * @return Set + */ + public static HashSet mkSet(T... elems) { + HashSet set = new HashSet(); + for (T e : elems) set.add(e); + return set; + } + + /** + * Makes a string of a comma separated list of collection elements + * @param collection the collection + * @param the type of collection elements + * @return String + */ + public static String mkString(Collection collection) { + StringBuilder sb = new StringBuilder(); + int count = collection.size(); + for (E elem : collection) { + sb.append(elem.toString()); + count--; + if (count > 0) sb.append(", "); + } + return sb.toString(); + } + + /** + * 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 rm(File file) { + if (file == null) { + return; + } else if (file.isDirectory()) { + File[] files = file.listFiles(); + if (files != null) { + for (File f : files) + rm(f); + } + file.delete(); + } else { + file.delete(); + } + } +>>>>>>> removing io.confluent imports: wip } diff --git a/stream/src/main/java/org/apache/kafka/stream/Chooser.java b/stream/src/main/java/org/apache/kafka/stream/Chooser.java index 4d5d0575d2f46..90953a08674f0 100644 --- a/stream/src/main/java/org/apache/kafka/stream/Chooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/Chooser.java @@ -1,5 +1,7 @@ package org.apache.kafka.stream; +import org.apache.kafka.stream.internal.RecordQueue; + /** * Created by yasuhiro on 6/25/15. */ diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java index 2118d38b84855..be69bc5567d77 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java @@ -1,13 +1,14 @@ package org.apache.kafka.stream; -import io.confluent.streaming.KStreamWindowed; -import io.confluent.streaming.KeyValueMapper; -import io.confluent.streaming.Predicate; -import io.confluent.streaming.Transformer; -import io.confluent.streaming.ValueMapper; -import io.confluent.streaming.Window; +import org.apache.kafka.clients.processor.Processor; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.stream.topology.KStreamWindowed; +import org.apache.kafka.stream.topology.KeyValueMapper; +import org.apache.kafka.stream.topology.Predicate; +import org.apache.kafka.stream.topology.Transformer; +import org.apache.kafka.stream.topology.ValueMapper; +import org.apache.kafka.stream.topology.Window; /** * KStream is an abstraction of a stream of key-value pairs. diff --git a/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java index 0832d7ba977bd..bbe5a6fc01d86 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java +++ b/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java @@ -17,6 +17,7 @@ package org.apache.kafka.stream; +<<<<<<< HEAD import io.confluent.streaming.internal.KStreamThread; import io.confluent.streaming.internal.ProcessorConfig; <<<<<<< HEAD @@ -54,7 +55,12 @@ >>>>>>> new api model ======= >>>>>>> wip +======= +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.stream.internal.ProcessorConfig; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.internal.KStreamThread; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java index 1d37c141cb789..2cf919c1bbb12 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java @@ -6,6 +6,7 @@ <<<<<<< HEAD ======= import org.apache.kafka.stream.KStream; +<<<<<<< HEAD >>>>>>> first re-org:stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java import io.confluent.streaming.KStreamTopology; ======= @@ -20,7 +21,15 @@ import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.Predicate; import io.confluent.streaming.StreamingConfig; +======= +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.stream.KafkaStreaming; +import org.apache.kafka.stream.StreamingConfig; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KeyValue; +import org.apache.kafka.stream.topology.KeyValueMapper; +import org.apache.kafka.stream.topology.Predicate; import java.util.Properties; diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java index 4c28753ef254a..47bb3e298cdd5 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java @@ -7,7 +7,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + public class IngestorImpl implements Ingestor { @@ -96,7 +101,7 @@ public void clear() { streamSynchronizers.clear(); } - boolean commitNeeded(Map offsets) { + public boolean commitNeeded(Map offsets) { for (TopicPartition tp : offsets.keySet()) { if (consumer.committed(tp) != offsets.get(tp)) { return true; @@ -105,7 +110,7 @@ boolean commitNeeded(Map offsets) { return false; } - void close() { + public void close() { consumer.close(); clear(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java b/stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java index 82025f0353a66..40621295582c2 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java @@ -3,11 +3,11 @@ /** * Created by yasuhiro on 6/19/15. */ -class PartitioningInfo { +public class PartitioningInfo { public final int numPartitions; - PartitioningInfo(int numPartitions) { + public PartitioningInfo(int numPartitions) { this.numPartitions = numPartitions; } } diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java index 9441ef8edc244..87d9774e28b02 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java @@ -1,7 +1,8 @@ package org.apache.kafka.stream.internal; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.Processor; +import org.apache.kafka.clients.processor.Processor; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; /** * Created by yasuhiro on 7/31/15. @@ -9,14 +10,14 @@ public class ProcessorNode implements Receiver { private final Processor processor; - private KStreamContext context; + private ProcessorContext context; - ProcessorNode(Processor processor) { + public ProcessorNode(Processor processor) { this.processor = processor; } @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { + public void bind(ProcessorContext context, KStreamMetadata metadata) { if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); this.context = context; diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java index 408bbd807d737..b226779204da2 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java @@ -16,14 +16,13 @@ */ package org.apache.kafka.stream.internal; -import io.confluent.streaming.StateStore; -import io.confluent.streaming.kv.internals.RestoreFunc; -import io.confluent.streaming.util.OffsetCheckpoint; -import kafka.cluster.Partition; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.processor.RestoreFunc; +import org.apache.kafka.clients.processor.StateStore; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.stream.util.OffsetCheckpoint; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java index ae79fe88dd75e..7f33571c49c93 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java @@ -1,9 +1,10 @@ package org.apache.kafka.stream.internal; -import io.confluent.streaming.Processor; -import io.confluent.streaming.util.Stamped; -public class PunctuationSchedule extends Stamped>{ +import org.apache.kafka.clients.processor.Processor; +import org.apache.kafka.stream.util.Stamped; + +public class PunctuationSchedule extends Stamped> { final long interval; diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java b/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java index 95b23b320206f..7779901395543 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java @@ -1,13 +1,14 @@ package org.apache.kafka.stream.internal; -import io.confluent.streaming.KStreamContext; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; /** * Created by yasuhiro on 6/17/15. */ public interface Receiver { - void bind(KStreamContext context, KStreamMetadata metadata); + void bind(ProcessorContext context, KStreamMetadata metadata); <<<<<<< HEAD <<<<<<< HEAD diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java index b39ba0adbc4b6..fbfb92ee0e5ee 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java @@ -1,6 +1,6 @@ package org.apache.kafka.stream.internal; -import io.confluent.streaming.RecordCollector; +import org.apache.kafka.clients.processor.RecordCollector; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -63,7 +63,7 @@ public void flush() { /** * Closes this RecordCollector */ - void close() { + public void close() { producer.close(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java b/stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java index 72f9b7487b80a..787c16fec1727 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java @@ -1,9 +1,9 @@ package org.apache.kafka.stream.internal; -import io.confluent.streaming.util.TimestampTracker; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.util.TimestampTracker; import java.util.ArrayDeque; diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java b/stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java index 55ce51e721995..9dd77b6e834e3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java @@ -1,7 +1,7 @@ package org.apache.kafka.stream.internal; -import io.confluent.streaming.util.Stamped; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.stream.util.Stamped; // TODO: making this class exposed to user in the lower-level Processor diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java b/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java index bbaee41a7be53..196b99d9f70e2 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java @@ -1,14 +1,15 @@ package org.apache.kafka.stream.internal; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.Processor; -import io.confluent.streaming.TimestampExtractor; -import io.confluent.streaming.util.MinTimestampTracker; -import io.confluent.streaming.util.ParallelExecutor; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.processor.Processor; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.Chooser; +import org.apache.kafka.stream.TimestampExtractor; import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.util.MinTimestampTracker; +import org.apache.kafka.stream.util.ParallelExecutor; import java.util.ArrayDeque; import java.util.Collections; @@ -34,7 +35,7 @@ public class StreamSynchronizer implements SyncGroup { public class StreamGroup implements ParallelExecutor.Task { >>>>>>> remove SyncGroup from user facing APIs:src/main/java/io/confluent/streaming/internal/StreamGroup.java - private final KStreamContext context; + private final ProcessorContext context; private final Ingestor ingestor; private final Chooser chooser; private final TimestampExtractor timestampExtractor; @@ -60,7 +61,7 @@ public class StreamGroup implements ParallelExecutor.Task { * @param timestampExtractor the instance of {@link TimestampExtractor} * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic */ - StreamGroup(KStreamContext context, + StreamGroup(ProcessorContext context, Ingestor ingestor, Chooser chooser, TimestampExtractor timestampExtractor, @@ -295,7 +296,7 @@ public void close() { } protected RecordQueue createRecordQueue(TopicPartition partition, KStreamSource stream) { - return new RecordQueue(partition, stream, new MinTimestampTracker>()); + return new RecordQueue(partition, stream, new MinTimestampTracker<>()); } private static class NewRecords { diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java index d0973be5498e4..b370c9e5f0f9e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java @@ -1,5 +1,7 @@ package org.apache.kafka.stream.internal; +import org.apache.kafka.stream.Chooser; + import java.util.Comparator; import java.util.PriorityQueue; diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java index 26e391d4603c9..c60404a88d89a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java @@ -1,8 +1,8 @@ package org.apache.kafka.stream.topology; -import io.confluent.streaming.internal.KStreamSource; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.KStream; +import org.apache.kafka.stream.topology.internal.KStreamSource; import java.util.ArrayList; import java.util.Collection; diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java index 4123ebcedea86..4beb11068fbbf 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java @@ -1,6 +1,9 @@ package org.apache.kafka.stream.topology.internal; +import org.apache.kafka.clients.processor.Processor; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.KStream; +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD <<<<<<< HEAD @@ -18,8 +21,14 @@ import io.confluent.streaming.Transformer; import io.confluent.streaming.ValueMapper; import io.confluent.streaming.Window; +======= +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.stream.internal.ProcessorNode; +import org.apache.kafka.stream.internal.Receiver; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KStreamWindowed; import org.apache.kafka.stream.topology.KeyValueMapper; import org.apache.kafka.stream.topology.Predicate; import org.apache.kafka.stream.topology.Transformer; @@ -38,7 +47,7 @@ abstract class KStreamImpl implements KStream, Receiver { <<<<<<< HEAD <<<<<<< HEAD protected KStreamTopology topology; - protected KStreamContext context; + protected ProcessorContext context; protected KStreamMetadata metadata; protected KStreamImpl(KStreamTopology topology) { @@ -46,7 +55,7 @@ protected KStreamImpl(KStreamTopology topology) { } @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { + public void bind(ProcessorContext context, KStreamMetadata metadata) { if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); this.context = context; this.metadata = metadata; @@ -231,10 +240,10 @@ public void sendTo(String topic, Serializer keySerializer, Serializer valS @SuppressWarnings("unchecked") private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { return new Processor() { - private KStreamContext context; + private ProcessorContext context; @Override - public void init(KStreamContext context) { + public void init(ProcessorContext context) { this.context = context; } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java index 97fc7b0b18840..31f59a97b59d8 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java @@ -21,7 +21,7 @@ public static KStreamMetadata unjoinable() { public StreamGroup streamGroup; public final Map topicPartitionInfos; - KStreamMetadata(Map topicPartitionInfos) { + public KStreamMetadata(Map topicPartitionInfos) { this.topicPartitionInfos = topicPartitionInfos; } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java index 5a38969635adb..7cbf96aed8763 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamContext; @@ -13,7 +14,11 @@ >>>>>>> new api model ======= >>>>>>> wip +======= +import org.apache.kafka.clients.processor.ProcessorContext; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.topology.KStreamTopology; import java.util.Arrays; import java.util.HashSet; @@ -29,8 +34,12 @@ public class KStreamSource extends KStreamImpl { <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD String[] topics; +======= + public String[] topics; +>>>>>>> removing io.confluent imports: wip public KStreamSource(String[] topics, KStreamTopology topology) { this(topics, null, null, topology); @@ -78,7 +87,7 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { >>>>>>> wip @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { + public void bind(ProcessorContext context, KStreamMetadata metadata) { if (keyDeserializer == null) keyDeserializer = (Deserializer) context.keyDeserializer(); if (valueDeserializer == null) valueDeserializer = (Deserializer) context.valueDeserializer(); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java index 1ae2cb6724715..38a5af386b26b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java @@ -17,6 +17,7 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamTopology; @@ -30,6 +31,8 @@ import io.confluent.streaming.StreamingConfig; import io.confluent.streaming.util.ParallelExecutor; import io.confluent.streaming.util.Util; +======= +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -49,9 +52,15 @@ import org.apache.kafka.common.serialization.Serializer; 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.stream.StreamingConfig; +import org.apache.kafka.stream.internal.IngestorImpl; +import org.apache.kafka.stream.internal.ProcessorContextImpl; import org.apache.kafka.stream.internal.ProcessorConfig; import org.apache.kafka.stream.internal.RecordCollectorImpl; import org.apache.kafka.stream.internal.StreamGroup; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.util.ParallelExecutor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,7 +79,7 @@ public class KStreamThread extends Thread { private final KStreamTopology topology; private final ArrayList streamGroups = new ArrayList<>(); private final ParallelExecutor parallelExecutor; - private final Map kstreamContexts = new HashMap<>(); + private final Map kstreamContexts = new HashMap<>(); private final IngestorImpl ingestor; private final RecordCollectorImpl collector; private final StreamingConfig streamingConfig; @@ -196,7 +205,7 @@ private void maybeCommit() { private void commitAll(long now) { Map commit = new HashMap<>(); - for (KStreamContextImpl context : kstreamContexts.values()) { + for (ProcessorContextImpl context : kstreamContexts.values()) { context.flush(); commit.putAll(context.consumedOffsets()); } @@ -222,11 +231,11 @@ private void maybeCleanState() { Integer id = Integer.parseInt(dir.getName()); if(!kstreamContexts.keySet().contains(id)) { log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); - Util.rm(dir); + Utils.rm(dir); } } catch(NumberFormatException e) { log.warn("Deleting unknown directory in state directory {}.", dir.getAbsolutePath()); - Util.rm(dir); + Utils.rm(dir); } } } @@ -256,10 +265,10 @@ private void addPartitions(Collection assignment) { ======= for (TopicPartition partition : partitions) { final Integer id = partition.partition(); // TODO: switch this to the group id - KStreamContextImpl context = kstreamContexts.get(id); + ProcessorContextImpl context = kstreamContexts.get(id); if (context == null) { try { - context = new KStreamContextImpl(id, ingestor, collector, streamingConfig, config, metrics); + context = new ProcessorContextImpl(id, ingestor, collector, streamingConfig, config, metrics); context.init(topology.sourceStreams()); >>>>>>> new api model @@ -277,7 +286,7 @@ private void addPartitions(Collection assignment) { } private void removePartitions() { - for (KStreamContextImpl context : kstreamContexts.values()) { + for (ProcessorContextImpl context : kstreamContexts.values()) { log.info("Removing task context {}", context.id()); try { context.close(); diff --git a/stream/src/main/java/org/apache/kafka/stream/util/Util.java b/stream/src/main/java/org/apache/kafka/stream/util/Util.java deleted file mode 100644 index e4a5200a8f14f..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/util/Util.java +++ /dev/null @@ -1,61 +0,0 @@ -package org.apache.kafka.stream.util; - -import java.io.File; -import java.lang.reflect.Field; -import java.util.Collection; -import java.util.HashSet; - -/** - * Created by yasuhiro on 6/22/15. - */ -public class Util { - - /** - * Creates a set - * @param elems the elements - * @param the type of element - * @return Set - */ - public static HashSet mkSet(T... elems) { - HashSet set = new HashSet(); - for (T e : elems) set.add(e); - return set; - } - - /** - * Makes a string of a comma separated list of collection elements - * @param collection the collection - * @param the type of collection elements - * @return String - */ - public static String mkString(Collection collection) { - StringBuilder sb = new StringBuilder(); - int count = collection.size(); - for (E elem : collection) { - sb.append(elem.toString()); - count--; - if (count > 0) sb.append(", "); - } - return sb.toString(); - } - - /** - * 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 rm(File file) { - if (file == null) { - return; - } else if (file.isDirectory()) { - File[] files = file.listFiles(); - if (files != null) { - for (File f : files) - rm(f); - } - file.delete(); - } else { - file.delete(); - } - } -} diff --git a/temp b/temp index e30005a96ec13..af827a1f872f5 100644 --- a/temp +++ b/temp @@ -1,8 +1,8 @@ -src/main/java/io/confluent/streaming/internal/KStreamBranch.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamFilter.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamJoin.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamMap.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamMapValues.java: needs merge -src/main/java/io/confluent/streaming/internal/KStreamSource.java: needs merge +clients/src/main/java/org/apache/kafka/clients/processor/Processor.java: needs merge +clients/src/main/java/org/apache/kafka/common/utils/Utils.java: needs merge +stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java: needs merge +stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java: needs merge +stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java: needs merge diff --git a/temp2 b/temp2 index a5478493edd05..f63967a305e3f 100644 --- a/temp2 +++ b/temp2 @@ -1,8 +1,8 @@ -src/main/java/io/confluent/streaming/internal/KStreamBranch.java -src/main/java/io/confluent/streaming/internal/KStreamFilter.java -src/main/java/io/confluent/streaming/internal/KStreamFlatMap.java -src/main/java/io/confluent/streaming/internal/KStreamFlatMapValues.java -src/main/java/io/confluent/streaming/internal/KStreamJoin.java -src/main/java/io/confluent/streaming/internal/KStreamMap.java -src/main/java/io/confluent/streaming/internal/KStreamMapValues.java -src/main/java/io/confluent/streaming/internal/KStreamSource.java +clients/src/main/java/org/apache/kafka/clients/processor/Processor.java +clients/src/main/java/org/apache/kafka/common/utils/Utils.java +stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java +stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java +stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java From 336a346282b744771f5a8fb2e803ef3587894ac6 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 4 Aug 2015 18:57:28 -0700 Subject: [PATCH 132/275] removing io.confluent imports: wip --- .../stream/examples/PrintKStreamJob.java | 14 +++++------ .../stream/examples/StatefulKStreamJob.java | 23 ++++++++++--------- .../topology/SingleProcessorTopology.java | 1 + 3 files changed, 20 insertions(+), 18 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java index 74d410a5c4a9d..7cfede884ee53 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java @@ -1,10 +1,10 @@ package org.apache.kafka.stream.examples; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KafkaStreaming; -import io.confluent.streaming.Processor; -import io.confluent.streaming.SingleProcessorTopology; -import io.confluent.streaming.StreamingConfig; +import org.apache.kafka.clients.processor.Processor; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KafkaStreaming; +import org.apache.kafka.stream.StreamingConfig; +import org.apache.kafka.stream.topology.SingleProcessorTopology; import java.util.Properties; @@ -13,10 +13,10 @@ */ public class PrintKStreamJob implements Processor { - private KStreamContext context; + private ProcessorContext context; @Override - public void init(KStreamContext context) { + public void init(ProcessorContext context) { this.context = context; } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java index 503b25e88e61e..bb393a5b80bbd 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java @@ -1,14 +1,15 @@ package org.apache.kafka.stream.examples; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KafkaStreaming; -import io.confluent.streaming.Processor; -import io.confluent.streaming.SingleProcessorTopology; -import io.confluent.streaming.StreamingConfig; -import io.confluent.streaming.kv.Entry; -import io.confluent.streaming.kv.InMemoryKeyValueStore; -import io.confluent.streaming.kv.KeyValueIterator; -import io.confluent.streaming.kv.KeyValueStore; + +import org.apache.kafka.clients.processor.Processor; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KafkaStreaming; +import org.apache.kafka.stream.StreamingConfig; +import org.apache.kafka.stream.kv.Entry; +import org.apache.kafka.stream.kv.InMemoryKeyValueStore; +import org.apache.kafka.stream.kv.KeyValueIterator; +import org.apache.kafka.stream.kv.KeyValueStore; +import org.apache.kafka.stream.topology.SingleProcessorTopology; import java.util.Properties; @@ -18,11 +19,11 @@ public class StatefulKStreamJob implements Processor { - private KStreamContext context; + private ProcessorContext context; private KeyValueStore kvStore; @Override - public void init(KStreamContext context) { + public void init(ProcessorContext context) { this.context = context; this.context.schedule(this, 1000); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java index e7365dcbc8387..f3e254569d06f 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology; +import org.apache.kafka.clients.processor.Processor; import org.apache.kafka.common.utils.Utils; /** From 13a670fe9c422db5106e815ea435b77fd1ffbd9e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 4 Aug 2015 19:07:25 -0700 Subject: [PATCH 133/275] removing io.confluent imports: wip --- .../kafka/stream/internal/ProcessorConfig.java | 2 +- .../kafka/stream/internal/PunctuationQueue.java | 2 -- .../kafka/stream/internal/RoundRobinChooser.java | 2 ++ .../kafka/stream/topology/SlidingWindow.java | 7 ++++--- .../kafka/stream/topology/Transformer.java | 9 +++++---- .../org/apache/kafka/stream/topology/Window.java | 5 ++++- .../stream/topology/internal/KStreamBranch.java | 10 ++++++++++ .../stream/topology/internal/KStreamFilter.java | 4 ++++ .../stream/topology/internal/KStreamFlatMap.java | 8 +++++++- .../topology/internal/KStreamFlatMapValues.java | 4 ++++ .../stream/topology/internal/KStreamJoin.java | 11 +++++++++-- .../stream/topology/internal/KStreamMap.java | 8 +++++++- .../topology/internal/KStreamMapValues.java | 4 ++++ .../topology/internal/KStreamTransform.java | 7 +++---- .../topology/internal/KStreamWindowedImpl.java | 8 +++++++- temp | 16 ++++++++-------- temp2 | 16 ++++++++-------- 17 files changed, 87 insertions(+), 36 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java index 6cb659fecbfe1..b2717a76ec9d8 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java @@ -17,11 +17,11 @@ package org.apache.kafka.stream.internal; -import io.confluent.streaming.StreamingConfig; 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 org.apache.kafka.stream.StreamingConfig; import java.io.File; import java.util.Properties; diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java index f081ace464969..2bf3ebacc77d9 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java @@ -1,7 +1,5 @@ package org.apache.kafka.stream.internal; -import io.confluent.streaming.Processor; - import java.util.PriorityQueue; /** diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java index ec56e254204f1..a702a662391ad 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java @@ -1,5 +1,7 @@ package org.apache.kafka.stream.internal; +import org.apache.kafka.stream.Chooser; + import java.util.ArrayDeque; /** diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java index df17fbcdb5fe0..0d91d623b61f6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java @@ -1,7 +1,8 @@ package org.apache.kafka.stream.topology; -import io.confluent.streaming.util.FilteredIterator; -import io.confluent.streaming.util.Stamped; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.util.FilteredIterator; +import org.apache.kafka.stream.util.Stamped; import java.util.HashMap; import java.util.Iterator; @@ -25,7 +26,7 @@ public SlidingWindow(String name, long duration, int maxCount) { } @Override - public void init(KStreamContext context) { + public void init(ProcessorContext context) { } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java b/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java index 540dd87742dec..48a9d959b6797 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java @@ -1,14 +1,15 @@ package org.apache.kafka.stream.topology; +import org.apache.kafka.clients.processor.Processor; + /** * Created by yasuhiro on 6/17/15. */ -public interface Transformer extends Processor { +public interface Transformer extends Processor { interface Forwarder { - void send(K key, V value, long timestamp); + public void send(K key, V value, long timestamp); } - void forwarder(Forwarder forwarder); - + public void forwarder(Forwarder forwarder); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Window.java b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java index 6e329b5c89608..25941b4a586b3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Window.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java @@ -1,5 +1,8 @@ package org.apache.kafka.stream.topology; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.StateStore; + import java.util.Iterator; /** @@ -7,7 +10,7 @@ */ public interface Window extends StateStore { - void init(KStreamContext context); + void init(ProcessorContext context); Iterator find(K key, long timestamp); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java index 3f2499bb37643..3ed8154b64a3c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD <<<<<<< HEAD @@ -11,6 +12,11 @@ import io.confluent.streaming.KStreamTopology; >>>>>>> wip import io.confluent.streaming.Predicate; +======= +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.internal.Receiver; +import org.apache.kafka.stream.topology.KStreamTopology; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.topology.Predicate; import java.lang.reflect.Array; @@ -54,6 +60,7 @@ class KStreamBranch implements Receiver { } @Override +<<<<<<< HEAD public void bind(KStreamContext context, KStreamMetadata metadata) { <<<<<<< HEAD <<<<<<< HEAD @@ -64,6 +71,9 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { stream.bind(context, metadata); >>>>>>> new api model ======= +======= + public void bind(ProcessorContext context, KStreamMetadata metadata) { +>>>>>>> removing io.confluent imports: wip for (KStreamSource branch : branches) { branch.bind(context, metadata); >>>>>>> wip diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java index aca026479a283..ead6b56f7a4d6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamTopology; @@ -10,6 +11,9 @@ import io.confluent.streaming.KStreamTopology; >>>>>>> wip import io.confluent.streaming.Predicate; +======= +import org.apache.kafka.stream.topology.KStreamTopology; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.topology.Predicate; /** diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java index 96f55a4f48a08..b72504040f3ba 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD <<<<<<< HEAD @@ -12,6 +13,11 @@ >>>>>>> wip import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; +======= +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KeyValue; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.topology.KeyValueMapper; /** @@ -41,7 +47,7 @@ class KStreamFlatMap extends KStreamImpl { } @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { + public void bind(ProcessorContext context, KStreamMetadata metadata) { super.bind(context, KStreamMetadata.unjoinable()); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java index d35f62bd22938..b18ac9c080430 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamTopology; @@ -10,6 +11,9 @@ import io.confluent.streaming.KStreamTopology; >>>>>>> wip import io.confluent.streaming.ValueMapper; +======= +import org.apache.kafka.stream.topology.KStreamTopology; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.topology.ValueMapper; /** diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java index 27036d532f298..3d92e4513173c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD <<<<<<< HEAD @@ -13,6 +14,12 @@ import io.confluent.streaming.NotCopartitionedException; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; +======= +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.NotCopartitionedException; +import org.apache.kafka.stream.internal.Receiver; +import org.apache.kafka.stream.topology.KStreamTopology; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.topology.ValueJoiner; import org.apache.kafka.stream.topology.Window; @@ -85,7 +92,7 @@ Iterator find(K key, long timestamp) { } @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { + public void bind(ProcessorContext context, KStreamMetadata metadata) { super.bind(context, metadata); thisMetadata = metadata; @@ -106,7 +113,7 @@ public void receive(Object key, Object value, long timestamp) { private Receiver getReceiverForOther() { return new Receiver() { @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { + public void bind(ProcessorContext context, KStreamMetadata metadata) { otherMetadata = metadata; if (thisMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) throw new NotCopartitionedException(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java index b8aca99af0b9c..12aee6cdefc1a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD <<<<<<< HEAD @@ -12,6 +13,11 @@ >>>>>>> wip import io.confluent.streaming.KeyValueMapper; import io.confluent.streaming.KeyValue; +======= +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KeyValue; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.topology.KeyValueMapper; /** @@ -41,7 +47,7 @@ class KStreamMap extends KStreamImpl { } @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { + public void bind(ProcessorContext context, KStreamMetadata metadata) { super.bind(context, KStreamMetadata.unjoinable()); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java index 0c8db36601c0a..28df6b3837f69 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamTopology; @@ -10,6 +11,9 @@ import io.confluent.streaming.KStreamTopology; >>>>>>> wip import io.confluent.streaming.ValueMapper; +======= +import org.apache.kafka.stream.topology.KStreamTopology; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.topology.ValueMapper; /** diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java index c4ec1f3ecdbc7..7920860b14d69 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java @@ -1,8 +1,7 @@ package org.apache.kafka.stream.topology.internal; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamTopology; -import io.confluent.streaming.Transformer; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Transformer; /** @@ -18,7 +17,7 @@ class KStreamTransform extends KStreamImpl implements Transf } @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { + public void bind(ProcessorContext context, KStreamMetadata metadata) { transformer.init(context); transformer.forwarder(this); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java index 342edba1ae2e2..fee1af3163880 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java @@ -1,7 +1,9 @@ package org.apache.kafka.stream.topology.internal; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.KStream; +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD <<<<<<< HEAD @@ -15,6 +17,10 @@ import io.confluent.streaming.KStreamWindowed; import io.confluent.streaming.ValueJoiner; import io.confluent.streaming.Window; +======= +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KStreamWindowed; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.topology.ValueJoiner; import org.apache.kafka.stream.topology.Window; @@ -39,7 +45,7 @@ public class KStreamWindowedImpl extends KStreamImpl implements KStr } @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { + public void bind(ProcessorContext context, KStreamMetadata metadata) { super.bind(context, metadata); window.init(context); } diff --git a/temp b/temp index af827a1f872f5..3852c77d64253 100644 --- a/temp +++ b/temp @@ -1,8 +1,8 @@ -clients/src/main/java/org/apache/kafka/clients/processor/Processor.java: needs merge -clients/src/main/java/org/apache/kafka/common/utils/Utils.java: needs merge -stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java: needs merge -stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java: needs merge -stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java: needs merge diff --git a/temp2 b/temp2 index f63967a305e3f..34a305c483bdf 100644 --- a/temp2 +++ b/temp2 @@ -1,8 +1,8 @@ -clients/src/main/java/org/apache/kafka/clients/processor/Processor.java -clients/src/main/java/org/apache/kafka/common/utils/Utils.java -stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java -stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java -stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java From 6320cf9eeaf97ff958ec661abbb308d8c3ee327f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 5 Aug 2015 13:19:04 -0700 Subject: [PATCH 134/275] removing io.confluent imports: wip --- .../java/org/apache/kafka/stream/KStream.java | 2 +- .../stream/examples/PrintKStreamJob.java | 8 +-- .../stream/examples/StatefulKStreamJob.java | 8 +-- .../kafka/stream/internal/ProcessorNode.java | 8 +-- .../internal/ProcessorStateManager.java | 4 +- .../stream/internal/PunctuationSchedule.java | 2 +- .../kafka/stream/internal/Receiver.java | 4 +- .../stream/internal/RecordCollectorImpl.java | 2 +- .../kafka/stream/internal/StreamGroup.java | 8 +-- .../stream/kv/InMemoryKeyValueStore.java | 10 ++-- .../apache/kafka/stream/kv/KeyValueStore.java | 2 +- .../kafka/stream/kv/RocksDBKeyValueStore.java | 10 ++-- .../kv/internal/MeteredKeyValueStore.java | 10 ++-- .../topology/SingleProcessorTopology.java | 1 - .../kafka/stream/topology/SlidingWindow.java | 4 +- .../kafka/stream/topology/Transformer.java | 2 - .../apache/kafka/stream/topology/Window.java | 6 +- .../topology/internal/KStreamBranch.java | 8 +++ .../topology/internal/KStreamFlatMap.java | 6 +- .../stream/topology/internal/KStreamImpl.java | 12 ++-- .../stream/topology/internal/KStreamJoin.java | 8 ++- .../stream/topology/internal/KStreamMap.java | 6 +- .../topology/internal/KStreamSource.java | 6 +- .../topology/internal/KStreamThread.java | 12 ++-- .../topology/internal/KStreamTransform.java | 4 +- .../internal/KStreamWindowedImpl.java | 4 +- .../kafka/stream/FilteredIteratorTest.java | 1 + .../kafka/stream/KStreamBranchTest.java | 22 ++++--- .../kafka/stream/KStreamFilterTest.java | 22 ++++--- .../kafka/stream/KStreamFlatMapTest.java | 23 ++++--- .../stream/KStreamFlatMapValuesTest.java | 16 +++-- .../apache/kafka/stream/KStreamJoinTest.java | 60 ++++++++++++++----- .../apache/kafka/stream/KStreamMapTest.java | 17 ++++-- .../kafka/stream/KStreamMapValuesTest.java | 15 ++++- .../kafka/stream/KStreamSourceTest.java | 19 ++++-- .../kafka/stream/KStreamTransformTest.java | 23 +++---- .../kafka/stream/KStreamWindowedTest.java | 12 ++-- .../kafka/stream/MinTimestampTrackerTest.java | 4 +- .../kafka/stream/ParallelExecutorTest.java | 1 + .../apache/kafka/stream/StreamGroupTest.java | 10 ++-- .../org/apache/kafka/test/MockIngestor.java | 4 +- .../apache/kafka/test/MockKStreamContext.java | 13 ++++ .../kafka/test/MockKStreamTopology.java | 3 +- .../org/apache/kafka/test/MockProcessor.java | 4 +- .../apache/kafka/test/UnlimitedWindow.java | 17 +++--- temp | 11 ++-- temp2 | 11 ++-- 47 files changed, 295 insertions(+), 170 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java index be69bc5567d77..84aebde78977c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java @@ -1,11 +1,11 @@ package org.apache.kafka.stream; -import org.apache.kafka.clients.processor.Processor; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.stream.topology.KStreamWindowed; import org.apache.kafka.stream.topology.KeyValueMapper; import org.apache.kafka.stream.topology.Predicate; +import org.apache.kafka.stream.topology.Processor; import org.apache.kafka.stream.topology.Transformer; import org.apache.kafka.stream.topology.ValueMapper; import org.apache.kafka.stream.topology.Window; diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java index 7cfede884ee53..ce7027a618caa 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java @@ -1,7 +1,7 @@ package org.apache.kafka.stream.examples; -import org.apache.kafka.clients.processor.Processor; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KafkaStreaming; import org.apache.kafka.stream.StreamingConfig; import org.apache.kafka.stream.topology.SingleProcessorTopology; @@ -13,10 +13,10 @@ */ public class PrintKStreamJob implements Processor { - private ProcessorContext context; + private KStreamContext context; @Override - public void init(ProcessorContext context) { + public void init(KStreamContext context) { this.context = context; } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java index bb393a5b80bbd..21714e0e6b644 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java @@ -1,8 +1,8 @@ package org.apache.kafka.stream.examples; -import org.apache.kafka.clients.processor.Processor; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KafkaStreaming; import org.apache.kafka.stream.StreamingConfig; import org.apache.kafka.stream.kv.Entry; @@ -19,11 +19,11 @@ public class StatefulKStreamJob implements Processor { - private ProcessorContext context; + private KStreamContext context; private KeyValueStore kvStore; @Override - public void init(ProcessorContext context) { + public void init(KStreamContext context) { this.context = context; this.context.schedule(this, 1000); diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java index 87d9774e28b02..67e16d4738eaf 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java @@ -1,7 +1,7 @@ package org.apache.kafka.stream.internal; -import org.apache.kafka.clients.processor.Processor; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.topology.internal.KStreamMetadata; /** @@ -10,14 +10,14 @@ public class ProcessorNode implements Receiver { private final Processor processor; - private ProcessorContext context; + private KStreamContext context; public ProcessorNode(Processor processor) { this.processor = processor; } @Override - public void bind(ProcessorContext context, KStreamMetadata metadata) { + public void bind(KStreamContext context, KStreamMetadata metadata) { if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); this.context = context; diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java index b226779204da2..411f3faa7ec99 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java @@ -18,8 +18,8 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.processor.RestoreFunc; -import org.apache.kafka.clients.processor.StateStore; +import org.apache.kafka.stream.RestoreFunc; +import org.apache.kafka.stream.StateStore; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.stream.util.OffsetCheckpoint; diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java index 7f33571c49c93..9868865f162c9 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java @@ -1,7 +1,7 @@ package org.apache.kafka.stream.internal; -import org.apache.kafka.clients.processor.Processor; +import org.apache.kafka.stream.topology.Processor; import org.apache.kafka.stream.util.Stamped; public class PunctuationSchedule extends Stamped> { diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java b/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java index 7779901395543..680488f01f395 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java @@ -1,6 +1,6 @@ package org.apache.kafka.stream.internal; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.topology.internal.KStreamMetadata; /** @@ -8,7 +8,7 @@ */ public interface Receiver { - void bind(ProcessorContext context, KStreamMetadata metadata); + void bind(KStreamContext context, KStreamMetadata metadata); <<<<<<< HEAD <<<<<<< HEAD diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java index fbfb92ee0e5ee..b3cf6796d96a4 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java @@ -1,6 +1,6 @@ package org.apache.kafka.stream.internal; -import org.apache.kafka.clients.processor.RecordCollector; +import org.apache.kafka.stream.RecordCollector; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java b/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java index 196b99d9f70e2..ef03c493c651b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java @@ -1,8 +1,8 @@ package org.apache.kafka.stream.internal; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.processor.Processor; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.Chooser; @@ -35,7 +35,7 @@ public class StreamSynchronizer implements SyncGroup { public class StreamGroup implements ParallelExecutor.Task { >>>>>>> remove SyncGroup from user facing APIs:src/main/java/io/confluent/streaming/internal/StreamGroup.java - private final ProcessorContext context; + private final KStreamContext context; private final Ingestor ingestor; private final Chooser chooser; private final TimestampExtractor timestampExtractor; @@ -61,7 +61,7 @@ public class StreamGroup implements ParallelExecutor.Task { * @param timestampExtractor the instance of {@link TimestampExtractor} * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic */ - StreamGroup(ProcessorContext context, + public StreamGroup(KStreamContext context, Ingestor ingestor, Chooser chooser, TimestampExtractor timestampExtractor, diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java index c395d1d871ba6..fb3a790c9c20c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java @@ -1,6 +1,6 @@ package org.apache.kafka.stream.kv; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.stream.kv.internal.MeteredKeyValueStore; @@ -19,11 +19,11 @@ */ public class InMemoryKeyValueStore extends MeteredKeyValueStore { - public InMemoryKeyValueStore(String name, ProcessorContext context) { + public InMemoryKeyValueStore(String name, KStreamContext context) { this(name, context, new SystemTime()); } - public InMemoryKeyValueStore(String name, ProcessorContext context, Time time) { + public InMemoryKeyValueStore(String name, KStreamContext context, Time time) { super(name, new MemoryStore(name, context), context, "kafka-streams", time); } @@ -31,10 +31,10 @@ private static class MemoryStore implements KeyValueStore { private final String name; private final NavigableMap map; - private final ProcessorContext context; + private final KStreamContext context; @SuppressWarnings("unchecked") - public MemoryStore(String name, ProcessorContext context) { + public MemoryStore(String name, KStreamContext context) { super(); this.name = name; this.map = new TreeMap<>(); diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java index 61ca072606016..bf45a1ef88a52 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java @@ -19,7 +19,7 @@ package org.apache.kafka.stream.kv; -import org.apache.kafka.clients.processor.StateStore; +import org.apache.kafka.stream.StateStore; import java.util.List; diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java index 127e143e0109a..35850ff9cec9e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java @@ -1,6 +1,6 @@ package org.apache.kafka.stream.kv; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.SystemTime; @@ -26,11 +26,11 @@ */ public class RocksDBKeyValueStore extends MeteredKeyValueStore { - public RocksDBKeyValueStore(String name, ProcessorContext context) { + public RocksDBKeyValueStore(String name, KStreamContext context) { this(name, context, new SystemTime()); } - public RocksDBKeyValueStore(String name, ProcessorContext context, Time time) { + public RocksDBKeyValueStore(String name, KStreamContext context, Time time) { super(name, new RocksDBStore(name, context), context, "kafka-streams", time); } @@ -50,7 +50,7 @@ private static class RocksDBStore implements KeyValueStore { private final String topic; private final int partition; - private final ProcessorContext context; + private final KStreamContext context; private final Options options; private final WriteOptions wOptions; @@ -62,7 +62,7 @@ private static class RocksDBStore implements KeyValueStore { private RocksDB db; @SuppressWarnings("unchecked") - public RocksDBStore(String name, ProcessorContext context) { + public RocksDBStore(String name, KStreamContext context) { this.topic = name; this.partition = context.id(); this.context = context; diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java index b60e4b997a8c2..7e3033a19d9c6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java @@ -1,8 +1,8 @@ package org.apache.kafka.stream.kv.internal; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.clients.processor.RecordCollector; -import org.apache.kafka.clients.processor.RestoreFunc; +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.RecordCollector; +import org.apache.kafka.stream.RestoreFunc; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.MeasurableStat; @@ -43,10 +43,10 @@ public class MeteredKeyValueStore implements KeyValueStore { private final int partition; private final Set dirty; private final int maxDirty; - private final ProcessorContext context; + private final KStreamContext context; // always wrap the logged store with the metered store - public MeteredKeyValueStore(final String name, final KeyValueStore inner, ProcessorContext context, String group, Time time) { + public MeteredKeyValueStore(final String name, final KeyValueStore inner, KStreamContext context, String group, Time time) { this.inner = inner; this.time = time; diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java index f3e254569d06f..e7365dcbc8387 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java @@ -1,6 +1,5 @@ package org.apache.kafka.stream.topology; -import org.apache.kafka.clients.processor.Processor; import org.apache.kafka.common.utils.Utils; /** diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java index 0d91d623b61f6..50938e8d2a714 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java @@ -1,6 +1,6 @@ package org.apache.kafka.stream.topology; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.util.FilteredIterator; import org.apache.kafka.stream.util.Stamped; @@ -26,7 +26,7 @@ public SlidingWindow(String name, long duration, int maxCount) { } @Override - public void init(ProcessorContext context) { + public void init(KStreamContext context) { } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java b/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java index 48a9d959b6797..c4e232524f2dc 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java @@ -1,7 +1,5 @@ package org.apache.kafka.stream.topology; -import org.apache.kafka.clients.processor.Processor; - /** * Created by yasuhiro on 6/17/15. */ diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Window.java b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java index 25941b4a586b3..3d623551cdede 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Window.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java @@ -1,7 +1,7 @@ package org.apache.kafka.stream.topology; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.clients.processor.StateStore; +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.StateStore; import java.util.Iterator; @@ -10,7 +10,7 @@ */ public interface Window extends StateStore { - void init(ProcessorContext context); + void init(KStreamContext context); Iterator find(K key, long timestamp); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java index 3ed8154b64a3c..7828b67fc22c3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD @@ -14,6 +15,9 @@ import io.confluent.streaming.Predicate; ======= import org.apache.kafka.clients.processor.ProcessorContext; +======= +import org.apache.kafka.stream.KStreamContext; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.internal.Receiver; import org.apache.kafka.stream.topology.KStreamTopology; >>>>>>> removing io.confluent imports: wip @@ -60,6 +64,7 @@ class KStreamBranch implements Receiver { } @Override +<<<<<<< HEAD <<<<<<< HEAD public void bind(KStreamContext context, KStreamMetadata metadata) { <<<<<<< HEAD @@ -73,6 +78,9 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { ======= ======= public void bind(ProcessorContext context, KStreamMetadata metadata) { +>>>>>>> removing io.confluent imports: wip +======= + public void bind(KStreamContext context, KStreamMetadata metadata) { >>>>>>> removing io.confluent imports: wip for (KStreamSource branch : branches) { branch.bind(context, metadata); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java index b72504040f3ba..63c30ce9b0e78 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD @@ -15,6 +16,9 @@ import io.confluent.streaming.KeyValue; ======= import org.apache.kafka.clients.processor.ProcessorContext; +======= +import org.apache.kafka.stream.KStreamContext; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KeyValue; >>>>>>> removing io.confluent imports: wip @@ -47,7 +51,7 @@ class KStreamFlatMap extends KStreamImpl { } @Override - public void bind(ProcessorContext context, KStreamMetadata metadata) { + public void bind(KStreamContext context, KStreamMetadata metadata) { super.bind(context, KStreamMetadata.unjoinable()); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java index 4beb11068fbbf..8b4462ae01b1c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java @@ -1,7 +1,7 @@ package org.apache.kafka.stream.topology.internal; -import org.apache.kafka.clients.processor.Processor; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KStream; <<<<<<< HEAD import io.confluent.streaming.KStreamContext; @@ -47,7 +47,7 @@ abstract class KStreamImpl implements KStream, Receiver { <<<<<<< HEAD <<<<<<< HEAD protected KStreamTopology topology; - protected ProcessorContext context; + protected KStreamContext context; protected KStreamMetadata metadata; protected KStreamImpl(KStreamTopology topology) { @@ -55,7 +55,7 @@ protected KStreamImpl(KStreamTopology topology) { } @Override - public void bind(ProcessorContext context, KStreamMetadata metadata) { + public void bind(KStreamContext context, KStreamMetadata metadata) { if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); this.context = context; this.metadata = metadata; @@ -240,10 +240,10 @@ public void sendTo(String topic, Serializer keySerializer, Serializer valS @SuppressWarnings("unchecked") private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { return new Processor() { - private ProcessorContext context; + private KStreamContext context; @Override - public void init(ProcessorContext context) { + public void init(KStreamContext context) { this.context = context; } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java index 3d92e4513173c..d2058e8cb9dd2 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD @@ -16,6 +17,9 @@ import io.confluent.streaming.Window; ======= import org.apache.kafka.clients.processor.ProcessorContext; +======= +import org.apache.kafka.stream.KStreamContext; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.NotCopartitionedException; import org.apache.kafka.stream.internal.Receiver; import org.apache.kafka.stream.topology.KStreamTopology; @@ -92,7 +96,7 @@ Iterator find(K key, long timestamp) { } @Override - public void bind(ProcessorContext context, KStreamMetadata metadata) { + public void bind(KStreamContext context, KStreamMetadata metadata) { super.bind(context, metadata); thisMetadata = metadata; @@ -113,7 +117,7 @@ public void receive(Object key, Object value, long timestamp) { private Receiver getReceiverForOther() { return new Receiver() { @Override - public void bind(ProcessorContext context, KStreamMetadata metadata) { + public void bind(KStreamContext context, KStreamMetadata metadata) { otherMetadata = metadata; if (thisMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) throw new NotCopartitionedException(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java index 12aee6cdefc1a..be73101fe7ba8 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream.topology.internal; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD @@ -15,6 +16,9 @@ import io.confluent.streaming.KeyValue; ======= import org.apache.kafka.clients.processor.ProcessorContext; +======= +import org.apache.kafka.stream.KStreamContext; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KeyValue; >>>>>>> removing io.confluent imports: wip @@ -47,7 +51,7 @@ class KStreamMap extends KStreamImpl { } @Override - public void bind(ProcessorContext context, KStreamMetadata metadata) { + public void bind(KStreamContext context, KStreamMetadata metadata) { super.bind(context, KStreamMetadata.unjoinable()); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java index 7cbf96aed8763..670f1ae316d27 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java @@ -3,6 +3,7 @@ <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; import io.confluent.streaming.KStreamTopology; <<<<<<< HEAD @@ -17,6 +18,9 @@ ======= import org.apache.kafka.clients.processor.ProcessorContext; >>>>>>> removing io.confluent imports: wip +======= +import org.apache.kafka.stream.KStreamContext; +>>>>>>> removing io.confluent imports: wip import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.topology.KStreamTopology; @@ -87,7 +91,7 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { >>>>>>> wip @Override - public void bind(ProcessorContext context, KStreamMetadata metadata) { + public void bind(KStreamContext context, KStreamMetadata metadata) { if (keyDeserializer == null) keyDeserializer = (Deserializer) context.keyDeserializer(); if (valueDeserializer == null) valueDeserializer = (Deserializer) context.valueDeserializer(); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java index 38a5af386b26b..78a30c838679c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java @@ -55,7 +55,7 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.stream.StreamingConfig; import org.apache.kafka.stream.internal.IngestorImpl; -import org.apache.kafka.stream.internal.ProcessorContextImpl; +import org.apache.kafka.stream.internal.KStreamContextImpl; import org.apache.kafka.stream.internal.ProcessorConfig; import org.apache.kafka.stream.internal.RecordCollectorImpl; import org.apache.kafka.stream.internal.StreamGroup; @@ -79,7 +79,7 @@ public class KStreamThread extends Thread { private final KStreamTopology topology; private final ArrayList streamGroups = new ArrayList<>(); private final ParallelExecutor parallelExecutor; - private final Map kstreamContexts = new HashMap<>(); + private final Map kstreamContexts = new HashMap<>(); private final IngestorImpl ingestor; private final RecordCollectorImpl collector; private final StreamingConfig streamingConfig; @@ -205,7 +205,7 @@ private void maybeCommit() { private void commitAll(long now) { Map commit = new HashMap<>(); - for (ProcessorContextImpl context : kstreamContexts.values()) { + for (KStreamContextImpl context : kstreamContexts.values()) { context.flush(); commit.putAll(context.consumedOffsets()); } @@ -265,10 +265,10 @@ private void addPartitions(Collection assignment) { ======= for (TopicPartition partition : partitions) { final Integer id = partition.partition(); // TODO: switch this to the group id - ProcessorContextImpl context = kstreamContexts.get(id); + KStreamContextImpl context = kstreamContexts.get(id); if (context == null) { try { - context = new ProcessorContextImpl(id, ingestor, collector, streamingConfig, config, metrics); + context = new KStreamContextImpl(id, ingestor, collector, streamingConfig, config, metrics); context.init(topology.sourceStreams()); >>>>>>> new api model @@ -286,7 +286,7 @@ private void addPartitions(Collection assignment) { } private void removePartitions() { - for (ProcessorContextImpl context : kstreamContexts.values()) { + for (KStreamContextImpl context : kstreamContexts.values()) { log.info("Removing task context {}", context.id()); try { context.close(); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java index 7920860b14d69..d868f96084f52 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java @@ -1,6 +1,6 @@ package org.apache.kafka.stream.topology.internal; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Transformer; @@ -17,7 +17,7 @@ class KStreamTransform extends KStreamImpl implements Transf } @Override - public void bind(ProcessorContext context, KStreamMetadata metadata) { + public void bind(KStreamContext context, KStreamMetadata metadata) { transformer.init(context); transformer.forwarder(this); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java index fee1af3163880..e097bfb11b490 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java @@ -1,7 +1,7 @@ package org.apache.kafka.stream.topology.internal; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KStream; <<<<<<< HEAD import io.confluent.streaming.KStreamContext; @@ -45,7 +45,7 @@ public class KStreamWindowedImpl extends KStreamImpl implements KStr } @Override - public void bind(ProcessorContext context, KStreamMetadata metadata) { + public void bind(KStreamContext context, KStreamMetadata metadata) { super.bind(context, metadata); window.init(context); } diff --git a/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java b/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java index 65777c31e0375..78f015108c662 100644 --- a/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java @@ -2,6 +2,7 @@ import static org.junit.Assert.assertEquals; +import org.apache.kafka.stream.util.FilteredIterator; import org.junit.Test; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java index dd62600cc5a08..c786e2c607f48 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java @@ -1,9 +1,13 @@ package org.apache.kafka.stream; -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; -import io.confluent.streaming.testutil.TestProcessor; -import org.apache.kafka.clients.processor.KStreamContext; +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.Predicate; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockKStreamContext; +import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.MockProcessor; import org.junit.Test; import java.lang.reflect.Array; @@ -57,16 +61,16 @@ public boolean apply(Integer key, String value) { >>>>>>> wip KStreamSource stream; KStream[] branches; - TestProcessor[] processors; + MockProcessor[] processors; stream = new KStreamSource<>(null, initializer); branches = stream.branch(isEven, isMultipleOfThree, isOdd); assertEquals(3, branches.length); - processors = (TestProcessor[]) Array.newInstance(TestProcessor.class, branches.length); + processors = (MockProcessor[]) Array.newInstance(MockProcessor.class, branches.length); for (int i = 0; i < branches.length; i++) { - processors[i] = new TestProcessor<>(); + processors[i] = new MockProcessor<>(); branches[i].process(processors[i]); } @@ -83,9 +87,9 @@ public boolean apply(Integer key, String value) { assertEquals(3, branches.length); - processors = (TestProcessor[]) Array.newInstance(TestProcessor.class, branches.length); + processors = (MockProcessor[]) Array.newInstance(MockProcessor.class, branches.length); for (int i = 0; i < branches.length; i++) { - processors[i] = new TestProcessor<>(); + processors[i] = new MockProcessor<>(); branches[i].process(processors[i]); } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java index a240baf39e521..5c3f616010800 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java @@ -1,11 +1,15 @@ package org.apache.kafka.stream; -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; -import io.confluent.streaming.testutil.TestProcessor; -import org.apache.kafka.clients.processor.KStreamContext; -import org.junit.Test; +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.Predicate; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.MockKStreamContext; +import org.junit.Test; import java.util.Collections; import static org.junit.Assert.assertEquals; @@ -41,9 +45,9 @@ public void testFilter() { KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; - TestProcessor processor; + MockProcessor processor; - processor = new TestProcessor<>(); + processor = new MockProcessor<>(); stream = new KStreamSource<>(null, initializer); stream.filter(isMultipleOfThree).process(processor); @@ -74,9 +78,9 @@ public void testFilterOut() { KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; - TestProcessor processor; + MockProcessor processor; - processor = new TestProcessor<>(); + processor = new MockProcessor<>(); stream = new KStreamSource<>(null, initializer); stream.filterOut(isMultipleOfThree).process(processor); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java index b1a09267beb9d..d400c2dc27e74 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java @@ -1,16 +1,21 @@ package org.apache.kafka.stream; -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; -import io.confluent.streaming.testutil.TestProcessor; -import org.apache.kafka.clients.processor.KStreamContext; +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KeyValue; +import org.apache.kafka.stream.topology.KeyValueMapper; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.MockKStreamContext; import org.junit.Test; +import static org.junit.Assert.assertEquals; + import java.util.ArrayList; import java.util.Collections; -import static org.junit.Assert.assertEquals; - public class KStreamFlatMapTest { private String topicName = "topic"; @@ -50,8 +55,9 @@ public KeyValue> apply(Integer key, String value) { >>>>>>> wip KStreamSource stream; - TestProcessor processor; + MockProcessor processor; +<<<<<<< HEAD processor = new TestProcessor<>(); <<<<<<< HEAD <<<<<<< HEAD @@ -60,6 +66,9 @@ public KeyValue> apply(Integer key, String value) { stream = new KStreamSource<>(null, initializer); >>>>>>> new api model ======= +======= + processor = new MockProcessor<>(); +>>>>>>> removing io.confluent imports: wip stream = new KStreamSource<>(null, topology); >>>>>>> wip stream.flatMap(mapper).process(processor); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java index 5ef445d3fac68..28a96347b8631 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java @@ -1,9 +1,13 @@ package org.apache.kafka.stream; -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; -import io.confluent.streaming.testutil.TestProcessor; -import org.apache.kafka.clients.processor.KStreamContext; +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.ValueMapper; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.MockKStreamContext; import org.junit.Test; import java.util.ArrayList; @@ -47,9 +51,9 @@ public Iterable apply(String value) { KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; - TestProcessor processor; + MockProcessor processor; - processor = new TestProcessor<>(); + processor = new MockProcessor<>(); stream = new KStreamSource<>(null, initializer); stream.flatMapValues(mapper).process(processor); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java index 5f307967f3a6c..c4ce68ff573f2 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java @@ -1,11 +1,19 @@ package org.apache.kafka.stream; -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; -import io.confluent.streaming.testutil.TestProcessor; -import io.confluent.streaming.testutil.UnlimitedWindow; -import io.confluent.streaming.util.Util; -import org.apache.kafka.clients.processor.KStreamContext; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KStreamWindowed; +import org.apache.kafka.stream.topology.KeyValue; +import org.apache.kafka.stream.topology.KeyValueMapper; +import org.apache.kafka.stream.topology.ValueJoiner; +import org.apache.kafka.stream.topology.ValueMapper; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.MockKStreamContext; +import org.apache.kafka.test.UnlimitedWindow; import org.junit.Test; import java.util.Collections; @@ -37,7 +45,7 @@ public String apply(String value) { private ValueMapper, String> valueMapper2 = new ValueMapper, String>() { @Override public Iterable apply(String value) { - return (Iterable) Util.mkSet(value); + return (Iterable) Utils.mkSet(value); } }; @@ -53,7 +61,7 @@ public KeyValue apply(Integer key, String value) { new KeyValueMapper, Integer, String>() { @Override public KeyValue> apply(Integer key, String value) { - return KeyValue.pair(key, (Iterable) Util.mkSet(value)); + return KeyValue.pair(key, (Iterable) Utils.mkSet(value)); } }; @@ -66,13 +74,14 @@ public void testJoin() { KStreamSource stream2; KStreamWindowed windowed1; KStreamWindowed windowed2; - TestProcessor processor; + MockProcessor processor; String[] expected; <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +<<<<<<< HEAD ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model @@ -83,6 +92,9 @@ public void testJoin() { KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip processor = new TestProcessor<>(); +======= + processor = new MockProcessor<>(); +>>>>>>> removing io.confluent imports: wip stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); windowed1 = stream1.with(new UnlimitedWindow()); @@ -168,7 +180,7 @@ public void testJoinPrior() { KStreamSource stream2; KStreamWindowed windowed1; KStreamWindowed windowed2; - TestProcessor processor; + MockProcessor processor; String[] expected; <<<<<<< HEAD @@ -183,8 +195,12 @@ public void testJoinPrior() { >>>>>>> wip ======= KStreamTopology initializer = new MockKStreamTopology(); +<<<<<<< HEAD >>>>>>> wip processor = new TestProcessor<>(); +======= + processor = new MockProcessor<>(); +>>>>>>> removing io.confluent imports: wip stream1 = new KStreamSource<>(null, initializer); stream2 = new KStreamSource<>(null, initializer); windowed1 = stream1.with(new UnlimitedWindow()); @@ -265,12 +281,13 @@ public void testMap() { KStream mapped2; KStreamWindowed windowed1; KStreamWindowed windowed2; - TestProcessor processor; + MockProcessor processor; <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +<<<<<<< HEAD processor = new TestProcessor<>(); ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); @@ -287,6 +304,9 @@ public void testMap() { KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); >>>>>>> wip +======= + processor = new MockProcessor<>(); +>>>>>>> removing io.confluent imports: wip boolean exceptionRaised; @@ -365,7 +385,7 @@ public void testFlatMap() { KStream mapped2; KStreamWindowed windowed1; KStreamWindowed windowed2; - TestProcessor processor; + MockProcessor processor; <<<<<<< HEAD <<<<<<< HEAD @@ -385,8 +405,12 @@ public void testFlatMap() { >>>>>>> new api model ======= KStreamTopology initializer = new MockKStreamTopology(); +<<<<<<< HEAD processor = new TestProcessor<>(); >>>>>>> wip +======= + processor = new MockProcessor<>(); +>>>>>>> removing io.confluent imports: wip boolean exceptionRaised; @@ -465,7 +489,7 @@ public void testMapValues() { KStream mapped2; KStreamWindowed windowed1; KStreamWindowed windowed2; - TestProcessor processor; + MockProcessor processor; <<<<<<< HEAD <<<<<<< HEAD @@ -485,8 +509,12 @@ public void testMapValues() { >>>>>>> new api model ======= KStreamTopology initializer = new MockKStreamTopology(); +<<<<<<< HEAD processor = new TestProcessor<>(); >>>>>>> wip +======= + processor = new MockProcessor<>(); +>>>>>>> removing io.confluent imports: wip boolean exceptionRaised; @@ -565,7 +593,7 @@ public void testFlatMapValues() { KStream mapped2; KStreamWindowed windowed1; KStreamWindowed windowed2; - TestProcessor processor; + MockProcessor processor; <<<<<<< HEAD <<<<<<< HEAD @@ -585,8 +613,12 @@ public void testFlatMapValues() { >>>>>>> new api model ======= KStreamTopology initializer = new MockKStreamTopology(); +<<<<<<< HEAD processor = new TestProcessor<>(); >>>>>>> wip +======= + processor = new MockProcessor<>(); +>>>>>>> removing io.confluent imports: wip boolean exceptionRaised; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java index 1797a9eb93f92..0ec3d83de099a 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java @@ -1,9 +1,14 @@ package org.apache.kafka.stream; -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; -import io.confluent.streaming.testutil.TestProcessor; -import org.apache.kafka.clients.processor.KStreamContext; +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KeyValue; +import org.apache.kafka.stream.topology.KeyValueMapper; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.MockKStreamContext; import org.junit.Test; import java.util.Collections; @@ -43,9 +48,9 @@ public KeyValue apply(Integer key, String value) { KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; - TestProcessor processor; + MockProcessor processor; - processor = new TestProcessor<>(); + processor = new MockProcessor<>(); stream = new KStreamSource<>(null, initializer); stream.map(mapper).process(processor); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java index 9ac9593b17b3a..11aed82b229cd 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream; +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD <<<<<<< HEAD @@ -15,6 +16,16 @@ import io.confluent.streaming.testutil.MockKStreamTopology; import io.confluent.streaming.testutil.TestProcessor; import org.apache.kafka.clients.processor.KStreamContext; +======= +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.ValueMapper; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.MockKStreamContext; +>>>>>>> removing io.confluent imports: wip import org.junit.Test; import java.util.Collections; @@ -54,9 +65,9 @@ public Integer apply(String value) { KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip KStreamSource stream; - TestProcessor processor; + MockProcessor processor; - processor = new TestProcessor<>(); + processor = new MockProcessor<>(); stream = new KStreamSource<>(null, initializer); stream.mapValues(mapper).process(processor); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java index 4fa357e3336a3..67099ae85c249 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java @@ -1,15 +1,18 @@ package org.apache.kafka.stream; -import static org.junit.Assert.assertEquals; - -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; -import io.confluent.streaming.testutil.TestProcessor; -import org.apache.kafka.clients.processor.KStreamContext; +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.MockKStreamContext; import org.junit.Test; import java.util.Collections; +import static org.junit.Assert.assertEquals; + public class KStreamSourceTest { private String topicName = "topic"; @@ -23,6 +26,7 @@ public void testKStreamSource() { <<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); +<<<<<<< HEAD ======= KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); >>>>>>> new api model @@ -33,6 +37,9 @@ public void testKStreamSource() { KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip TestProcessor processor = new TestProcessor<>(); +======= + MockProcessor processor = new MockProcessor<>(); +>>>>>>> removing io.confluent imports: wip KStreamSource stream = new KStreamSource<>(null, initializer); stream.process(processor); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java index f9406f0496be8..ecfb24cded090 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java @@ -1,12 +1,13 @@ package org.apache.kafka.stream; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamTopology; -import io.confluent.streaming.Transformer; -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; -import io.confluent.streaming.testutil.TestProcessor; -import org.apache.kafka.clients.processor.KStreamContext; +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.Transformer; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockKStreamContext; +import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.MockProcessor; import org.junit.Test; import java.util.Collections; @@ -25,10 +26,10 @@ public void testTransform() { KStreamTopology topology = new MockKStreamTopology(); KStreamSource stream; - TestProcessor processor; + MockProcessor processor; Transformer transformer; - processor = new TestProcessor<>(); + processor = new MockProcessor<>(); transformer = new Transformer() { KStreamContext context; Forwarder forwarder; @@ -67,10 +68,10 @@ public void testTransformEmitOnPuncutation() { KStreamTopology topology = new MockKStreamTopology(); KStreamSource stream; - TestProcessor processor; + MockProcessor processor; Transformer transformer; - processor = new TestProcessor<>(); + processor = new MockProcessor<>(); transformer = new Transformer() { KStreamContext context; Forwarder forwarder; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java index d03701a7301aa..8b67c6add8138 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java @@ -1,9 +1,13 @@ package org.apache.kafka.stream; -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; -import io.confluent.streaming.testutil.UnlimitedWindow; -import org.apache.kafka.clients.processor.KStreamContext; +import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.Window; +import org.apache.kafka.stream.topology.internal.KStreamMetadata; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockKStreamContext; +import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.UnlimitedWindow; import org.junit.Test; import java.util.Collections; diff --git a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java index 996c7bdd7fe7d..273978f173cf3 100644 --- a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java @@ -2,12 +2,14 @@ import static org.junit.Assert.assertEquals; +import org.apache.kafka.stream.util.MinTimestampTracker; +import org.apache.kafka.stream.util.Stamped; import org.junit.Test; public class MinTimestampTrackerTest { private Stamped elem(long timestamp) { - return new Stamped("", timestamp); + return new Stamped<>("", timestamp); } @SuppressWarnings("unchecked") diff --git a/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java b/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java index 3730a4700e335..800ccaec9ffc8 100644 --- a/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java @@ -1,5 +1,6 @@ package org.apache.kafka.stream; +import org.apache.kafka.stream.util.ParallelExecutor; import org.junit.Test; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java index 5fc848a75e556..07b4ca5ce38ce 100644 --- a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java @@ -1,15 +1,17 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.TimestampExtractor; -import io.confluent.streaming.testutil.MockIngestor; -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; 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.stream.internal.StreamGroup; +import org.apache.kafka.stream.internal.TimeBasedChooser; +import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.test.MockIngestor; +import org.apache.kafka.test.MockKStreamContext; +import org.apache.kafka.test.MockKStreamTopology; import org.junit.Test; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java index 1ea8a355fd49c..56bf9946e81d2 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java @@ -1,9 +1,9 @@ package org.apache.kafka.test; -import io.confluent.streaming.internal.Ingestor; -import io.confluent.streaming.internal.StreamGroup; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.stream.internal.Ingestor; +import org.apache.kafka.stream.internal.StreamGroup; import java.util.HashMap; import java.util.HashSet; diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java index 1835f1bb8f859..30451defba772 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java @@ -1,6 +1,7 @@ package org.apache.kafka.test; +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; <<<<<<< HEAD @@ -31,9 +32,17 @@ >>>>>>> removed ProcessorContext import io.confluent.streaming.kv.internals.RestoreFunc; >>>>>>> new api model +======= +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.RecordCollector; +import org.apache.kafka.stream.RestoreFunc; +import org.apache.kafka.stream.StateStore; +>>>>>>> removing io.confluent imports: wip 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.stream.internal.PunctuationQueue; import java.io.File; import java.util.Map; @@ -80,6 +89,7 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer public Metrics metrics() { throw new UnsupportedOperationException("metrics() not supported."); } @Override +<<<<<<< HEAD public void restore(StateStore store, RestoreFunc func) { throw new UnsupportedOperationException("restore() not supported."); } <<<<<<< HEAD @@ -88,6 +98,9 @@ public MockKStreamContext(Serializer serializer, Deserializer deserializer >>>>>>> new api model public void register(StateStore store) { throw new UnsupportedOperationException("restore() not supported."); } +======= + public void register(StateStore store, RestoreFunc func) { throw new UnsupportedOperationException("restore() not supported."); } +>>>>>>> removing io.confluent imports: wip @Override public void flush() { throw new UnsupportedOperationException("flush() not supported."); } diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java index 8272fdbfb6cef..07b0bf75536c7 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java @@ -1,6 +1,7 @@ package org.apache.kafka.test; -import io.confluent.streaming.KStreamTopology; + +import org.apache.kafka.stream.topology.KStreamTopology; /** * Created by yasuhiro on 7/31/15. diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java index 51703266c23ea..ba2c2e7d519fe 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -1,7 +1,7 @@ package org.apache.kafka.test; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.Processor; +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java index f2da677ba651d..9d77505d71ac7 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java @@ -1,17 +1,18 @@ package org.apache.kafka.test; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KeyValue; -import io.confluent.streaming.Window; -import io.confluent.streaming.util.FilteredIterator; -import io.confluent.streaming.util.Stamped; + +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.topology.KeyValue; +import org.apache.kafka.stream.topology.Window; +import org.apache.kafka.stream.util.FilteredIterator; +import org.apache.kafka.stream.util.Stamped; import java.util.Iterator; import java.util.LinkedList; public class UnlimitedWindow implements Window { - private LinkedList>> list = new LinkedList>>(); + private LinkedList>> list = new LinkedList<>(); @Override public void init(KStreamContext context) { @@ -60,10 +61,6 @@ public void flush() { public void close() { } - @Override - public void restore() { - } - @Override public boolean persistent() { return false; diff --git a/temp b/temp index 3852c77d64253..de4d210a795df 100644 --- a/temp +++ b/temp @@ -1,8 +1,11 @@ +clients/src/main/java/org/apache/kafka/clients/processor/Processor.java: needs merge stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java: needs merge stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java: needs merge stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java: needs merge stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java: needs merge +stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java: needs merge diff --git a/temp2 b/temp2 index 34a305c483bdf..a8d47f2c55784 100644 --- a/temp2 +++ b/temp2 @@ -1,8 +1,11 @@ +clients/src/main/java/org/apache/kafka/clients/processor/Processor.java stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java +stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java From 680fe3d008041ba8265ee54ebd9083360d4029b4 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 5 Aug 2015 14:23:09 -0700 Subject: [PATCH 135/275] compile and test passed --- build.gradle | 11 +- checkstyle/checkstyle.xml | 8 +- checkstyle/import-control.xml | 15 +- kafka-work.iws | 208 +++-- .../java/org/apache/kafka/stream/Chooser.java | 28 +- .../java/org/apache/kafka/stream/KStream.java | 272 +++--- .../apache/kafka/stream/KStreamException.java | 44 +- .../apache/kafka/stream/KafkaStreaming.java | 26 +- .../stream/NotCopartitionedException.java | 12 - .../apache/kafka/stream/StreamingConfig.java | 6 +- .../kafka/stream/TimestampExtractor.java | 35 +- .../kafka/stream/examples/MapKStreamJob.java | 98 ++- .../stream/examples/PrintKStreamJob.java | 68 +- .../stream/examples/StatefulKStreamJob.java | 101 ++- .../kafka/stream/internal/Ingestor.java | 27 - .../kafka/stream/internal/IngestorImpl.java | 117 --- .../stream/internal/PartitioningInfo.java | 13 - .../stream/internal/ProcessorConfig.java | 104 --- .../stream/internal/PunctuationQueue.java | 38 - .../stream/internal/PunctuationSchedule.java | 24 - .../stream/internal/RecordCollectorImpl.java | 77 -- .../stream/internal/RoundRobinChooser.java | 33 - .../kafka/stream/internal/StampedRecord.java | 29 - .../stream/internal/TimeBasedChooser.java | 47 - .../kafka/stream/internals/Ingestor.java | 41 + .../kafka/stream/internals/IngestorImpl.java | 133 +++ .../stream/internals/PartitioningInfo.java | 27 + .../stream/internals/ProcessorConfig.java | 104 +++ .../kafka/stream/internals/ProcessorNode.java | 52 ++ .../ProcessorStateManager.java | 23 +- .../stream/internals/PunctuationQueue.java | 52 ++ .../stream/internals/PunctuationSchedule.java | 40 + .../kafka/stream/internals/Receiver.java | 31 + .../stream/internals/RecordCollectorImpl.java | 92 ++ .../kafka/stream/internals/RecordQueue.java | 123 +++ .../stream/internals/RoundRobinChooser.java | 47 + .../kafka/stream/internals/StampedRecord.java | 49 ++ .../kafka/stream/internals/StreamGroup.java | 283 ++++++ .../stream/internals/TimeBasedChooser.java | 61 ++ .../org/apache/kafka/stream/kv/Entry.java | 28 - .../org/apache/kafka/stream/state/Entry.java | 42 + .../{kv => state}/InMemoryKeyValueStore.java | 28 +- .../{kv => state}/KeyValueIterator.java | 8 +- .../stream/{kv => state}/KeyValueStore.java | 8 +- .../{kv => state}/RocksDBKeyValueStore.java | 28 +- .../internals}/MeteredKeyValueStore.java | 43 +- .../stream/topology/KStreamTopology.java | 175 ++-- .../stream/topology/KStreamWindowed.java | 67 +- .../kafka/stream/topology/KeyValue.java | 40 +- .../kafka/stream/topology/KeyValueMapper.java | 22 +- .../topology/NotCopartitionedException.java | 28 + .../kafka/stream/topology/Predicate.java | 22 +- .../topology/SingleProcessorTopology.java | 51 +- .../kafka/stream/topology/SlidingWindow.java | 220 ++--- .../kafka/stream/topology/Transformer.java | 28 +- .../kafka/stream/topology/ValueJoiner.java | 22 +- .../kafka/stream/topology/ValueMapper.java | 22 +- .../apache/kafka/stream/topology/Window.java | 30 +- .../topology/internal/KStreamMetadata.java | 53 -- .../topology/internal/KStreamTransform.java | 39 - .../topology/internals/KStreamBranch.java | 68 ++ .../topology/internals/KStreamFilter.java | 42 + .../topology/internals/KStreamFlatMap.java | 50 ++ .../internals/KStreamFlatMapValues.java | 43 + .../topology/internals/KStreamImpl.java | 191 +++++ .../topology/internals/KStreamJoin.java | 129 +++ .../stream/topology/internals/KStreamMap.java | 48 ++ .../topology/internals/KStreamMapValues.java | 41 + .../topology/internals/KStreamMetadata.java | 67 ++ .../topology/internals/KStreamSource.java | 75 ++ .../KStreamThread.java | 40 +- .../topology/internals/KStreamTransform.java | 53 ++ .../internals/KStreamWindowedImpl.java | 78 ++ .../kafka/stream/util/FilteredIterator.java | 80 +- .../stream/util/MinTimestampTracker.java | 59 +- .../kafka/stream/util/OffsetCheckpoint.java | 21 +- .../kafka/stream/util/ParallelExecutor.java | 231 ++--- .../org/apache/kafka/stream/util/Stamped.java | 44 +- .../kafka/stream/util/TimestampTracker.java | 61 +- .../kafka/stream/FilteredIteratorTest.java | 109 ++- .../kafka/stream/KStreamBranchTest.java | 125 ++- .../kafka/stream/KStreamFilterTest.java | 110 ++- .../kafka/stream/KStreamFlatMapTest.java | 86 +- .../stream/KStreamFlatMapValuesTest.java | 85 +- .../apache/kafka/stream/KStreamJoinTest.java | 810 ++++++++++-------- .../apache/kafka/stream/KStreamMapTest.java | 79 +- .../kafka/stream/KStreamMapValuesTest.java | 81 +- .../kafka/stream/KStreamSourceTest.java | 62 +- .../kafka/stream/KStreamTransformTest.java | 212 +++-- .../kafka/stream/KStreamWindowedTest.java | 99 ++- .../kafka/stream/MinTimestampTrackerTest.java | 117 +-- .../kafka/stream/ParallelExecutorTest.java | 192 +++-- .../apache/kafka/stream/StreamGroupTest.java | 258 +++--- .../org/apache/kafka/test/MockIngestor.java | 81 +- .../apache/kafka/test/MockKStreamContext.java | 129 ++- .../kafka/test/MockKStreamTopology.java | 27 +- .../org/apache/kafka/test/MockProcessor.java | 57 +- .../apache/kafka/test/UnlimitedWindow.java | 126 +-- temp | 21 +- temp2 | 21 +- 100 files changed, 5250 insertions(+), 2661 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/Ingestor.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/Ingestor.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/IngestorImpl.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/PartitioningInfo.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/ProcessorConfig.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/ProcessorNode.java rename stream/src/main/java/org/apache/kafka/stream/{internal => internals}/ProcessorStateManager.java (94%) create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/PunctuationQueue.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/PunctuationSchedule.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/Receiver.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/RecordCollectorImpl.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/RecordQueue.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/RoundRobinChooser.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/StampedRecord.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/TimeBasedChooser.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/kv/Entry.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/state/Entry.java rename stream/src/main/java/org/apache/kafka/stream/{kv => state}/InMemoryKeyValueStore.java (76%) rename stream/src/main/java/org/apache/kafka/stream/{kv => state}/KeyValueIterator.java (90%) rename stream/src/main/java/org/apache/kafka/stream/{kv => state}/KeyValueStore.java (96%) rename stream/src/main/java/org/apache/kafka/stream/{kv => state}/RocksDBKeyValueStore.java (89%) rename stream/src/main/java/org/apache/kafka/stream/{kv/internal => state/internals}/MeteredKeyValueStore.java (84%) create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java rename stream/src/main/java/org/apache/kafka/stream/topology/{internal => internals}/KStreamThread.java (92%) create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamTransform.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindowedImpl.java diff --git a/build.gradle b/build.gradle index cf75ffe161c27..081d79a1012a8 100644 --- a/build.gradle +++ b/build.gradle @@ -215,25 +215,22 @@ for ( sv in ['2_10_5', '2_11_7'] ) { } def copycatPkgs = ['copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file'] -def pkgs = ['clients', 'examples', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'log4j-appender', 'tools'] + copycatPkgs +def pkgs = ['clients', 'examples', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'log4j-appender', 'tools', 'streams'] + copycatPkgs tasks.create(name: "jarCopycat", dependsOn: copycatPkgs.collect { it + ":jar" }) {} -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7'] + pkgs.collect { it + ":jar" }) { -} +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7'] + pkgs.collect { it + ":jar" }) { } tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_10_5', 'srcJar_2_11_7'] + pkgs.collect { it + ":srcJar" }) { } tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_10_5', 'docsJar_2_11_7'] + pkgs.collect { it + ":docsJar" }) { } tasks.create(name: "testCopycat", dependsOn: copycatPkgs.collect { it + ":test" }) {} -tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7'] + pkgs.collect { it + ":test" }) { -} +tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7'] + pkgs.collect { it + ":test" }) { } tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7'] + pkgs.collect { it + ":uploadArchives" }) { -} +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7'] + pkgs.collect { it + ":uploadArchives" }) { } project(':core') { println "Building project 'core' with Scala version $scalaVersion" diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index a215ff36e9252..999fd6ca1dde9 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -53,9 +53,13 @@ - - + + + + + + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index eb682f4eb37a0..624ef38bcc244 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -87,8 +87,8 @@ - + @@ -108,6 +108,19 @@ + + + + + + + + + + + + + diff --git a/kafka-work.iws b/kafka-work.iws index c09380d8c3020..46c931c0f0708 100644 --- a/kafka-work.iws +++ b/kafka-work.iws @@ -2,28 +2,117 @@ - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - - + + @@ -109,28 +198,9 @@ - - - - - - - - - - - - - - - - - - - - - - + + + @@ -155,11 +225,21 @@
- + - - + + + + + + + + + + + + @@ -205,7 +285,6 @@ @@ -904,14 +984,6 @@ - - - - - - - - @@ -933,8 +1005,7 @@ - - + @@ -944,8 +1015,8 @@ - - + + @@ -968,28 +1039,9 @@ - - - - - - - - - - - - - - - - - - - - - - + + + @@ -1011,8 +1063,16 @@ - - + + + + + + + + + + diff --git a/stream/src/main/java/org/apache/kafka/stream/Chooser.java b/stream/src/main/java/org/apache/kafka/stream/Chooser.java index 90953a08674f0..c735517730ab3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/Chooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/Chooser.java @@ -1,16 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.kafka.stream; -import org.apache.kafka.stream.internal.RecordQueue; +import org.apache.kafka.stream.internals.RecordQueue; -/** - * Created by yasuhiro on 6/25/15. - */ public interface Chooser { - void add(RecordQueue queue); + void add(RecordQueue queue); - RecordQueue next(); + RecordQueue next(); - void close(); + void close(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java index 84aebde78977c..79f09dc9d943c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java @@ -1,3 +1,20 @@ +/** + * 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.stream; import org.apache.kafka.common.serialization.Deserializer; @@ -15,126 +32,139 @@ */ public interface KStream { - /** - * Creates a new stream consists of all elements of this stream which satisfy a predicate - * @param predicate the instance of Predicate - * @return KStream - */ - KStream filter(Predicate predicate); - - /** - * Creates a new stream consists all elements of this stream which do not satisfy a predicate - * @param predicate the instance of Predicate - * @return KStream - */ - KStream filterOut(Predicate predicate); - - /** - * Creates a new stream by transforming key-value pairs by a mapper to all elements of this stream - * @param mapper the instance of KeyValueMapper - * @param the key type of the new stream - * @param the value type of the new stream - * @return KStream - */ - KStream map(KeyValueMapper mapper); - - /** - * Creates a new stream by transforming valuesa by a mapper to all values of this stream - * @param mapper the instance of ValueMapper - * @param the value type of the new stream - * @return KStream - */ - KStream mapValues(ValueMapper mapper); - - /** - * Creates a new stream by applying a mapper to all elements of this stream and using the values in the resulting Iterable - * @param mapper the instance of KeyValueMapper - * @param the key type of the new stream - * @param the value type of the new stream - * @return KStream - */ - KStream flatMap(KeyValueMapper, K, V> mapper); - - /** - * Creates a new stream by applying a mapper to all values of this stream and using the values in the resulting Iterable - * @param processor the instance of Processor - * @param the value type of the new stream - * @return KStream - */ - KStream flatMapValues(ValueMapper, V> processor); - - /** - * Creates a new windowed stream using a specified window instance. - * @param window the instance of Window - * @return KStream - */ - KStreamWindowed with(Window window); - - /** - * Creates an array of streams from this stream. Each stream in the array coresponds to a predicate in - * supplied predicates in the same order. Predicates are evaluated in order. An element is streamed to - * a corresponding stream for the first predicate is evaluated true. - * An element will be dropped if none of the predicates evaluate true. - * @param predicates Instances of Predicate - * @return KStream - */ - KStream[] branch(Predicate... predicates); - - /** - * Sends key-value to a topic, also creates a new stream from the topic. - * The created stream is added to the default synchronization group. - * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). - * @param topic the topic name - * @return KStream - */ - KStream through(String topic); - - /** - * Sends key-value to a topic, also creates a new stream from the topic. - * The created stream is added to the default synchronization group. - * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). - * @param topic the topic name - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used - * @param keyDeserializer key deserializer used to create the new KStream, - * if not specified the default deserializer defined in the configs will be used - * @param valDeserializer value deserializer used to create the new KStream, - * if not specified the default deserializer defined in the configs will be used - * @param the key type of the new stream - * @param the value type of the new stream - * - * @return KStream - */ - KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); - - /** - * Sends key-value to a topic. - * @param topic the topic name - */ - void sendTo(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 sendTo(String topic, Serializer keySerializer, Serializer valSerializer); - - /** - * Processes all elements in this stream by applying a processor. - * @param processor the instance of Processor - */ - void process(Processor processor); - - /** - * Transform all elements in this stream by applying a tranformer. - * @param transformer the instance of Transformer - */ - KStream transform(Transformer transformer); + /** + * Creates a new stream consists of all elements of this stream which satisfy a predicate + * + * @param predicate the instance of Predicate + * @return KStream + */ + KStream filter(Predicate predicate); + + /** + * Creates a new stream consists all elements of this stream which do not satisfy a predicate + * + * @param predicate the instance of Predicate + * @return KStream + */ + KStream filterOut(Predicate predicate); + + /** + * Creates a new stream by transforming key-value pairs by a mapper to all elements of this stream + * + * @param mapper the instance of KeyValueMapper + * @param the key type of the new stream + * @param the value type of the new stream + * @return KStream + */ + KStream map(KeyValueMapper mapper); + + /** + * Creates a new stream by transforming valuesa by a mapper to all values of this stream + * + * @param mapper the instance of ValueMapper + * @param the value type of the new stream + * @return KStream + */ + KStream mapValues(ValueMapper mapper); + + /** + * Creates a new stream by applying a mapper to all elements of this stream and using the values in the resulting Iterable + * + * @param mapper the instance of KeyValueMapper + * @param the key type of the new stream + * @param the value type of the new stream + * @return KStream + */ + KStream flatMap(KeyValueMapper, K, V> mapper); + + /** + * Creates a new stream by applying a mapper to all values of this stream and using the values in the resulting Iterable + * + * @param processor the instance of Processor + * @param the value type of the new stream + * @return KStream + */ + KStream flatMapValues(ValueMapper, V> processor); + + /** + * Creates a new windowed stream using a specified window instance. + * + * @param window the instance of Window + * @return KStream + */ + KStreamWindowed with(Window window); + + /** + * Creates an array of streams from this stream. Each stream in the array coresponds to a predicate in + * supplied predicates in the same order. Predicates are evaluated in order. An element is streamed to + * a corresponding stream for the first predicate is evaluated true. + * An element will be dropped if none of the predicates evaluate true. + * + * @param predicates Instances of Predicate + * @return KStream + */ + KStream[] branch(Predicate... predicates); + + /** + * Sends key-value to a topic, also creates a new stream from the topic. + * The created stream is added to the default synchronization group. + * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). + * + * @param topic the topic name + * @return KStream + */ + KStream through(String topic); + + /** + * Sends key-value to a topic, also creates a new stream from the topic. + * The created stream is added to the default synchronization group. + * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). + * + * @param topic the topic name + * @param keySerializer key serializer used to send key-value pairs, + * if not specified the default serializer defined in the configs will be used + * @param valSerializer value serializer used to send key-value pairs, + * if not specified the default serializer defined in the configs will be used + * @param keyDeserializer key deserializer used to create the new KStream, + * if not specified the default deserializer defined in the configs will be used + * @param valDeserializer value deserializer used to create the new KStream, + * if not specified the default deserializer defined in the configs will be used + * @param the key type of the new stream + * @param the value type of the new stream + * @return KStream + */ + KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + + /** + * Sends key-value to a topic. + * + * @param topic the topic name + */ + void sendTo(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 sendTo(String topic, Serializer keySerializer, Serializer valSerializer); + + /** + * Processes all elements in this stream by applying a processor. + * + * @param processor the instance of Processor + */ + void process(Processor processor); + + /** + * Transform all elements in this stream by applying a tranformer. + * + * @param transformer the instance of Transformer + */ + KStream transform(Transformer transformer); } diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamException.java b/stream/src/main/java/org/apache/kafka/stream/KStreamException.java index 2556e66c0ce66..66b5930795d3c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamException.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamException.java @@ -1,23 +1,37 @@ -package org.apache.kafka.stream; - /** - * Created by yasuhiro on 7/2/15. + * 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.stream; + public class KStreamException extends RuntimeException { - public KStreamException() { - super(); - } + public KStreamException() { + super(); + } - public KStreamException(String msg) { - super(msg); - } + public KStreamException(String msg) { + super(msg); + } - public KStreamException(Throwable t) { - super(t); - } + public KStreamException(Throwable t) { + super(t); + } - public KStreamException(String msg, Throwable t) { - super(msg, t); - } + public KStreamException(String msg, Throwable t) { + super(msg, t); + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java index bbe5a6fc01d86..f8e93295ce8bb 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java +++ b/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java @@ -5,9 +5,9 @@ * 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 - * + *

+ * 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. @@ -58,9 +58,9 @@ ======= >>>>>>> removing io.confluent imports: wip import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.stream.internal.ProcessorConfig; +import org.apache.kafka.stream.internals.ProcessorConfig; import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.internal.KStreamThread; +import org.apache.kafka.stream.topology.internals.KStreamThread; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -137,11 +137,18 @@ public class KafkaStreaming implements Runnable { // ======= // Container States +<<<<<<< HEAD >>>>>>> fixed comments private final int CREATED = 0; private final int RUNNING = 1; private final int STOPPING = 2; private final int STOPPED = 3; +======= + private static final int CREATED = 0; + private static final int RUNNING = 1; + private static final int STOPPING = 2; + private static final int STOPPED = 3; +>>>>>>> compile and test passed private int state = CREATED; >>>>>>> removed Coordinator @@ -202,7 +209,8 @@ public KafkaStreaming(KStreamTopology topology, StreamingConfig streamingConfig) public KafkaStreaming(KStreamTopology topology, StreamingConfig streamingConfig) { >>>>>>> wip - if (streamingConfig.timestampExtractor() == null) throw new NullPointerException("timestamp extractor is missing"); + if (streamingConfig.timestampExtractor() == null) + throw new NullPointerException("timestamp extractor is missing"); this.config = new ProcessorConfig(streamingConfig.config()); <<<<<<< HEAD @@ -400,8 +408,7 @@ private void commitAll(long now) { >>>>>>> removed Coordinator try { lock.wait(); - } - catch (InterruptedException ex) { + } catch (InterruptedException ex) { Thread.interrupted(); } >>>>>>> added KStreamThread @@ -476,8 +483,7 @@ public void close() { while (state == STOPPING) { try { lock.wait(); - } - catch (InterruptedException ex) { + } catch (InterruptedException ex) { Thread.interrupted(); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java b/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java deleted file mode 100644 index 1786535e1b717..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java +++ /dev/null @@ -1,12 +0,0 @@ -package org.apache.kafka.stream; - -/** - * Created by yasuhiro on 6/19/15. - */ -public class NotCopartitionedException extends KStreamException { - - public NotCopartitionedException() { - super(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java b/stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java index 08355c4042283..fca6775d9ab3e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java +++ b/stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java @@ -5,9 +5,9 @@ * 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 - * + *

+ * 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. diff --git a/stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java b/stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java index 2f43758aa6c2f..2e076ce76310a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java +++ b/stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java @@ -1,3 +1,20 @@ +/** + * 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.stream; /** @@ -5,13 +22,13 @@ */ public interface TimestampExtractor { - /** - * Extracts a timestamp from a key-value pair from a topic - * @param topic the topic name - * @param key the key object - * @param value the value object - * @return timestamp - */ - long extract(String topic, Object key, Object value); - + /** + * Extracts a timestamp from a key-value pair from a topic + * + * @param topic the topic name + * @param key the key object + * @param value the value object + * @return timestamp + */ + long extract(String topic, Object key, Object value); } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java index 2cf919c1bbb12..801f127f33ea0 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java @@ -1,3 +1,20 @@ +/** + * 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.stream.examples; <<<<<<< HEAD:src/main/java/io/confluent/streaming/examples/MapKStreamJob.java @@ -33,11 +50,9 @@ import java.util.Properties; -/** - * Created by guozhang on 7/14/15. - */ public class MapKStreamJob extends KStreamTopology { +<<<<<<< HEAD @SuppressWarnings("unchecked") @Override <<<<<<< HEAD @@ -49,46 +64,51 @@ public void init(KStreamInitializer context) { ======= public void topology() { >>>>>>> wip +======= + @SuppressWarnings("unchecked") + @Override + public void topology() { +>>>>>>> compile and test passed - // With overriden de-serializer - KStream stream1 = from(new StringDeserializer(), new StringDeserializer(), "topic1"); + // With overriden de-serializer + KStream stream1 = from(new StringDeserializer(), new StringDeserializer(), "topic1"); - stream1.map(new KeyValueMapper() { - @Override - public KeyValue apply(String key, String value) { - return new KeyValue<>(key, new Integer(value)); - } - }).filter(new Predicate() { - @Override - public boolean apply(String key, Integer value) { - return true; - } - }).sendTo("topic2"); + stream1.map(new KeyValueMapper() { + @Override + public KeyValue apply(String key, String value) { + return new KeyValue<>(key, new Integer(value)); + } + }).filter(new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + }).sendTo("topic2"); - // Without overriden de-serialzier - KStream stream2 = (KStream)from("topic2"); + // Without overriden de-serialzier + KStream stream2 = (KStream) from("topic2"); - KStream[] streams = stream2.branch( - new Predicate() { - @Override - public boolean apply(String key, Integer value) { - return true; - } - }, - new Predicate() { - @Override - public boolean apply(String key, Integer value) { - return true; - } - } - ); + KStream[] streams = stream2.branch( + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + }, + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + } + ); - streams[0].sendTo("topic3"); - streams[1].sendTo("topic4"); - } + streams[0].sendTo("topic3"); + streams[1].sendTo("topic4"); + } - public static void main(String[] args) { - KafkaStreaming kstream = new KafkaStreaming(new MapKStreamJob(), new StreamingConfig(new Properties())); - kstream.run(); - } + public static void main(String[] args) { + KafkaStreaming kstream = new KafkaStreaming(new MapKStreamJob(), new StreamingConfig(new Properties())); + kstream.run(); + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java index ce7027a618caa..5d272bd4e72f7 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java @@ -1,3 +1,20 @@ +/** + * 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.stream.examples; import org.apache.kafka.stream.topology.Processor; @@ -8,37 +25,35 @@ import java.util.Properties; -/** - * Created by guozhang on 7/14/15. - */ public class PrintKStreamJob implements Processor { - private KStreamContext context; + private KStreamContext context; - @Override - public void init(KStreamContext context) { - this.context = context; - } + @Override + public void init(KStreamContext context) { + this.context = context; + } - @Override - public void process(K key, V value) { - System.out.println("[" + key + ", " + value + "]"); + @Override + public void process(K key, V value) { + System.out.println("[" + key + ", " + value + "]"); - context.commit(); + context.commit(); - context.send("topic", key, value); - } + context.send("topic", key, value); + } - @Override - public void punctuate(long streamTime) { - // do nothing - } + @Override + public void punctuate(long streamTime) { + // do nothing + } - @Override - public void close() { - // do nothing - } + @Override + public void close() { + // do nothing + } +<<<<<<< HEAD public static void main(String[] args) { <<<<<<< HEAD <<<<<<< HEAD @@ -61,4 +76,13 @@ public static void main(String[] args) { ======= >>>>>>> fix examples } +======= + public static void main(String[] args) { + KafkaStreaming streaming = new KafkaStreaming( + new SingleProcessorTopology(PrintKStreamJob.class, args), + new StreamingConfig(new Properties()) + ); + streaming.run(); + } +>>>>>>> compile and test passed } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java index 21714e0e6b644..3f3d000c13285 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java @@ -1,32 +1,40 @@ -package org.apache.kafka.stream.examples; +/** + * 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.stream.examples; import org.apache.kafka.stream.topology.Processor; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KafkaStreaming; import org.apache.kafka.stream.StreamingConfig; -import org.apache.kafka.stream.kv.Entry; -import org.apache.kafka.stream.kv.InMemoryKeyValueStore; -import org.apache.kafka.stream.kv.KeyValueIterator; -import org.apache.kafka.stream.kv.KeyValueStore; +import org.apache.kafka.stream.state.Entry; +import org.apache.kafka.stream.state.InMemoryKeyValueStore; +import org.apache.kafka.stream.state.KeyValueIterator; +import org.apache.kafka.stream.state.KeyValueStore; import org.apache.kafka.stream.topology.SingleProcessorTopology; import java.util.Properties; -/** - * Created by guozhang on 7/27/15. - */ - public class StatefulKStreamJob implements Processor { - private KStreamContext context; - private KeyValueStore kvStore; - - @Override - public void init(KStreamContext context) { - this.context = context; - this.context.schedule(this, 1000); + private KStreamContext context; + private KeyValueStore kvStore; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD this.kvStore = new InMemoryKeyValueStore<>("local-state", context); @@ -46,34 +54,44 @@ public void init(KStreamContext context) { ======= >>>>>>> remove restore function } +======= + @Override + public void init(KStreamContext context) { + this.context = context; + this.context.schedule(this, 1000); +>>>>>>> compile and test passed - @Override - public void process(String key, Integer value) { - Integer oldValue = this.kvStore.get(key); - if (oldValue == null) { - this.kvStore.put(key, value); - } else { - int newValue = oldValue + value; - this.kvStore.put(key, newValue); + this.kvStore = new InMemoryKeyValueStore<>("local-state", context); } - context.commit(); - } + @Override + public void process(String key, Integer value) { + Integer oldValue = this.kvStore.get(key); + if (oldValue == null) { + this.kvStore.put(key, value); + } else { + int newValue = oldValue + value; + this.kvStore.put(key, newValue); + } - @Override - public void punctuate(long streamTime) { - KeyValueIterator iter = this.kvStore.all(); - while (iter.hasNext()) { - Entry entry = iter.next(); - System.out.println("[" + entry.key() + ", " + entry.value() + "]"); + context.commit(); } - } - @Override - public void close() { - // do nothing - } + @Override + public void punctuate(long streamTime) { + KeyValueIterator iter = this.kvStore.all(); + while (iter.hasNext()) { + Entry entry = iter.next(); + System.out.println("[" + entry.key() + ", " + entry.value() + "]"); + } + } + @Override + public void close() { + // do nothing + } + +<<<<<<< HEAD public static void main(String[] args) { <<<<<<< HEAD <<<<<<< HEAD @@ -96,4 +114,13 @@ public static void main(String[] args) { ======= >>>>>>> fix examples } +======= + public static void main(String[] args) { + KafkaStreaming streaming = new KafkaStreaming( + new SingleProcessorTopology(StatefulKStreamJob.class, args), + new StreamingConfig(new Properties()) + ); + streaming.run(); + } +>>>>>>> compile and test passed } diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/Ingestor.java b/stream/src/main/java/org/apache/kafka/stream/internal/Ingestor.java deleted file mode 100644 index eeb793f6ddf22..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/Ingestor.java +++ /dev/null @@ -1,27 +0,0 @@ -package org.apache.kafka.stream.internal; - -import org.apache.kafka.common.TopicPartition; - -import java.util.Map; -import java.util.Set; - -/** - * Created by yasuhiro on 6/30/15. - */ -public interface Ingestor { - - Set topics(); - - void poll(long timeoutMs); - - void pause(TopicPartition partition); - - void unpause(TopicPartition partition, long offset); - - void commit(Map offsets); - - int numPartitions(String topic); - - void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition); - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java deleted file mode 100644 index 47bb3e298cdd5..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/IngestorImpl.java +++ /dev/null @@ -1,117 +0,0 @@ -package org.apache.kafka.stream.internal; - -import org.apache.kafka.clients.consumer.CommitType; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.common.TopicPartition; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - - -public class IngestorImpl implements Ingestor { - - private static final Logger log = LoggerFactory.getLogger(IngestorImpl.class); - - private final Set topics; - private final Consumer consumer; - private final Set unpaused = new HashSet<>(); - private final Map streamSynchronizers = new HashMap<>(); - - public IngestorImpl(Consumer consumer, Set topics) { - this.consumer = consumer; - this.topics = Collections.unmodifiableSet(topics); - for (String topic : this.topics) consumer.subscribe(topic); - } - - public void open() { - for (String topic : this.topics) consumer.subscribe(topic); - } - - public void init() { - unpaused.clear(); - unpaused.addAll(consumer.subscriptions()); - } - - @Override - public Set topics() { - return topics; - } - - @Override - public void poll(long timeoutMs) { - synchronized (this) { - ConsumerRecords records = consumer.poll(timeoutMs); - - for (TopicPartition partition : unpaused) { - StreamGroup streamGroup = streamSynchronizers.get(partition); - - if (streamGroup != null) - streamGroup.addRecords(partition, records.records(partition).iterator()); - else - log.warn("unused topic: " + partition.topic()); - } - } - } - - @Override - public void pause(TopicPartition partition) { - synchronized (this) { - consumer.seek(partition, Long.MAX_VALUE); // hack: stop consuming from this partition by setting a big offset - unpaused.remove(partition); - } - } - - @Override - public void unpause(TopicPartition partition, long lastOffset) { - synchronized (this) { - consumer.seek(partition, lastOffset); - unpaused.add(partition); - } - } - - @Override - public void commit(Map offsets) { - synchronized (this) { - consumer.commit(offsets, CommitType.SYNC); - } - } - - @Override - public int numPartitions(String topic) { - return consumer.partitionsFor(topic).size(); - } - - @SuppressWarnings("unchecked") - @Override - public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition) { - synchronized (this) { - streamSynchronizers.put(partition, streamGroup); - unpaused.add(partition); - } - } - - public void clear() { - unpaused.clear(); - streamSynchronizers.clear(); - } - - public boolean commitNeeded(Map offsets) { - for (TopicPartition tp : offsets.keySet()) { - if (consumer.committed(tp) != offsets.get(tp)) { - return true; - } - } - return false; - } - - public void close() { - consumer.close(); - clear(); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java b/stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java deleted file mode 100644 index 40621295582c2..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/PartitioningInfo.java +++ /dev/null @@ -1,13 +0,0 @@ -package org.apache.kafka.stream.internal; - -/** - * Created by yasuhiro on 6/19/15. - */ -public class PartitioningInfo { - - public final int numPartitions; - - public PartitioningInfo(int numPartitions) { - this.numPartitions = numPartitions; - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java deleted file mode 100644 index b2717a76ec9d8..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorConfig.java +++ /dev/null @@ -1,104 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internal; - -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 org.apache.kafka.stream.StreamingConfig; - -import java.io.File; -import java.util.Properties; - -public class ProcessorConfig extends AbstractConfig { - - private static final ConfigDef CONFIG; - - static { - CONFIG = new ConfigDef().define(StreamingConfig.TOPICS_CONFIG, - Type.STRING, - "", - Importance.HIGH, - "All the possible topic names this job need to interact with") - .define(StreamingConfig.STATE_DIR_CONFIG, - Type.STRING, - System.getProperty("java.io.tmpdir"), - Importance.MEDIUM, - "") - .define(StreamingConfig.POLL_TIME_MS_CONFIG, - Type.LONG, - 100, - Importance.LOW, - "The amount of time to block waiting for input.") - .define(StreamingConfig.COMMIT_TIME_MS_CONFIG, - Type.LONG, - 30000, - Importance.HIGH, - "The frequency with which to save the position of the processor.") - .define(StreamingConfig.WINDOW_TIME_MS_CONFIG, - Type.LONG, - -1L, - Importance.MEDIUM, - "Setting this to a non-negative value will cause the processor to get called with this frequency even if there is no message.") - .define(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, - Type.INT, - 1000, - Importance.LOW, - "The maximum number of records to buffer per partition") - .define(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG, - Type.LONG, - 60000, - Importance.LOW, - "The amount of time to wait before deleting state when a partition has migrated.") - .define(StreamingConfig.TOTAL_RECORDS_TO_PROCESS, - Type.LONG, - -1L, - Importance.LOW, - "Exit after processing this many records.") - .define(StreamingConfig.NUM_STREAM_THREADS, - Type.INT, - 1, - Importance.LOW, - "The number of threads to execute stream processing."); - } - - public final String topics; - public final File stateDir; - public final long pollTimeMs; - public final long commitTimeMs; - public final long windowTimeMs; - public final int bufferedRecordsPerPartition; - public final long stateCleanupDelay; - public final long totalRecordsToProcess; - public final int numStreamThreads; - - public ProcessorConfig(Properties processor) { - super(CONFIG, processor); - this.topics = this.getString(StreamingConfig.TOPICS_CONFIG); - this.stateDir = new File(this.getString(StreamingConfig.STATE_DIR_CONFIG)); - this.pollTimeMs = this.getLong(StreamingConfig.POLL_TIME_MS_CONFIG); - this.commitTimeMs = this.getLong(StreamingConfig.COMMIT_TIME_MS_CONFIG); - this.windowTimeMs = this.getLong(StreamingConfig.WINDOW_TIME_MS_CONFIG); - this.bufferedRecordsPerPartition = this.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); - this.stateCleanupDelay = this.getLong(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG); - this.totalRecordsToProcess = this.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); - this.numStreamThreads = this.getInt(StreamingConfig.NUM_STREAM_THREADS); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java deleted file mode 100644 index 2bf3ebacc77d9..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationQueue.java +++ /dev/null @@ -1,38 +0,0 @@ -package org.apache.kafka.stream.internal; - -import java.util.PriorityQueue; - -/** - * Created by yasuhiro on 6/29/15. - */ -public class PunctuationQueue { - - private PriorityQueue pq = new PriorityQueue<>(); - - public void schedule(PunctuationSchedule sched) { - synchronized (pq) { - pq.add(sched); - } - } - - public void close() { - synchronized (pq) { - pq.clear(); - } - } - - public void mayPunctuate(long streamTime) { - synchronized (pq) { - PunctuationSchedule top = pq.peek(); - while (top != null && top.timestamp <= streamTime) { - PunctuationSchedule sched = top; - pq.poll(); - sched.processor().punctuate(streamTime); - pq.add(sched.next()); - - top = pq.peek(); - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java deleted file mode 100644 index 9868865f162c9..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/PunctuationSchedule.java +++ /dev/null @@ -1,24 +0,0 @@ -package org.apache.kafka.stream.internal; - - -import org.apache.kafka.stream.topology.Processor; -import org.apache.kafka.stream.util.Stamped; - -public class PunctuationSchedule extends Stamped> { - - final long interval; - - public PunctuationSchedule(Processor processor, long interval) { - super(processor, System.currentTimeMillis() + interval); - this.interval = interval; - } - - public Processor processor() { - return value; - } - - public PunctuationSchedule next() { - return new PunctuationSchedule(value, timestamp + interval); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java deleted file mode 100644 index b3cf6796d96a4..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/RecordCollectorImpl.java +++ /dev/null @@ -1,77 +0,0 @@ -package org.apache.kafka.stream.internal; - -import org.apache.kafka.stream.RecordCollector; -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; - -/** - * Created by yasuhiro on 6/26/15. - */ -public class RecordCollectorImpl implements RecordCollector { - - private static final Logger log = LoggerFactory.getLogger(RecordCollectorImpl.class); - - private final Producer producer; - private final Map offsets; - private final Callback callback = new Callback(){ - public void onCompletion(RecordMetadata metadata, Exception exception) { - if(exception == null) { - TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); - offsets.put(tp, metadata.offset()); - } else { - log.error("Error sending record: ", exception); - } - } - }; - private final Serializer keySerializer; - private final Serializer valueSerializer; - - - public RecordCollectorImpl(Producer producer, Serializer keySerializer, Serializer valueSerializer) { - this.producer = producer; - this.offsets = new HashMap<>(); - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - } - - @Override - public void send(ProducerRecord record) { - send(record, this.keySerializer, this.valueSerializer); - } - - @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { - byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); - byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); - this.producer.send(new ProducerRecord<>(record.topic(), keyBytes, valBytes), callback); - } - - @Override - public void flush() { - this.producer.flush(); - } - - /** - * Closes this RecordCollector - */ - public void close() { - producer.close(); - } - - /** - * The last ack'd offset from the producer - * @return the map from TopicPartition to offset - */ - Map offsets() { - return this.offsets; - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java deleted file mode 100644 index a702a662391ad..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/RoundRobinChooser.java +++ /dev/null @@ -1,33 +0,0 @@ -package org.apache.kafka.stream.internal; - -import org.apache.kafka.stream.Chooser; - -import java.util.ArrayDeque; - -/** - * Created by yasuhiro on 6/25/15. - */ -public class RoundRobinChooser implements Chooser { - - private final ArrayDeque deque; - - public RoundRobinChooser() { - deque = new ArrayDeque<>(); - } - - @Override - public void add(RecordQueue queue) { - deque.offer(queue); - } - - @Override - public RecordQueue next() { - return deque.poll(); - } - - @Override - public void close() { - deque.clear(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java b/stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java deleted file mode 100644 index 9dd77b6e834e3..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/StampedRecord.java +++ /dev/null @@ -1,29 +0,0 @@ -package org.apache.kafka.stream.internal; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.stream.util.Stamped; - - -// TODO: making this class exposed to user in the lower-level Processor -public class StampedRecord extends Stamped> { - - StampedRecord(ConsumerRecord record, long timestamp) { - super(record, timestamp); - } - - public String topic() { return value.topic(); } - - public int partition() { return value.partition(); } - - public Object key() { - return value.key(); - } - - public Object value() { - return value.value(); - } - - public long offset() { - return value.offset(); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java deleted file mode 100644 index b370c9e5f0f9e..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/TimeBasedChooser.java +++ /dev/null @@ -1,47 +0,0 @@ -package org.apache.kafka.stream.internal; - -import org.apache.kafka.stream.Chooser; - -import java.util.Comparator; -import java.util.PriorityQueue; - -/** - * Created by yasuhiro on 6/25/15. - */ -public class TimeBasedChooser implements Chooser { - - private final PriorityQueue pq; - - public TimeBasedChooser() { - this(new Comparator() { - public int compare(RecordQueue queue1, RecordQueue queue2) { - long time1 = queue1.trackedTimestamp(); - long time2 = queue2.trackedTimestamp(); - - if (time1 < time2) return -1; - if (time1 > time2) return 1; - return 0; - } - }); - } - - private TimeBasedChooser(Comparator comparator) { - pq = new PriorityQueue<>(3, comparator); - } - - @Override - public void add(RecordQueue queue) { - pq.offer(queue); - } - - @Override - public RecordQueue next() { - return pq.poll(); - } - - @Override - public void close() { - pq.clear(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/Ingestor.java b/stream/src/main/java/org/apache/kafka/stream/internals/Ingestor.java new file mode 100644 index 0000000000000..0d9154c7899f1 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/Ingestor.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; +import java.util.Set; + +public interface Ingestor { + + Set topics(); + + void poll(long timeoutMs); + + void pause(TopicPartition partition); + + void unpause(TopicPartition partition, long offset); + + void commit(Map offsets); + + int numPartitions(String topic); + + void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition); + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/IngestorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/IngestorImpl.java new file mode 100644 index 0000000000000..878f6fe91f691 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/IngestorImpl.java @@ -0,0 +1,133 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.consumer.CommitType; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +public class IngestorImpl implements Ingestor { + + private static final Logger log = LoggerFactory.getLogger(IngestorImpl.class); + + private final Set topics; + private final Consumer consumer; + private final Set unpaused = new HashSet<>(); + private final Map streamSynchronizers = new HashMap<>(); + + public IngestorImpl(Consumer consumer, Set topics) { + this.consumer = consumer; + this.topics = Collections.unmodifiableSet(topics); + for (String topic : this.topics) consumer.subscribe(topic); + } + + public void open() { + for (String topic : this.topics) consumer.subscribe(topic); + } + + public void init() { + unpaused.clear(); + unpaused.addAll(consumer.subscriptions()); + } + + @Override + public Set topics() { + return topics; + } + + @Override + public void poll(long timeoutMs) { + synchronized (this) { + ConsumerRecords records = consumer.poll(timeoutMs); + + for (TopicPartition partition : unpaused) { + StreamGroup streamGroup = streamSynchronizers.get(partition); + + if (streamGroup != null) + streamGroup.addRecords(partition, records.records(partition).iterator()); + else + log.warn("unused topic: " + partition.topic()); + } + } + } + + @Override + public void pause(TopicPartition partition) { + synchronized (this) { + consumer.seek(partition, Long.MAX_VALUE); // hack: stop consuming from this partition by setting a big offset + unpaused.remove(partition); + } + } + + @Override + public void unpause(TopicPartition partition, long lastOffset) { + synchronized (this) { + consumer.seek(partition, lastOffset); + unpaused.add(partition); + } + } + + @Override + public void commit(Map offsets) { + synchronized (this) { + consumer.commit(offsets, CommitType.SYNC); + } + } + + @Override + public int numPartitions(String topic) { + return consumer.partitionsFor(topic).size(); + } + + @SuppressWarnings("unchecked") + @Override + public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition) { + synchronized (this) { + streamSynchronizers.put(partition, streamGroup); + unpaused.add(partition); + } + } + + public void clear() { + unpaused.clear(); + streamSynchronizers.clear(); + } + + public boolean commitNeeded(Map offsets) { + for (TopicPartition tp : offsets.keySet()) { + if (consumer.committed(tp) != offsets.get(tp)) { + return true; + } + } + return false; + } + + public void close() { + consumer.close(); + clear(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/PartitioningInfo.java b/stream/src/main/java/org/apache/kafka/stream/internals/PartitioningInfo.java new file mode 100644 index 0000000000000..7840dab82a930 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/PartitioningInfo.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +public class PartitioningInfo { + + public final int numPartitions; + + public PartitioningInfo(int numPartitions) { + this.numPartitions = numPartitions; + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorConfig.java new file mode 100644 index 0000000000000..b34e17f67d537 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorConfig.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +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 org.apache.kafka.stream.StreamingConfig; + +import java.io.File; +import java.util.Properties; + +public class ProcessorConfig extends AbstractConfig { + + private static final ConfigDef CONFIG; + + static { + CONFIG = new ConfigDef().define(StreamingConfig.TOPICS_CONFIG, + Type.STRING, + "", + Importance.HIGH, + "All the possible topic names this job need to interact with") + .define(StreamingConfig.STATE_DIR_CONFIG, + Type.STRING, + System.getProperty("java.io.tmpdir"), + Importance.MEDIUM, + "") + .define(StreamingConfig.POLL_TIME_MS_CONFIG, + Type.LONG, + 100, + Importance.LOW, + "The amount of time to block waiting for input.") + .define(StreamingConfig.COMMIT_TIME_MS_CONFIG, + Type.LONG, + 30000, + Importance.HIGH, + "The frequency with which to save the position of the processor.") + .define(StreamingConfig.WINDOW_TIME_MS_CONFIG, + Type.LONG, + -1L, + Importance.MEDIUM, + "Setting this to a non-negative value will cause the processor to get called with this frequency even if there is no message.") + .define(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + 1000, + Importance.LOW, + "The maximum number of records to buffer per partition") + .define(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG, + Type.LONG, + 60000, + Importance.LOW, + "The amount of time to wait before deleting state when a partition has migrated.") + .define(StreamingConfig.TOTAL_RECORDS_TO_PROCESS, + Type.LONG, + -1L, + Importance.LOW, + "Exit after processing this many records.") + .define(StreamingConfig.NUM_STREAM_THREADS, + Type.INT, + 1, + Importance.LOW, + "The number of threads to execute stream processing."); + } + + public final String topics; + public final File stateDir; + public final long pollTimeMs; + public final long commitTimeMs; + public final long windowTimeMs; + public final int bufferedRecordsPerPartition; + public final long stateCleanupDelay; + public final long totalRecordsToProcess; + public final int numStreamThreads; + + public ProcessorConfig(Properties processor) { + super(CONFIG, processor); + this.topics = this.getString(StreamingConfig.TOPICS_CONFIG); + this.stateDir = new File(this.getString(StreamingConfig.STATE_DIR_CONFIG)); + this.pollTimeMs = this.getLong(StreamingConfig.POLL_TIME_MS_CONFIG); + this.commitTimeMs = this.getLong(StreamingConfig.COMMIT_TIME_MS_CONFIG); + this.windowTimeMs = this.getLong(StreamingConfig.WINDOW_TIME_MS_CONFIG); + this.bufferedRecordsPerPartition = this.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + this.stateCleanupDelay = this.getLong(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG); + this.totalRecordsToProcess = this.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); + this.numStreamThreads = this.getInt(StreamingConfig.NUM_STREAM_THREADS); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorNode.java new file mode 100644 index 0000000000000..600f2938ab2eb --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorNode.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; + +public class ProcessorNode implements Receiver { + + private final Processor processor; + private KStreamContext context; + + public ProcessorNode(Processor processor) { + this.processor = processor; + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); + + this.context = context; + processor.init(context); + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + processor.process((K) key, (V) value); + } + + @Override + public void close() { + processor.close(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorStateManager.java similarity index 94% rename from stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java rename to stream/src/main/java/org/apache/kafka/stream/internals/ProcessorStateManager.java index 411f3faa7ec99..265ab778c6385 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorStateManager.java +++ b/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorStateManager.java @@ -5,16 +5,17 @@ * 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 - * + *

+ * 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.stream.internal; + +package org.apache.kafka.stream.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -72,7 +73,7 @@ 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())) + if (this.stores.containsKey(store.name())) throw new IllegalArgumentException("Store " + store.name() + " has already been registered."); // ---- register the store ---- // @@ -121,7 +122,7 @@ public void register(StateStore store, RestoreFunc restoreFunc) { // restore its state from changelog records; while restoring the log end offset // should not change since it is only written by this thread. while (true) { - for(ConsumerRecord record: restoreConsumer.poll(100)) { + for (ConsumerRecord record : restoreConsumer.poll(100)) { restoreFunc.apply(record.key(), record.value()); } @@ -142,8 +143,8 @@ public void register(StateStore store, RestoreFunc restoreFunc) { 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())) { + for (File file : this.baseDir.listFiles()) { + if (!this.stores.containsKey(file.getName())) { log.info("Deleting state directory {}", file.getAbsolutePath()); file.delete(); } @@ -151,7 +152,7 @@ public void cleanup() throws IOException { } public void flush() { - if(!this.stores.isEmpty()) { + if (!this.stores.isEmpty()) { log.debug("Flushing stores."); for (StateStore engine : this.stores.values()) engine.flush(); @@ -159,7 +160,7 @@ public void flush() { } public void close(Map ackedOffsets) throws IOException { - if(!stores.isEmpty()) { + if (!stores.isEmpty()) { log.debug("Closing stores."); for (Map.Entry entry : stores.entrySet()) { log.debug("Closing storage engine {}", entry.getKey()); @@ -168,7 +169,7 @@ public void close(Map ackedOffsets) throws IOException { } Map checkpointOffsets = new HashMap(restoredOffsets); - for(String storeName: stores.keySet()) { + for (String storeName : stores.keySet()) { TopicPartition part = new TopicPartition(storeName, id); // only checkpoint the offset to the offsets file if it is persistent; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationQueue.java b/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationQueue.java new file mode 100644 index 0000000000000..6e2b53f326518 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationQueue.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import java.util.PriorityQueue; + +public class PunctuationQueue { + + private PriorityQueue pq = new PriorityQueue<>(); + + public void schedule(PunctuationSchedule sched) { + synchronized (pq) { + pq.add(sched); + } + } + + public void close() { + synchronized (pq) { + pq.clear(); + } + } + + public void mayPunctuate(long streamTime) { + synchronized (pq) { + PunctuationSchedule top = pq.peek(); + while (top != null && top.timestamp <= streamTime) { + PunctuationSchedule sched = top; + pq.poll(); + sched.processor().punctuate(streamTime); + pq.add(sched.next()); + + top = pq.peek(); + } + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationSchedule.java new file mode 100644 index 0000000000000..8dcfc1e65c363 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationSchedule.java @@ -0,0 +1,40 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.util.Stamped; + +public class PunctuationSchedule extends Stamped> { + + final long interval; + + public PunctuationSchedule(Processor processor, long interval) { + super(processor, System.currentTimeMillis() + interval); + this.interval = interval; + } + + public Processor processor() { + return value; + } + + public PunctuationSchedule next() { + return new PunctuationSchedule(value, timestamp + interval); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/Receiver.java b/stream/src/main/java/org/apache/kafka/stream/internals/Receiver.java new file mode 100644 index 0000000000000..ef933bd5ad4b6 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/Receiver.java @@ -0,0 +1,31 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; + +public interface Receiver { + + void bind(KStreamContext context, KStreamMetadata metadata); + + void receive(Object key, Object value, long timestamp); + + void close(); + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/RecordCollectorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/RecordCollectorImpl.java new file mode 100644 index 0000000000000..6948d7b2f8154 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/RecordCollectorImpl.java @@ -0,0 +1,92 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.stream.RecordCollector; +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 RecordCollectorImpl implements RecordCollector { + + private static final Logger log = LoggerFactory.getLogger(RecordCollectorImpl.class); + + private final Producer producer; + private final Map offsets; + private final Callback callback = new Callback() { + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception == null) { + TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); + offsets.put(tp, metadata.offset()); + } else { + log.error("Error sending record: ", exception); + } + } + }; + private final Serializer keySerializer; + private final Serializer valueSerializer; + + + public RecordCollectorImpl(Producer producer, Serializer keySerializer, Serializer valueSerializer) { + this.producer = producer; + this.offsets = new HashMap<>(); + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + } + + @Override + public void send(ProducerRecord record) { + send(record, this.keySerializer, this.valueSerializer); + } + + @Override + public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); + byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); + this.producer.send(new ProducerRecord<>(record.topic(), keyBytes, valBytes), callback); + } + + @Override + public void flush() { + this.producer.flush(); + } + + /** + * Closes this RecordCollector + */ + public void close() { + producer.close(); + } + + /** + * The last ack'd offset from the producer + * + * @return the map from TopicPartition to offset + */ + Map offsets() { + return this.offsets; + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/stream/internals/RecordQueue.java new file mode 100644 index 0000000000000..4db0448bbdf89 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/RecordQueue.java @@ -0,0 +1,123 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.util.TimestampTracker; + +import java.util.ArrayDeque; + +/** + * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). It is intended to be used in + * {@link StreamGroup}. + */ +public class RecordQueue { + + private final ArrayDeque queue = new ArrayDeque<>(); + public final KStreamSource stream; + private final TopicPartition partition; + private TimestampTracker> timestampTracker; + private long offset; + + /** + * Creates a new instance of RecordQueue + * + * @param partition partition + * @param stream the instance of KStreamImpl that receives records + * @param timestampTracker TimestampTracker + */ + public RecordQueue(TopicPartition partition, KStreamSource stream, TimestampTracker> timestampTracker) { + this.partition = partition; + this.stream = stream; + this.timestampTracker = timestampTracker; + } + + /** + * Returns the partition with which this queue is associated + * + * @return TopicPartition + */ + public TopicPartition partition() { + return partition; + } + + /** + * Adds a StampedRecord to the queue + * + * @param record StampedRecord + */ + public void add(StampedRecord record) { + queue.addLast(record); + timestampTracker.addStampedElement(record); + offset = record.offset(); + } + + /** + * Returns the next record fro the queue + * + * @return StampedRecord + */ + public StampedRecord next() { + StampedRecord elem = queue.pollFirst(); + + if (elem == null) return null; + + timestampTracker.removeStampedElement(elem); + + return elem; + } + + /** + * Returns the highest offset in the queue + * + * @return offset + */ + public long offset() { + return offset; + } + + /** + * Returns the number of records in the queue + * + * @return the number of records + */ + public int size() { + return queue.size(); + } + + /** + * Tests if the queue is empty + * + * @return true if the queue is empty, otherwise false + */ + public boolean isEmpty() { + return queue.isEmpty(); + } + + /** + * Returns a timestamp tracked by the TimestampTracker + * + * @return timestamp + */ + public long trackedTimestamp() { + return timestampTracker.get(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/stream/internals/RoundRobinChooser.java new file mode 100644 index 0000000000000..bb8cb88973c2e --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/RoundRobinChooser.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import org.apache.kafka.stream.Chooser; + +import java.util.ArrayDeque; + +public class RoundRobinChooser implements Chooser { + + private final ArrayDeque deque; + + public RoundRobinChooser() { + deque = new ArrayDeque<>(); + } + + @Override + public void add(RecordQueue queue) { + deque.offer(queue); + } + + @Override + public RecordQueue next() { + return deque.poll(); + } + + @Override + public void close() { + deque.clear(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/StampedRecord.java b/stream/src/main/java/org/apache/kafka/stream/internals/StampedRecord.java new file mode 100644 index 0000000000000..9817021ce140c --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/StampedRecord.java @@ -0,0 +1,49 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.stream.util.Stamped; + +// TODO: making this class exposed to user in the lower-level Processor +public class StampedRecord extends Stamped> { + + StampedRecord(ConsumerRecord record, long timestamp) { + super(record, timestamp); + } + + public String topic() { + return value.topic(); + } + + public int partition() { + return value.partition(); + } + + public Object key() { + return value.key(); + } + + public Object value() { + return value.value(); + } + + public long offset() { + return value.offset(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java new file mode 100644 index 0000000000000..7138668ec5127 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java @@ -0,0 +1,283 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.Chooser; +import org.apache.kafka.stream.TimestampExtractor; +import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.util.MinTimestampTracker; +import org.apache.kafka.stream.util.ParallelExecutor; + +import java.util.ArrayDeque; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** + * A StreamGroup is composed of multiple streams from different topics that need to be synchronized. + */ +public class StreamGroup implements ParallelExecutor.Task { + + private final KStreamContext context; + private final Ingestor ingestor; + private final Chooser chooser; + private final TimestampExtractor timestampExtractor; + private final Map stash = new HashMap<>(); + + private final int desiredUnprocessed; + + // TODO: merge stash, consumedOffset, and newRecordBuffer into sth. like partition metadata + private final Map consumedOffsets; + private final PunctuationQueue punctuationQueue = new PunctuationQueue(); + private final ArrayDeque newRecordBuffer = new ArrayDeque<>(); + + private long streamTime = -1; + private boolean commitRequested = false; + private StampedRecord currRecord = null; + private volatile int buffered = 0; + + /** + * Creates StreamGroup + * + * @param context the task context + * @param ingestor the instance of {@link Ingestor} + * @param chooser the instance of {@link Chooser} + * @param timestampExtractor the instance of {@link TimestampExtractor} + * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic + */ + public StreamGroup(KStreamContext context, + Ingestor ingestor, + Chooser chooser, + TimestampExtractor timestampExtractor, + int desiredUnprocessedPerPartition) { + this.context = context; + this.ingestor = ingestor; + this.chooser = chooser; + this.timestampExtractor = timestampExtractor; + this.desiredUnprocessed = desiredUnprocessedPerPartition; + this.consumedOffsets = new HashMap<>(); + } + + public StampedRecord record() { + return currRecord; + } + + /** + * Merges a stream group into this group + */ + public void mergeStreamGroup(StreamGroup other) { + // check these groups have the same ingestor + if (ingestor != other.ingestor) + throw new IllegalArgumentException("groups with different ingestors cannot be merged"); + + // check these group have the same chooser and time extractor types + if (!this.chooser.getClass().equals(other.chooser.getClass())) + throw new IllegalArgumentException("groups with different type of choosers cannot be merged"); + + if (!this.timestampExtractor.getClass().equals(other.timestampExtractor.getClass())) + throw new IllegalArgumentException("groups with different type of time extractors cannot be merged"); + + // add all the other's groups partitions + this.stash.putAll(other.stash); + } + + /** + * Adds a partition and its receiver to this stream synchronizer + * + * @param partition the partition + * @param stream the instance of KStreamImpl + */ + @SuppressWarnings("unchecked") + public void addPartition(TopicPartition partition, KStreamSource stream) { + synchronized (this) { + RecordQueue recordQueue = stash.get(partition); + + if (recordQueue == null) { + stash.put(partition, createRecordQueue(partition, stream)); + } else { + throw new IllegalStateException("duplicate partition"); + } + } + } + + /** + * Adds records + * + * @param partition the partition + * @param iterator the iterator of records + */ + @SuppressWarnings("unchecked") + public void addRecords(TopicPartition partition, Iterator> iterator) { + synchronized (this) { + newRecordBuffer.addLast(new NewRecords(partition, iterator)); + } + } + + @SuppressWarnings("unchecked") + private void ingestNewRecords() { + for (NewRecords newRecords : newRecordBuffer) { + TopicPartition partition = newRecords.partition; + Iterator> iterator = newRecords.iterator; + + RecordQueue recordQueue = stash.get(partition); + if (recordQueue != null) { + boolean wasEmpty = recordQueue.isEmpty(); + + while (iterator.hasNext()) { + ConsumerRecord record = iterator.next(); + + // deserialize the raw record, extract the timestamp and put into the queue + Deserializer keyDeserializer = recordQueue.stream.keyDeserializer(); + Deserializer valDeserializer = recordQueue.stream.valueDeserializer(); + + Object key = keyDeserializer.deserialize(record.topic(), record.key()); + Object value = valDeserializer.deserialize(record.topic(), record.value()); + ConsumerRecord deserializedRecord = new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), key, value); + + long timestamp = timestampExtractor.extract(record.topic(), key, value); + recordQueue.add(new StampedRecord(deserializedRecord, timestamp)); + buffered++; + } + + int queueSize = recordQueue.size(); + if (wasEmpty && queueSize > 0) chooser.add(recordQueue); + + // if we have buffered enough for this partition, pause + if (queueSize >= this.desiredUnprocessed) { + ingestor.pause(partition); + } + } + } + newRecordBuffer.clear(); + } + + /** + * Schedules a punctuation for the processor + * + * @param processor the processor requesting scheduler + * @param interval the interval in milliseconds + */ + public void schedule(Processor processor, long interval) { + punctuationQueue.schedule(new PunctuationSchedule(processor, interval)); + } + + /** + * Processes one record + */ + @SuppressWarnings("unchecked") + @Override + public boolean process() { + synchronized (this) { + boolean readyForNextExecution = false; + ingestNewRecords(); + + RecordQueue recordQueue = chooser.next(); + if (recordQueue == null) { + return false; + } + + if (recordQueue.size() == 0) throw new IllegalStateException("empty record queue"); + + if (recordQueue.size() == this.desiredUnprocessed) { + ingestor.unpause(recordQueue.partition(), recordQueue.offset()); + } + + long trackedTimestamp = recordQueue.trackedTimestamp(); + currRecord = recordQueue.next(); + + if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; + + recordQueue.stream.receive(currRecord.key(), currRecord.value(), currRecord.timestamp); + consumedOffsets.put(recordQueue.partition(), currRecord.offset()); + + // TODO: local state flush and downstream producer flush + // need to be done altogether with offset commit atomically + if (commitRequested) { + // flush local state + context.flush(); + + // flush produced records in the downstream + context.recordCollector().flush(); + + // commit consumed offsets + ingestor.commit(consumedOffsets()); + } + + if (commitRequested) ingestor.commit(Collections.singletonMap( + new TopicPartition(currRecord.topic(), currRecord.partition()), + currRecord.offset())); + + if (recordQueue.size() > 0) { + readyForNextExecution = true; + chooser.add(recordQueue); + } + + buffered--; + currRecord = null; + + punctuationQueue.mayPunctuate(streamTime); + + return readyForNextExecution; + } + } + + /** + * Returns consumed offsets + * + * @return the map of partition to consumed offset + */ + public Map consumedOffsets() { + return this.consumedOffsets; + } + + /** + * Request committing the current record's offset + */ + public void commitOffset() { + this.commitRequested = true; + } + + public int buffered() { + return buffered; + } + + public void close() { + chooser.close(); + stash.clear(); + } + + protected RecordQueue createRecordQueue(TopicPartition partition, KStreamSource stream) { + return new RecordQueue(partition, stream, new MinTimestampTracker>()); + } + + private static class NewRecords { + final TopicPartition partition; + final Iterator> iterator; + + NewRecords(TopicPartition partition, Iterator> iterator) { + this.partition = partition; + this.iterator = iterator; + } + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/stream/internals/TimeBasedChooser.java new file mode 100644 index 0000000000000..58cf73384d31c --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/TimeBasedChooser.java @@ -0,0 +1,61 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.stream.Chooser; + +import java.util.Comparator; +import java.util.PriorityQueue; + +public class TimeBasedChooser implements Chooser { + + private final PriorityQueue pq; + + public TimeBasedChooser() { + this(new Comparator() { + public int compare(RecordQueue queue1, RecordQueue queue2) { + long time1 = queue1.trackedTimestamp(); + long time2 = queue2.trackedTimestamp(); + + if (time1 < time2) return -1; + if (time1 > time2) return 1; + return 0; + } + }); + } + + private TimeBasedChooser(Comparator comparator) { + pq = new PriorityQueue<>(3, comparator); + } + + @Override + public void add(RecordQueue queue) { + pq.offer(queue); + } + + @Override + public RecordQueue next() { + return pq.poll(); + } + + @Override + public void close() { + pq.clear(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/Entry.java b/stream/src/main/java/org/apache/kafka/stream/kv/Entry.java deleted file mode 100644 index efafa0c7bfb9a..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/kv/Entry.java +++ /dev/null @@ -1,28 +0,0 @@ -package org.apache.kafka.stream.kv; - -/** - * Created by yasuhiro on 6/26/15. - */ -public class Entry { - - private final K key; - private final V value; - - public Entry(K key, V value) { - this.key = key; - this.value = value; - } - - public K key() { - return key; - } - - public V value() { - return value; - } - - public String toString() { - return "Entry(" + key() + ", " + value() + ")"; - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/state/Entry.java b/stream/src/main/java/org/apache/kafka/stream/state/Entry.java new file mode 100644 index 0000000000000..c02ffac4a18d8 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/state/Entry.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.state; + +public class Entry { + + private final K key; + private final V value; + + public Entry(K key, V value) { + this.key = key; + this.value = value; + } + + public K key() { + return key; + } + + public V value() { + return value; + } + + public String toString() { + return "Entry(" + key() + ", " + value() + ")"; + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java similarity index 76% rename from stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java rename to stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java index fb3a790c9c20c..62323462dadf0 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/InMemoryKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java @@ -1,9 +1,26 @@ -package org.apache.kafka.stream.kv; +/** + * 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.stream.state; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.stream.kv.internal.MeteredKeyValueStore; +import org.apache.kafka.stream.state.internals.MeteredKeyValueStore; import java.util.Iterator; import java.util.List; @@ -47,7 +64,9 @@ public String name() { } @Override - public boolean persistent() { return false; } + public boolean persistent() { + return false; + } @Override public V get(K key) { @@ -119,7 +138,8 @@ public void remove() { } @Override - public void close() {} + public void close() { + } } } diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueIterator.java b/stream/src/main/java/org/apache/kafka/stream/state/KeyValueIterator.java similarity index 90% rename from stream/src/main/java/org/apache/kafka/stream/kv/KeyValueIterator.java rename to stream/src/main/java/org/apache/kafka/stream/state/KeyValueIterator.java index 8473096777556..50cc76dd15e2b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueIterator.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/KeyValueIterator.java @@ -6,9 +6,9 @@ * 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 - * + *

+ * 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 @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.kafka.stream.kv; +package org.apache.kafka.stream.state; import java.io.Closeable; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java rename to stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java index bf45a1ef88a52..b3739f6a46c3b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/KeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java @@ -6,9 +6,9 @@ * 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 - * + *

+ * 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 @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.kafka.stream.kv; +package org.apache.kafka.stream.state; import org.apache.kafka.stream.StateStore; diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java similarity index 89% rename from stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java rename to stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java index 35850ff9cec9e..155c39d084d4e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/RocksDBKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java @@ -1,11 +1,28 @@ -package org.apache.kafka.stream.kv; +/** + * 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.stream.state; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.stream.kv.internal.MeteredKeyValueStore; +import org.apache.kafka.stream.state.internals.MeteredKeyValueStore; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.CompactionStyle; import org.rocksdb.CompressionType; @@ -21,9 +38,6 @@ import java.util.List; import java.util.NoSuchElementException; -/** - * Created by guozhang on 7/30/15. - */ public class RocksDBKeyValueStore extends MeteredKeyValueStore { public RocksDBKeyValueStore(String name, KStreamContext context) { @@ -228,8 +242,8 @@ private static class LexicographicComparator implements Comparator { @Override public int compare(byte[] left, byte[] right) { for (int i = 0, j = 0; i < left.length && j < right.length; i++, j++) { - int leftByte = (left[i] & 0xff); - int rightByte = (right[j] & 0xff); + int leftByte = left[i] & 0xff; + int rightByte = right[j] & 0xff; if (leftByte != rightByte) { return leftByte - rightByte; } diff --git a/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java similarity index 84% rename from stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java rename to stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java index 7e3033a19d9c6..1cbdd4f1de1af 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kv/internal/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java @@ -1,4 +1,21 @@ -package org.apache.kafka.stream.kv.internal; +/** + * 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.stream.state.internals; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.RecordCollector; @@ -15,9 +32,9 @@ 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.stream.kv.Entry; -import org.apache.kafka.stream.kv.KeyValueIterator; -import org.apache.kafka.stream.kv.KeyValueStore; +import org.apache.kafka.stream.state.Entry; +import org.apache.kafka.stream.state.KeyValueIterator; +import org.apache.kafka.stream.state.KeyValueStore; import java.util.HashSet; import java.util.List; @@ -25,7 +42,7 @@ public class MeteredKeyValueStore implements KeyValueStore { - protected final KeyValueStore inner; + protected final KeyValueStore inner; private final Time time; private final String group; @@ -46,7 +63,7 @@ public class MeteredKeyValueStore implements KeyValueStore { private final KStreamContext context; // always wrap the logged store with the metered store - public MeteredKeyValueStore(final String name, final KeyValueStore inner, KStreamContext context, String group, Time time) { + public MeteredKeyValueStore(final String name, final KeyValueStore inner, KStreamContext context, String group, Time time) { this.inner = inner; this.time = time; @@ -102,7 +119,7 @@ private void addLatencyMetrics(Sensor sensor, String opName, String... kvs) { } private void maybeAddMetric(Sensor sensor, MetricName name, MeasurableStat stat) { - if(!metrics.metrics().containsKey(name)) + if (!metrics.metrics().containsKey(name)) sensor.add(name, stat); } @@ -173,16 +190,18 @@ public void delete(K key) { @Override public KeyValueIterator range(K from, K to) { - return new MeteredKeyValueIterator(this.inner.range(from, to), this.rangeTime); + return new MeteredKeyValueIterator(this.inner.range(from, to), this.rangeTime); } @Override public KeyValueIterator all() { - return new MeteredKeyValueIterator(this.inner.all(), this.allTime); + return new MeteredKeyValueIterator(this.inner.all(), this.allTime); } @Override - public void close() { inner.close(); } + public void close() { + inner.close(); + } @Override public void flush() { @@ -200,7 +219,7 @@ private void logChange() { Serializer keySerializer = (Serializer) context.keySerializer(); Serializer valueSerializer = (Serializer) context.valueSerializer(); - if(collector != null) { + 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); @@ -231,7 +250,7 @@ public boolean hasNext() { } @Override - public Entry next() { + public Entry next() { return iter.next(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java index c60404a88d89a..a19eec3bd272e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java @@ -1,8 +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.stream.topology; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamSource; import java.util.ArrayList; import java.util.Collection; @@ -11,96 +28,96 @@ /** * KStreamTopology is the class that allows an implementation of {@link KStreamTopology#topology()} to create KStream instances. - * */ public abstract class KStreamTopology { - private final ArrayList> streams = new ArrayList<>(); + private final ArrayList> streams = new ArrayList<>(); - /** - * Initializes a stream processing topology. This method may be called multiple times. - * An application constructs a processing logic using KStream API. - *

- * For example, - *

- *
-   *   KStreamTopology topology = new KStreamTopology() {
-   *     public void topology() {
-   *       KStream<Integer, PageView> pageViewStream = from("pageView").mapValues(...);
-   *       KStream<Integer, AdClick> adClickStream = from("adClick").join(pageViewStream, ...).process(...);
-   *     }
-   *   }
-   *
-   *   KafkaStreaming streaming = new KafkaStreaming(topology, streamingConfig)
-   *   streaming.run();
-   * 
- */ - public abstract void topology(); + /** + * Initializes a stream processing topology. This method may be called multiple times. + * An application constructs a processing logic using KStream API. + *

+ * For example, + *

+ *
+     *   KStreamTopology topology = new KStreamTopology() {
+     *     public void topology() {
+     *       KStream<Integer, PageView> pageViewStream = from("pageView").mapValues(...);
+     *       KStream<Integer, AdClick> adClickStream = from("adClick").join(pageViewStream, ...).process(...);
+     *     }
+     *   }
+     *
+     *   KafkaStreaming streaming = new KafkaStreaming(topology, streamingConfig)
+     *   streaming.run();
+     * 
+ */ + public abstract void topology(); - /** - * Extracts topics used in the KStream topology. This method calls {@link KStreamTopology#topology()} method. - * @return - */ - public final Set topics() { - synchronized (streams) { - try { - streams.clear(); - topology(); - Set topics = new HashSet<>(); - for (KStreamSource stream : streams) { - topics.addAll(stream.topics()); + /** + * Extracts topics used in the KStream topology. This method calls {@link KStreamTopology#topology()} method. + * + * @return + */ + public final Set topics() { + synchronized (streams) { + try { + streams.clear(); + topology(); + Set topics = new HashSet<>(); + for (KStreamSource stream : streams) { + topics.addAll(stream.topics()); + } + return topics; + } finally { + streams.clear(); + } } - return topics; - } - finally { - streams.clear(); - } } - } - /** - * Returns source streams in the KStream topology. This method calls {@link KStreamTopology#topology()} method. - * This method may be called multiple times. - */ - public final Collection> sourceStreams() { - synchronized (streams) { - try { - streams.clear(); - topology(); - return new ArrayList<>(streams); - } - finally { - streams.clear(); - } + /** + * Returns source streams in the KStream topology. This method calls {@link KStreamTopology#topology()} method. + * This method may be called multiple times. + */ + public final Collection> sourceStreams() { + synchronized (streams) { + try { + streams.clear(); + topology(); + return new ArrayList<>(streams); + } finally { + streams.clear(); + } + } } - } - // TODO: support regex topic matching in from() calls, for example: - // context.from("Topic*PageView") + // TODO: support regex topic matching in from() calls, for example: + // context.from("Topic*PageView") - /** - * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream - */ - public KStream from(String... topics) { - return from(null, null, topics); - } + /** + * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. + * + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(String... topics) { + return from(null, null, topics); + } - /** - * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. - * @param keyDeserializer key deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param valDeserializer value deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream - */ - public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KStreamSource stream = new KStreamSource<>(topics, keyDeserializer, valDeserializer, this); - streams.add(stream); - return stream; - } + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * + * @param keyDeserializer key deserializer used to read this source KStream, + * if not specified the default deserializer defined in the configs will be used + * @param valDeserializer value deserializer used to read this source KStream, + * if not specified the default deserializer defined in the configs will be used + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + KStreamSource stream = new KStreamSource<>(topics, keyDeserializer, valDeserializer, this); + streams.add(stream); + return stream; + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java index 89967b75b826d..d4ddda679ddbb 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java @@ -1,3 +1,20 @@ +/** + * 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.stream.topology; import org.apache.kafka.stream.KStream; @@ -7,31 +24,31 @@ */ public interface KStreamWindowed extends KStream { - /** - * Creates a new stream by joining this windowed stream with the other windowed stream. - * Each element arrived from either of the streams is joined with elements in a window of each other. - * The resulting values are computed by applying a joiner. - * - * @param other the other windowed stream - * @param joiner ValueJoiner - * @param the value type of the other stream - * @param the value type of the new stream - * @return KStream - */ - KStream join(KStreamWindowed other, ValueJoiner joiner); + /** + * Creates a new stream by joining this windowed stream with the other windowed stream. + * Each element arrived from either of the streams is joined with elements in a window of each other. + * The resulting values are computed by applying a joiner. + * + * @param other the other windowed stream + * @param joiner ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream + * @return KStream + */ + KStream join(KStreamWindowed other, ValueJoiner joiner); - /** - * 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 if - * the element from the other stream has an older timestamp. - * The resulting values are computed by applying a joiner. - * - * @param other the other windowed stream - * @param joiner the instance ValueJoiner - * @param the value type of the other stream - * @param the value type of the new stream - * @return KStream - */ - KStream joinPrior(KStreamWindowed other, ValueJoiner joiner); + /** + * 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 if + * the element from the other stream has an older timestamp. + * The resulting values are computed by applying a joiner. + * + * @param other the other windowed stream + * @param joiner the instance ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream + * @return KStream + */ + KStream joinPrior(KStreamWindowed other, ValueJoiner joiner); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java index 43809d1068e38..37a99d062db69 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java @@ -1,20 +1,34 @@ -package org.apache.kafka.stream.topology; - /** - * Created by yasuhiro on 6/17/15. + * 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. */ -public class KeyValue { - public final K key; - public final V value; +package org.apache.kafka.stream.topology; + +public class KeyValue { + + public final K key; + public final V value; - public KeyValue(K key, V value) { - this.key = key; - this.value = value; - } + public KeyValue(K key, V value) { + this.key = key; + this.value = value; + } - public static KeyValue pair(K key, V value) { - return new KeyValue(key, value); - } + public static KeyValue pair(K key, V value) { + return new KeyValue(key, value); + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java index bc0825e31b70b..be08861c835ed 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java @@ -1,10 +1,24 @@ -package org.apache.kafka.stream.topology; - /** - * Created by yasuhiro on 6/17/15. + * 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.stream.topology; + public interface KeyValueMapper { - KeyValue apply(K key, V value); + KeyValue apply(K key, V value); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java b/stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java new file mode 100644 index 0000000000000..3b5e253a36426 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.topology; + +import org.apache.kafka.stream.KStreamException; + +public class NotCopartitionedException extends KStreamException { + + public NotCopartitionedException() { + super(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java b/stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java index 7a2574545edbc..ae9b1e8072fbe 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java @@ -1,10 +1,24 @@ -package org.apache.kafka.stream.topology; - /** - * Created by yasuhiro on 6/17/15. + * 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.stream.topology; + public interface Predicate { - boolean apply(K key, V value); + boolean apply(K key, V value); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java index e7365dcbc8387..4febc6e82949f 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java @@ -1,27 +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.stream.topology; import org.apache.kafka.common.utils.Utils; -/** - * Created by guozhang on 7/14/15. - */ public class SingleProcessorTopology extends KStreamTopology { - private final Class processorClass; - private final String[] topics; + private final Class processorClass; + private final String[] topics; + + public SingleProcessorTopology(Class processorClass, String... topics) { + this.processorClass = processorClass; + this.topics = topics; + } - public SingleProcessorTopology(Class processorClass, String... topics) { - this.processorClass = processorClass; - this.topics = topics; - } - @SuppressWarnings("unchecked") - @Override - public void topology() { - from(topics).process(newProcessor()); - } + @SuppressWarnings("unchecked") + @Override + public void topology() { + from(topics).process(newProcessor()); + } - @SuppressWarnings("unchecked") - private Processor newProcessor() { - return (Processor) Utils.newInstance(processorClass); - } + @SuppressWarnings("unchecked") + private Processor newProcessor() { + return (Processor) Utils.newInstance(processorClass); + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java index 50938e8d2a714..feb9a5d3cdd06 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java @@ -1,3 +1,20 @@ +/** + * 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.stream.topology; import org.apache.kafka.stream.KStreamContext; @@ -8,128 +25,123 @@ import java.util.Iterator; import java.util.LinkedList; -/** - * Created by yasuhiro on 6/18/15. - */ public class SlidingWindow implements Window { - private String name; - private final long duration; - private final int maxCount; - private LinkedList list = new LinkedList(); - private HashMap>> map = new HashMap>>(); - - public SlidingWindow(String name, long duration, int maxCount) { - this.name = name; - this.duration = duration; - this.maxCount = maxCount; - } - - @Override - public void init(KStreamContext context) { - } - - @Override - public Iterator findAfter(K key, final long timestamp) { - return find(key, timestamp, timestamp + duration); - } - - @Override - public Iterator findBefore(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp); - } - - @Override - public Iterator find(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp + duration); - } - - /* - * finds items in the window between startTime and endTime (both inclusive) - */ - private Iterator find(K key, final long startTime, final long endTime) { - final LinkedList> values = map.get(key); - - if (values == null) { - return null; + private String name; + private final long duration; + private final int maxCount; + private LinkedList list = new LinkedList(); + private HashMap>> map = new HashMap>>(); + + public SlidingWindow(String name, long duration, int maxCount) { + this.name = name; + this.duration = duration; + this.maxCount = maxCount; + } + + @Override + public void init(KStreamContext context) { + } + + @Override + public Iterator findAfter(K key, final long timestamp) { + return find(key, timestamp, timestamp + duration); + } + + @Override + public Iterator findBefore(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp); + } + + @Override + public Iterator find(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp + duration); } - else { - return new FilteredIterator>(values.iterator()) { - @Override - protected V filter(Stamped item) { - if (startTime <= item.timestamp && item.timestamp <= endTime) - return item.value; - else + + /* + * finds items in the window between startTime and endTime (both inclusive) + */ + private Iterator find(K key, final long startTime, final long endTime) { + final LinkedList> values = map.get(key); + + if (values == null) { return null; + } else { + return new FilteredIterator>(values.iterator()) { + @Override + protected V filter(Stamped item) { + if (startTime <= item.timestamp && item.timestamp <= endTime) + return item.value; + else + return null; + } + }; } - }; } - } - @Override - public void put(K key, V value, long timestamp) { - list.offerLast(key); + @Override + public void put(K key, V value, long timestamp) { + list.offerLast(key); + + LinkedList> values = map.get(key); + if (values == null) { + values = new LinkedList>(); + map.put(key, values); + } - LinkedList> values = map.get(key); - if (values == null) { - values = new LinkedList>(); - map.put(key, values); + values.offerLast(new Stamped(value, timestamp)); + + evictExcess(); + evictExpired(timestamp - duration); } - values.offerLast(new Stamped(value, timestamp)); + private void evictExcess() { + while (list.size() > maxCount) { + K oldestKey = list.pollFirst(); - evictExcess(); - evictExpired(timestamp - duration); - } + LinkedList> values = map.get(oldestKey); + values.removeFirst(); - private void evictExcess() { - while (list.size() > maxCount) { - K oldestKey = list.pollFirst(); + if (values.isEmpty()) map.remove(oldestKey); + } + } - LinkedList> values = map.get(oldestKey); - values.removeFirst(); + private void evictExpired(long cutoffTime) { + while (true) { + K oldestKey = list.peekFirst(); - if (values.isEmpty()) map.remove(oldestKey); + LinkedList> values = map.get(oldestKey); + Stamped oldestValue = values.peekFirst(); + + if (oldestValue.timestamp < cutoffTime) { + list.pollFirst(); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } else { + break; + } + } } - } - private void evictExpired(long cutoffTime) { - while (true) { - K oldestKey = list.peekFirst(); + @Override + public String name() { + return name; + } - LinkedList> values = map.get(oldestKey); - Stamped oldestValue = values.peekFirst(); + @Override + public void flush() { + // TODO + } - if (oldestValue.timestamp < cutoffTime) { - list.pollFirst(); - values.removeFirst(); + @Override + public void close() { + // TODO + } - if (values.isEmpty()) map.remove(oldestKey); - } - else { - break; - } + @Override + public boolean persistent() { + // TODO: should not be persistent, right? + return false; } - } - - @Override - public String name() { - return name; - } - - @Override - public void flush() { - // TODO - } - - @Override - public void close() { - // TODO - } - - @Override - public boolean persistent() { - // TODO: should not be persistent, right? - return false; - } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java b/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java index c4e232524f2dc..24b01cdc24405 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java @@ -1,13 +1,27 @@ -package org.apache.kafka.stream.topology; - /** - * Created by yasuhiro on 6/17/15. + * 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.stream.topology; + public interface Transformer extends Processor { - interface Forwarder { - public void send(K key, V value, long timestamp); - } + interface Forwarder { + public void send(K key, V value, long timestamp); + } - public void forwarder(Forwarder forwarder); + public void forwarder(Forwarder forwarder); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java b/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java index 6f43b9c68bc28..7f8fff0a98e5d 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java @@ -1,10 +1,24 @@ -package org.apache.kafka.stream.topology; - /** - * Created by yasuhiro on 6/17/15. + * 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.stream.topology; + public interface ValueJoiner { - R apply(V1 value1, V2 value2); + R apply(V1 value1, V2 value2); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java index cf7ecd866ed98..c08a749c4279b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java @@ -1,10 +1,24 @@ -package org.apache.kafka.stream.topology; - /** - * Created by yasuhiro on 6/17/15. + * 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.stream.topology; + public interface ValueMapper { - R apply(V value); + R apply(V value); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Window.java b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java index 3d623551cdede..8d7bb3da0a03a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Window.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java @@ -1,3 +1,20 @@ +/** + * 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.stream.topology; import org.apache.kafka.stream.KStreamContext; @@ -5,19 +22,16 @@ import java.util.Iterator; -/** - * Created by yasuhiro on 6/17/15. - */ public interface Window extends StateStore { - void init(KStreamContext context); + void init(KStreamContext context); - Iterator find(K key, long timestamp); + Iterator find(K key, long timestamp); - Iterator findAfter(K key, long timestamp); + Iterator findAfter(K key, long timestamp); - Iterator findBefore(K key, long timestamp); + Iterator findBefore(K key, long timestamp); - void put(K key, V value, long timestamp); + void put(K key, V value, long timestamp); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java deleted file mode 100644 index 31f59a97b59d8..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMetadata.java +++ /dev/null @@ -1,53 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -import org.apache.kafka.stream.internal.PartitioningInfo; -import org.apache.kafka.stream.internal.StreamGroup; - -import java.util.Collections; -import java.util.Map; - -/** - * Created by guozhang on 7/13/15. - */ -public class KStreamMetadata { - - public static String UNKNOWN_TOPICNAME = "__UNKNOWN_TOPIC__"; - public static int UNKNOWN_PARTITION = -1; - - public static KStreamMetadata unjoinable() { - return new KStreamMetadata(Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(UNKNOWN_PARTITION))); - } - - public StreamGroup streamGroup; - public final Map topicPartitionInfos; - - public KStreamMetadata(Map topicPartitionInfos) { - this.topicPartitionInfos = topicPartitionInfos; - } - - boolean isJoinCompatibleWith(KStreamMetadata other) { - // the two streams should only be joinable if they are inside the same sync group - // and their contained streams all have the same number of partitions - if (this.streamGroup != other.streamGroup) - return false; - - int numPartitions = -1; - for (PartitioningInfo partitionInfo : this.topicPartitionInfos.values()) { - if (partitionInfo.numPartitions < 0) { - return false; - } else if (numPartitions >= 0) { - if (partitionInfo.numPartitions != numPartitions) - return false; - } else { - numPartitions = partitionInfo.numPartitions; - } - } - - for (PartitioningInfo partitionInfo : other.topicPartitionInfos.values()) { - if (partitionInfo.numPartitions != numPartitions) - return false; - } - - return true; - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java deleted file mode 100644 index d868f96084f52..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamTransform.java +++ /dev/null @@ -1,39 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.Transformer; - -/** - * Created by yasuhiro on 6/17/15. - */ -class KStreamTransform extends KStreamImpl implements Transformer.Forwarder { - - private final Transformer transformer; - - KStreamTransform(Transformer transformer, KStreamTopology topology) { - super(topology); - this.transformer = transformer; - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - transformer.init(context); - transformer.forwarder(this); - - super.bind(context, KStreamMetadata.unjoinable()); - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - transformer.process((K1) key, (V1) value); - } - } - - @Override - public void send(K key, V value, long timestamp) { - forward(key, value, timestamp); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java new file mode 100644 index 0000000000000..5ac6d1eca1978 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.topology.internals; + +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.internals.Receiver; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.Predicate; + +import java.lang.reflect.Array; +import java.util.Arrays; + +class KStreamBranch implements Receiver { + + private final Predicate[] predicates; + final KStreamSource[] branches; + + @SuppressWarnings("unchecked") + KStreamBranch(Predicate[] predicates, KStreamTopology topology) { + this.predicates = Arrays.copyOf(predicates, predicates.length); + this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); + for (int i = 0; i < branches.length; i++) { + branches[i] = new KStreamSource<>(null, topology); + } + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + for (KStreamSource branch : branches) { + branch.bind(context, metadata); + } + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + for (int i = 0; i < predicates.length; i++) { + Predicate predicate = predicates[i]; + if (predicate.apply((K) key, (V) value)) { + branches[i].receive(key, value, timestamp); + return; + } + } + } + + @Override + public void close() { + for (KStreamSource branch : branches) { + branch.close(); + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java new file mode 100644 index 0000000000000..683485d62e61c --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.topology.internals; + +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.Predicate; + +class KStreamFilter extends KStreamImpl { + + private final Predicate predicate; + + KStreamFilter(Predicate predicate, KStreamTopology topology) { + super(topology); + this.predicate = predicate; + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + synchronized (this) { + if (predicate.apply((K) key, (V) value)) { + forward(key, value, timestamp); + } + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java new file mode 100644 index 0000000000000..271ab6e625724 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java @@ -0,0 +1,50 @@ +/** + * 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.stream.topology.internals; + +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KeyValue; +import org.apache.kafka.stream.topology.KeyValueMapper; + +class KStreamFlatMap extends KStreamImpl { + + private final KeyValueMapper, K1, V1> mapper; + + KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology topology) { + super(topology); + this.mapper = mapper; + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + super.bind(context, KStreamMetadata.unjoinable()); + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + synchronized (this) { + KeyValue> newPair = mapper.apply((K1) key, (V1) value); + for (V v : newPair.value) { + forward(newPair.key, v, timestamp); + } + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java new file mode 100644 index 0000000000000..ea9c05bca9c54 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java @@ -0,0 +1,43 @@ +/** + * 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.stream.topology.internals; + +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.ValueMapper; + +class KStreamFlatMapValues extends KStreamImpl { + + private final ValueMapper, V1> mapper; + + KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology topology) { + super(topology); + this.mapper = mapper; + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + synchronized (this) { + Iterable newValues = mapper.apply((V1) value); + for (V v : newValues) { + forward(key, v, timestamp); + } + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java new file mode 100644 index 0000000000000..6f5304103b170 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java @@ -0,0 +1,191 @@ +/** + * 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.stream.topology.internals; + +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.KStream; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.stream.internals.ProcessorNode; +import org.apache.kafka.stream.internals.Receiver; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KStreamWindowed; +import org.apache.kafka.stream.topology.KeyValueMapper; +import org.apache.kafka.stream.topology.Predicate; +import org.apache.kafka.stream.topology.Transformer; +import org.apache.kafka.stream.topology.ValueMapper; +import org.apache.kafka.stream.topology.Window; + +import java.util.ArrayList; + +abstract class KStreamImpl implements KStream, Receiver { + + private final ArrayList nextReceivers = new ArrayList<>(1); + protected KStreamTopology topology; + protected KStreamContext context; + protected KStreamMetadata metadata; + + protected KStreamImpl(KStreamTopology topology) { + this.topology = topology; + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); + this.context = context; + this.metadata = metadata; + int numReceivers = nextReceivers.size(); + for (int i = 0; i < numReceivers; i++) { + nextReceivers.get(i).bind(context, metadata); + } + } + + @Override + public void close() { + int numReceivers = nextReceivers.size(); + for (int i = 0; i < numReceivers; i++) { + nextReceivers.get(i).close(); + } + } + + @Override + public KStream filter(Predicate predicate) { + return chain(new KStreamFilter(predicate, topology)); + } + + @Override + public KStream filterOut(final Predicate predicate) { + return filter(new Predicate() { + public boolean apply(K key, V value) { + return !predicate.apply(key, value); + } + }); + } + + @Override + public KStream map(KeyValueMapper mapper) { + return chain(new KStreamMap(mapper, topology)); + } + + @Override + public KStream mapValues(ValueMapper mapper) { + return chain(new KStreamMapValues(mapper, topology)); + } + + @Override + public KStream flatMap(KeyValueMapper, K, V> mapper) { + return chain(new KStreamFlatMap(mapper, topology)); + } + + @Override + public KStream flatMapValues(ValueMapper, V> mapper) { + return chain(new KStreamFlatMapValues(mapper, topology)); + } + + @Override + public KStreamWindowed with(Window window) { + return (KStreamWindowed) chain(new KStreamWindowedImpl<>(window, topology)); + } + + @Override + public KStream[] branch(Predicate... predicates) { + KStreamBranch branch = new KStreamBranch<>(predicates, topology); + registerReceiver(branch); + return branch.branches; + } + + @SuppressWarnings("unchecked") + @Override + public KStream through(String topic) { + return through(topic, null, null, null, null); + } + + @SuppressWarnings("unchecked") + @Override + public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { + process(this.getSendProcessor(topic, keySerializer, valSerializer)); + return topology.from(keyDeserializer, valDeserializer, topic); + } + + @Override + public void sendTo(String topic) { + process(this.getSendProcessor(topic, null, null)); + } + + @Override + public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { + process(this.getSendProcessor(topic, keySerializer, valSerializer)); + } + + @SuppressWarnings("unchecked") + private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { + return new Processor() { + private KStreamContext context; + + @Override + public void init(KStreamContext context) { + this.context = context; + } + + @Override + public void process(K key, V value) { + this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); + } + + @Override + public void punctuate(long streamTime) { + } + + @Override + public void close() { + } + }; + } + + @SuppressWarnings("unchecked") + @Override + public void process(Processor processor) { + registerReceiver(new ProcessorNode<>(processor)); + } + + @SuppressWarnings("unchecked") + @Override + public KStream transform(Transformer transformer) { + return chain(new KStreamTransform<>(transformer, topology)); + } + + void registerReceiver(Receiver receiver) { + nextReceivers.add(receiver); + } + + protected void forward(Object key, Object value, long timestamp) { + int numReceivers = nextReceivers.size(); + for (int i = 0; i < numReceivers; i++) { + nextReceivers.get(i).receive(key, value, timestamp); + } + } + + protected KStream chain(KStreamImpl kstream) { + synchronized (this) { + nextReceivers.add(kstream); + return kstream; + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java new file mode 100644 index 0000000000000..9aa5d14933192 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java @@ -0,0 +1,129 @@ +/** + * 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.stream.topology.internals; + +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.internals.Receiver; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.NotCopartitionedException; +import org.apache.kafka.stream.topology.ValueJoiner; +import org.apache.kafka.stream.topology.Window; + +import java.util.Iterator; + +class KStreamJoin extends KStreamImpl { + + private static abstract class Finder { + abstract Iterator find(K key, long timestamp); + } + + private final Finder finder1; + private final Finder finder2; + private final ValueJoiner joiner; + final Receiver receiverForOtherStream; + private KStreamMetadata thisMetadata; + private KStreamMetadata otherMetadata; + + KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology topology) { + super(topology); + + final Window window1 = stream1.window; + final Window window2 = stream2.window; + + if (prior) { + this.finder1 = new Finder() { + Iterator find(K key, long timestamp) { + return window1.findAfter(key, timestamp); + } + }; + this.finder2 = new Finder() { + Iterator find(K key, long timestamp) { + return window2.findBefore(key, timestamp); + } + }; + } else { + this.finder1 = new Finder() { + Iterator find(K key, long timestamp) { + return window1.find(key, timestamp); + } + }; + this.finder2 = new Finder() { + Iterator find(K key, long timestamp) { + return window2.find(key, timestamp); + } + }; + } + + this.joiner = joiner; + + this.receiverForOtherStream = getReceiverForOther(); + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + super.bind(context, metadata); + + thisMetadata = metadata; + if (otherMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) + throw new NotCopartitionedException(); + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + Iterator iter = finder2.find((K) key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + doJoin((K) key, (V1) value, iter.next(), timestamp); + } + } + } + + private Receiver getReceiverForOther() { + return new Receiver() { + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + otherMetadata = metadata; + if (thisMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) + throw new NotCopartitionedException(); + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value2, long timestamp) { + Iterator iter = finder1.find((K) key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + doJoin((K) key, iter.next(), (V2) value2, timestamp); + } + } + } + + @Override + public void close() { + // down stream instances are close when the primary stream is closed + } + }; + } + + // TODO: use the "outer-stream" topic as the resulted join stream topic + private void doJoin(K key, V1 value1, V2 value2, long timestamp) { + forward(key, joiner.apply(value1, value2), timestamp); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java new file mode 100644 index 0000000000000..fb371f1dd95c0 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.topology.internals; + +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KeyValue; +import org.apache.kafka.stream.topology.KeyValueMapper; + +class KStreamMap extends KStreamImpl { + + private final KeyValueMapper mapper; + + KStreamMap(KeyValueMapper mapper, KStreamTopology topology) { + super(topology); + this.mapper = mapper; + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + super.bind(context, KStreamMetadata.unjoinable()); + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + synchronized (this) { + KeyValue newPair = mapper.apply((K1) key, (V1) value); + forward(newPair.key, newPair.value, timestamp); + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java new file mode 100644 index 0000000000000..9dd0d6bc10040 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.topology.internals; + +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.ValueMapper; + +class KStreamMapValues extends KStreamImpl { + + private final ValueMapper mapper; + + KStreamMapValues(ValueMapper mapper, KStreamTopology topology) { + super(topology); + this.mapper = mapper; + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + synchronized (this) { + V newValue = mapper.apply((V1) value); + forward(key, newValue, timestamp); + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java new file mode 100644 index 0000000000000..6f3a0996d1622 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.topology.internals; + +import org.apache.kafka.stream.internals.PartitioningInfo; +import org.apache.kafka.stream.internals.StreamGroup; + +import java.util.Collections; +import java.util.Map; + +public class KStreamMetadata { + + public static final String UNKNOWN_TOPICNAME = "__UNKNOWN_TOPIC__"; + public static final int UNKNOWN_PARTITION = -1; + + public static KStreamMetadata unjoinable() { + return new KStreamMetadata(Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(UNKNOWN_PARTITION))); + } + + public StreamGroup streamGroup; + public final Map topicPartitionInfos; + + public KStreamMetadata(Map topicPartitionInfos) { + this.topicPartitionInfos = topicPartitionInfos; + } + + boolean isJoinCompatibleWith(KStreamMetadata other) { + // the two streams should only be joinable if they are inside the same sync group + // and their contained streams all have the same number of partitions + if (this.streamGroup != other.streamGroup) + return false; + + int numPartitions = -1; + for (PartitioningInfo partitionInfo : this.topicPartitionInfos.values()) { + if (partitionInfo.numPartitions < 0) { + return false; + } else if (numPartitions >= 0) { + if (partitionInfo.numPartitions != numPartitions) + return false; + } else { + numPartitions = partitionInfo.numPartitions; + } + } + + for (PartitioningInfo partitionInfo : other.topicPartitionInfos.values()) { + if (partitionInfo.numPartitions != numPartitions) + return false; + } + + return true; + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java new file mode 100644 index 0000000000000..f1c8260acee0d --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java @@ -0,0 +1,75 @@ +/** + * 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.stream.topology.internals; + +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.topology.KStreamTopology; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +public class KStreamSource extends KStreamImpl { + + private Deserializer keyDeserializer; + private Deserializer valueDeserializer; + + public String[] topics; + + public KStreamSource(String[] topics, KStreamTopology topology) { + this(topics, null, null, topology); + } + + public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology topology) { + super(topology); + this.topics = topics; + this.keyDeserializer = keyDeserializer; + this.valueDeserializer = valueDeserializer; + } + + @SuppressWarnings("unchecked") + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + if (keyDeserializer == null) keyDeserializer = (Deserializer) context.keyDeserializer(); + if (valueDeserializer == null) valueDeserializer = (Deserializer) context.valueDeserializer(); + + super.bind(context, metadata); + } + + @Override + public void receive(Object key, Object value, long timestamp) { + synchronized (this) { + // KStream needs to forward the topic name since it is directly from the Kafka source + forward(key, value, timestamp); + } + } + + public Deserializer keyDeserializer() { + return keyDeserializer; + } + + public Deserializer valueDeserializer() { + return valueDeserializer; + } + + public Set topics() { + return new HashSet<>(Arrays.asList(topics)); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java similarity index 92% rename from stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java rename to stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java index 78a30c838679c..fbb91b92c83e7 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java @@ -5,9 +5,9 @@ * 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 - * + *

+ * 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. @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.topology.internal; +package org.apache.kafka.stream.topology.internals; <<<<<<< HEAD <<<<<<< HEAD @@ -54,11 +54,11 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.stream.StreamingConfig; -import org.apache.kafka.stream.internal.IngestorImpl; -import org.apache.kafka.stream.internal.KStreamContextImpl; -import org.apache.kafka.stream.internal.ProcessorConfig; -import org.apache.kafka.stream.internal.RecordCollectorImpl; -import org.apache.kafka.stream.internal.StreamGroup; +import org.apache.kafka.stream.internals.IngestorImpl; +import org.apache.kafka.stream.internals.KStreamContextImpl; +import org.apache.kafka.stream.internals.ProcessorConfig; +import org.apache.kafka.stream.internals.RecordCollectorImpl; +import org.apache.kafka.stream.internals.StreamGroup; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.util.ParallelExecutor; import org.slf4j.Logger; @@ -120,7 +120,7 @@ public KStreamThread(KStreamTopology topology, Set topics, StreamingConf this.ingestor = new IngestorImpl(consumer, topics); Producer producer = new KafkaProducer<>(streamingConfig.config(), new ByteArraySerializer(), new ByteArraySerializer()); - this.collector = new RecordCollectorImpl(producer, (Serializer)streamingConfig.keySerializer(), (Serializer)streamingConfig.valueSerializer()); + this.collector = new RecordCollectorImpl(producer, (Serializer) streamingConfig.keySerializer(), (Serializer) streamingConfig.valueSerializer()); this.running = true; this.lastCommit = 0; @@ -184,11 +184,11 @@ private void runLoop() { } private boolean stillRunning() { - if(!running) { + if (!running) { log.debug("Shutting down at user request."); return false; } - if(config.totalRecordsToProcess >= 0 && recordsProcessed >= config.totalRecordsToProcess) { + if (config.totalRecordsToProcess >= 0 && recordsProcessed >= config.totalRecordsToProcess) { log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", config.totalRecordsToProcess); return false; } @@ -223,17 +223,17 @@ private void commitAll(long now) { /* delete any state dirs that aren't for active contexts */ private void maybeCleanState() { long now = time.milliseconds(); - if(now > nextStateCleaning) { + if (now > nextStateCleaning) { File[] stateDirs = config.stateDir.listFiles(); - if(stateDirs != null) { - for(File dir: stateDirs) { + if (stateDirs != null) { + for (File dir : stateDirs) { try { Integer id = Integer.parseInt(dir.getName()); - if(!kstreamContexts.keySet().contains(id)) { + if (!kstreamContexts.keySet().contains(id)) { log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); Utils.rm(dir); } - } catch(NumberFormatException e) { + } catch (NumberFormatException e) { log.warn("Deleting unknown directory in state directory {}.", dir.getAbsolutePath()); Utils.rm(dir); } @@ -273,8 +273,7 @@ private void addPartitions(Collection assignment) { >>>>>>> new api model kstreamContexts.put(id, context); - } - catch (Exception e) { + } catch (Exception e) { throw new KafkaException(e); } @@ -290,8 +289,7 @@ private void removePartitions() { log.info("Removing task context {}", context.id()); try { context.close(); - } - catch (Exception e) { + } catch (Exception e) { throw new KafkaException(e); } streamingMetrics.processorDestruction.record(); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamTransform.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamTransform.java new file mode 100644 index 0000000000000..bf09d7d24a32a --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamTransform.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.topology.internals; + +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.Transformer; + +class KStreamTransform extends KStreamImpl implements Transformer.Forwarder { + + private final Transformer transformer; + + KStreamTransform(Transformer transformer, KStreamTopology topology) { + super(topology); + this.transformer = transformer; + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + transformer.init(context); + transformer.forwarder(this); + + super.bind(context, KStreamMetadata.unjoinable()); + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + synchronized (this) { + transformer.process((K1) key, (V1) value); + } + } + + @Override + public void send(K key, V value, long timestamp) { + forward(key, value, timestamp); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindowedImpl.java new file mode 100644 index 0000000000000..6f67f1c06d394 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindowedImpl.java @@ -0,0 +1,78 @@ +/** + * 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.stream.topology.internals; + +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.KStream; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KStreamWindowed; +import org.apache.kafka.stream.topology.ValueJoiner; +import org.apache.kafka.stream.topology.Window; + +public class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { + + final Window window; + + KStreamWindowedImpl(Window window, KStreamTopology initializer) { + super(initializer); + this.window = window; + } + + @Override + public void bind(KStreamContext context, KStreamMetadata metadata) { + super.bind(context, metadata); + window.init(context); + } + + @SuppressWarnings("unchecked") + @Override + public void receive(Object key, Object value, long timestamp) { + synchronized (this) { + window.put((K) key, (V) value, timestamp); + // KStreamWindowed needs to forward the topic name since it may receive directly from KStreamSource + forward(key, value, timestamp); + } + } + + @Override + public void close() { + window.close(); + super.close(); + } + + @Override + public KStream join(KStreamWindowed other, ValueJoiner processor) { + return join(other, false, processor); + } + + @Override + public KStream joinPrior(KStreamWindowed other, ValueJoiner processor) { + return join(other, true, processor); + } + + private KStream join(KStreamWindowed other, boolean prior, ValueJoiner processor) { + + KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; + + KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, topology); + otherImpl.registerReceiver(stream.receiverForOtherStream); + + return chain(stream); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/util/FilteredIterator.java b/stream/src/main/java/org/apache/kafka/stream/util/FilteredIterator.java index c9e67112c8382..daff513d328cd 100644 --- a/stream/src/main/java/org/apache/kafka/stream/util/FilteredIterator.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/FilteredIterator.java @@ -1,49 +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.stream.util; import java.util.Iterator; -/** - * Created by yasuhiro on 6/29/15. - */ public abstract class FilteredIterator implements Iterator { - private Iterator inner; - private T nextValue = null; + private Iterator inner; + private T nextValue = null; - public FilteredIterator(Iterator inner) { - this.inner = inner; + public FilteredIterator(Iterator inner) { + this.inner = inner; - findNext(); - } + findNext(); + } - @Override - public boolean hasNext() { - return nextValue != null; - } + @Override + public boolean hasNext() { + return nextValue != null; + } - @Override - public T next() { - T value = nextValue; - findNext(); + @Override + public T next() { + T value = nextValue; + findNext(); - return value; - } + return value; + } - @Override - public void remove() { - throw new UnsupportedOperationException(); - } + @Override + public void remove() { + throw new UnsupportedOperationException(); + } - private void findNext() { - while (inner.hasNext()) { - S item = inner.next(); - nextValue = filter(item); - if (nextValue != null) { - return; - } + private void findNext() { + while (inner.hasNext()) { + S item = inner.next(); + nextValue = filter(item); + if (nextValue != null) { + return; + } + } + nextValue = null; } - nextValue = null; - } - protected abstract T filter(S item); + protected abstract T filter(S item); } diff --git a/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java index f9daab2a8fc8a..55ec056f4987c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java @@ -1,3 +1,20 @@ +/** + * 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.stream.util; import java.util.LinkedList; @@ -9,33 +26,33 @@ */ public class MinTimestampTracker implements TimestampTracker { - private final LinkedList> descendingSubsequence = new LinkedList>(); + private final LinkedList> descendingSubsequence = new LinkedList>(); - public void addStampedElement(Stamped elem) { - if (elem == null) throw new NullPointerException(); + public void addStampedElement(Stamped elem) { + if (elem == null) throw new NullPointerException(); - Stamped minElem = descendingSubsequence.peekLast(); - while (minElem != null && minElem.timestamp >= elem.timestamp) { - descendingSubsequence.removeLast(); - minElem = descendingSubsequence.peekLast(); + Stamped minElem = descendingSubsequence.peekLast(); + while (minElem != null && minElem.timestamp >= elem.timestamp) { + descendingSubsequence.removeLast(); + minElem = descendingSubsequence.peekLast(); + } + descendingSubsequence.offerLast(elem); } - descendingSubsequence.offerLast(elem); - } - public void removeStampedElement(Stamped elem) { - if (elem != null && descendingSubsequence.peekFirst() == elem) - descendingSubsequence.removeFirst(); - } + public void removeStampedElement(Stamped elem) { + if (elem != null && descendingSubsequence.peekFirst() == elem) + descendingSubsequence.removeFirst(); + } - public int size() { - return descendingSubsequence.size(); - } + public int size() { + return descendingSubsequence.size(); + } - public long get() { - Stamped stamped = descendingSubsequence.peekFirst(); - if (stamped == null) return -1L; + public long get() { + Stamped stamped = descendingSubsequence.peekFirst(); + if (stamped == null) return -1L; - return stamped.timestamp; - } + return stamped.timestamp; + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java b/stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java index 6138237ecad57..4a68a1868e500 100644 --- a/stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java @@ -5,15 +5,16 @@ * 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 - * + *

+ * 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.stream.util; import org.apache.kafka.common.TopicPartition; @@ -79,8 +80,8 @@ public void write(Map offsets) throws IOException { file.delete(); if (!temp.renameTo(file)) throw new IOException(String.format("File rename from %s to %s failed.", - temp.getAbsolutePath(), - file.getAbsolutePath())); + temp.getAbsolutePath(), + file.getAbsolutePath())); } } } @@ -104,7 +105,7 @@ public Map read() throws IOException { BufferedReader reader = null; try { reader = new BufferedReader(new FileReader(file)); - } catch(FileNotFoundException e) { + } catch (FileNotFoundException e) { return Collections.emptyMap(); } @@ -119,7 +120,7 @@ public Map read() throws IOException { String[] pieces = line.split("\\s+"); if (pieces.length != 3) throw new IOException(String.format("Malformed line in offset checkpoint file: '%s'.", - line)); + line)); String topic = pieces[0]; int partition = Integer.parseInt(pieces[1]); @@ -129,15 +130,15 @@ public Map read() throws IOException { } if (offsets.size() != expectedSize) throw new IOException(String.format("Expected %d entries but found only %d", - expectedSize, - offsets.size())); + expectedSize, + offsets.size())); return offsets; default: throw new IllegalArgumentException("Unknown offset checkpoint version: " + version); } } finally { - if(reader != null) + if (reader != null) reader.close(); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java b/stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java index 1cace23052360..4acc83df624c3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java @@ -1,3 +1,20 @@ +/** + * 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.stream.util; import java.util.ArrayList; @@ -10,128 +27,126 @@ */ public class ParallelExecutor { - /** - * A parallel task must implement this interface - */ - public interface Task { /** - * Executes a task - * @return boolean true if the task are ready for next execution + * A parallel task must implement this interface */ - boolean process(); - } - - private final WorkerThread[] workerThreads; - private final AtomicInteger taskIndex = new AtomicInteger(0); - private volatile ArrayList tasks = new ArrayList<>(); - private volatile CountDownLatch latch; - private volatile boolean readyForNextExecution = true; - private volatile boolean running = true; - private volatile Exception exception; - - public ParallelExecutor(int parallelDegree) { - parallelDegree = Math.max(parallelDegree, 1); - workerThreads = new WorkerThread[parallelDegree - 1]; - for (int i = 0; i < workerThreads.length; i++) { - workerThreads[i] = new WorkerThread(); - workerThreads[i].start(); + public interface Task { + /** + * Executes a task + * + * @return boolean true if the task are ready for next execution + */ + boolean process(); + } + + private final WorkerThread[] workerThreads; + private final AtomicInteger taskIndex = new AtomicInteger(0); + private volatile ArrayList tasks = new ArrayList<>(); + private volatile CountDownLatch latch; + private volatile boolean readyForNextExecution = true; + private volatile boolean running = true; + private volatile Exception exception; + + public ParallelExecutor(int parallelDegree) { + parallelDegree = Math.max(parallelDegree, 1); + workerThreads = new WorkerThread[parallelDegree - 1]; + for (int i = 0; i < workerThreads.length; i++) { + workerThreads[i] = new WorkerThread(); + workerThreads[i].start(); + } } - } - - /** - * Executes tasks in parallel. While this method is executing, other execute call will be blocked. - * @param tasks a list of tasks executed in parallel - * @return boolean true if at least one task is ready for next execution, otherwise false - * @throws Exception an exception thrown by a failed task - */ - public boolean execute(ArrayList tasks) throws Exception { - synchronized (this) { - try { - int numTasks = tasks.size(); - exception = null; - readyForNextExecution = false; - if (numTasks > 0) { - this.tasks = tasks; - this.latch = new CountDownLatch(numTasks); - - taskIndex.set(numTasks); - wakeUpWorkers(Math.min(numTasks - 1, workerThreads.length)); - - // the calling thread also picks up tasks - if (taskIndex.get() > 0) doProcess(); - - while (true) { + + /** + * Executes tasks in parallel. While this method is executing, other execute call will be blocked. + * + * @param tasks a list of tasks executed in parallel + * @return boolean true if at least one task is ready for next execution, otherwise false + * @throws Exception an exception thrown by a failed task + */ + public boolean execute(ArrayList tasks) throws Exception { + synchronized (this) { try { - latch.await(); - break; - } catch (InterruptedException ex) { - Thread.interrupted(); + int numTasks = tasks.size(); + exception = null; + readyForNextExecution = false; + if (numTasks > 0) { + this.tasks = tasks; + this.latch = new CountDownLatch(numTasks); + + taskIndex.set(numTasks); + wakeUpWorkers(Math.min(numTasks - 1, workerThreads.length)); + + // the calling thread also picks up tasks + if (taskIndex.get() > 0) doProcess(); + + while (true) { + try { + latch.await(); + break; + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + } + if (exception != null) throw exception; + } finally { + this.tasks = null; + this.latch = null; + this.exception = null; } - } + return readyForNextExecution; } - if (exception != null) throw exception; - } - finally { - this.tasks = null; - this.latch = null; - this.exception = null; - } - return readyForNextExecution; - } - } - - /** - * Shuts this parallel executor down - */ - public void shutdown() { - synchronized (this) { - running = false; - // wake up all workers - wakeUpWorkers(workerThreads.length); - } - } - - private void doProcess() { - int index = taskIndex.decrementAndGet(); - if (index >= 0) { - try { - if (tasks.get(index).process()) - this.readyForNextExecution = true; - } - catch (Exception ex) { - exception = ex; - } - finally { - latch.countDown(); - } } - } - private void wakeUpWorkers(int numWorkers) { - for (int i = 0; i < numWorkers; i++) - LockSupport.unpark(workerThreads[i]); - } + /** + * Shuts this parallel executor down + */ + public void shutdown() { + synchronized (this) { + running = false; + // wake up all workers + wakeUpWorkers(workerThreads.length); + } + } - private class WorkerThread extends Thread { + private void doProcess() { + int index = taskIndex.decrementAndGet(); + if (index >= 0) { + try { + if (tasks.get(index).process()) + this.readyForNextExecution = true; + } catch (Exception ex) { + exception = ex; + } finally { + latch.countDown(); + } + } + } - WorkerThread() { - super(); - setDaemon(true); + private void wakeUpWorkers(int numWorkers) { + for (int i = 0; i < numWorkers; i++) + LockSupport.unpark(workerThreads[i]); } - @Override - public void run() { - while (running) { - if (taskIndex.get() > 0) { - doProcess(); + private class WorkerThread extends Thread { + + WorkerThread() { + super(); + setDaemon(true); } - else { - // no more work. park this thread. - LockSupport.park(); - Thread.interrupted(); + + @Override + public void run() { + while (running) { + if (taskIndex.get() > 0) { + doProcess(); + } else { + // no more work. park this thread. + LockSupport.park(); + Thread.interrupted(); + } + } } - } } - } } diff --git a/stream/src/main/java/org/apache/kafka/stream/util/Stamped.java b/stream/src/main/java/org/apache/kafka/stream/util/Stamped.java index f0ceaeda9b27b..ed5d4b50040cc 100644 --- a/stream/src/main/java/org/apache/kafka/stream/util/Stamped.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/Stamped.java @@ -1,24 +1,38 @@ -package org.apache.kafka.stream.util; - /** - * Created by yasuhiro on 6/23/15. + * 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.stream.util; + public class Stamped implements Comparable { - public final V value; - public final long timestamp; + public final V value; + public final long timestamp; - public Stamped(V value, long timestamp) { - this.value = value; - this.timestamp = timestamp; - } + public Stamped(V value, long timestamp) { + this.value = value; + this.timestamp = timestamp; + } - public int compareTo(Object other) { - long otherTimestamp = ((Stamped) other).timestamp; + public int compareTo(Object other) { + long otherTimestamp = ((Stamped) other).timestamp; - if (timestamp < otherTimestamp) return -1; - else if (timestamp > otherTimestamp) return 1; - return 0; - } + if (timestamp < otherTimestamp) return -1; + else if (timestamp > otherTimestamp) return 1; + return 0; + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java index 9e57cd8568aff..c887383d96676 100644 --- a/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java @@ -1,3 +1,20 @@ +/** + * 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.stream.util; /** @@ -8,28 +25,32 @@ */ public interface TimestampTracker { - /** - * Adds a stamped elements to this tracker. - * @param elem the added element - */ - void addStampedElement(Stamped elem); + /** + * Adds a stamped elements to this tracker. + * + * @param elem the added element + */ + void addStampedElement(Stamped elem); - /** - * Removed a stamped elements to this tracker. - * @param elem the removed element - */ - void removeStampedElement(Stamped elem); + /** + * Removed a stamped elements to this tracker. + * + * @param elem the removed element + */ + void removeStampedElement(Stamped elem); - /** - * Returns the timestamp - * @return timestamp, or -1L when empty - */ - long get(); + /** + * Returns the timestamp + * + * @return timestamp, or -1L when empty + */ + long get(); - /** - * Returns the size of internal structure. The meaning of "size" depends on the implementation. - * @return size - */ - int size(); + /** + * Returns the size of internal structure. The meaning of "size" depends on the implementation. + * + * @return size + */ + int size(); } diff --git a/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java b/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java index 78f015108c662..6b0c679279b7b 100644 --- a/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java @@ -1,3 +1,20 @@ +/** + * 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.stream; import static org.junit.Assert.assertEquals; @@ -12,67 +29,67 @@ public class FilteredIteratorTest { - @Test - public void testFiltering() { - List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + @Test + public void testFiltering() { + List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 3 == 0) return i.toString(); + return null; + } + }; - Iterator filtered = new FilteredIterator(list.iterator()) { - protected String filter(Integer i) { - if (i % 3 == 0) return i.toString(); - return null; - } - }; + List expected = Arrays.asList("3", "9", "6", "3"); + List result = new ArrayList(); - List expected = Arrays.asList("3", "9", "6", "3"); - List result = new ArrayList(); + while (filtered.hasNext()) { + result.add(filtered.next()); + } - while (filtered.hasNext()) { - result.add(filtered.next()); + assertEquals(expected, result); } - assertEquals(expected, result); - } + @Test + public void testEmptySource() { + List list = new ArrayList(); - @Test - public void testEmptySource() { - List list = new ArrayList(); + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 3 == 0) return i.toString(); + return null; + } + }; - Iterator filtered = new FilteredIterator(list.iterator()) { - protected String filter(Integer i) { - if (i % 3 == 0) return i.toString(); - return null; - } - }; + List expected = new ArrayList(); + List result = new ArrayList(); - List expected = new ArrayList(); - List result = new ArrayList(); + while (filtered.hasNext()) { + result.add(filtered.next()); + } - while (filtered.hasNext()) { - result.add(filtered.next()); + assertEquals(expected, result); } - assertEquals(expected, result); - } + @Test + public void testNoMatch() { + List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); - @Test - public void testNoMatch() { - List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 7 == 0) return i.toString(); + return null; + } + }; - Iterator filtered = new FilteredIterator(list.iterator()) { - protected String filter(Integer i) { - if (i % 7 == 0) return i.toString(); - return null; - } - }; + List expected = new ArrayList(); + List result = new ArrayList(); - List expected = new ArrayList(); - List result = new ArrayList(); + while (filtered.hasNext()) { + result.add(filtered.next()); + } - while (filtered.hasNext()) { - result.add(filtered.next()); + assertEquals(expected, result); } - assertEquals(expected, result); - } - } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java index c786e2c607f48..a4c7d3b8618e2 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java @@ -1,10 +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.stream; -import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Predicate; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockKStreamContext; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; @@ -17,6 +34,7 @@ public class KStreamBranchTest { +<<<<<<< HEAD private String topicName = "topic"; private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); @@ -76,32 +94,81 @@ public boolean apply(Integer key, String value) { for (int i = 0; i < expectedKeys.length; i++) { stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); +======= + private String topicName = "topic"; + + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + + @SuppressWarnings("unchecked") + @Test + public void testKStreamBranch() { + + Predicate isEven = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 2) == 0; + } + }; + Predicate isMultipleOfThree = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 3) == 0; + } + }; + Predicate isOdd = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 2) != 0; + } + }; + + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + + KStreamTopology initializer = new MockKStreamTopology(); + KStreamSource stream; + KStream[] branches; + MockProcessor[] processors; + + stream = new KStreamSource<>(null, initializer); + branches = stream.branch(isEven, isMultipleOfThree, isOdd); + + assertEquals(3, branches.length); + + processors = (MockProcessor[]) Array.newInstance(MockProcessor.class, branches.length); + for (int i = 0; i < branches.length; i++) { + processors[i] = new MockProcessor<>(); + branches[i].process(processors[i]); + } + + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(3, processors[0].processed.size()); + assertEquals(1, processors[1].processed.size()); + assertEquals(3, processors[2].processed.size()); + + stream = new KStreamSource<>(null, initializer); + branches = stream.branch(isEven, isOdd, isMultipleOfThree); + + assertEquals(3, branches.length); + + processors = (MockProcessor[]) Array.newInstance(MockProcessor.class, branches.length); + for (int i = 0; i < branches.length; i++) { + processors[i] = new MockProcessor<>(); + branches[i].process(processors[i]); + } + + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(3, processors[0].processed.size()); + assertEquals(4, processors[1].processed.size()); + assertEquals(0, processors[2].processed.size()); +>>>>>>> compile and test passed } - assertEquals(3, processors[0].processed.size()); - assertEquals(1, processors[1].processed.size()); - assertEquals(3, processors[2].processed.size()); - - stream = new KStreamSource<>(null, initializer); - branches = stream.branch(isEven, isOdd, isMultipleOfThree); - - assertEquals(3, branches.length); - - processors = (MockProcessor[]) Array.newInstance(MockProcessor.class, branches.length); - for (int i = 0; i < branches.length; i++) { - processors[i] = new MockProcessor<>(); - branches[i].process(processors[i]); - } - - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); - } - - assertEquals(3, processors[0].processed.size()); - assertEquals(4, processors[1].processed.size()); - assertEquals(0, processors[2].processed.size()); - } - } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java index 5c3f616010800..ae9bdec644d2d 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java @@ -1,38 +1,57 @@ +/** + * 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.stream; -import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Predicate; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; import org.junit.Test; + import java.util.Collections; import static org.junit.Assert.assertEquals; public class KStreamFilterTest { - private String topicName = "topic"; + private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - private Predicate isMultipleOfThree = new Predicate() { - @Override - public boolean apply(Integer key, String value) { - return (key % 3) == 0; - } - }; + private Predicate isMultipleOfThree = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 3) == 0; + } + }; - @Test - public void testFilter() { - final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; + @Test + public void testFilter() { + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -46,26 +65,36 @@ public void testFilter() { >>>>>>> wip KStreamSource stream; MockProcessor processor; - - processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, initializer); - stream.filter(isMultipleOfThree).process(processor); - - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); +======= + KStreamTopology initializer = new MockKStreamTopology(); + KStreamSource stream; + MockProcessor processor; +>>>>>>> compile and test passed + + processor = new MockProcessor<>(); + stream = new KStreamSource<>(null, initializer); + stream.filter(isMultipleOfThree).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(2, processor.processed.size()); } - assertEquals(2, processor.processed.size()); - } + @Test + public void testFilterOut() { + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - @Test - public void testFilterOut() { - final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; + KStreamTopology initializer = new MockKStreamTopology(); + KStreamSource stream; + MockProcessor processor; <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -79,18 +108,19 @@ public void testFilterOut() { >>>>>>> wip KStreamSource stream; MockProcessor processor; - - processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, initializer); - stream.filterOut(isMultipleOfThree).process(processor); - - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); +======= + processor = new MockProcessor<>(); + stream = new KStreamSource<>(null, initializer); + stream.filterOut(isMultipleOfThree).process(processor); +>>>>>>> compile and test passed + + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(5, processor.processed.size()); } - assertEquals(5, processor.processed.size()); - } - } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java index d400c2dc27e74..b227f72a36743 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java @@ -1,11 +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.stream; -import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.KeyValueMapper; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; @@ -18,29 +35,30 @@ public class KStreamFlatMapTest { - private String topicName = "topic"; + private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - @Test - public void testFlatMap() { + @Test + public void testFlatMap() { - KeyValueMapper, Integer, String> mapper = - new KeyValueMapper, Integer, String>() { - @Override - public KeyValue> apply(Integer key, String value) { - ArrayList result = new ArrayList(); - for (int i = 0; i < key; i++) { - result.add(value); - } - return KeyValue.pair(Integer.toString(key * 10), (Iterable)result); - } - }; + KeyValueMapper, Integer, String> mapper = + new KeyValueMapper, Integer, String>() { + @Override + public KeyValue> apply(Integer key, String value) { + ArrayList result = new ArrayList(); + for (int i = 0; i < key; i++) { + result.add(value); + } + return KeyValue.pair(Integer.toString(key * 10), (Iterable) result); + } + }; - final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology topology = new MockKStreamTopology(); @@ -72,20 +90,30 @@ public KeyValue> apply(Integer key, String value) { stream = new KStreamSource<>(null, topology); >>>>>>> wip stream.flatMap(mapper).process(processor); +======= + KStreamTopology topology = new MockKStreamTopology(); - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); - } + KStreamSource stream; + MockProcessor processor; - assertEquals(6, processor.processed.size()); + processor = new MockProcessor<>(); + stream = new KStreamSource<>(null, topology); + stream.flatMap(mapper).process(processor); +>>>>>>> compile and test passed - String[] expected = new String[] { "10:V1", "20:V2", "20:V2", "30:V3", "30:V3", "30:V3" }; + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } + + assertEquals(6, processor.processed.size()); - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); + String[] expected = new String[]{"10:V1", "20:V2", "20:V2", "30:V3", "30:V3", "30:V3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } } - } } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java index 28a96347b8631..b5a9acee45690 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java @@ -1,10 +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.stream; -import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.ValueMapper; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; @@ -17,28 +34,29 @@ public class KStreamFlatMapValuesTest { - private String topicName = "topic"; + private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - @Test - public void testFlatMapValues() { + @Test + public void testFlatMapValues() { - ValueMapper, String> mapper = - new ValueMapper, String>() { - @Override - public Iterable apply(String value) { - ArrayList result = new ArrayList(); - result.add(value.toLowerCase()); - result.add(value); - return result; - } - }; + ValueMapper, String> mapper = + new ValueMapper, String>() { + @Override + public Iterable apply(String value) { + ArrayList result = new ArrayList(); + result.add(value.toLowerCase()); + result.add(value); + return result; + } + }; - final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -52,24 +70,29 @@ public Iterable apply(String value) { >>>>>>> wip KStreamSource stream; MockProcessor processor; +======= + KStreamTopology initializer = new MockKStreamTopology(); + KStreamSource stream; + MockProcessor processor; +>>>>>>> compile and test passed - processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, initializer); - stream.flatMapValues(mapper).process(processor); + processor = new MockProcessor<>(); + stream = new KStreamSource<>(null, initializer); + stream.flatMapValues(mapper).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); - } + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } - assertEquals(8, processor.processed.size()); + assertEquals(8, processor.processed.size()); - String[] expected = new String[] { "0:v0", "0:V0", "1:v1", "1:V1", "2:v2", "2:V2", "3:v3", "3:V3" }; + String[] expected = new String[]{"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)); + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } } - } } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java index c4ce68ff573f2..9228e01fda4a5 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java @@ -1,18 +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.stream; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KStreamWindowed; import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.KeyValueMapper; import org.apache.kafka.stream.topology.ValueJoiner; import org.apache.kafka.stream.topology.ValueMapper; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; +import org.apache.kafka.stream.topology.NotCopartitionedException; import org.apache.kafka.test.UnlimitedWindow; import org.junit.Test; @@ -24,61 +42,53 @@ public class KStreamJoinTest { - private String topicName = "topic"; - - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - - private ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; - } - }; + private String topicName = "topic"; - private ValueMapper valueMapper = new ValueMapper() { - @Override - public String apply(String value) { - return "#" + value; - } - }; + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - private ValueMapper, String> valueMapper2 = new ValueMapper, String>() { - @Override - public Iterable apply(String value) { - return (Iterable) Utils.mkSet(value); - } - }; - - private KeyValueMapper keyValueMapper = - new KeyValueMapper() { - @Override - public KeyValue apply(Integer key, String value) { - return KeyValue.pair(key, value); - } + private ValueJoiner joiner = new ValueJoiner() { + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } }; - KeyValueMapper, Integer, String> keyValueMapper2 = - new KeyValueMapper, Integer, String>() { - @Override - public KeyValue> apply(Integer key, String value) { - return KeyValue.pair(key, (Iterable) Utils.mkSet(value)); - } + private ValueMapper valueMapper = new ValueMapper() { + @Override + public String apply(String value) { + return "#" + value; + } }; - @Test - public void testJoin() { - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + private ValueMapper, String> valueMapper2 = new ValueMapper, String>() { + @Override + public Iterable apply(String value) { + return (Iterable) Utils.mkSet(value); + } + }; - KStreamSource stream1; - KStreamSource stream2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; - String[] expected; + private KeyValueMapper keyValueMapper = + new KeyValueMapper() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(key, value); + } + }; + + KeyValueMapper, Integer, String> keyValueMapper2 = + new KeyValueMapper, Integer, String>() { + @Override + public KeyValue> apply(Integer key, String value) { + return KeyValue.pair(key, (Iterable) Utils.mkSet(value)); + } + }; + + @Test + public void testJoin() { <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); <<<<<<< HEAD @@ -99,92 +109,110 @@ public void testJoin() { stream2 = new KStreamSource<>(null, initializer); windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); +======= + final int[] expectedKeys = new int[]{0, 1, 2, 3}; +>>>>>>> compile and test passed - boolean exceptionRaised = false; + KStreamSource stream1; + KStreamSource stream2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + MockProcessor processor; + String[] expected; - try { - windowed1.join(windowed2, joiner).process(processor); + KStreamTopology initializer = new MockKStreamTopology(); + processor = new MockProcessor<>(); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + boolean exceptionRaised = false; - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + try { + windowed1.join(windowed2, joiner).process(processor); - assertFalse(exceptionRaised); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - // push two items to the main stream. the other stream's window is empty + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - for (int i = 0; i < 2; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); - } + assertFalse(exceptionRaised); - assertEquals(0, processor.processed.size()); + // push two items to the main stream. the other stream's window is empty - // push two items to the other stream. the main stream's window has two items + for (int i = 0; i < 2; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + } - for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); - } + assertEquals(0, processor.processed.size()); - assertEquals(2, processor.processed.size()); + // push two items to the other stream. the main stream's window has two items - expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; + for (int i = 0; i < 2; i++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + } - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); - } + assertEquals(2, processor.processed.size()); - processor.processed.clear(); + expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; - // push all items to the main stream. this should produce two items. + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } - for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); - } + processor.processed.clear(); - assertEquals(2, processor.processed.size()); + // push all items to the main stream. this should produce two items. - expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; + for (int i = 0; i < expectedKeys.length; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + } - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); - } + assertEquals(2, processor.processed.size()); - processor.processed.clear(); + expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; - // there will be previous two items + all items in the main stream's window, thus two are duplicates. + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } - // push all items to the other stream. this should produce 6 items - for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); - } + processor.processed.clear(); + + // there will be previous two items + all items in the main stream's window, thus two are duplicates. - assertEquals(6, processor.processed.size()); + // push all items to the other stream. this should produce 6 items + for (int i = 0; i < expectedKeys.length; i++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + } - expected = new String[]{"0:X0+Y0", "0:X0+Y0", "1:X1+Y1", "1:X1+Y1", "2:X2+Y2", "3:X3+Y3"}; + assertEquals(6, processor.processed.size()); - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); + 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)); + } } - } - @Test - public void testJoinPrior() { + @Test + public void testJoinPrior() { - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KStreamSource stream1; - KStreamSource stream2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; - String[] expected; + KStreamSource stream1; + KStreamSource stream2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + MockProcessor processor; + String[] expected; <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -205,86 +233,98 @@ public void testJoinPrior() { stream2 = new KStreamSource<>(null, initializer); windowed1 = stream1.with(new UnlimitedWindow()); windowed2 = stream2.with(new UnlimitedWindow()); +======= + KStreamTopology initializer = new MockKStreamTopology(); + processor = new MockProcessor<>(); + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); +>>>>>>> compile and test passed - boolean exceptionRaised = false; + boolean exceptionRaised = false; - try { - windowed1.joinPrior(windowed2, joiner).process(processor); + try { + windowed1.joinPrior(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - assertFalse(exceptionRaised); + assertFalse(exceptionRaised); - // push two items to the main stream. the other stream's window is empty + // push two items to the main stream. the other stream's window is empty - for (int i = 0; i < 2; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i); - } + for (int i = 0; i < 2; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i); + } - assertEquals(0, processor.processed.size()); + assertEquals(0, processor.processed.size()); - // push two items to the other stream. the main stream's window has two items - // no corresponding item in the main window has a newer timestamp + // push two items to the other stream. the main stream's window has two items + // no corresponding item in the main window has a newer timestamp - for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i + 1); - } + for (int i = 0; i < 2; i++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i + 1); + } - assertEquals(0, processor.processed.size()); + assertEquals(0, processor.processed.size()); - processor.processed.clear(); + processor.processed.clear(); - // push all items with newer timestamps to the main stream. this should produce two items. + // push all items with newer timestamps to the main stream. this should produce two items. - for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i + 2); - } + for (int i = 0; i < expectedKeys.length; i++) { + stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i + 2); + } - assertEquals(2, processor.processed.size()); + assertEquals(2, processor.processed.size()); - expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; + expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); - } + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } - processor.processed.clear(); + processor.processed.clear(); - // there will be previous two items + all items in the main stream's window, thus two are duplicates. + // there will be previous two items + all items in the main stream's window, thus two are duplicates. - // push all items with older timestamps to the other stream. this should produce six items - for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i); - } + // push all items with older timestamps to the other stream. this should produce six items + for (int i = 0; i < expectedKeys.length; i++) { + stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i); + } + + assertEquals(6, processor.processed.size()); - 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"}; - 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)); + } - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); } - } + @Test + public void testMap() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + MockProcessor processor; - @Test - public void testMap() { - KStreamSource stream1; - KStreamSource stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; + KStreamTopology initializer = new MockKStreamTopology(); + processor = new MockProcessor<>(); <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); <<<<<<< HEAD @@ -307,88 +347,93 @@ public void testMap() { ======= processor = new MockProcessor<>(); >>>>>>> removing io.confluent imports: wip +======= + boolean exceptionRaised; +>>>>>>> compile and test passed - boolean exceptionRaised; + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.map(keyValueMapper); + mapped2 = stream2.map(keyValueMapper); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.map(keyValueMapper); - mapped2 = stream2.map(keyValueMapper); + exceptionRaised = false; + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + assertTrue(exceptionRaised); - assertTrue(exceptionRaised); + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.map(keyValueMapper); + mapped2 = stream2.map(keyValueMapper); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.map(keyValueMapper); - mapped2 = stream2.map(keyValueMapper); + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + assertTrue(exceptionRaised); - assertTrue(exceptionRaised); + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.map(keyValueMapper); + mapped2 = stream2.map(keyValueMapper); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.map(keyValueMapper); - mapped2 = stream2.map(keyValueMapper); + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; + assertTrue(exceptionRaised); } - assertTrue(exceptionRaised); - } + @Test + public void testFlatMap() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + MockProcessor processor; - @Test - public void testFlatMap() { - KStreamSource stream1; - KStreamSource stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; + KStreamTopology initializer = new MockKStreamTopology(); + processor = new MockProcessor<>(); <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); @@ -411,88 +456,93 @@ public void testFlatMap() { ======= processor = new MockProcessor<>(); >>>>>>> removing io.confluent imports: wip +======= + boolean exceptionRaised; +>>>>>>> compile and test passed - boolean exceptionRaised; + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMap(keyValueMapper2); + mapped2 = stream2.flatMap(keyValueMapper2); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.flatMap(keyValueMapper2); - mapped2 = stream2.flatMap(keyValueMapper2); + exceptionRaised = false; + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + assertTrue(exceptionRaised); - assertTrue(exceptionRaised); + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMap(keyValueMapper2); + mapped2 = stream2.flatMap(keyValueMapper2); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.flatMap(keyValueMapper2); - mapped2 = stream2.flatMap(keyValueMapper2); + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + assertTrue(exceptionRaised); - assertTrue(exceptionRaised); + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMap(keyValueMapper2); + mapped2 = stream2.flatMap(keyValueMapper2); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.flatMap(keyValueMapper2); - mapped2 = stream2.flatMap(keyValueMapper2); + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; + assertTrue(exceptionRaised); } - assertTrue(exceptionRaised); - } + @Test + public void testMapValues() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + MockProcessor processor; - @Test - public void testMapValues() { - KStreamSource stream1; - KStreamSource stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; + KStreamTopology initializer = new MockKStreamTopology(); + processor = new MockProcessor<>(); <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); @@ -515,88 +565,93 @@ public void testMapValues() { ======= processor = new MockProcessor<>(); >>>>>>> removing io.confluent imports: wip +======= + boolean exceptionRaised; +>>>>>>> compile and test passed - boolean exceptionRaised; + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.mapValues(valueMapper); + mapped2 = stream2.mapValues(valueMapper); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.mapValues(valueMapper); - mapped2 = stream2.mapValues(valueMapper); + exceptionRaised = false; + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + assertFalse(exceptionRaised); - assertFalse(exceptionRaised); + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.mapValues(valueMapper); + mapped2 = stream2.mapValues(valueMapper); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.mapValues(valueMapper); - mapped2 = stream2.mapValues(valueMapper); + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + assertFalse(exceptionRaised); - assertFalse(exceptionRaised); + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.mapValues(valueMapper); + mapped2 = stream2.mapValues(valueMapper); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.mapValues(valueMapper); - mapped2 = stream2.mapValues(valueMapper); + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; + assertFalse(exceptionRaised); } - assertFalse(exceptionRaised); - } + @Test + public void testFlatMapValues() { + KStreamSource stream1; + KStreamSource stream2; + KStream mapped1; + KStream mapped2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + MockProcessor processor; - @Test - public void testFlatMapValues() { - KStreamSource stream1; - KStreamSource stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; + KStreamTopology initializer = new MockKStreamTopology(); + processor = new MockProcessor<>(); <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); processor = new TestProcessor<>(); @@ -619,74 +674,75 @@ public void testFlatMapValues() { ======= processor = new MockProcessor<>(); >>>>>>> removing io.confluent imports: wip +======= + boolean exceptionRaised; +>>>>>>> compile and test passed - boolean exceptionRaised; + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMapValues(valueMapper2); + mapped2 = stream2.flatMapValues(valueMapper2); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.flatMapValues(valueMapper2); - mapped2 = stream2.flatMapValues(valueMapper2); + exceptionRaised = false; + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + assertFalse(exceptionRaised); - assertFalse(exceptionRaised); + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMapValues(valueMapper2); + mapped2 = stream2.flatMapValues(valueMapper2); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.flatMapValues(valueMapper2); - mapped2 = stream2.flatMapValues(valueMapper2); + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + assertFalse(exceptionRaised); - assertFalse(exceptionRaised); + try { + stream1 = new KStreamSource<>(null, initializer); + stream2 = new KStreamSource<>(null, initializer); + mapped1 = stream1.flatMapValues(valueMapper2); + mapped2 = stream2.flatMapValues(valueMapper2); - try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.flatMapValues(valueMapper2); - mapped2 = stream2.flatMapValues(valueMapper2); + exceptionRaised = false; + windowed1 = mapped1.with(new UnlimitedWindow()); + windowed2 = mapped2.with(new UnlimitedWindow()); - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1.join(windowed2, joiner).process(processor); - windowed1.join(windowed2, joiner).process(processor); + KStreamContext context = new MockKStreamContext(null, null); + stream1.bind(context, streamMetadata); + stream2.bind(context, streamMetadata); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + } catch (NotCopartitionedException e) { + exceptionRaised = true; + } - } catch (NotCopartitionedException e) { - exceptionRaised = true; + assertFalse(exceptionRaised); } - assertFalse(exceptionRaised); - } - } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java index 0ec3d83de099a..f738e62798f54 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java @@ -1,11 +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.stream; -import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.KeyValueMapper; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; @@ -17,25 +34,26 @@ public class KStreamMapTest { - private String topicName = "topic"; + private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - @Test - public void testMap() { + @Test + public void testMap() { - KeyValueMapper mapper = - new KeyValueMapper() { - @Override - public KeyValue apply(Integer key, String value) { - return KeyValue.pair(value, key); - } - }; + KeyValueMapper mapper = + new KeyValueMapper() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(value, key); + } + }; - final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -49,24 +67,29 @@ public KeyValue apply(Integer key, String value) { >>>>>>> wip KStreamSource stream; MockProcessor processor; +======= + KStreamTopology initializer = new MockKStreamTopology(); + KStreamSource stream; + MockProcessor processor; +>>>>>>> compile and test passed - processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, initializer); - stream.map(mapper).process(processor); + processor = new MockProcessor<>(); + stream = new KStreamSource<>(null, initializer); + stream.map(mapper).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); - } + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } - assertEquals(4, processor.processed.size()); + assertEquals(4, processor.processed.size()); - String[] expected = new String[] { "V0:0", "V1:1", "V2:2", "V3:3" }; + 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)); + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } } - } } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java index 11aed82b229cd..d0da399c0f38a 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java @@ -1,5 +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.stream; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD @@ -18,10 +36,13 @@ import org.apache.kafka.clients.processor.KStreamContext; ======= import org.apache.kafka.stream.internal.PartitioningInfo; +======= +import org.apache.kafka.stream.internals.PartitioningInfo; +>>>>>>> compile and test passed import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.ValueMapper; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; @@ -34,25 +55,26 @@ public class KStreamMapValuesTest { - private String topicName = "topic"; + private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - @Test - public void testFlatMapValues() { + @Test + public void testFlatMapValues() { - ValueMapper mapper = - new ValueMapper() { - @Override - public Integer apply(String value) { - return value.length(); - } - }; + ValueMapper mapper = + new ValueMapper() { + @Override + public Integer apply(String value) { + return value.length(); + } + }; - final int[] expectedKeys = new int[] { 1, 10, 100, 1000 }; + final int[] expectedKeys = new int[]{1, 10, 100, 1000}; <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -66,24 +88,29 @@ public Integer apply(String value) { >>>>>>> wip KStreamSource stream; MockProcessor processor; +======= + KStreamTopology initializer = new MockKStreamTopology(); + KStreamSource stream; + MockProcessor processor; +>>>>>>> compile and test passed - processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, initializer); - stream.mapValues(mapper).process(processor); + processor = new MockProcessor<>(); + stream = new KStreamSource<>(null, initializer); + stream.mapValues(mapper).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L); - } + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L); + } - assertEquals(4, processor.processed.size()); + assertEquals(4, processor.processed.size()); - String[] expected = new String[] { "1:1", "10:2", "100:3", "1000:4" }; + String[] expected = new String[]{"1:1", "10:2", "100:3", "1000:4"}; - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } } - } } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java index 67099ae85c249..45e061d048b1a 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java @@ -1,9 +1,26 @@ +/** + * 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.stream; -import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; @@ -15,15 +32,16 @@ public class KStreamSourceTest { - private String topicName = "topic"; + private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - @Test - public void testKStreamSource() { + @Test + public void testKStreamSource() { <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); <<<<<<< HEAD @@ -40,24 +58,28 @@ public void testKStreamSource() { ======= MockProcessor processor = new MockProcessor<>(); >>>>>>> removing io.confluent imports: wip +======= + KStreamTopology initializer = new MockKStreamTopology(); + MockProcessor processor = new MockProcessor<>(); +>>>>>>> compile and test passed - KStreamSource stream = new KStreamSource<>(null, initializer); - stream.process(processor); + KStreamSource stream = new KStreamSource<>(null, initializer); + stream.process(processor); - final String[] expectedKeys = new String[] { "k1", "k2", "k3" }; - final String[] expectedValues = new String[] { "v1", "v2", "v3" }; + final String[] expectedKeys = new String[]{"k1", "k2", "k3"}; + final String[] expectedValues = new String[]{"v1", "v2", "v3"}; - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], expectedValues[i], 0L); - } + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], expectedValues[i], 0L); + } - assertEquals(3, processor.processed.size()); + assertEquals(3, processor.processed.size()); - for (int i = 0; i < expectedKeys.length; i++) { - assertEquals(expectedKeys[i] + ":" + expectedValues[i], processor.processed.get(i)); + for (int i = 0; i < expectedKeys.length; i++) { + assertEquals(expectedKeys[i] + ":" + expectedValues[i], processor.processed.get(i)); + } } - } } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java index ecfb24cded090..8faeeacda5b41 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java @@ -1,10 +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.stream; -import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Transformer; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockKStreamContext; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; @@ -16,97 +33,110 @@ public class KStreamTransformTest { - private String topicName = "topic"; - - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - - @Test - public void testTransform() { - final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - - KStreamTopology topology = new MockKStreamTopology(); - KStreamSource stream; - MockProcessor processor; - Transformer transformer; - - processor = new MockProcessor<>(); - transformer = new Transformer() { - KStreamContext context; - Forwarder forwarder; - public void init(KStreamContext context) { - this.context = context; - } - public void forwarder(Forwarder forwarder) { - this.forwarder = forwarder; - } - public void process(String key, String value) { - forwarder.send(Integer.parseInt(value), 0, 0L); - } - public void punctuate(long timestamp) {} - public void close() {} - }; - - stream = new KStreamSource<>(null, topology); - - stream.transform(transformer).process(processor); - - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(null, Integer.toString(expectedKeys[i]), 0L); - } + private String topicName = "topic"; - assertEquals(expectedKeys.length, processor.processed.size()); - for (int i = 0; i < expectedKeys.length; i++) { - assertEquals(expectedKeys[i]+ ":" + 0, processor.processed.get(i)); - } - } - - @Test - public void testTransformEmitOnPuncutation() { - final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - - KStreamTopology topology = new MockKStreamTopology(); - KStreamSource stream; - MockProcessor processor; - Transformer transformer; - - processor = new MockProcessor<>(); - transformer = new Transformer() { - KStreamContext context; - Forwarder forwarder; - Integer currentKey; - public void init(KStreamContext context) { - this.context = context; - } - public void forwarder(Forwarder forwarder) { - this.forwarder = forwarder; - } - public void process(Integer key, String value) { - currentKey = Integer.parseInt(value); - } - public void punctuate(long timestamp) { - forwarder.send(currentKey, 0, 0L); - } - public void close() {} - }; - - stream = new KStreamSource<>(null, topology); - - stream.transform(transformer).process(processor); - - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(null, Integer.toString(expectedKeys[i]), 0L); - if (i % 3 == 2) transformer.punctuate(0L); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + + @Test + public void testTransform() { + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + + KStreamTopology topology = new MockKStreamTopology(); + KStreamSource stream; + MockProcessor processor; + Transformer transformer; + + processor = new MockProcessor<>(); + transformer = new Transformer() { + KStreamContext context; + Forwarder forwarder; + + public void init(KStreamContext context) { + this.context = context; + } + + public void forwarder(Forwarder forwarder) { + this.forwarder = forwarder; + } + + public void process(String key, String value) { + forwarder.send(Integer.parseInt(value), 0, 0L); + } + + public void punctuate(long timestamp) { + } + + public void close() { + } + }; + + stream = new KStreamSource<>(null, topology); + + stream.transform(transformer).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(null, Integer.toString(expectedKeys[i]), 0L); + } + + assertEquals(expectedKeys.length, processor.processed.size()); + for (int i = 0; i < expectedKeys.length; i++) { + assertEquals(expectedKeys[i] + ":" + 0, processor.processed.get(i)); + } } - final int[] expected = new int[] { 3, 6 }; - assertEquals(2, processor.processed.size()); - for (int i = 0; i < 2; i++) { - assertEquals(expected[i] + ":" + 0, processor.processed.get(i)); + @Test + public void testTransformEmitOnPuncutation() { + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + + KStreamTopology topology = new MockKStreamTopology(); + KStreamSource stream; + MockProcessor processor; + Transformer transformer; + + processor = new MockProcessor<>(); + transformer = new Transformer() { + KStreamContext context; + Forwarder forwarder; + Integer currentKey; + + public void init(KStreamContext context) { + this.context = context; + } + + public void forwarder(Forwarder forwarder) { + this.forwarder = forwarder; + } + + public void process(Integer key, String value) { + currentKey = Integer.parseInt(value); + } + + public void punctuate(long timestamp) { + forwarder.send(currentKey, 0, 0L); + } + + public void close() { + } + }; + + stream = new KStreamSource<>(null, topology); + + stream.transform(transformer).process(processor); + + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(null, Integer.toString(expectedKeys[i]), 0L); + if (i % 3 == 2) transformer.punctuate(0L); + } + + final int[] expected = new int[]{3, 6}; + assertEquals(2, processor.processed.size()); + for (int i = 0; i < 2; i++) { + assertEquals(expected[i] + ":" + 0, processor.processed.get(i)); + } } - } } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java index 8b67c6add8138..0d201f2b20b7d 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java @@ -1,10 +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.stream; -import org.apache.kafka.stream.internal.PartitioningInfo; +import org.apache.kafka.stream.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Window; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockKStreamContext; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.UnlimitedWindow; @@ -17,15 +34,16 @@ public class KStreamWindowedTest { - private String topicName = "topic"; + private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - @Test - public void testWindowedStream() { + @Test + public void testWindowedStream() { - final int[] expectedKeys = new int[] { 0, 1, 2, 3 }; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; +<<<<<<< HEAD KStreamSource stream; Window window; <<<<<<< HEAD @@ -42,46 +60,51 @@ public void testWindowedStream() { ======= KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> wip +======= + KStreamSource stream; + Window window; + KStreamTopology initializer = new MockKStreamTopology(); +>>>>>>> compile and test passed - window = new UnlimitedWindow<>(); - stream = new KStreamSource<>(null, initializer); - stream.with(window); + window = new UnlimitedWindow<>(); + stream = new KStreamSource<>(null, initializer); + stream.with(window); - boolean exceptionRaised = false; + boolean exceptionRaised = false; - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); + KStreamContext context = new MockKStreamContext(null, null); + stream.bind(context, streamMetadata); - // two items in the window + // two items in the window - for (int i = 0; i < 2; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); - } + for (int i = 0; i < 2; i++) { + stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + } - 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))); + 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 + // previous two items + all items, thus two are duplicates, in the window - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); - } + for (int i = 0; i < expectedKeys.length; i++) { + stream.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + } - 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))); - } + assertEquals(2, countItem(window.find(0, 0L))); + assertEquals(2, countItem(window.find(1, 0L))); + assertEquals(1, countItem(window.find(2, 0L))); + assertEquals(1, countItem(window.find(3, 0L))); + } - private int countItem(Iterator iter) { - int i = 0; - while (iter.hasNext()) { - i++; - iter.next(); + private int countItem(Iterator iter) { + int i = 0; + while (iter.hasNext()) { + i++; + iter.next(); + } + return i; } - return i; - } } diff --git a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java index 273978f173cf3..8e58407950dea 100644 --- a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java @@ -1,3 +1,20 @@ +/** + * 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.stream; import static org.junit.Assert.assertEquals; @@ -8,71 +25,71 @@ public class MinTimestampTrackerTest { - private Stamped elem(long timestamp) { - return new Stamped<>("", timestamp); - } + private Stamped elem(long timestamp) { + return new Stamped<>("", timestamp); + } - @SuppressWarnings("unchecked") - @Test - public void testTracking() { - MinTimestampTracker tracker = new MinTimestampTracker(); + @SuppressWarnings("unchecked") + @Test + public void testTracking() { + MinTimestampTracker tracker = new MinTimestampTracker(); - Object[] elems = new Object[] { - elem(100), elem(101), elem(102), elem(98), elem(99), elem(100) - }; + Object[] elems = new Object[]{ + elem(100), elem(101), elem(102), elem(98), elem(99), elem(100) + }; - int insertionIndex = 0; - int removalIndex = 0; + int insertionIndex = 0; + int removalIndex = 0; - // add 100 - tracker.addStampedElement((Stamped)elems[insertionIndex++]); - assertEquals(100L, tracker.get()); + // add 100 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); - // add 101 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(100L, tracker.get()); + // add 101 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); - // remove 100 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(101L, tracker.get()); + // remove 100 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(101L, tracker.get()); - // add 102 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(101L, tracker.get()); + // add 102 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(101L, tracker.get()); - // add 98 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); + // add 98 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); - // add 99 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); + // add 99 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); - // add 100 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); + // add 100 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); - // remove 101 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(98L, tracker.get()); + // remove 101 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); - // remove 102 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(98L, tracker.get()); + // remove 102 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); - // remove 98 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(99L, tracker.get()); + // remove 98 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(99L, tracker.get()); - // remove 99 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(100L, tracker.get()); + // remove 99 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(100L, tracker.get()); - // remove 100 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(-1L, tracker.get()); + // remove 100 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(-1L, tracker.get()); - assertEquals(insertionIndex, removalIndex); - } + assertEquals(insertionIndex, removalIndex); + } } diff --git a/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java b/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java index 800ccaec9ffc8..31b5e324a5dcd 100644 --- a/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java @@ -1,3 +1,20 @@ +/** + * 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.stream; import org.apache.kafka.stream.util.ParallelExecutor; @@ -12,112 +29,109 @@ public class ParallelExecutorTest { - @Test - public void testExecutingShortTaskList() throws Exception { - ParallelExecutor parallelExecutor = new ParallelExecutor(10); - ArrayList taskList = new ArrayList<>(); - AtomicInteger counter = new AtomicInteger(0); + @Test + public void testExecutingShortTaskList() throws Exception { + ParallelExecutor parallelExecutor = new ParallelExecutor(10); + ArrayList taskList = new ArrayList<>(); + AtomicInteger counter = new AtomicInteger(0); - for (int i = 0; i < 5; i++) { - taskList.add(new TestTask(counter)); - } + for (int i = 0; i < 5; i++) { + taskList.add(new TestTask(counter)); + } - parallelExecutor.execute(taskList); + parallelExecutor.execute(taskList); - for (TestTask task : taskList) { - assertEquals(task.executionCount, 1); - } - assertEquals(counter.get(), taskList.size()); + for (TestTask task : taskList) { + assertEquals(task.executionCount, 1); + } + assertEquals(counter.get(), taskList.size()); - parallelExecutor.execute(taskList); + parallelExecutor.execute(taskList); - for (TestTask task : taskList) { - assertEquals(task.executionCount, 2); + for (TestTask task : taskList) { + assertEquals(task.executionCount, 2); + } + assertEquals(counter.get(), taskList.size() * 2); } - assertEquals(counter.get(), taskList.size() * 2); - } - @Test - public void testExecutingLongTaskList() throws Exception { - ParallelExecutor parallelExecutor = new ParallelExecutor(10); - ArrayList taskList = new ArrayList<>(); - AtomicInteger counter = new AtomicInteger(0); + @Test + public void testExecutingLongTaskList() throws Exception { + ParallelExecutor parallelExecutor = new ParallelExecutor(10); + ArrayList taskList = new ArrayList<>(); + AtomicInteger counter = new AtomicInteger(0); - for (int i = 0; i < 20; i++) { - taskList.add(new TestTask(counter)); - } + for (int i = 0; i < 20; i++) { + taskList.add(new TestTask(counter)); + } - parallelExecutor.execute(taskList); + parallelExecutor.execute(taskList); - for (TestTask task : taskList) { - assertEquals(task.executionCount, 1); - } - assertEquals(counter.get(), taskList.size()); + for (TestTask task : taskList) { + assertEquals(task.executionCount, 1); + } + assertEquals(counter.get(), taskList.size()); - parallelExecutor.execute(taskList); + parallelExecutor.execute(taskList); - for (TestTask task : taskList) { - assertEquals(task.executionCount, 2); - } - assertEquals(counter.get(), taskList.size() * 2); - } - - @Test - public void testException() { - ParallelExecutor parallelExecutor = new ParallelExecutor(10); - ArrayList taskList = new ArrayList<>(); - AtomicInteger counter = new AtomicInteger(0); - - for (int i = 0; i < 20; i++) { - if (i == 15) { - taskList.add(new TestTask(counter) { - @Override - public boolean process() { - throw new TestException(); - } - }); - } - else { - taskList.add(new TestTask(counter)); - } + for (TestTask task : taskList) { + assertEquals(task.executionCount, 2); + } + assertEquals(counter.get(), taskList.size() * 2); } - Exception exception = null; - try { - parallelExecutor.execute(taskList); - } - catch (Exception ex) { - exception = ex; + @Test + public void testException() { + ParallelExecutor parallelExecutor = new ParallelExecutor(10); + ArrayList taskList = new ArrayList<>(); + AtomicInteger counter = new AtomicInteger(0); + + for (int i = 0; i < 20; i++) { + if (i == 15) { + taskList.add(new TestTask(counter) { + @Override + public boolean process() { + throw new TestException(); + } + }); + } else { + taskList.add(new TestTask(counter)); + } + } + + Exception exception = null; + try { + parallelExecutor.execute(taskList); + } catch (Exception ex) { + exception = ex; + } + + assertEquals(counter.get(), taskList.size() - 1); + assertFalse(exception == null); + assertTrue(exception instanceof TestException); } - assertEquals(counter.get(), taskList.size() - 1); - assertFalse(exception == null); - assertTrue(exception instanceof TestException); - } - - private static class TestTask implements ParallelExecutor.Task { - public volatile int executionCount = 0; - private AtomicInteger counter; - - TestTask(AtomicInteger counter) { - this.counter = counter; + private static class TestTask implements ParallelExecutor.Task { + public volatile int executionCount = 0; + private AtomicInteger counter; + + TestTask(AtomicInteger counter) { + this.counter = counter; + } + + @Override + public boolean process() { + try { + Thread.sleep(20); + executionCount++; + } catch (Exception ex) { + // ignore + } + counter.incrementAndGet(); + + return true; + } } - @Override - public boolean process() { - try { - Thread.sleep(20); - executionCount++; - } - catch (Exception ex) { - // ignore - } - counter.incrementAndGet(); - - return true; + private static class TestException extends RuntimeException { } - } - - private static class TestException extends RuntimeException { - } } diff --git a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java index 07b4ca5ce38ce..92821cc719e2f 100644 --- a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java @@ -1,3 +1,20 @@ +/** + * 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.stream; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -6,9 +23,9 @@ 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.stream.internal.StreamGroup; -import org.apache.kafka.stream.internal.TimeBasedChooser; -import org.apache.kafka.stream.topology.internal.KStreamSource; +import org.apache.kafka.stream.internals.StreamGroup; +import org.apache.kafka.stream.internals.TimeBasedChooser; +import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockIngestor; import org.apache.kafka.test.MockKStreamContext; import org.apache.kafka.test.MockKStreamTopology; @@ -18,21 +35,14 @@ import java.util.Arrays; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class StreamGroupTest { - private static Serializer serializer = new IntegerSerializer(); - private static Deserializer deserializer = new IntegerDeserializer(); - - private static class MockKStreamSource extends KStreamSource { - - public int numReceived = 0; - public ArrayList keys = new ArrayList<>(); - public ArrayList values = new ArrayList<>(); - public ArrayList timestamps = new ArrayList<>(); + private static Serializer serializer = new IntegerSerializer(); + private static Deserializer deserializer = new IntegerDeserializer(); +<<<<<<< HEAD public MockKStreamSource() { <<<<<<< HEAD <<<<<<< HEAD @@ -48,136 +58,148 @@ public MockKStreamSource() { super(null, deserializer, deserializer, new MockKStreamTopology()); >>>>>>> wip } +======= + private static class MockKStreamSource extends KStreamSource { +>>>>>>> compile and test passed - @Override - public void receive(Object key, Object value, long timestamp) { - this.numReceived++; - this.keys.add(key); - this.values.add(value); - this.timestamps.add(timestamp); - } + public int numReceived = 0; + public ArrayList keys = new ArrayList<>(); + public ArrayList values = new ArrayList<>(); + public ArrayList timestamps = new ArrayList<>(); - } + public MockKStreamSource() { + super(null, deserializer, deserializer, new MockKStreamTopology()); + } - @SuppressWarnings("unchecked") - @Test - public void testAddPartition() { + @Override + public void receive(Object key, Object value, long timestamp) { + this.numReceived++; + this.keys.add(key); + this.values.add(value); + this.timestamps.add(timestamp); + } - MockIngestor mockIngestor = new MockIngestor(); + } - StreamGroup streamGroup = new StreamGroup( - new MockKStreamContext(serializer, deserializer), - mockIngestor, - new TimeBasedChooser(), - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - if (topic.equals("topic1")) - return ((Integer)key).longValue(); - else - return ((Integer)key).longValue() / 10L + 5L; + @SuppressWarnings("unchecked") + @Test + public void testAddPartition() { + + MockIngestor mockIngestor = new MockIngestor(); + + StreamGroup streamGroup = new StreamGroup( + new MockKStreamContext(serializer, deserializer), + mockIngestor, + new TimeBasedChooser(), + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { + if (topic.equals("topic1")) + return ((Integer) key).longValue(); + else + return ((Integer) key).longValue() / 10L + 5L; + } + }, + 3 + ); + + TopicPartition partition1 = new TopicPartition("topic1", 1); + TopicPartition partition2 = new TopicPartition("topic2", 1); + MockKStreamSource stream1 = new MockKStreamSource(); + MockKStreamSource stream2 = new MockKStreamSource(); + MockKStreamSource stream3 = new MockKStreamSource(); + + streamGroup.addPartition(partition1, stream1); + mockIngestor.addPartitionStreamToGroup(streamGroup, partition1); + + streamGroup.addPartition(partition2, stream2); + mockIngestor.addPartitionStreamToGroup(streamGroup, partition2); + + Exception exception = null; + try { + streamGroup.addPartition(partition1, stream3); + } catch (Exception ex) { + exception = ex; } - }, - 3 - ); - - TopicPartition partition1 = new TopicPartition("topic1", 1); - TopicPartition partition2 = new TopicPartition("topic2", 1); - MockKStreamSource stream1 = new MockKStreamSource(); - MockKStreamSource stream2 = new MockKStreamSource(); - MockKStreamSource stream3 = new MockKStreamSource(); - - streamGroup.addPartition(partition1, stream1); - mockIngestor.addPartitionStreamToGroup(streamGroup, partition1); - - streamGroup.addPartition(partition2, stream2); - mockIngestor.addPartitionStreamToGroup(streamGroup, partition2); - - Exception exception = null; - try { - streamGroup.addPartition(partition1, stream3); - } catch (Exception ex) { - exception = ex; - } - assertTrue(exception != null); + assertTrue(exception != null); - byte[] recordValue = serializer.serialize(null, new Integer(10)); + byte[] recordValue = serializer.serialize(null, new Integer(10)); - mockIngestor.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 1, serializer.serialize(partition1.topic(), new Integer(10)), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 2, serializer.serialize(partition1.topic(), new Integer(20)), recordValue) - )); + mockIngestor.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 1, serializer.serialize(partition1.topic(), new Integer(10)), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 2, serializer.serialize(partition1.topic(), new Integer(20)), recordValue) + )); - mockIngestor.addRecords(partition2, records( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 1, serializer.serialize(partition1.topic(), new Integer(300)), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 2, serializer.serialize(partition1.topic(), new Integer(400)), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 3, serializer.serialize(partition1.topic(), new Integer(500)), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 4, serializer.serialize(partition1.topic(), new Integer(600)), recordValue) - )); + mockIngestor.addRecords(partition2, records( + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 1, serializer.serialize(partition1.topic(), new Integer(300)), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 2, serializer.serialize(partition1.topic(), new Integer(400)), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 3, serializer.serialize(partition1.topic(), new Integer(500)), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 4, serializer.serialize(partition1.topic(), new Integer(600)), recordValue) + )); - streamGroup.process(); - assertEquals(stream1.numReceived, 1); - assertEquals(stream2.numReceived, 0); + streamGroup.process(); + assertEquals(stream1.numReceived, 1); + assertEquals(stream2.numReceived, 0); - assertEquals(mockIngestor.paused.size(), 1); - assertTrue(mockIngestor.paused.contains(partition2)); + assertEquals(mockIngestor.paused.size(), 1); + assertTrue(mockIngestor.paused.contains(partition2)); - mockIngestor.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 3, serializer.serialize(partition1.topic(), new Integer(30)), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 4, serializer.serialize(partition1.topic(), new Integer(40)), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 5, serializer.serialize(partition1.topic(), new Integer(50)), recordValue) - )); + mockIngestor.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 3, serializer.serialize(partition1.topic(), new Integer(30)), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 4, serializer.serialize(partition1.topic(), new Integer(40)), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 5, serializer.serialize(partition1.topic(), new Integer(50)), recordValue) + )); - streamGroup.process(); - assertEquals(stream1.numReceived, 2); - assertEquals(stream2.numReceived, 0); + streamGroup.process(); + assertEquals(stream1.numReceived, 2); + assertEquals(stream2.numReceived, 0); - assertEquals(mockIngestor.paused.size(), 2); - assertTrue(mockIngestor.paused.contains(partition1)); - assertTrue(mockIngestor.paused.contains(partition2)); + assertEquals(mockIngestor.paused.size(), 2); + assertTrue(mockIngestor.paused.contains(partition1)); + assertTrue(mockIngestor.paused.contains(partition2)); - streamGroup.process(); - assertEquals(stream1.numReceived, 3); - assertEquals(stream2.numReceived, 0); + streamGroup.process(); + assertEquals(stream1.numReceived, 3); + assertEquals(stream2.numReceived, 0); - streamGroup.process(); - assertEquals(stream1.numReceived, 3); - assertEquals(stream2.numReceived, 1); + streamGroup.process(); + assertEquals(stream1.numReceived, 3); + assertEquals(stream2.numReceived, 1); - assertEquals(mockIngestor.paused.size(), 1); - assertTrue(mockIngestor.paused.contains(partition2)); + assertEquals(mockIngestor.paused.size(), 1); + assertTrue(mockIngestor.paused.contains(partition2)); - streamGroup.process(); - assertEquals(stream1.numReceived, 4); - assertEquals(stream2.numReceived, 1); + streamGroup.process(); + assertEquals(stream1.numReceived, 4); + assertEquals(stream2.numReceived, 1); - assertEquals(mockIngestor.paused.size(), 1); + assertEquals(mockIngestor.paused.size(), 1); - streamGroup.process(); - assertEquals(stream1.numReceived, 4); - assertEquals(stream2.numReceived, 2); + streamGroup.process(); + assertEquals(stream1.numReceived, 4); + assertEquals(stream2.numReceived, 2); - assertEquals(mockIngestor.paused.size(), 0); + assertEquals(mockIngestor.paused.size(), 0); - streamGroup.process(); - assertEquals(stream1.numReceived, 5); - assertEquals(stream2.numReceived, 2); + streamGroup.process(); + assertEquals(stream1.numReceived, 5); + assertEquals(stream2.numReceived, 2); - streamGroup.process(); - assertEquals(stream1.numReceived, 5); - assertEquals(stream2.numReceived, 3); + streamGroup.process(); + assertEquals(stream1.numReceived, 5); + assertEquals(stream2.numReceived, 3); - streamGroup.process(); - assertEquals(stream1.numReceived, 5); - assertEquals(stream2.numReceived, 4); + streamGroup.process(); + assertEquals(stream1.numReceived, 5); + assertEquals(stream2.numReceived, 4); - assertEquals(mockIngestor.paused.size(), 0); + assertEquals(mockIngestor.paused.size(), 0); - streamGroup.process(); - assertEquals(stream1.numReceived, 5); - assertEquals(stream2.numReceived, 4); - } + streamGroup.process(); + assertEquals(stream1.numReceived, 5); + assertEquals(stream2.numReceived, 4); + } - private Iterable> records(ConsumerRecord... recs) { - return Arrays.asList(recs); - } + private Iterable> records(ConsumerRecord... recs) { + return Arrays.asList(recs); + } } diff --git a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java index 56bf9946e81d2..4d1048a90c32a 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java @@ -1,9 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.kafka.test; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.stream.internal.Ingestor; -import org.apache.kafka.stream.internal.StreamGroup; +import org.apache.kafka.stream.internals.Ingestor; +import org.apache.kafka.stream.internals.StreamGroup; import java.util.HashMap; import java.util.HashSet; @@ -12,44 +29,44 @@ public class MockIngestor implements Ingestor { - private HashMap streamSynchronizers = new HashMap<>(); + private HashMap streamSynchronizers = new HashMap<>(); - public HashSet paused = new HashSet<>(); + public HashSet paused = new HashSet<>(); - @Override - public Set topics() { - return null; - } + @Override + public Set topics() { + return null; + } - @Override - public void poll(long timeoutMs) { - } + @Override + public void poll(long timeoutMs) { + } - @Override - public void pause(TopicPartition partition) { - paused.add(partition); - } + @Override + public void pause(TopicPartition partition) { + paused.add(partition); + } - @Override - public void unpause(TopicPartition partition, long offset) { - paused.remove(partition); - } + @Override + public void unpause(TopicPartition partition, long offset) { + paused.remove(partition); + } - @Override - public void commit(Map offsets) { /* do nothing */} + @Override + public void commit(Map offsets) { /* do nothing */} - @Override - public int numPartitions(String topic) { - return 1; - } + @Override + public int numPartitions(String topic) { + return 1; + } - @Override - public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition) { - streamSynchronizers.put(partition, streamGroup); - } + @Override + public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition) { + streamSynchronizers.put(partition, streamGroup); + } - public void addRecords(TopicPartition partition, Iterable> records) { - streamSynchronizers.get(partition).addRecords(partition, records.iterator()); - } + public void addRecords(TopicPartition partition, Iterable> records) { + streamSynchronizers.get(partition).addRecords(partition, records.iterator()); + } } diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java index 30451defba772..5116f413a8555 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java @@ -1,5 +1,21 @@ -package org.apache.kafka.test; +/** + * 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; <<<<<<< HEAD import io.confluent.streaming.KStreamContext; @@ -42,16 +58,14 @@ 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.stream.internal.PunctuationQueue; +import org.apache.kafka.stream.internals.PunctuationQueue; import java.io.File; import java.util.Map; -/** - * Created by guozhang on 7/11/15. - */ public class MockKStreamContext implements KStreamContext { +<<<<<<< HEAD Serializer serializer; Deserializer deserializer; private final PunctuationQueue punctuationQueue = new PunctuationQueue(); @@ -148,5 +162,110 @@ public void schedule(Processor processor, long interval) { @Override public long timestamp() { throw new UnsupportedOperationException("timestamp() not supported."); } +======= + Serializer serializer; + Deserializer deserializer; + private final PunctuationQueue punctuationQueue = new PunctuationQueue(); + + public MockKStreamContext(Serializer serializer, Deserializer deserializer) { + this.serializer = serializer; + this.deserializer = deserializer; + } + + @Override + public int id() { + return -1; + } + + @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 RecordCollector recordCollector() { + throw new UnsupportedOperationException("recordCollector() not supported."); + } + + @Override + public Map getContext() { + throw new UnsupportedOperationException("getContext() not supported."); + } + + @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) { + throw new UnsupportedOperationException("restore() not supported."); + } + + @Override + public void flush() { + throw new UnsupportedOperationException("flush() not supported."); + } + + @Override + public void send(String topic, Object key, Object value) { + throw new UnsupportedOperationException("send() not supported."); + } + + @Override + public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { + throw new UnsupportedOperationException("send() not supported."); + } + + @Override + public void schedule(Processor processor, long interval) { + throw new UnsupportedOperationException("schedule() not supported"); + } + + @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() { + throw new UnsupportedOperationException("timestamp() not supported."); + } +>>>>>>> compile and test passed } diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java index 07b0bf75536c7..c1ef5b221aa2d 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java @@ -1,13 +1,26 @@ -package org.apache.kafka.test; +/** + * 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.stream.topology.KStreamTopology; -/** - * Created by yasuhiro on 7/31/15. - */ public class MockKStreamTopology extends KStreamTopology { - @Override - public void topology() { - } + @Override + public void topology() { + } } diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java index ba2c2e7d519fe..3b1e9065aad5a 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -1,3 +1,20 @@ +/** + * 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.stream.topology.Processor; @@ -6,25 +23,25 @@ import java.util.ArrayList; public class MockProcessor implements Processor { - public final ArrayList processed = new ArrayList<>(); - public final ArrayList punctuated = new ArrayList<>(); - - @Override - public void process(K key, V value) { - processed.add(key + ":" + value); - } - - @Override - public void init(KStreamContext context) { - } - - @Override - public void punctuate(long streamTime) { - punctuated.add(streamTime); - } - - @Override - public void close() { - } + public final ArrayList processed = new ArrayList<>(); + public final ArrayList punctuated = new ArrayList<>(); + + @Override + public void process(K key, V value) { + processed.add(key + ":" + value); + } + + @Override + public void init(KStreamContext context) { + } + + @Override + public void punctuate(long streamTime) { + punctuated.add(streamTime); + } + + @Override + public void close() { + } } diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java index 9d77505d71ac7..f11b9df73faab 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java @@ -1,5 +1,21 @@ -package org.apache.kafka.test; +/** + * 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.stream.KStreamContext; import org.apache.kafka.stream.topology.KeyValue; @@ -12,57 +28,59 @@ public class UnlimitedWindow implements Window { - private LinkedList>> list = new LinkedList<>(); - - @Override - public void init(KStreamContext context) { - - } - @Override - public Iterator find(final K key, long timestamp) { - return find(key, Long.MIN_VALUE, timestamp); - } - - @Override - public Iterator findAfter(final K key, long timestamp) { - return find(key, timestamp, Long.MAX_VALUE); - } - - @Override - public Iterator findBefore(final K key, long timestamp) { - return find(key, Long.MIN_VALUE, Long.MAX_VALUE); - } - - private Iterator find(final K key, final long startTime, final long endTime) { - return new FilteredIterator>>(list.iterator()) { - protected V filter(Stamped> item) { - if (item.value.key.equals(key) && startTime <= item.timestamp && item.timestamp <= endTime) - return item.value.value; - else - return null; - } - }; - } - @Override - public void put(K key, V value, long timestamp) { - list.add(new Stamped>(KeyValue.pair(key, value), timestamp)); - } - - @Override - public String name() { - return null; - } - - @Override - public void flush() { - } - - @Override - public void close() { - } - - @Override - public boolean persistent() { - return false; - } + private LinkedList>> list = new LinkedList<>(); + + @Override + public void init(KStreamContext context) { + + } + + @Override + public Iterator find(final K key, long timestamp) { + return find(key, Long.MIN_VALUE, timestamp); + } + + @Override + public Iterator findAfter(final K key, long timestamp) { + return find(key, timestamp, Long.MAX_VALUE); + } + + @Override + public Iterator findBefore(final K key, long timestamp) { + return find(key, Long.MIN_VALUE, Long.MAX_VALUE); + } + + private Iterator find(final K key, final long startTime, final long endTime) { + return new FilteredIterator>>(list.iterator()) { + protected V filter(Stamped> item) { + if (item.value.key.equals(key) && startTime <= item.timestamp && item.timestamp <= endTime) + return item.value.value; + else + return null; + } + }; + } + + @Override + public void put(K key, V value, long timestamp) { + list.add(new Stamped>(KeyValue.pair(key, value), timestamp)); + } + + @Override + public String name() { + return null; + } + + @Override + public void flush() { + } + + @Override + public void close() { + } + + @Override + public boolean persistent() { + return false; + } } diff --git a/temp b/temp index de4d210a795df..175295436b35b 100644 --- a/temp +++ b/temp @@ -1,11 +1,30 @@ -clients/src/main/java/org/apache/kafka/clients/processor/Processor.java: needs merge +build.gradle: needs merge +stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java: needs merge +stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java: needs merge +stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java: needs merge +stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java: needs merge +stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java: needs merge +stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java: needs merge +stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java: needs merge +stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java: needs merge stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java: needs merge stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java: needs merge stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java: needs merge stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java: needs merge stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java: needs merge stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java: needs merge stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java: needs merge stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java: needs merge stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java: needs merge stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java: needs merge diff --git a/temp2 b/temp2 index a8d47f2c55784..1b9255223536b 100644 --- a/temp2 +++ b/temp2 @@ -1,11 +1,30 @@ -clients/src/main/java/org/apache/kafka/clients/processor/Processor.java +build.gradle +stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java +stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java +stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java +stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java +stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java +stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java +stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java +stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java +stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java +stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java +stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java From 44cd09f432e03291dd38a03de0d4498909b9d0e1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 5 Aug 2015 14:40:48 -0700 Subject: [PATCH 136/275] added missing files --- .../apache/kafka/stream/KStreamContext.java | 119 ++++++++ .../apache/kafka/stream/RecordCollector.java | 30 ++ .../org/apache/kafka/stream/RestoreFunc.java | 24 ++ .../org/apache/kafka/stream/StateStore.java | 52 ++++ .../stream/internals/KStreamContextImpl.java | 287 ++++++++++++++++++ .../kafka/stream/topology/Processor.java | 31 ++ 6 files changed, 543 insertions(+) create mode 100644 stream/src/main/java/org/apache/kafka/stream/RecordCollector.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/RestoreFunc.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/StateStore.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamContextImpl.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/Processor.java diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java b/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java index d10db1a4bcf6f..363d313c45e40 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java @@ -1,9 +1,35 @@ +<<<<<<< HEAD package org.apache.kafka.stream; import io.confluent.streaming.kv.internals.RestoreFunc; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +======= +/** + * 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.stream; + +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.stream.topology.Processor; +>>>>>>> added missing files import java.io.File; import java.util.Map; @@ -14,6 +40,7 @@ */ public interface KStreamContext { +<<<<<<< HEAD /** * Returns the partition group id * @return partition group id @@ -122,5 +149,97 @@ public interface KStreamContext { long offset(); long timestamp(); +======= + /** + * 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 a RecordCollector + * + * @return RecordCollector + */ + RecordCollector recordCollector(); + + /** + * Returns an application context registered to {@link StreamingConfig}. + * + * @return an application context + */ + Map getContext(); + + /** + * Returns the state directory for the partition. + * + * @return the state directory + */ + File stateDir(); + + /** + * Returns Metrics instance + * + * @return Metrics + */ + Metrics metrics(); + + /** + * Registers and possibly restores the specified storage engine. + * + * @param store the storage engine + */ + void register(StateStore store, RestoreFunc restoreFunc); + + /** + * Flush the local state of this context + */ + void flush(); + + void send(String topic, Object key, Object value); + + void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); + + void schedule(Processor processor, long interval); + + void commit(); + + String topic(); + + int partition(); + + long offset(); + + long timestamp(); +>>>>>>> added missing files } diff --git a/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java b/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java new file mode 100644 index 0000000000000..e32b6c8c017cf --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serializer; + +public interface RecordCollector { + + void send(ProducerRecord record); + + void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer); + + void flush(); +} diff --git a/stream/src/main/java/org/apache/kafka/stream/RestoreFunc.java b/stream/src/main/java/org/apache/kafka/stream/RestoreFunc.java new file mode 100644 index 0000000000000..61540062988c1 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/RestoreFunc.java @@ -0,0 +1,24 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +// TODO: this should be removed once we move to Java 8 +public interface RestoreFunc { + + void apply(byte[] key, byte[] value); +} diff --git a/stream/src/main/java/org/apache/kafka/stream/StateStore.java b/stream/src/main/java/org/apache/kafka/stream/StateStore.java new file mode 100644 index 0000000000000..6f66c7ab05553 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/StateStore.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +/** + * A storage engine for managing state maintained by a stream processor. + * + *

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

+ */ +public interface StateStore { + + /** + * The name of this store. + * @return the storage name + */ + String name(); + + /** + * Flush any cached data + */ + void flush(); + + /** + * Close the storage engine + */ + void close(); + + /** + * If the storage is persistent + */ + boolean persistent(); +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamContextImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamContextImpl.java new file mode 100644 index 0000000000000..0e2b905fe97ce --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamContextImpl.java @@ -0,0 +1,287 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.RecordCollector; +import org.apache.kafka.stream.RestoreFunc; +import org.apache.kafka.stream.StateStore; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.stream.KStreamException; +import org.apache.kafka.stream.StreamingConfig; +import org.apache.kafka.stream.TimestampExtractor; +import org.apache.kafka.stream.topology.internals.KStreamMetadata; +import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Map; +import java.util.Set; + +public class KStreamContextImpl implements KStreamContext { + + private static final Logger log = LoggerFactory.getLogger(KStreamContextImpl.class); + + public final int id; + public final StreamGroup streamGroup; + public final Ingestor ingestor; + + private final RecordCollectorImpl collector; + private final HashMap> sourceStreams = new HashMap<>(); + private final HashMap partitioningInfos = new HashMap<>(); + private final TimestampExtractor timestampExtractor; + private final StreamingConfig streamingConfig; + private final ProcessorConfig processorConfig; + private final Metrics metrics; + private final ProcessorStateManager stateMgr; + + private boolean initialized = false; + + @SuppressWarnings("unchecked") + public KStreamContextImpl(int id, + Ingestor ingestor, + RecordCollectorImpl collector, + StreamingConfig streamingConfig, + ProcessorConfig processorConfig, + Metrics metrics) { + this.id = id; + this.ingestor = ingestor; + this.collector = collector; + this.streamingConfig = streamingConfig; + this.processorConfig = processorConfig; + this.timestampExtractor = this.streamingConfig.timestampExtractor(); + this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id)), + new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer())); + this.metrics = metrics; + this.streamGroup = new StreamGroup(this, this.ingestor, new TimeBasedChooser(), this.timestampExtractor, this.processorConfig.bufferedRecordsPerPartition); + } + + @Override + public int id() { + return id; + } + + @Override + public Serializer keySerializer() { + return streamingConfig.keySerializer(); + } + + @Override + public Serializer valueSerializer() { + return streamingConfig.valueSerializer(); + } + + @Override + public Deserializer keyDeserializer() { + return streamingConfig.keyDeserializer(); + } + + @Override + public Deserializer valueDeserializer() { + return streamingConfig.valueDeserializer(); + } + + @Override + public RecordCollector recordCollector() { + return collector; + } + + @Override + public Map getContext() { + return streamingConfig.context(); + } + + @Override + public File stateDir() { + return stateMgr.baseDir(); + } + + @Override + public Metrics metrics() { + return metrics; + } + + @Override + public void register(StateStore store, RestoreFunc restoreFunc) { + ensureInitialization(); + + stateMgr.register(store, restoreFunc); + } + + public void ensureInitialization() { + if (!initialized) + throw new IllegalStateException("context initialization is already finished"); + } + + @Override + public void flush() { + stateMgr.flush(); + } + + public String topic() { + if (streamGroup.record() == null) + throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); + + return streamGroup.record().topic(); + } + + @Override + public int partition() { + if (streamGroup.record() == null) + throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); + + return streamGroup.record().partition(); + } + + @Override + public long offset() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); + + return this.streamGroup.record().offset(); + } + + @Override + public long timestamp() { + if (streamGroup.record() == null) + throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); + + return streamGroup.record().timestamp; + } + + @Override + public void send(String topic, Object key, Object value) { + collector.send(new ProducerRecord<>(topic, key, value)); + } + + @Override + public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { + if (keySerializer == null || valSerializer == null) + throw new IllegalStateException("key and value serializers must be specified"); + + collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); + } + + @Override + public void commit() { + streamGroup.commitOffset(); + } + + @Override + public void schedule(Processor processor, long interval) { + streamGroup.schedule(processor, interval); + } + + public void init(Collection> streams) throws IOException { + stateMgr.init(); + + for (KStreamSource stream : streams) { + KStreamMetadata metadata = linkStreamToTopics(stream); + + stream.bind(this, metadata); + } + + // add partition -> stream group mappings to the ingestor + for (Map.Entry> entry : sourceStreams.entrySet()) { + TopicPartition partition = new TopicPartition(entry.getKey(), id); + ingestor.addPartitionStreamToGroup(streamGroup, partition); + } + + if (!ingestor.topics().equals(sourceStreams.keySet())) { + LinkedList unusedTopics = new LinkedList<>(); + for (String topic : ingestor.topics()) { + if (!sourceStreams.containsKey(topic)) + unusedTopics.add(topic); + } + throw new KStreamException("unused topics: " + Utils.mkString(unusedTopics)); + } + + initialized = true; + } + + private KStreamMetadata linkStreamToTopics(KStreamSource stream) { + ensureInitialization(); + + Set fromTopics; + + synchronized (this) { + // if topics not specified, use all the topics be default + if (stream.topics == null || stream.topics.length == 0) { + fromTopics = ingestor.topics(); + } else { + fromTopics = Collections.unmodifiableSet(Utils.mkSet(stream.topics)); + } + + // iterate over the topics and check if the stream has already been created for them + for (String topic : fromTopics) { + if (!ingestor.topics().contains(topic)) + throw new IllegalArgumentException("topic not subscribed: " + topic); + + if (sourceStreams.containsKey(topic)) + throw new IllegalArgumentException("another stream created with the same topic " + topic); + } + + // create stream metadata + Map topicPartitionInfos = new HashMap<>(); + for (String topic : fromTopics) { + PartitioningInfo partitioningInfo = this.partitioningInfos.get(topic); + + if (partitioningInfo == null) { + partitioningInfo = new PartitioningInfo(ingestor.numPartitions(topic)); + this.partitioningInfos.put(topic, partitioningInfo); + } + + topicPartitionInfos.put(topic, partitioningInfo); + } + + // update source stream map + for (String topic : fromTopics) { + sourceStreams.put(topic, stream); + + TopicPartition partition = new TopicPartition(topic, id); + streamGroup.addPartition(partition, stream); + } + + return new KStreamMetadata(topicPartitionInfos); + } + } + + public Map consumedOffsets() { + return streamGroup.consumedOffsets(); + } + + public void close() throws Exception { + stateMgr.close(collector.offsets()); + streamGroup.close(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Processor.java b/stream/src/main/java/org/apache/kafka/stream/topology/Processor.java new file mode 100644 index 0000000000000..01fe371983bb0 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Processor.java @@ -0,0 +1,31 @@ +/** + * 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.stream.topology; + +import org.apache.kafka.stream.KStreamContext; + +public interface Processor { + + void init(KStreamContext context); + + void process(K key, V value); + + void punctuate(long streamTime); + + void close(); +} From c7cffddc6bff863e9ea52366cafc18d40ed616ee Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 5 Aug 2015 17:17:28 -0700 Subject: [PATCH 137/275] SingleProcessorTopology implements Processor --- .../apache/kafka/stream/examples/MapKStreamJob.java | 2 +- .../kafka/stream/examples/PrintKStreamJob.java | 9 ++++++--- .../kafka/stream/examples/StatefulKStreamJob.java | 12 +++++++++--- .../stream/topology/SingleProcessorTopology.java | 13 +++---------- 4 files changed, 19 insertions(+), 17 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java index 801f127f33ea0..5d436151e7a68 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java @@ -70,7 +70,7 @@ public void topology() { public void topology() { >>>>>>> compile and test passed - // With overriden de-serializer + // With overridden de-serializer KStream stream1 = from(new StringDeserializer(), new StringDeserializer(), "topic1"); stream1.map(new KeyValueMapper() { diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java index 5d272bd4e72f7..0377986494bfd 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java @@ -17,7 +17,6 @@ package org.apache.kafka.stream.examples; -import org.apache.kafka.stream.topology.Processor; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KafkaStreaming; import org.apache.kafka.stream.StreamingConfig; @@ -25,10 +24,14 @@ import java.util.Properties; -public class PrintKStreamJob implements Processor { +public class PrintKStreamJob extends SingleProcessorTopology { private KStreamContext context; + public PrintKStreamJob(String... topics) { + super(topics); + } + @Override public void init(KStreamContext context) { this.context = context; @@ -79,7 +82,7 @@ public static void main(String[] args) { ======= public static void main(String[] args) { KafkaStreaming streaming = new KafkaStreaming( - new SingleProcessorTopology(PrintKStreamJob.class, args), + new PrintKStreamJob(args), new StreamingConfig(new Properties()) ); streaming.run(); diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java index 3f3d000c13285..003aecb96a358 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java @@ -17,7 +17,6 @@ package org.apache.kafka.stream.examples; -import org.apache.kafka.stream.topology.Processor; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KafkaStreaming; import org.apache.kafka.stream.StreamingConfig; @@ -29,13 +28,14 @@ import java.util.Properties; -public class StatefulKStreamJob implements Processor { +public class StatefulKStreamJob extends SingleProcessorTopology { private KStreamContext context; private KeyValueStore kvStore; <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD this.kvStore = new InMemoryKeyValueStore<>("local-state", context); <<<<<<< HEAD @@ -55,6 +55,12 @@ public class StatefulKStreamJob implements Processor { >>>>>>> remove restore function } ======= +======= + public StatefulKStreamJob(String... topics) { + super(topics); + } + +>>>>>>> SingleProcessorTopology implements Processor @Override public void init(KStreamContext context) { this.context = context; @@ -117,7 +123,7 @@ public static void main(String[] args) { ======= public static void main(String[] args) { KafkaStreaming streaming = new KafkaStreaming( - new SingleProcessorTopology(StatefulKStreamJob.class, args), + new StatefulKStreamJob(args), new StreamingConfig(new Properties()) ); streaming.run(); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java index 4febc6e82949f..98aadf7ae5086 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java @@ -19,24 +19,17 @@ import org.apache.kafka.common.utils.Utils; -public class SingleProcessorTopology extends KStreamTopology { +abstract public class SingleProcessorTopology extends KStreamTopology implements Processor { - private final Class processorClass; private final String[] topics; - public SingleProcessorTopology(Class processorClass, String... topics) { - this.processorClass = processorClass; + public SingleProcessorTopology(String... topics) { this.topics = topics; } @SuppressWarnings("unchecked") @Override public void topology() { - from(topics).process(newProcessor()); - } - - @SuppressWarnings("unchecked") - private Processor newProcessor() { - return (Processor) Utils.newInstance(processorClass); + from(topics).process((Processor)this); } } From a329fce7af95b923dd19ce07bf34388b4efd4d0d Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 6 Aug 2015 11:22:26 -0700 Subject: [PATCH 138/275] implement StateStore methods in SlidingWindow --- .../kafka/stream/internals/StreamGroup.java | 3 +- .../kafka/stream/topology/SlidingWindow.java | 139 +++++++++++++-- .../topology/internals/WindowSupport.java | 159 ++++++++++++++++++ 3 files changed, 282 insertions(+), 19 deletions(-) create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java index 7138668ec5127..5f7b52c0824e6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java @@ -263,8 +263,9 @@ public int buffered() { } public void close() { - chooser.close(); + for (RecordQueue queue : stash.values()) queue.stream.close(); stash.clear(); + chooser.close(); } protected RecordQueue createRecordQueue(TopicPartition partition, KStreamSource stream) { diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java index feb9a5d3cdd06..081a1c3b2702e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java @@ -17,30 +17,66 @@ package org.apache.kafka.stream.topology; +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.stream.KStreamContext; +import org.apache.kafka.stream.RecordCollector; +import org.apache.kafka.stream.RestoreFunc; +import org.apache.kafka.stream.topology.internals.WindowSupport; import org.apache.kafka.stream.util.FilteredIterator; import org.apache.kafka.stream.util.Stamped; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; +import java.util.Map; -public class SlidingWindow implements Window { +public class SlidingWindow extends WindowSupport implements Window { + private final Object lock = new Object(); + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + private KStreamContext context; + private int slotNum; private String name; private final long duration; private final int maxCount; private LinkedList list = new LinkedList(); - private HashMap>> map = new HashMap>>(); - - public SlidingWindow(String name, long duration, int maxCount) { + private HashMap> map = new HashMap<>(); + + public SlidingWindow( + String name, + long duration, + int maxCount, + Serializer keySerializer, + Serializer valueSerializer, + Deserializer keyDeseriaizer, + Deserializer valueDeserializer) { this.name = name; this.duration = duration; this.maxCount = maxCount; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.keyDeserializer = keyDeseriaizer; + this.valueDeserializer = valueDeserializer; } @Override public void init(KStreamContext context) { + this.context = context; + RestoreFuncImpl restoreFunc = new RestoreFuncImpl(); + context.register(this, restoreFunc); + + for (ValueList valueList : map.values()) { + valueList.clearDirtyValues(); + } + this.slotNum = restoreFunc.slotNum; } @Override @@ -62,14 +98,14 @@ public Iterator find(K key, final long timestamp) { * finds items in the window between startTime and endTime (both inclusive) */ private Iterator find(K key, final long startTime, final long endTime) { - final LinkedList> values = map.get(key); + final ValueList values = map.get(key); if (values == null) { return null; } else { - return new FilteredIterator>(values.iterator()) { + return new FilteredIterator>(values.iterator()) { @Override - protected V filter(Stamped item) { + protected V filter(Value item) { if (startTime <= item.timestamp && item.timestamp <= endTime) return item.value; else @@ -81,16 +117,19 @@ protected V filter(Stamped item) { @Override public void put(K key, V value, long timestamp) { - list.offerLast(key); + synchronized (lock) { + slotNum++; - LinkedList> values = map.get(key); - if (values == null) { - values = new LinkedList>(); - map.put(key, values); - } + list.offerLast(key); - values.offerLast(new Stamped(value, timestamp)); + ValueList values = map.get(key); + if (values == null) { + values = new ValueList<>(); + map.put(key, values); + } + values.add(slotNum, value, timestamp); + } evictExcess(); evictExpired(timestamp - duration); } @@ -99,7 +138,7 @@ private void evictExcess() { while (list.size() > maxCount) { K oldestKey = list.pollFirst(); - LinkedList> values = map.get(oldestKey); + ValueList values = map.get(oldestKey); values.removeFirst(); if (values.isEmpty()) map.remove(oldestKey); @@ -110,8 +149,8 @@ private void evictExpired(long cutoffTime) { while (true) { K oldestKey = list.peekFirst(); - LinkedList> values = map.get(oldestKey); - Stamped oldestValue = values.peekFirst(); + ValueList values = map.get(oldestKey); + Stamped oldestValue = values.first(); if (oldestValue.timestamp < cutoffTime) { list.pollFirst(); @@ -131,7 +170,38 @@ public String name() { @Override public void flush() { - // TODO + IntegerSerializer intSerializer = new IntegerSerializer(); + ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); + + RecordCollector collector = context.recordCollector(); + + for (Map.Entry> entry : map.entrySet()) { + ValueList values = entry.getValue(); + if (values.hasDirtyValues()) { + K key = entry.getKey(); + + byte[] keyBytes = keySerializer.serialize(name, key); + + Iterator> iterator = values.dirtyValueIterator(); + while (iterator.hasNext()) { + Value dirtyValue = iterator.next(); + byte[] slot = intSerializer.serialize("", dirtyValue.slotNum); + byte[] valBytes = valueSerializer.serialize(name, dirtyValue.value); + + byte[] combined = new byte[8 + 4 + keyBytes.length + 4 + valBytes.length]; + + int offset = 0; + offset += putLong(combined, offset, dirtyValue.timestamp); + offset += puts(combined, offset, keyBytes); + offset += puts(combined, offset, valBytes); + + if (offset != combined.length) throw new IllegalStateException("serialized length does not match"); + + collector.send(new ProducerRecord<>(name, context.id(), slot, combined), byteArraySerializer, byteArraySerializer); + } + values.clearDirtyValues(); + } + } } @Override @@ -144,4 +214,37 @@ public boolean persistent() { // TODO: should not be persistent, right? return false; } + + private class RestoreFuncImpl implements RestoreFunc { + + final IntegerDeserializer intDeserializer; + int slotNum = 0; + + RestoreFuncImpl() { + intDeserializer = new IntegerDeserializer(); + } + + @Override + public void apply(byte[] slot, byte[] bytes) { + + slotNum = intDeserializer.deserialize("", slot); + + int offset = 0; + // timestamp + long timestamp = getLong(bytes, offset); + offset += 8; + // key + int length = getInt(bytes, offset); + offset += 4; + K key = deserialize(bytes, offset, length, name, keyDeserializer); + offset += length; + // value + length = getInt(bytes, offset); + offset += 4; + V value = deserialize(bytes, offset, length, name, valueDeserializer); + + put(key, value, timestamp); + } + } + } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java new file mode 100644 index 0000000000000..5ce2c7b75e363 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.topology.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.util.Stamped; + +import java.util.Iterator; + +public class WindowSupport { + + public static class ValueList { + Value head = null; + Value tail = null; + Value dirty = null; + + public void add(int slotNum, V value, long timestamp) { + Value v = new Value<>(slotNum, value, timestamp); + if (tail != null) { + tail.next = v; + } else { + head = v; + } + tail = v; + if (dirty == null) dirty = v; + } + + public Value first() { + return head; + } + + public void removeFirst() { + if (head != null) { + if (head == tail) tail = null; + head = head.next; + } + } + + public boolean isEmpty() { + return head == null; + } + + public boolean hasDirtyValues() { + return dirty != null; + } + + public void clearDirtyValues() { + dirty = null; + } + + public Iterator> iterator() { + return new ValueListIterator(head); + } + + public Iterator> dirtyValueIterator() { + return new ValueListIterator(dirty); + } + + } + + private static class ValueListIterator implements Iterator> { + + Value ptr; + + ValueListIterator(Value start) { + ptr = start; + } + + @Override + public boolean hasNext() { + return ptr != null; + } + + @Override + public Value next() { + Value value = ptr; + if (value != null) ptr = value.next; + return value; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + } + + public static class Value extends Stamped { + public final int slotNum; + private Value next = null; + + Value(int slotNum, V value, long timestamp) { + super(value, timestamp); + this.slotNum = slotNum; + } + } + + + public static long getLong(byte[] bytes, int offset) { + long value = 0; + for (int i = 0; i < 8; i++) { + value = (value << 8) | bytes[offset + i]; + } + return value; + } + + public static int getInt(byte[] bytes, int offset) { + int value = 0; + for (int i = 0; i < 4; i++) { + value = (value << 8) | bytes[offset + i]; + } + return value; + } + + public static int putLong(byte[] bytes, int offset, long value) { + for (int i = 7; i >= 0; i--) { + bytes[offset + i] = (byte) (value & 0xFF); + value = value >> 8; + } + return 8; + } + + public static int putInt(byte[] bytes, int offset, int value) { + for (int i = 3; i >= 0; i--) { + bytes[offset + i] = (byte) (value & 0xFF); + value = value >> 8; + } + return 4; + } + + public static int puts(byte[] bytes, int offset, byte[] value) { + offset += putInt(bytes, offset, value.length); + System.arraycopy(bytes, offset, value, 0, value.length); + return 4 + value.length; + } + + + public static T deserialize(byte[] bytes, int offset, int length, String topic, Deserializer deserializer) { + byte[] buf = new byte[length]; + System.arraycopy(bytes, offset, buf, 0, length); + return deserializer.deserialize(topic, buf); + } + +} From 41b78556d24dcff468b63990b36ed555a0e93c23 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 6 Aug 2015 14:30:48 -0700 Subject: [PATCH 139/275] Address Yasu's comments --- .../stream/examples/PrintKStreamJob.java | 49 +++++++------- .../stream/examples/StatefulKStreamJob.java | 67 ++++++++++++------- .../topology/SingleProcessorTopology.java | 35 ---------- 3 files changed, 67 insertions(+), 84 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java index 0377986494bfd..4558af947cb7c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java @@ -20,41 +20,44 @@ import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KafkaStreaming; import org.apache.kafka.stream.StreamingConfig; -import org.apache.kafka.stream.topology.SingleProcessorTopology; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.Processor; import java.util.Properties; -public class PrintKStreamJob extends SingleProcessorTopology { +public class PrintKStreamJob extends KStreamTopology { - private KStreamContext context; + private class MyProcessor implements Processor { + private KStreamContext context; - public PrintKStreamJob(String... topics) { - super(topics); - } + @Override + public void init(KStreamContext context) { + this.context = context; + } - @Override - public void init(KStreamContext context) { - this.context = context; - } + @Override + public void process(K key, V value) { + System.out.println("[" + key + ", " + value + "]"); - @Override - public void process(K key, V value) { - System.out.println("[" + key + ", " + value + "]"); + context.commit(); - context.commit(); + context.send("topic", key, value); + } - context.send("topic", key, value); - } + @Override + public void punctuate(long streamTime) { + // do nothing + } - @Override - public void punctuate(long streamTime) { - // do nothing + @Override + public void close() { + // do nothing + } } + @SuppressWarnings("unchecked") @Override - public void close() { - // do nothing - } + public void topology() { from("topic").process(new MyProcessor()); } <<<<<<< HEAD public static void main(String[] args) { @@ -82,7 +85,7 @@ public static void main(String[] args) { ======= public static void main(String[] args) { KafkaStreaming streaming = new KafkaStreaming( - new PrintKStreamJob(args), + new PrintKStreamJob(), new StreamingConfig(new Properties()) ); streaming.run(); diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java index 003aecb96a358..2ef09b846d5b9 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java @@ -17,6 +17,7 @@ package org.apache.kafka.stream.examples; +import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KafkaStreaming; import org.apache.kafka.stream.StreamingConfig; @@ -24,18 +25,21 @@ import org.apache.kafka.stream.state.InMemoryKeyValueStore; import org.apache.kafka.stream.state.KeyValueIterator; import org.apache.kafka.stream.state.KeyValueStore; -import org.apache.kafka.stream.topology.SingleProcessorTopology; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.Processor; import java.util.Properties; -public class StatefulKStreamJob extends SingleProcessorTopology { +public class StatefulKStreamJob extends KStreamTopology { - private KStreamContext context; - private KeyValueStore kvStore; + private class MyProcessor implements Processor { + private KStreamContext context; + private KeyValueStore kvStore; <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD this.kvStore = new InMemoryKeyValueStore<>("local-state", context); <<<<<<< HEAD @@ -66,36 +70,47 @@ public void init(KStreamContext context) { this.context = context; this.context.schedule(this, 1000); >>>>>>> compile and test passed +======= + @Override + public void init(KStreamContext context) { + this.context = context; + this.context.schedule(this, 1000); - this.kvStore = new InMemoryKeyValueStore<>("local-state", context); - } + this.kvStore = new InMemoryKeyValueStore<>("local-state", context); + } +>>>>>>> Address Yasu's comments - @Override - public void process(String key, Integer value) { - Integer oldValue = this.kvStore.get(key); - if (oldValue == null) { - this.kvStore.put(key, value); - } else { - int newValue = oldValue + value; - this.kvStore.put(key, newValue); + @Override + public void process(String key, Integer value) { + Integer oldValue = this.kvStore.get(key); + if (oldValue == null) { + this.kvStore.put(key, value); + } else { + int newValue = oldValue + value; + this.kvStore.put(key, newValue); + } + + context.commit(); } - context.commit(); - } + @Override + public void punctuate(long streamTime) { + KeyValueIterator iter = this.kvStore.all(); + while (iter.hasNext()) { + Entry entry = iter.next(); + System.out.println("[" + entry.key() + ", " + entry.value() + "]"); + } + } - @Override - public void punctuate(long streamTime) { - KeyValueIterator iter = this.kvStore.all(); - while (iter.hasNext()) { - Entry entry = iter.next(); - System.out.println("[" + entry.key() + ", " + entry.value() + "]"); + @Override + public void close() { + // do nothing } } + @SuppressWarnings("unchecked") @Override - public void close() { - // do nothing - } + public void topology() { ((KStream) from("topic")).process(new MyProcessor()); } <<<<<<< HEAD public static void main(String[] args) { @@ -123,7 +138,7 @@ public static void main(String[] args) { ======= public static void main(String[] args) { KafkaStreaming streaming = new KafkaStreaming( - new StatefulKStreamJob(args), + new StatefulKStreamJob(), new StreamingConfig(new Properties()) ); streaming.run(); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java deleted file mode 100644 index 98aadf7ae5086..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SingleProcessorTopology.java +++ /dev/null @@ -1,35 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -import org.apache.kafka.common.utils.Utils; - -abstract public class SingleProcessorTopology extends KStreamTopology implements Processor { - - private final String[] topics; - - public SingleProcessorTopology(String... topics) { - this.topics = topics; - } - - @SuppressWarnings("unchecked") - @Override - public void topology() { - from(topics).process((Processor)this); - } -} From 4ea4aad8c9b110e6297af8a8f38bc2a55bd129f8 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 6 Aug 2015 17:53:33 -0700 Subject: [PATCH 140/275] wip --- .../clients/processor/KafkaProcessor.java | 56 ++++++++++ .../kafka/clients/processor/PTopology.java | 103 ++++++++++++++++++ .../kafka/clients/processor/Processor.java | 29 +++++ .../clients/processor/ProcessorContext.java | 22 ++++ .../kafka/clients/processor/Receiver.java | 23 ++++ 5 files changed, 233 insertions(+) create mode 100644 clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/processor/Receiver.java diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java b/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java new file mode 100644 index 0000000000000..2a5b0e0afbc4c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +import java.util.ArrayList; +import java.util.List; + +public abstract class KafkaProcessor implements Processor { + + private final List> children; + + private final String name; + + public KafkaProcessor(String name) { + this.name = name; + this.children = new ArrayList<>(); + } + + public String name() { return name; } + + public final void receive(K1 key, V1 value) { + this.process(key, value); + } + + public final void chain(KafkaProcessor child) { + children.add(child); + } + + public final void forward(K2 key, V2 value) { + for (KafkaProcessor child : children) { + child.receive(key, value); + } + } + + @Override + public void close() { + for (KafkaProcessor child : children) { + child.close(); + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java new file mode 100644 index 0000000000000..236ecb2e9e2db --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -0,0 +1,103 @@ +/** + * 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.clients.processor; + +import org.apache.kafka.common.serialization.Deserializer; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public abstract class PTopology { + + private class KeyValueDeserializers { + public Deserializer keyDeserializer; + public Deserializer valDeserializer; + + public KeyValueDeserializers(Deserializer keyDeserializer, Deserializer valDeserializer) { + this.keyDeserializer = keyDeserializer; + this.valDeserializer = valDeserializer; + } + } + + List sources = new ArrayList<>(); + List processors = new ArrayList<>(); + Map topicDesers = new HashMap<>(); + + public List sources() { + return sources; + } + + public Set topics() { + return topicDesers.keySet(); + } + + public Deserializer keyDeser(String topic) { + KeyValueDeserializers desers = topicDesers.get(topic); + + if (desers == null) + throw new IllegalStateException("The topic " + topic + " is unknown."); + + return desers.keyDeserializer; + } + + public Deserializer valueDeser(String topic) { + KeyValueDeserializers desers = topicDesers.get(topic); + + if (desers == null) + throw new IllegalStateException("The topic " + topic + " is unknown."); + + return desers.valDeserializer; + } + + public final void addProcessor(KafkaProcessor processor, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + if (processors.contains(processor)) + throw new IllegalArgumentException("Processor " + processor.name() + " is already added."); + + processors.add(processor); + + for (String topic : topics) { + if (topicDesers.containsKey(topic)) + throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); + + topicDesers.put(topic, new KeyValueDeserializers(keyDeserializer, valDeserializer)); + } + + sources.add(processor); + } + + public final void addProcessor(KafkaProcessor processor, KafkaProcessor... parents) { + if (processors.contains(processor)) + throw new IllegalArgumentException("Processor " + processor.name() + " is already added."); + + processors.add(processor); + + if (parents != null) { + for (KafkaProcessor parent : parents) { + if (!processors.contains(parent)) + throw new IllegalArgumentException("Parent processor " + parent.name() + " is not added yet."); + + parent.chain(processor); + } + } + } + + public abstract void build(); +} diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java b/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java index 2dc0bb76d98a8..677c5c9ed4ff1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java @@ -1,3 +1,4 @@ +<<<<<<< HEAD package org.apache.kafka.clients.processor; /** @@ -46,4 +47,32 @@ public interface ProcessorContext { void punctuate(long streamTime); void close(); +======= +/** + * 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.clients.processor; + +public interface Processor { + + void init(ProcessorContext context); + + void process(K key, V value); + + void close(); +>>>>>>> wip } diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java b/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java new file mode 100644 index 0000000000000..11c83f327e3ff --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java @@ -0,0 +1,22 @@ +/** + * 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.clients.processor; + +public interface ProcessorContext { + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/Receiver.java b/clients/src/main/java/org/apache/kafka/clients/processor/Receiver.java new file mode 100644 index 0000000000000..d4f5ca62dd44a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/processor/Receiver.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +public interface Receiver { + + void receive(K key, V value, long timestamp); +} \ No newline at end of file From 3df37562f153f23bed739e560750c2c6320f0e34 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 6 Aug 2015 19:52:58 -0700 Subject: [PATCH 141/275] wip --- .../kafka/clients/processor/PTopology.java | 20 +++++- .../kafka/clients/processor/Receiver.java | 23 ------ .../java/org/apache/kafka/stream/KStream.java | 2 +- .../apache/kafka/stream/KafkaStreaming.java | 2 +- .../kafka/stream/examples/MapKStreamJob.java | 4 ++ .../stream/examples/PrintKStreamJob.java | 36 ++++++---- .../stream/examples/StatefulKStreamJob.java | 2 +- .../stream/topology/KStreamTopology.java | 70 ++++--------------- .../kafka/stream/topology/KeyValueMapper.java | 4 +- .../kafka/stream/topology/ValueMapper.java | 4 +- .../topology/internals/KStreamFilter.java | 37 +++++++--- .../topology/internals/KStreamFlatMap.java | 33 +++++---- .../topology/internals/KStreamImpl.java | 52 +++++++++----- .../stream/topology/internals/KStreamMap.java | 32 +++++---- .../topology/internals/KStreamMapValues.java | 32 ++++++--- .../topology/internals/KStreamSource.java | 43 +----------- .../kafka/test/MockKStreamTopology.java | 2 +- 17 files changed, 190 insertions(+), 208 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/processor/Receiver.java diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java index 236ecb2e9e2db..7e9fc8b25ecca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -25,7 +25,7 @@ import java.util.Map; import java.util.Set; -public abstract class PTopology { +public class PTopology { private class KeyValueDeserializers { public Deserializer keyDeserializer; @@ -41,15 +41,26 @@ public KeyValueDeserializers(Deserializer keyDeserializer, Deserializer va List processors = new ArrayList<>(); Map topicDesers = new HashMap<>(); + boolean built = false; + public List sources() { + if (!built) + throw new IllegalStateException("Topology has not been built."); + return sources; } public Set topics() { + if (!built) + throw new IllegalStateException("Topology has not been built."); + return topicDesers.keySet(); } public Deserializer keyDeser(String topic) { + if (!built) + throw new IllegalStateException("Topology has not been built."); + KeyValueDeserializers desers = topicDesers.get(topic); if (desers == null) @@ -59,6 +70,9 @@ public Deserializer keyDeser(String topic) { } public Deserializer valueDeser(String topic) { + if (!built) + throw new IllegalStateException("Topology has not been built."); + KeyValueDeserializers desers = topicDesers.get(topic); if (desers == null) @@ -99,5 +113,7 @@ public final void addProcessor(KafkaProcessor processor, Kafk } } - public abstract void build(); + public void build() { + built = true; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/Receiver.java b/clients/src/main/java/org/apache/kafka/clients/processor/Receiver.java deleted file mode 100644 index d4f5ca62dd44a..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/processor/Receiver.java +++ /dev/null @@ -1,23 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.clients.processor; - -public interface Receiver { - - void receive(K key, V value, long timestamp); -} \ No newline at end of file diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java index 79f09dc9d943c..1b6cc70d926af 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java @@ -56,7 +56,7 @@ public interface KStream { * @param the value type of the new stream * @return KStream */ - KStream map(KeyValueMapper mapper); + KStream map(KeyValueMapper mapper); /** * Creates a new stream by transforming valuesa by a mapper to all values of this stream diff --git a/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java index f8e93295ce8bb..a49efc4109ddc 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java +++ b/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java @@ -70,7 +70,7 @@ * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and * sends output to zero or more output topics. *

- * This processing is defined by extending the {@link KStreamTopology} abstract class to specify the transformation operator topology. The + * This processing is defined by extending the {@link KStreamTopology} abstract class to specify the transformation operator build. 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. *

diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java index 5d436151e7a68..526d4c45012ac 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java @@ -67,8 +67,12 @@ public void topology() { ======= @SuppressWarnings("unchecked") @Override +<<<<<<< HEAD public void topology() { >>>>>>> compile and test passed +======= + public void build() { +>>>>>>> wip // With overridden de-serializer KStream stream1 = from(new StringDeserializer(), new StringDeserializer(), "topic1"); diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java index 4558af947cb7c..90f1fc1ea0b49 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java @@ -17,36 +17,37 @@ package org.apache.kafka.stream.examples; -import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.KafkaStreaming; import org.apache.kafka.stream.StreamingConfig; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.Processor; import java.util.Properties; -public class PrintKStreamJob extends KStreamTopology { +public class PrintKStreamJob { - private class MyProcessor implements Processor { - private KStreamContext context; + private static class MyProcessor extends KafkaProcessor { + ProcessorContext context; + + public MyProcessor(String name) { + super(name); + } @Override - public void init(KStreamContext context) { + public void init(ProcessorContext context) { this.context = context; } @Override - public void process(K key, V value) { + public void process(String key, Integer value) { System.out.println("[" + key + ", " + value + "]"); context.commit(); - context.send("topic", key, value); - } - - @Override - public void punctuate(long streamTime) { - // do nothing + context.send("topic-dest", key, value); } @Override @@ -55,6 +56,7 @@ public void close() { } } +<<<<<<< HEAD @SuppressWarnings("unchecked") @Override public void topology() { from("topic").process(new MyProcessor()); } @@ -83,7 +85,13 @@ public static void main(String[] args) { >>>>>>> fix examples } ======= +======= +>>>>>>> wip public static void main(String[] args) { + PTopology topology = new PTopology(); + topology.addProcessor(new MyProcessor("processor"), new StringDeserializer(), new IntegerDeserializer(), "topic-source"); + topology.build(); + KafkaStreaming streaming = new KafkaStreaming( new PrintKStreamJob(), new StreamingConfig(new Properties()) diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java index 2ef09b846d5b9..7b176f765d1df 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java @@ -110,7 +110,7 @@ public void close() { @SuppressWarnings("unchecked") @Override - public void topology() { ((KStream) from("topic")).process(new MyProcessor()); } + public void build() { ((KStream) from("topic")).process(new MyProcessor()); } <<<<<<< HEAD public static void main(String[] args) { diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java index a19eec3bd272e..9933c85018709 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java @@ -17,21 +17,20 @@ package org.apache.kafka.stream.topology; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.topology.internals.KStreamSource; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.Set; +import org.apache.kafka.stream.topology.internals.SourceProcessor; /** - * KStreamTopology is the class that allows an implementation of {@link KStreamTopology#topology()} to create KStream instances. + * KStreamTopology is the class that allows an implementation of {@link KStreamTopology#build()} to create KStream instances. */ public abstract class KStreamTopology { - private final ArrayList> streams = new ArrayList<>(); + private PTopology topology = new PTopology(); /** * Initializes a stream processing topology. This method may be called multiple times. @@ -41,58 +40,17 @@ public abstract class KStreamTopology { *

*
      *   KStreamTopology topology = new KStreamTopology() {
-     *     public void topology() {
+     *     public void build() {
      *       KStream<Integer, PageView> pageViewStream = from("pageView").mapValues(...);
      *       KStream<Integer, AdClick> adClickStream = from("adClick").join(pageViewStream, ...).process(...);
      *     }
      *   }
      *
-     *   KafkaStreaming streaming = new KafkaStreaming(topology, streamingConfig)
+     *   KafkaStreaming streaming = new KafkaStreaming(build, streamingConfig)
      *   streaming.run();
      * 
*/ - public abstract void topology(); - - /** - * Extracts topics used in the KStream topology. This method calls {@link KStreamTopology#topology()} method. - * - * @return - */ - public final Set topics() { - synchronized (streams) { - try { - streams.clear(); - topology(); - Set topics = new HashSet<>(); - for (KStreamSource stream : streams) { - topics.addAll(stream.topics()); - } - return topics; - } finally { - streams.clear(); - } - } - } - - /** - * Returns source streams in the KStream topology. This method calls {@link KStreamTopology#topology()} method. - * This method may be called multiple times. - */ - public final Collection> sourceStreams() { - synchronized (streams) { - try { - streams.clear(); - topology(); - return new ArrayList<>(streams); - } finally { - streams.clear(); - } - } - } - - - // TODO: support regex topic matching in from() calls, for example: - // context.from("Topic*PageView") + public abstract void build(); /** * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. @@ -115,9 +73,11 @@ public final Set topics() { * @return KStream */ public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KStreamSource stream = new KStreamSource<>(topics, keyDeserializer, valDeserializer, this); - streams.add(stream); - return stream; - } + // TODO + SourceProcessor source = new SourceProcessor<>("KAFKA-SOURCE"); + + topology.addProcessor(source, keyDeserializer, valDeserializer, topics); + return new KStreamSource<>(topology, source); + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java index be08861c835ed..d4b1aed703831 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java @@ -17,8 +17,8 @@ package org.apache.kafka.stream.topology; -public interface KeyValueMapper { +public interface KeyValueMapper { - KeyValue apply(K key, V value); + KeyValue apply(K1 key, V1 value); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java index c08a749c4279b..8d69a45daf86e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java @@ -17,8 +17,8 @@ package org.apache.kafka.stream.topology; -public interface ValueMapper { +public interface ValueMapper { - R apply(V value); + V2 apply(V1 value); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java index 683485d62e61c..fab55b3c9dc62 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java @@ -17,26 +17,43 @@ package org.apache.kafka.stream.topology.internals; -import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.topology.Predicate; -class KStreamFilter extends KStreamImpl { +class KStreamFilter extends KafkaProcessor { + + private static final String FILTER_NAME = "KAFKA-FILTER"; private final Predicate predicate; + private final boolean filterOut; + + public KStreamFilter(Predicate predicate) { + this(predicate, false); + } + + public KStreamFilter(Predicate predicate, boolean filterOut) { + super(FILTER_NAME); - KStreamFilter(Predicate predicate, KStreamTopology topology) { - super(topology); this.predicate = predicate; + this.filterOut = filterOut; } - @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - if (predicate.apply((K) key, (V) value)) { - forward(key, value, timestamp); - } + public void init(ProcessorContext context) { + // do nothing + } + + @Override + public void process(K key, V value) { + if ((!filterOut && predicate.apply(key, value)) + || (filterOut && !predicate.apply(key, value))) { + forward(key, value); } } + @Override + public void close() { + // do nothing + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java index 271ab6e625724..72d8d66b039db 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java @@ -17,34 +17,41 @@ package org.apache.kafka.stream.topology.internals; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.KeyValueMapper; -class KStreamFlatMap extends KStreamImpl { +class KStreamFlatMap extends KafkaProcessor { - private final KeyValueMapper, K1, V1> mapper; + private static final String FLATMAP_NAME = "KAFKA-FLATMAP"; + + private final KeyValueMapper> mapper; + + KStreamFlatMap(KeyValueMapper> mapper) { + super(FLATMAP_NAME); - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology topology) { - super(topology); this.mapper = mapper; } @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - super.bind(context, KStreamMetadata.unjoinable()); + public void init(ProcessorContext context) { + // do nothing } - @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - KeyValue> newPair = mapper.apply((K1) key, (V1) value); - for (V v : newPair.value) { - forward(newPair.key, v, timestamp); - } + public void process(K1 key, V1 value) { + KeyValue> newPair = mapper.apply(key, value); + for (V2 v : newPair.value) { + forward(newPair.key, v); } } + @Override + public void close() { + // do nothing + } + } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java index 6f5304103b170..a85e8f3212a38 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java @@ -17,6 +17,8 @@ package org.apache.kafka.stream.topology.internals; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; import org.apache.kafka.stream.topology.Processor; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KStream; @@ -24,7 +26,6 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.stream.internals.ProcessorNode; import org.apache.kafka.stream.internals.Receiver; -import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KStreamWindowed; import org.apache.kafka.stream.topology.KeyValueMapper; import org.apache.kafka.stream.topology.Predicate; @@ -32,17 +33,16 @@ import org.apache.kafka.stream.topology.ValueMapper; import org.apache.kafka.stream.topology.Window; -import java.util.ArrayList; - abstract class KStreamImpl implements KStream, Receiver { - private final ArrayList nextReceivers = new ArrayList<>(1); - protected KStreamTopology topology; + protected PTopology topology; + protected KafkaProcessor processor; protected KStreamContext context; protected KStreamMetadata metadata; - protected KStreamImpl(KStreamTopology topology) { + protected KStreamImpl(PTopology topology, KafkaProcessor processor) { this.topology = topology; + this.processor = processor; } @Override @@ -66,31 +66,47 @@ public void close() { @Override public KStream filter(Predicate predicate) { - return chain(new KStreamFilter(predicate, topology)); + KStreamFilter filter = new KStreamFilter<>(predicate); + + topology.addProcessor(filter, processor); + + return new KStreamImpl(topology, filter); } @Override public KStream filterOut(final Predicate predicate) { - return filter(new Predicate() { - public boolean apply(K key, V value) { - return !predicate.apply(key, value); - } - }); + KStreamFilter filter = new KStreamFilter<>(predicate, true); + + topology.addProcessor(filter, processor); + + return new KStreamImpl(topology, filter); } @Override - public KStream map(KeyValueMapper mapper) { - return chain(new KStreamMap(mapper, topology)); + public KStream map(KeyValueMapper mapper) { + KStreamMap map = new KStreamMap<>(mapper); + + topology.addProcessor(map, processor); + + return new KStreamImpl(topology, map); } @Override - public KStream mapValues(ValueMapper mapper) { - return chain(new KStreamMapValues(mapper, topology)); + public KStream mapValues(ValueMapper mapper) { + KStreamMapValues map = new KStreamMapValues<>(mapper); + + topology.addProcessor(map, processor); + + return new KStreamImpl(topology, map); } @Override - public KStream flatMap(KeyValueMapper, K, V> mapper) { - return chain(new KStreamFlatMap(mapper, topology)); + public KStream flatMap(KeyValueMapper> mapper) { + KStreamFlatMap map = new KStreamFlatMap<>(mapper); + + topology.addProcessor(map, processor); + + return new KStreamImpl(topology, map); } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java index fb371f1dd95c0..620aa8b994da4 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java @@ -17,32 +17,36 @@ package org.apache.kafka.stream.topology.internals; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.KeyValueMapper; -class KStreamMap extends KStreamImpl { +class KStreamMap extends KafkaProcessor { - private final KeyValueMapper mapper; + private static final String MAP_NAME = "KAFKA-MAP"; + + private final KeyValueMapper mapper; + + public KStreamMap(KeyValueMapper mapper) { + super(MAP_NAME); - KStreamMap(KeyValueMapper mapper, KStreamTopology topology) { - super(topology); this.mapper = mapper; } @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - super.bind(context, KStreamMetadata.unjoinable()); + public void init(ProcessorContext context) { + // do nothing } - @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - KeyValue newPair = mapper.apply((K1) key, (V1) value); - forward(newPair.key, newPair.value, timestamp); - } + public void process(K1 key, V1 value) { + KeyValue newPair = mapper.apply(key, value); + forward(newPair.key, newPair.value); } + @Override + public void close() { + // do nothing + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java index 9dd0d6bc10040..3edfd9a153cfa 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java @@ -17,25 +17,35 @@ package org.apache.kafka.stream.topology.internals; -import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.topology.ValueMapper; -class KStreamMapValues extends KStreamImpl { +class KStreamMapValues extends KafkaProcessor { - private final ValueMapper mapper; + private static final String MAPVALUES_NAME = "KAFKA-MAPVALUES"; + + private final ValueMapper mapper; + + public KStreamMapValues(ValueMapper mapper) { + super(MAPVALUES_NAME); - KStreamMapValues(ValueMapper mapper, KStreamTopology topology) { - super(topology); this.mapper = mapper; } - @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - V newValue = mapper.apply((V1) value); - forward(key, newValue, timestamp); - } + public void init(ProcessorContext context) { + // do nothing } + @Override + public void process(K1 key, V1 value) { + V2 newValue = mapper.apply(value); + forward(key, newValue); + } + + @Override + public void close() { + // do nothing + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java index f1c8260acee0d..3e3a595676636 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java @@ -17,30 +17,14 @@ package org.apache.kafka.stream.topology.internals; +import org.apache.kafka.clients.processor.PTopology; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.stream.topology.KStreamTopology; - -import java.util.Arrays; -import java.util.HashSet; -import java.util.Set; public class KStreamSource extends KStreamImpl { - private Deserializer keyDeserializer; - private Deserializer valueDeserializer; - - public String[] topics; - - public KStreamSource(String[] topics, KStreamTopology topology) { - this(topics, null, null, topology); - } - - public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology topology) { - super(topology); - this.topics = topics; - this.keyDeserializer = keyDeserializer; - this.valueDeserializer = valueDeserializer; + public KStreamSource(PTopology topology, SourceProcessor source) { + super(topology, source); } @SuppressWarnings("unchecked") @@ -51,25 +35,4 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { super.bind(context, metadata); } - - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - // KStream needs to forward the topic name since it is directly from the Kafka source - forward(key, value, timestamp); - } - } - - public Deserializer keyDeserializer() { - return keyDeserializer; - } - - public Deserializer valueDeserializer() { - return valueDeserializer; - } - - public Set topics() { - return new HashSet<>(Arrays.asList(topics)); - } - } diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java index c1ef5b221aa2d..bf61a6ecbfe94 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java @@ -21,6 +21,6 @@ public class MockKStreamTopology extends KStreamTopology { @Override - public void topology() { + public void build() { } } From 0d9bcba24a77ea39cec461bb537cf329f6a9f993 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 6 Aug 2015 19:53:14 -0700 Subject: [PATCH 142/275] add one file --- .../topology/internals/SourceProcessor.java | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/SourceProcessor.java diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/SourceProcessor.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/SourceProcessor.java new file mode 100644 index 0000000000000..ed6713c9bc01a --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/SourceProcessor.java @@ -0,0 +1,40 @@ +/** + * 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.stream.topology.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.ProcessorContext; + +public class SourceProcessor extends KafkaProcessor { + public SourceProcessor(String name) { + super(name); + } + + @Override + public void init(ProcessorContext context) { + // do nothing + } + + @Override + public void process(K key, V value) { forward(key, value); } + + @Override + public void close() { + // do nothing + } +} From 23f32596d548b44b847b27582c67534a6f12785d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 7 Aug 2015 00:38:11 -0700 Subject: [PATCH 143/275] wip --- .../clients/processor/KafkaProcessor.java | 10 +- .../clients/processor/ProcessorContext.java | 90 ++++++ .../java/org/apache/kafka/stream/Chooser.java | 2 +- .../java/org/apache/kafka/stream/KStream.java | 19 +- .../apache/kafka/stream/KStreamContext.java | 4 + .../apache/kafka/stream/RecordCollector.java | 30 -- .../org/apache/kafka/stream/RestoreFunc.java | 24 -- .../org/apache/kafka/stream/StateStore.java | 52 ---- .../apache/kafka/stream/StreamingConfig.java | 142 --------- .../kafka/stream/TimestampExtractor.java | 34 --- .../kafka/stream/examples/MapKStreamJob.java | 4 +- .../stream/examples/PrintKStreamJob.java | 4 +- .../stream/examples/StatefulKStreamJob.java | 4 +- .../kafka/stream/internals/Ingestor.java | 41 --- .../kafka/stream/internals/IngestorImpl.java | 133 -------- .../stream/internals/KStreamContextImpl.java | 287 ------------------ .../stream/internals/PartitioningInfo.java | 27 -- .../stream/internals/ProcessorConfig.java | 104 ------- .../kafka/stream/internals/ProcessorNode.java | 52 ---- .../internals/ProcessorStateManager.java | 194 ------------ .../stream/internals/PunctuationQueue.java | 52 ---- .../stream/internals/PunctuationSchedule.java | 40 --- .../stream/internals/RecordCollectorImpl.java | 92 ------ .../kafka/stream/internals/RecordQueue.java | 123 -------- .../stream/internals/RoundRobinChooser.java | 47 --- .../kafka/stream/internals/StampedRecord.java | 49 --- .../stream/internals/TimeBasedChooser.java | 61 ---- .../kafka/stream/state/KeyValueStore.java | 2 +- .../state/internals/MeteredKeyValueStore.java | 4 +- .../stream/topology/KStreamTopology.java | 10 +- .../kafka/stream/topology/KeyValueMapper.java | 1 - .../kafka/stream/topology/SlidingWindow.java | 2 +- .../kafka/stream/topology/ValueMapper.java | 1 - .../apache/kafka/stream/topology/Window.java | 2 +- .../topology/internals/KStreamBranch.java | 50 +-- .../topology/internals/KStreamFilter.java | 11 - .../topology/internals/KStreamFlatMap.java | 14 - .../internals/KStreamFlatMapValues.java | 24 +- .../topology/internals/KStreamImpl.java | 127 ++++---- .../stream/topology/internals/KStreamMap.java | 11 - .../topology/internals/KStreamMapValues.java | 11 - .../topology/internals/KStreamMetadata.java | 4 +- .../topology/internals/KStreamSource.java | 9 - .../topology/internals/KStreamTransform.java | 53 ---- .../stream/util/MinTimestampTracker.java | 2 + .../kafka/stream/util/ParallelExecutor.java | 152 ---------- .../org/apache/kafka/stream/util/Stamped.java | 38 --- .../kafka/stream/util/TimestampTracker.java | 2 + .../kafka/stream/KStreamBranchTest.java | 2 +- .../kafka/stream/KStreamFilterTest.java | 2 +- .../kafka/stream/KStreamFlatMapTest.java | 2 +- .../stream/KStreamFlatMapValuesTest.java | 2 +- .../apache/kafka/stream/KStreamJoinTest.java | 2 +- .../apache/kafka/stream/KStreamMapTest.java | 2 +- .../kafka/stream/KStreamMapValuesTest.java | 4 + .../kafka/stream/KStreamSourceTest.java | 2 +- .../kafka/stream/KStreamTransformTest.java | 2 +- .../kafka/stream/KStreamWindowedTest.java | 2 +- .../kafka/stream/MinTimestampTrackerTest.java | 2 +- .../kafka/stream/ParallelExecutorTest.java | 2 +- .../apache/kafka/stream/StreamGroupTest.java | 5 +- .../org/apache/kafka/test/MockIngestor.java | 4 +- .../apache/kafka/test/MockKStreamContext.java | 8 +- .../apache/kafka/test/UnlimitedWindow.java | 2 +- temp | 30 +- temp2 | 30 +- 66 files changed, 264 insertions(+), 2092 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/RecordCollector.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/RestoreFunc.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/StateStore.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/Ingestor.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/IngestorImpl.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamContextImpl.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/PartitioningInfo.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/ProcessorConfig.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/ProcessorNode.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/ProcessorStateManager.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/PunctuationQueue.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/PunctuationSchedule.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/RecordCollectorImpl.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/RecordQueue.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/RoundRobinChooser.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/StampedRecord.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/TimeBasedChooser.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamTransform.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/util/Stamped.java diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java b/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java index 2a5b0e0afbc4c..332730055fd01 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java @@ -20,9 +20,9 @@ import java.util.ArrayList; import java.util.List; -public abstract class KafkaProcessor implements Processor { +public abstract class KafkaProcessor implements Processor, Receiver { - private final List> children; + protected final List> children; private final String name; @@ -33,6 +33,7 @@ public KafkaProcessor(String name) { public String name() { return name; } + @Override public final void receive(K1 key, V1 value) { this.process(key, value); } @@ -47,6 +48,11 @@ public final void forward(K2 key, V2 value) { } } + @Override + public void init(ProcessorContext context) { + // do nothing + } + @Override public void close() { for (KafkaProcessor child : children) { diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java b/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java index 11c83f327e3ff..caf6417678550 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java @@ -17,6 +17,96 @@ package org.apache.kafka.clients.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; +import java.util.Map; + 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 a RecordCollector + * + * @return RecordCollector + */ + RecordCollector recordCollector(); + + /** + * Returns the state directory for the partition. + * + * @return the state directory + */ + File stateDir(); + + /** + * Returns Metrics instance + * + * @return Metrics + */ + Metrics metrics(); + + /** + * Registers and possibly restores the specified storage engine. + * + * @param store the storage engine + */ + void register(StateStore store, RestoreFunc restoreFunc); + + /** + * Flush the local state of this context + */ + void flush(); + + void send(String topic, Object key, Object value); + + void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); + + void schedule(Processor processor, long interval); + + void commit(); + + String topic(); + + int partition(); + + long offset(); + + long timestamp(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/Chooser.java b/stream/src/main/java/org/apache/kafka/stream/Chooser.java index c735517730ab3..9668027fcf632 100644 --- a/stream/src/main/java/org/apache/kafka/stream/Chooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/Chooser.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.internals.RecordQueue; +import org.apache.kafka.clients.processor.internals.RecordQueue; public interface Chooser { diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java index 1b6cc70d926af..8722539329e0f 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java @@ -17,13 +17,13 @@ package org.apache.kafka.stream; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.Processor; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.stream.topology.KStreamWindowed; import org.apache.kafka.stream.topology.KeyValueMapper; import org.apache.kafka.stream.topology.Predicate; -import org.apache.kafka.stream.topology.Processor; -import org.apache.kafka.stream.topology.Transformer; import org.apache.kafka.stream.topology.ValueMapper; import org.apache.kafka.stream.topology.Window; @@ -65,7 +65,7 @@ public interface KStream { * @param the value type of the new stream * @return KStream */ - KStream mapValues(ValueMapper mapper); + KStream mapValues(ValueMapper mapper); /** * Creates a new stream by applying a mapper to all elements of this stream and using the values in the resulting Iterable @@ -75,7 +75,7 @@ public interface KStream { * @param the value type of the new stream * @return KStream */ - KStream flatMap(KeyValueMapper, K, V> mapper); + KStream flatMap(KeyValueMapper> mapper); /** * Creates a new stream by applying a mapper to all values of this stream and using the values in the resulting Iterable @@ -84,7 +84,7 @@ public interface KStream { * @param the value type of the new stream * @return KStream */ - KStream flatMapValues(ValueMapper, V> processor); + KStream flatMapValues(ValueMapper> processor); /** * Creates a new windowed stream using a specified window instance. @@ -158,13 +158,12 @@ public interface KStream { * * @param processor the instance of Processor */ - void process(Processor processor); + KStream process(KafkaProcessor processor); /** - * Transform all elements in this stream by applying a tranformer. + * Processes all elements in this stream by applying a processor. * - * @param transformer the instance of Transformer + * @param processor the instance of Processor */ - KStream transform(Transformer transformer); - + void process(Processor processor); } diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java b/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java index 363d313c45e40..e11ccdb7c5971 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java @@ -25,6 +25,10 @@ package org.apache.kafka.stream; +import org.apache.kafka.clients.processor.RecordCollector; +import org.apache.kafka.clients.processor.RestoreFunc; +import org.apache.kafka.clients.processor.StateStore; +import org.apache.kafka.clients.processor.internals.StreamingConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; diff --git a/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java b/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java deleted file mode 100644 index e32b6c8c017cf..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/RecordCollector.java +++ /dev/null @@ -1,30 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.Serializer; - -public interface RecordCollector { - - void send(ProducerRecord record); - - void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer); - - void flush(); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/RestoreFunc.java b/stream/src/main/java/org/apache/kafka/stream/RestoreFunc.java deleted file mode 100644 index 61540062988c1..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/RestoreFunc.java +++ /dev/null @@ -1,24 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -// TODO: this should be removed once we move to Java 8 -public interface RestoreFunc { - - void apply(byte[] key, byte[] value); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/StateStore.java b/stream/src/main/java/org/apache/kafka/stream/StateStore.java deleted file mode 100644 index 6f66c7ab05553..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/StateStore.java +++ /dev/null @@ -1,52 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -/** - * A storage engine for managing state maintained by a stream processor. - * - *

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

- */ -public interface StateStore { - - /** - * The name of this store. - * @return the storage name - */ - String name(); - - /** - * Flush any cached data - */ - void flush(); - - /** - * Close the storage engine - */ - void close(); - - /** - * If the storage is persistent - */ - boolean persistent(); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java b/stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java deleted file mode 100644 index fca6775d9ab3e..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/StreamingConfig.java +++ /dev/null @@ -1,142 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; - -import java.util.*; - -/** - * Configuration information passed to the {@link KafkaStreaming} instance for configuring the associated - * {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer}, and - * {@link org.apache.kafka.clients.consumer.KafkaConsumer KafkaConsumer}, as - * well as the processor itself. - */ -public class StreamingConfig { - - /** topics */ - public static final String TOPICS_CONFIG = "topics"; - - /** state.dir */ - public static final String STATE_DIR_CONFIG = "state.dir"; - - /** poll.time.ms */ - public static final String POLL_TIME_MS_CONFIG = "poll.time.ms"; - - /** commit.time.ms */ - public static final String COMMIT_TIME_MS_CONFIG = "commit.time.ms"; - - /** window.time.ms */ - public static final String WINDOW_TIME_MS_CONFIG = "window.time.ms"; - - /** buffered.records.per.partition */ - public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; - - /** state.cleanup.delay */ - public static final String STATE_CLEANUP_DELAY_CONFIG = "state.cleanup.delay"; - - /** total.records.to.process */ - public static final String TOTAL_RECORDS_TO_PROCESS = "total.records.to.process"; - - /** num.of.stream.threads */ - public static final String NUM_STREAM_THREADS = "num.of.stream.threads"; - - private final Properties config; - private final Map context = new HashMap(); - private Serializer keySerializer; - private Serializer valSerializer; - private Deserializer keyDeserializer; - private Deserializer valDeserializer; - private TimestampExtractor timestampExtractor; - - public StreamingConfig(Properties config) { - this.config = config; - this.config.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - this.config.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); - this.config.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); - } - - @Override - public StreamingConfig clone() { - return new StreamingConfig(this.config); - } - - public void addContextObject(String key, Object value) { - this.context.put(key, value); - } - - @SuppressWarnings("all") - - public void serialization(Serializer serializer, Deserializer deserializer) { - keySerializer(serializer); - valueSerializer(serializer); - keyDeserializer(deserializer); - valueDeserializer(deserializer); - } - - public void keySerializer(Serializer serializer) { - this.keySerializer = serializer; - } - - public void valueSerializer(Serializer serializer) { - this.valSerializer = serializer; - } - - public void keyDeserializer(Deserializer deserializer) { - this.keyDeserializer = deserializer; - } - - public void valueDeserializer(Deserializer deserializer) { - this.valDeserializer = deserializer; - } - - public Properties config() { - return this.config; - } - - public Map context() { - return this.context; - } - - public Serializer keySerializer() { - return this.keySerializer; - } - - public Serializer valueSerializer() { - return this.valSerializer; - } - - public Deserializer keyDeserializer() { - return this.keyDeserializer; - } - - public Deserializer valueDeserializer() { - return this.valDeserializer; - } - - public void timestampExtractor(TimestampExtractor timestampExtractor) { - this.timestampExtractor = timestampExtractor; - } - - public TimestampExtractor timestampExtractor() { - return this.timestampExtractor; - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java b/stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java deleted file mode 100644 index 2e076ce76310a..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/TimestampExtractor.java +++ /dev/null @@ -1,34 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -/** - * An interface that allows the KStream framework to extract a timestamp from a key-value pair - */ -public interface TimestampExtractor { - - /** - * Extracts a timestamp from a key-value pair from a topic - * - * @param topic the topic name - * @param key the key object - * @param value the value object - * @return timestamp - */ - long extract(String topic, Object key, Object value); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java index 526d4c45012ac..87c728b1aa3b4 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java @@ -41,8 +41,8 @@ ======= >>>>>>> removing io.confluent imports: wip import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.KafkaStreaming; -import org.apache.kafka.stream.StreamingConfig; +import org.apache.kafka.clients.processor.KafkaStreaming; +import org.apache.kafka.clients.processor.internals.StreamingConfig; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.KeyValueMapper; diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java index 90f1fc1ea0b49..51216c0597529 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java @@ -22,8 +22,8 @@ import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.KafkaStreaming; -import org.apache.kafka.stream.StreamingConfig; +import org.apache.kafka.clients.processor.KafkaStreaming; +import org.apache.kafka.clients.processor.internals.StreamingConfig; import java.util.Properties; diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java index 7b176f765d1df..5d4f7c38cfabc 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java @@ -19,8 +19,8 @@ import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.KafkaStreaming; -import org.apache.kafka.stream.StreamingConfig; +import org.apache.kafka.clients.processor.KafkaStreaming; +import org.apache.kafka.clients.processor.internals.StreamingConfig; import org.apache.kafka.stream.state.Entry; import org.apache.kafka.stream.state.InMemoryKeyValueStore; import org.apache.kafka.stream.state.KeyValueIterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/Ingestor.java b/stream/src/main/java/org/apache/kafka/stream/internals/Ingestor.java deleted file mode 100644 index 0d9154c7899f1..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/Ingestor.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.common.TopicPartition; - -import java.util.Map; -import java.util.Set; - -public interface Ingestor { - - Set topics(); - - void poll(long timeoutMs); - - void pause(TopicPartition partition); - - void unpause(TopicPartition partition, long offset); - - void commit(Map offsets); - - int numPartitions(String topic); - - void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition); - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/IngestorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/IngestorImpl.java deleted file mode 100644 index 878f6fe91f691..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/IngestorImpl.java +++ /dev/null @@ -1,133 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.clients.consumer.CommitType; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.common.TopicPartition; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -public class IngestorImpl implements Ingestor { - - private static final Logger log = LoggerFactory.getLogger(IngestorImpl.class); - - private final Set topics; - private final Consumer consumer; - private final Set unpaused = new HashSet<>(); - private final Map streamSynchronizers = new HashMap<>(); - - public IngestorImpl(Consumer consumer, Set topics) { - this.consumer = consumer; - this.topics = Collections.unmodifiableSet(topics); - for (String topic : this.topics) consumer.subscribe(topic); - } - - public void open() { - for (String topic : this.topics) consumer.subscribe(topic); - } - - public void init() { - unpaused.clear(); - unpaused.addAll(consumer.subscriptions()); - } - - @Override - public Set topics() { - return topics; - } - - @Override - public void poll(long timeoutMs) { - synchronized (this) { - ConsumerRecords records = consumer.poll(timeoutMs); - - for (TopicPartition partition : unpaused) { - StreamGroup streamGroup = streamSynchronizers.get(partition); - - if (streamGroup != null) - streamGroup.addRecords(partition, records.records(partition).iterator()); - else - log.warn("unused topic: " + partition.topic()); - } - } - } - - @Override - public void pause(TopicPartition partition) { - synchronized (this) { - consumer.seek(partition, Long.MAX_VALUE); // hack: stop consuming from this partition by setting a big offset - unpaused.remove(partition); - } - } - - @Override - public void unpause(TopicPartition partition, long lastOffset) { - synchronized (this) { - consumer.seek(partition, lastOffset); - unpaused.add(partition); - } - } - - @Override - public void commit(Map offsets) { - synchronized (this) { - consumer.commit(offsets, CommitType.SYNC); - } - } - - @Override - public int numPartitions(String topic) { - return consumer.partitionsFor(topic).size(); - } - - @SuppressWarnings("unchecked") - @Override - public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition) { - synchronized (this) { - streamSynchronizers.put(partition, streamGroup); - unpaused.add(partition); - } - } - - public void clear() { - unpaused.clear(); - streamSynchronizers.clear(); - } - - public boolean commitNeeded(Map offsets) { - for (TopicPartition tp : offsets.keySet()) { - if (consumer.committed(tp) != offsets.get(tp)) { - return true; - } - } - return false; - } - - public void close() { - consumer.close(); - clear(); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamContextImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamContextImpl.java deleted file mode 100644 index 0e2b905fe97ce..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamContextImpl.java +++ /dev/null @@ -1,287 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.stream.topology.Processor; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.RecordCollector; -import org.apache.kafka.stream.RestoreFunc; -import org.apache.kafka.stream.StateStore; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.stream.KStreamException; -import org.apache.kafka.stream.StreamingConfig; -import org.apache.kafka.stream.TimestampExtractor; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.Map; -import java.util.Set; - -public class KStreamContextImpl implements KStreamContext { - - private static final Logger log = LoggerFactory.getLogger(KStreamContextImpl.class); - - public final int id; - public final StreamGroup streamGroup; - public final Ingestor ingestor; - - private final RecordCollectorImpl collector; - private final HashMap> sourceStreams = new HashMap<>(); - private final HashMap partitioningInfos = new HashMap<>(); - private final TimestampExtractor timestampExtractor; - private final StreamingConfig streamingConfig; - private final ProcessorConfig processorConfig; - private final Metrics metrics; - private final ProcessorStateManager stateMgr; - - private boolean initialized = false; - - @SuppressWarnings("unchecked") - public KStreamContextImpl(int id, - Ingestor ingestor, - RecordCollectorImpl collector, - StreamingConfig streamingConfig, - ProcessorConfig processorConfig, - Metrics metrics) { - this.id = id; - this.ingestor = ingestor; - this.collector = collector; - this.streamingConfig = streamingConfig; - this.processorConfig = processorConfig; - this.timestampExtractor = this.streamingConfig.timestampExtractor(); - this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id)), - new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer())); - this.metrics = metrics; - this.streamGroup = new StreamGroup(this, this.ingestor, new TimeBasedChooser(), this.timestampExtractor, this.processorConfig.bufferedRecordsPerPartition); - } - - @Override - public int id() { - return id; - } - - @Override - public Serializer keySerializer() { - return streamingConfig.keySerializer(); - } - - @Override - public Serializer valueSerializer() { - return streamingConfig.valueSerializer(); - } - - @Override - public Deserializer keyDeserializer() { - return streamingConfig.keyDeserializer(); - } - - @Override - public Deserializer valueDeserializer() { - return streamingConfig.valueDeserializer(); - } - - @Override - public RecordCollector recordCollector() { - return collector; - } - - @Override - public Map getContext() { - return streamingConfig.context(); - } - - @Override - public File stateDir() { - return stateMgr.baseDir(); - } - - @Override - public Metrics metrics() { - return metrics; - } - - @Override - public void register(StateStore store, RestoreFunc restoreFunc) { - ensureInitialization(); - - stateMgr.register(store, restoreFunc); - } - - public void ensureInitialization() { - if (!initialized) - throw new IllegalStateException("context initialization is already finished"); - } - - @Override - public void flush() { - stateMgr.flush(); - } - - public String topic() { - if (streamGroup.record() == null) - throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); - - return streamGroup.record().topic(); - } - - @Override - public int partition() { - if (streamGroup.record() == null) - throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); - - return streamGroup.record().partition(); - } - - @Override - public long offset() { - if (this.streamGroup.record() == null) - throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); - - return this.streamGroup.record().offset(); - } - - @Override - public long timestamp() { - if (streamGroup.record() == null) - throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); - - return streamGroup.record().timestamp; - } - - @Override - public void send(String topic, Object key, Object value) { - collector.send(new ProducerRecord<>(topic, key, value)); - } - - @Override - public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { - if (keySerializer == null || valSerializer == null) - throw new IllegalStateException("key and value serializers must be specified"); - - collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); - } - - @Override - public void commit() { - streamGroup.commitOffset(); - } - - @Override - public void schedule(Processor processor, long interval) { - streamGroup.schedule(processor, interval); - } - - public void init(Collection> streams) throws IOException { - stateMgr.init(); - - for (KStreamSource stream : streams) { - KStreamMetadata metadata = linkStreamToTopics(stream); - - stream.bind(this, metadata); - } - - // add partition -> stream group mappings to the ingestor - for (Map.Entry> entry : sourceStreams.entrySet()) { - TopicPartition partition = new TopicPartition(entry.getKey(), id); - ingestor.addPartitionStreamToGroup(streamGroup, partition); - } - - if (!ingestor.topics().equals(sourceStreams.keySet())) { - LinkedList unusedTopics = new LinkedList<>(); - for (String topic : ingestor.topics()) { - if (!sourceStreams.containsKey(topic)) - unusedTopics.add(topic); - } - throw new KStreamException("unused topics: " + Utils.mkString(unusedTopics)); - } - - initialized = true; - } - - private KStreamMetadata linkStreamToTopics(KStreamSource stream) { - ensureInitialization(); - - Set fromTopics; - - synchronized (this) { - // if topics not specified, use all the topics be default - if (stream.topics == null || stream.topics.length == 0) { - fromTopics = ingestor.topics(); - } else { - fromTopics = Collections.unmodifiableSet(Utils.mkSet(stream.topics)); - } - - // iterate over the topics and check if the stream has already been created for them - for (String topic : fromTopics) { - if (!ingestor.topics().contains(topic)) - throw new IllegalArgumentException("topic not subscribed: " + topic); - - if (sourceStreams.containsKey(topic)) - throw new IllegalArgumentException("another stream created with the same topic " + topic); - } - - // create stream metadata - Map topicPartitionInfos = new HashMap<>(); - for (String topic : fromTopics) { - PartitioningInfo partitioningInfo = this.partitioningInfos.get(topic); - - if (partitioningInfo == null) { - partitioningInfo = new PartitioningInfo(ingestor.numPartitions(topic)); - this.partitioningInfos.put(topic, partitioningInfo); - } - - topicPartitionInfos.put(topic, partitioningInfo); - } - - // update source stream map - for (String topic : fromTopics) { - sourceStreams.put(topic, stream); - - TopicPartition partition = new TopicPartition(topic, id); - streamGroup.addPartition(partition, stream); - } - - return new KStreamMetadata(topicPartitionInfos); - } - } - - public Map consumedOffsets() { - return streamGroup.consumedOffsets(); - } - - public void close() throws Exception { - stateMgr.close(collector.offsets()); - streamGroup.close(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/PartitioningInfo.java b/stream/src/main/java/org/apache/kafka/stream/internals/PartitioningInfo.java deleted file mode 100644 index 7840dab82a930..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/PartitioningInfo.java +++ /dev/null @@ -1,27 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -public class PartitioningInfo { - - public final int numPartitions; - - public PartitioningInfo(int numPartitions) { - this.numPartitions = numPartitions; - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorConfig.java deleted file mode 100644 index b34e17f67d537..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorConfig.java +++ /dev/null @@ -1,104 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -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 org.apache.kafka.stream.StreamingConfig; - -import java.io.File; -import java.util.Properties; - -public class ProcessorConfig extends AbstractConfig { - - private static final ConfigDef CONFIG; - - static { - CONFIG = new ConfigDef().define(StreamingConfig.TOPICS_CONFIG, - Type.STRING, - "", - Importance.HIGH, - "All the possible topic names this job need to interact with") - .define(StreamingConfig.STATE_DIR_CONFIG, - Type.STRING, - System.getProperty("java.io.tmpdir"), - Importance.MEDIUM, - "") - .define(StreamingConfig.POLL_TIME_MS_CONFIG, - Type.LONG, - 100, - Importance.LOW, - "The amount of time to block waiting for input.") - .define(StreamingConfig.COMMIT_TIME_MS_CONFIG, - Type.LONG, - 30000, - Importance.HIGH, - "The frequency with which to save the position of the processor.") - .define(StreamingConfig.WINDOW_TIME_MS_CONFIG, - Type.LONG, - -1L, - Importance.MEDIUM, - "Setting this to a non-negative value will cause the processor to get called with this frequency even if there is no message.") - .define(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, - Type.INT, - 1000, - Importance.LOW, - "The maximum number of records to buffer per partition") - .define(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG, - Type.LONG, - 60000, - Importance.LOW, - "The amount of time to wait before deleting state when a partition has migrated.") - .define(StreamingConfig.TOTAL_RECORDS_TO_PROCESS, - Type.LONG, - -1L, - Importance.LOW, - "Exit after processing this many records.") - .define(StreamingConfig.NUM_STREAM_THREADS, - Type.INT, - 1, - Importance.LOW, - "The number of threads to execute stream processing."); - } - - public final String topics; - public final File stateDir; - public final long pollTimeMs; - public final long commitTimeMs; - public final long windowTimeMs; - public final int bufferedRecordsPerPartition; - public final long stateCleanupDelay; - public final long totalRecordsToProcess; - public final int numStreamThreads; - - public ProcessorConfig(Properties processor) { - super(CONFIG, processor); - this.topics = this.getString(StreamingConfig.TOPICS_CONFIG); - this.stateDir = new File(this.getString(StreamingConfig.STATE_DIR_CONFIG)); - this.pollTimeMs = this.getLong(StreamingConfig.POLL_TIME_MS_CONFIG); - this.commitTimeMs = this.getLong(StreamingConfig.COMMIT_TIME_MS_CONFIG); - this.windowTimeMs = this.getLong(StreamingConfig.WINDOW_TIME_MS_CONFIG); - this.bufferedRecordsPerPartition = this.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); - this.stateCleanupDelay = this.getLong(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG); - this.totalRecordsToProcess = this.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); - this.numStreamThreads = this.getInt(StreamingConfig.NUM_STREAM_THREADS); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorNode.java deleted file mode 100644 index 600f2938ab2eb..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorNode.java +++ /dev/null @@ -1,52 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.stream.topology.Processor; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; - -public class ProcessorNode implements Receiver { - - private final Processor processor; - private KStreamContext context; - - public ProcessorNode(Processor processor) { - this.processor = processor; - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); - - this.context = context; - processor.init(context); - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - processor.process((K) key, (V) value); - } - - @Override - public void close() { - processor.close(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorStateManager.java deleted file mode 100644 index 265ab778c6385..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/ProcessorStateManager.java +++ /dev/null @@ -1,194 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.stream.RestoreFunc; -import org.apache.kafka.stream.StateStore; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.stream.util.OffsetCheckpoint; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -public class ProcessorStateManager { - - private static final Logger log = LoggerFactory.getLogger(ProcessorStateManager.class); - private static final String CHECKPOINT_FILE_NAME = ".checkpoint"; - - private final int id; - private final File baseDir; - private final Map stores; - private final Consumer restoreConsumer; - private final Map restoredOffsets; - private final Map checkpointedOffsets; - - public ProcessorStateManager(int id, File baseDir, Consumer restoreConsumer) { - this.id = id; - this.baseDir = baseDir; - this.stores = new HashMap<>(); - this.restoreConsumer = restoreConsumer; - this.restoredOffsets = new HashMap<>(); - this.checkpointedOffsets = new HashMap<>(); - } - - public File baseDir() { - return this.baseDir; - } - - public Consumer restoreConsumer() { - return this.restoreConsumer; - } - - public void init() throws IOException { - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); - this.checkpointedOffsets.putAll(checkpoint.read()); - - // delete the checkpoint file after finish loading its stored offsets - checkpoint.delete(); - } - - 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().keySet().contains(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.subscriptions().isEmpty()) { - throw new IllegalStateException("Restore consumer should have not subscribed to any partitions beforehand"); - } - restoreConsumer.subscribe(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 { - restoreConsumer.seekToBeginning(storePartition); - } - - // restore its state from changelog records; while restoring the log end offset - // should not change since it is only written by this thread. - while (true) { - for (ConsumerRecord record : restoreConsumer.poll(100)) { - 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-subscribe the change log partition - restoreConsumer.unsubscribe(storePartition); - } - - 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 engine : this.stores.values()) - engine.flush(); - } - } - - public void close(Map ackedOffsets) throws IOException { - if (!stores.isEmpty()) { - log.debug("Closing stores."); - for (Map.Entry entry : stores.entrySet()) { - log.debug("Closing storage engine {}", entry.getKey()); - entry.getValue().flush(); - entry.getValue().close(); - } - - Map checkpointOffsets = new HashMap(restoredOffsets); - 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()) { - if (ackedOffsets.containsKey(part)) - // store the last ack'd offset + 1 (the log position after restoration) - checkpointOffsets.put(part, ackedOffsets.get(part) + 1); - } else { - checkpointOffsets.remove(part); - } - } - - // write the checkpoint file before closing, to indicate clean shutdown - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); - checkpoint.write(checkpointOffsets); - } - - // close the restore consumer - restoreConsumer.close(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationQueue.java b/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationQueue.java deleted file mode 100644 index 6e2b53f326518..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationQueue.java +++ /dev/null @@ -1,52 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import java.util.PriorityQueue; - -public class PunctuationQueue { - - private PriorityQueue pq = new PriorityQueue<>(); - - public void schedule(PunctuationSchedule sched) { - synchronized (pq) { - pq.add(sched); - } - } - - public void close() { - synchronized (pq) { - pq.clear(); - } - } - - public void mayPunctuate(long streamTime) { - synchronized (pq) { - PunctuationSchedule top = pq.peek(); - while (top != null && top.timestamp <= streamTime) { - PunctuationSchedule sched = top; - pq.poll(); - sched.processor().punctuate(streamTime); - pq.add(sched.next()); - - top = pq.peek(); - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationSchedule.java deleted file mode 100644 index 8dcfc1e65c363..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/PunctuationSchedule.java +++ /dev/null @@ -1,40 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.stream.topology.Processor; -import org.apache.kafka.stream.util.Stamped; - -public class PunctuationSchedule extends Stamped> { - - final long interval; - - public PunctuationSchedule(Processor processor, long interval) { - super(processor, System.currentTimeMillis() + interval); - this.interval = interval; - } - - public Processor processor() { - return value; - } - - public PunctuationSchedule next() { - return new PunctuationSchedule(value, timestamp + interval); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/RecordCollectorImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/RecordCollectorImpl.java deleted file mode 100644 index 6948d7b2f8154..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/RecordCollectorImpl.java +++ /dev/null @@ -1,92 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.stream.RecordCollector; -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 RecordCollectorImpl implements RecordCollector { - - private static final Logger log = LoggerFactory.getLogger(RecordCollectorImpl.class); - - private final Producer producer; - private final Map offsets; - private final Callback callback = new Callback() { - public void onCompletion(RecordMetadata metadata, Exception exception) { - if (exception == null) { - TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); - offsets.put(tp, metadata.offset()); - } else { - log.error("Error sending record: ", exception); - } - } - }; - private final Serializer keySerializer; - private final Serializer valueSerializer; - - - public RecordCollectorImpl(Producer producer, Serializer keySerializer, Serializer valueSerializer) { - this.producer = producer; - this.offsets = new HashMap<>(); - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - } - - @Override - public void send(ProducerRecord record) { - send(record, this.keySerializer, this.valueSerializer); - } - - @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { - byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); - byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); - this.producer.send(new ProducerRecord<>(record.topic(), keyBytes, valBytes), callback); - } - - @Override - public void flush() { - this.producer.flush(); - } - - /** - * Closes this RecordCollector - */ - public void close() { - producer.close(); - } - - /** - * The last ack'd offset from the producer - * - * @return the map from TopicPartition to offset - */ - Map offsets() { - return this.offsets; - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/stream/internals/RecordQueue.java deleted file mode 100644 index 4db0448bbdf89..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/RecordQueue.java +++ /dev/null @@ -1,123 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.stream.topology.internals.KStreamSource; -import org.apache.kafka.stream.util.TimestampTracker; - -import java.util.ArrayDeque; - -/** - * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). It is intended to be used in - * {@link StreamGroup}. - */ -public class RecordQueue { - - private final ArrayDeque queue = new ArrayDeque<>(); - public final KStreamSource stream; - private final TopicPartition partition; - private TimestampTracker> timestampTracker; - private long offset; - - /** - * Creates a new instance of RecordQueue - * - * @param partition partition - * @param stream the instance of KStreamImpl that receives records - * @param timestampTracker TimestampTracker - */ - public RecordQueue(TopicPartition partition, KStreamSource stream, TimestampTracker> timestampTracker) { - this.partition = partition; - this.stream = stream; - this.timestampTracker = timestampTracker; - } - - /** - * Returns the partition with which this queue is associated - * - * @return TopicPartition - */ - public TopicPartition partition() { - return partition; - } - - /** - * Adds a StampedRecord to the queue - * - * @param record StampedRecord - */ - public void add(StampedRecord record) { - queue.addLast(record); - timestampTracker.addStampedElement(record); - offset = record.offset(); - } - - /** - * Returns the next record fro the queue - * - * @return StampedRecord - */ - public StampedRecord next() { - StampedRecord elem = queue.pollFirst(); - - if (elem == null) return null; - - timestampTracker.removeStampedElement(elem); - - return elem; - } - - /** - * Returns the highest offset in the queue - * - * @return offset - */ - public long offset() { - return offset; - } - - /** - * Returns the number of records in the queue - * - * @return the number of records - */ - public int size() { - return queue.size(); - } - - /** - * Tests if the queue is empty - * - * @return true if the queue is empty, otherwise false - */ - public boolean isEmpty() { - return queue.isEmpty(); - } - - /** - * Returns a timestamp tracked by the TimestampTracker - * - * @return timestamp - */ - public long trackedTimestamp() { - return timestampTracker.get(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/stream/internals/RoundRobinChooser.java deleted file mode 100644 index bb8cb88973c2e..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/RoundRobinChooser.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.stream.Chooser; - -import java.util.ArrayDeque; - -public class RoundRobinChooser implements Chooser { - - private final ArrayDeque deque; - - public RoundRobinChooser() { - deque = new ArrayDeque<>(); - } - - @Override - public void add(RecordQueue queue) { - deque.offer(queue); - } - - @Override - public RecordQueue next() { - return deque.poll(); - } - - @Override - public void close() { - deque.clear(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/StampedRecord.java b/stream/src/main/java/org/apache/kafka/stream/internals/StampedRecord.java deleted file mode 100644 index 9817021ce140c..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/StampedRecord.java +++ /dev/null @@ -1,49 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.stream.util.Stamped; - -// TODO: making this class exposed to user in the lower-level Processor -public class StampedRecord extends Stamped> { - - StampedRecord(ConsumerRecord record, long timestamp) { - super(record, timestamp); - } - - public String topic() { - return value.topic(); - } - - public int partition() { - return value.partition(); - } - - public Object key() { - return value.key(); - } - - public Object value() { - return value.value(); - } - - public long offset() { - return value.offset(); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/stream/internals/TimeBasedChooser.java deleted file mode 100644 index 58cf73384d31c..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/TimeBasedChooser.java +++ /dev/null @@ -1,61 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.stream.Chooser; - -import java.util.Comparator; -import java.util.PriorityQueue; - -public class TimeBasedChooser implements Chooser { - - private final PriorityQueue pq; - - public TimeBasedChooser() { - this(new Comparator() { - public int compare(RecordQueue queue1, RecordQueue queue2) { - long time1 = queue1.trackedTimestamp(); - long time2 = queue2.trackedTimestamp(); - - if (time1 < time2) return -1; - if (time1 > time2) return 1; - return 0; - } - }); - } - - private TimeBasedChooser(Comparator comparator) { - pq = new PriorityQueue<>(3, comparator); - } - - @Override - public void add(RecordQueue queue) { - pq.offer(queue); - } - - @Override - public RecordQueue next() { - return pq.poll(); - } - - @Override - public void close() { - pq.clear(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java index b3739f6a46c3b..1e3954a1b3b1a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java @@ -19,7 +19,7 @@ package org.apache.kafka.stream.state; -import org.apache.kafka.stream.StateStore; +import org.apache.kafka.clients.processor.StateStore; import java.util.List; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java index 1cbdd4f1de1af..74647d4881825 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java @@ -18,8 +18,8 @@ package org.apache.kafka.stream.state.internals; import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.RecordCollector; -import org.apache.kafka.stream.RestoreFunc; +import org.apache.kafka.clients.processor.RecordCollector; +import org.apache.kafka.clients.processor.RestoreFunc; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.MeasurableStat; diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java index 9933c85018709..a9066d8965cf0 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java @@ -17,9 +17,7 @@ package org.apache.kafka.stream.topology; -import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.topology.internals.KStreamSource; @@ -73,11 +71,17 @@ public abstract class KStreamTopology { * @return KStream */ public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - // TODO + SourceProcessor source = new SourceProcessor<>("KAFKA-SOURCE"); topology.addProcessor(source, keyDeserializer, valDeserializer, topics); return new KStreamSource<>(topology, source); } + + public PTopology get() { + this.topology.build(); + + return this.topology; + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java index d4b1aed703831..ccaf1f0b314f0 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java @@ -20,5 +20,4 @@ public interface KeyValueMapper { KeyValue apply(K1 key, V1 value); - } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java index 081a1c3b2702e..7bbe03ea58ceb 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java @@ -28,7 +28,7 @@ import org.apache.kafka.stream.RestoreFunc; import org.apache.kafka.stream.topology.internals.WindowSupport; import org.apache.kafka.stream.util.FilteredIterator; -import org.apache.kafka.stream.util.Stamped; +import org.apache.kafka.clients.processor.internals.Stamped; import java.util.HashMap; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java index 8d69a45daf86e..401c05e004caa 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java @@ -20,5 +20,4 @@ public interface ValueMapper { V2 apply(V1 value); - } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Window.java b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java index 8d7bb3da0a03a..708d5eb86831f 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Window.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java @@ -18,7 +18,7 @@ package org.apache.kafka.stream.topology; import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.StateStore; +import org.apache.kafka.clients.processor.StateStore; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java index 5ac6d1eca1978..35aa33e01e069 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java @@ -17,6 +17,10 @@ package org.apache.kafka.stream.topology.internals; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.internals.Receiver; import org.apache.kafka.stream.topology.KStreamTopology; @@ -24,45 +28,49 @@ import java.lang.reflect.Array; import java.util.Arrays; +import java.util.concurrent.atomic.AtomicInteger; -class KStreamBranch implements Receiver { +class KStreamBranch extends KafkaProcessor { + private static final String BRANCH_NAME = "KAFKA-BRANCH"; + private static final AtomicInteger BRANCH_INDEX = new AtomicInteger(1); + + private final PTopology topology; private final Predicate[] predicates; - final KStreamSource[] branches; + private final SourceProcessor[] branches; + private final KafkaProcessor parent; @SuppressWarnings("unchecked") - KStreamBranch(Predicate[] predicates, KStreamTopology topology) { + public KStreamBranch(Predicate[] predicates, PTopology topology, KafkaProcessor parent) { + super(BRANCH_NAME); + + this.parent = parent; + this.topology = topology; this.predicates = Arrays.copyOf(predicates, predicates.length); - this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); + this.branches = (SourceProcessor[]) Array.newInstance(SourceProcessor.class, predicates.length); for (int i = 0; i < branches.length; i++) { - branches[i] = new KStreamSource<>(null, topology); + branches[i] = new SourceProcessor<>(BRANCH_NAME + BRANCH_INDEX.getAndIncrement()); + topology.addProcessor(branches[i], parent); } } @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - for (KStreamSource branch : branches) { - branch.bind(context, metadata); - } - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { + public void process(K key, V value) { for (int i = 0; i < predicates.length; i++) { Predicate predicate = predicates[i]; - if (predicate.apply((K) key, (V) value)) { - branches[i].receive(key, value, timestamp); + if (predicate.apply( key, value)) { + branches[i].receive(key, value); return; } } } - @Override - public void close() { - for (KStreamSource branch : branches) { - branch.close(); + @SuppressWarnings("unchecked") + public KStream[] branches() { + KStream[] streams = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); + for (int i = 0; i < branches.length; i++) { + streams[i] = new KStreamSource<>(topology, branches[i]); } + return streams; } - } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java index fab55b3c9dc62..b22a87c86d5ed 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java @@ -18,7 +18,6 @@ package org.apache.kafka.stream.topology.internals; import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.topology.Predicate; class KStreamFilter extends KafkaProcessor { @@ -39,11 +38,6 @@ public KStreamFilter(Predicate predicate, boolean filterOut) { this.filterOut = filterOut; } - @Override - public void init(ProcessorContext context) { - // do nothing - } - @Override public void process(K key, V value) { if ((!filterOut && predicate.apply(key, value)) @@ -51,9 +45,4 @@ public void process(K key, V value) { forward(key, value); } } - - @Override - public void close() { - // do nothing - } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java index 72d8d66b039db..b48e700ebe179 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java @@ -18,9 +18,6 @@ package org.apache.kafka.stream.topology.internals; import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.KeyValueMapper; @@ -36,11 +33,6 @@ class KStreamFlatMap extends KafkaProcessor { this.mapper = mapper; } - @Override - public void init(ProcessorContext context) { - // do nothing - } - @Override public void process(K1 key, V1 value) { KeyValue> newPair = mapper.apply(key, value); @@ -48,10 +40,4 @@ public void process(K1 key, V1 value) { forward(newPair.key, v); } } - - @Override - public void close() { - // do nothing - } - } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java index ea9c05bca9c54..fe496fe44a027 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java @@ -17,27 +17,25 @@ package org.apache.kafka.stream.topology.internals; -import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.stream.topology.ValueMapper; -class KStreamFlatMapValues extends KStreamImpl { +class KStreamFlatMapValues extends KafkaProcessor { - private final ValueMapper, V1> mapper; + private static final String FLATMAPVALUES_NAME = "KAFKA-FLATMAPVALUES"; - KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology topology) { - super(topology); + private final ValueMapper> mapper; + + KStreamFlatMapValues(ValueMapper> mapper) { + super(FLATMAPVALUES_NAME); this.mapper = mapper; } - @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - Iterable newValues = mapper.apply((V1) value); - for (V v : newValues) { - forward(key, v, timestamp); - } + public void process(K1 key, V1 value) { + Iterable newValues = mapper.apply(value); + for (V2 v : newValues) { + forward(key, v); } } - } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java index a85e8f3212a38..2287231ef78a3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java @@ -19,99 +19,90 @@ import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.clients.processor.Processor; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KStream; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.stream.internals.ProcessorNode; -import org.apache.kafka.stream.internals.Receiver; import org.apache.kafka.stream.topology.KStreamWindowed; import org.apache.kafka.stream.topology.KeyValueMapper; import org.apache.kafka.stream.topology.Predicate; -import org.apache.kafka.stream.topology.Transformer; import org.apache.kafka.stream.topology.ValueMapper; import org.apache.kafka.stream.topology.Window; -abstract class KStreamImpl implements KStream, Receiver { +public class KStreamImpl implements KStream { + + private static final String PROCESSOR_NAME = "KAFKA-PROCESS"; protected PTopology topology; - protected KafkaProcessor processor; + protected KafkaProcessor processor; protected KStreamContext context; protected KStreamMetadata metadata; - protected KStreamImpl(PTopology topology, KafkaProcessor processor) { + protected KStreamImpl(PTopology topology, KafkaProcessor processor) { this.topology = topology; this.processor = processor; } - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); - this.context = context; - this.metadata = metadata; - int numReceivers = nextReceivers.size(); - for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).bind(context, metadata); - } - } - - @Override - public void close() { - int numReceivers = nextReceivers.size(); - for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).close(); - } - } - + @SuppressWarnings("unchecked") @Override public KStream filter(Predicate predicate) { KStreamFilter filter = new KStreamFilter<>(predicate); topology.addProcessor(filter, processor); - return new KStreamImpl(topology, filter); + return new KStreamImpl<>(topology, filter); } + @SuppressWarnings("unchecked") @Override public KStream filterOut(final Predicate predicate) { KStreamFilter filter = new KStreamFilter<>(predicate, true); topology.addProcessor(filter, processor); - return new KStreamImpl(topology, filter); + return new KStreamImpl<>(topology, filter); } + @SuppressWarnings("unchecked") @Override public KStream map(KeyValueMapper mapper) { KStreamMap map = new KStreamMap<>(mapper); topology.addProcessor(map, processor); - return new KStreamImpl(topology, map); + return new KStreamImpl<>(topology, map); } + @SuppressWarnings("unchecked") @Override public KStream mapValues(ValueMapper mapper) { KStreamMapValues map = new KStreamMapValues<>(mapper); topology.addProcessor(map, processor); - return new KStreamImpl(topology, map); + return new KStreamImpl<>(topology, map); } + @SuppressWarnings("unchecked") @Override - public KStream flatMap(KeyValueMapper> mapper) { - KStreamFlatMap map = new KStreamFlatMap<>(mapper); + public KStream flatMap(KeyValueMapper> mapper) { + KStreamFlatMap map = new KStreamFlatMap<>(mapper); topology.addProcessor(map, processor); - return new KStreamImpl(topology, map); + return new KStreamImpl<>(topology, map); } + @SuppressWarnings("unchecked") @Override - public KStream flatMapValues(ValueMapper, V> mapper) { - return chain(new KStreamFlatMapValues(mapper, topology)); + public KStream flatMapValues(ValueMapper> mapper) { + KStreamFlatMapValues map = new KStreamFlatMapValues<>(mapper); + + topology.addProcessor(map, processor); + + return new KStreamImpl<>(topology, map); } @Override @@ -121,9 +112,8 @@ public KStreamWindowed with(Window window) { @Override public KStream[] branch(Predicate... predicates) { - KStreamBranch branch = new KStreamBranch<>(predicates, topology); - registerReceiver(branch); - return branch.branches; + KStreamBranch branch = new KStreamBranch<>(predicates, topology, processor); + return branch.branches(); } @SuppressWarnings("unchecked") @@ -134,9 +124,18 @@ public KStream through(String topic) { @SuppressWarnings("unchecked") @Override - public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { + public KStream through(String topic, + Serializer keySerializer, + Serializer valSerializer, + Deserializer keyDeserializer, + Deserializer valDeserializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); - return topology.from(keyDeserializer, valDeserializer, topic); + + SourceProcessor source = new SourceProcessor<>("KAFKA-SOURCE"); + + topology.addProcessor(source, keyDeserializer, valDeserializer, topic); + + return new KStreamSource<>(topology, source); } @Override @@ -150,24 +149,20 @@ public void sendTo(String topic, Serializer keySerializer, Serializer valS } @SuppressWarnings("unchecked") - private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { - return new Processor() { - private KStreamContext context; + private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { + return new Processor() { + private ProcessorContext context; @Override - public void init(KStreamContext context) { + public void init(ProcessorContext context) { this.context = context; } @Override - public void process(K key, V value) { + public void process(K1 key, V1 value) { this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); } - @Override - public void punctuate(long streamTime) { - } - @Override public void close() { } @@ -176,32 +171,22 @@ public void close() { @SuppressWarnings("unchecked") @Override - public void process(Processor processor) { - registerReceiver(new ProcessorNode<>(processor)); + public KStream process(final KafkaProcessor current) { + topology.addProcessor(current, processor); + + return new KStreamImpl(topology, current); } @SuppressWarnings("unchecked") @Override - public KStream transform(Transformer transformer) { - return chain(new KStreamTransform<>(transformer, topology)); - } - - void registerReceiver(Receiver receiver) { - nextReceivers.add(receiver); - } - - protected void forward(Object key, Object value, long timestamp) { - int numReceivers = nextReceivers.size(); - for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).receive(key, value, timestamp); - } - } + public void process(final Processor current) { + KafkaProcessor wrapper = new KafkaProcessor(PROCESSOR_NAME) { + @Override + public void process(K key, V value) { + current.process(key, value); + } + }; - protected KStream chain(KStreamImpl kstream) { - synchronized (this) { - nextReceivers.add(kstream); - return kstream; - } + topology.addProcessor(wrapper, processor); } - } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java index 620aa8b994da4..1833e1f7391cd 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java @@ -18,7 +18,6 @@ package org.apache.kafka.stream.topology.internals; import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.KeyValueMapper; @@ -34,19 +33,9 @@ public KStreamMap(KeyValueMapper mapper) { this.mapper = mapper; } - @Override - public void init(ProcessorContext context) { - // do nothing - } - @Override public void process(K1 key, V1 value) { KeyValue newPair = mapper.apply(key, value); forward(newPair.key, newPair.value); } - - @Override - public void close() { - // do nothing - } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java index 3edfd9a153cfa..61e5d466ffb7d 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java @@ -18,7 +18,6 @@ package org.apache.kafka.stream.topology.internals; import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.topology.ValueMapper; class KStreamMapValues extends KafkaProcessor { @@ -33,19 +32,9 @@ public KStreamMapValues(ValueMapper mapper) { this.mapper = mapper; } - @Override - public void init(ProcessorContext context) { - // do nothing - } - @Override public void process(K1 key, V1 value) { V2 newValue = mapper.apply(value); forward(key, newValue); } - - @Override - public void close() { - // do nothing - } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java index 6f3a0996d1622..936525d8fa792 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java @@ -17,8 +17,8 @@ package org.apache.kafka.stream.topology.internals; -import org.apache.kafka.stream.internals.PartitioningInfo; -import org.apache.kafka.stream.internals.StreamGroup; +import org.apache.kafka.clients.processor.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.StreamGroup; import java.util.Collections; import java.util.Map; diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java index 3e3a595676636..7f6a7ea47481a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java @@ -26,13 +26,4 @@ public class KStreamSource extends KStreamImpl { public KStreamSource(PTopology topology, SourceProcessor source) { super(topology, source); } - - @SuppressWarnings("unchecked") - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - if (keyDeserializer == null) keyDeserializer = (Deserializer) context.keyDeserializer(); - if (valueDeserializer == null) valueDeserializer = (Deserializer) context.valueDeserializer(); - - super.bind(context, metadata); - } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamTransform.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamTransform.java deleted file mode 100644 index bf09d7d24a32a..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamTransform.java +++ /dev/null @@ -1,53 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.Transformer; - -class KStreamTransform extends KStreamImpl implements Transformer.Forwarder { - - private final Transformer transformer; - - KStreamTransform(Transformer transformer, KStreamTopology topology) { - super(topology); - this.transformer = transformer; - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - transformer.init(context); - transformer.forwarder(this); - - super.bind(context, KStreamMetadata.unjoinable()); - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - transformer.process((K1) key, (V1) value); - } - } - - @Override - public void send(K key, V value, long timestamp) { - forward(key, value, timestamp); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java index 55ec056f4987c..fb7fa8c0f5041 100644 --- a/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java @@ -17,6 +17,8 @@ package org.apache.kafka.stream.util; +import org.apache.kafka.clients.processor.internals.Stamped; + import java.util.LinkedList; /** diff --git a/stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java b/stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java deleted file mode 100644 index 4acc83df624c3..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/util/ParallelExecutor.java +++ /dev/null @@ -1,152 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.util; - -import java.util.ArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.LockSupport; - -/** - * A lightweight parallel executor - */ -public class ParallelExecutor { - - /** - * A parallel task must implement this interface - */ - public interface Task { - /** - * Executes a task - * - * @return boolean true if the task are ready for next execution - */ - boolean process(); - } - - private final WorkerThread[] workerThreads; - private final AtomicInteger taskIndex = new AtomicInteger(0); - private volatile ArrayList tasks = new ArrayList<>(); - private volatile CountDownLatch latch; - private volatile boolean readyForNextExecution = true; - private volatile boolean running = true; - private volatile Exception exception; - - public ParallelExecutor(int parallelDegree) { - parallelDegree = Math.max(parallelDegree, 1); - workerThreads = new WorkerThread[parallelDegree - 1]; - for (int i = 0; i < workerThreads.length; i++) { - workerThreads[i] = new WorkerThread(); - workerThreads[i].start(); - } - } - - /** - * Executes tasks in parallel. While this method is executing, other execute call will be blocked. - * - * @param tasks a list of tasks executed in parallel - * @return boolean true if at least one task is ready for next execution, otherwise false - * @throws Exception an exception thrown by a failed task - */ - public boolean execute(ArrayList tasks) throws Exception { - synchronized (this) { - try { - int numTasks = tasks.size(); - exception = null; - readyForNextExecution = false; - if (numTasks > 0) { - this.tasks = tasks; - this.latch = new CountDownLatch(numTasks); - - taskIndex.set(numTasks); - wakeUpWorkers(Math.min(numTasks - 1, workerThreads.length)); - - // the calling thread also picks up tasks - if (taskIndex.get() > 0) doProcess(); - - while (true) { - try { - latch.await(); - break; - } catch (InterruptedException ex) { - Thread.interrupted(); - } - } - } - if (exception != null) throw exception; - } finally { - this.tasks = null; - this.latch = null; - this.exception = null; - } - return readyForNextExecution; - } - } - - /** - * Shuts this parallel executor down - */ - public void shutdown() { - synchronized (this) { - running = false; - // wake up all workers - wakeUpWorkers(workerThreads.length); - } - } - - private void doProcess() { - int index = taskIndex.decrementAndGet(); - if (index >= 0) { - try { - if (tasks.get(index).process()) - this.readyForNextExecution = true; - } catch (Exception ex) { - exception = ex; - } finally { - latch.countDown(); - } - } - } - - private void wakeUpWorkers(int numWorkers) { - for (int i = 0; i < numWorkers; i++) - LockSupport.unpark(workerThreads[i]); - } - - private class WorkerThread extends Thread { - - WorkerThread() { - super(); - setDaemon(true); - } - - @Override - public void run() { - while (running) { - if (taskIndex.get() > 0) { - doProcess(); - } else { - // no more work. park this thread. - LockSupport.park(); - Thread.interrupted(); - } - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/util/Stamped.java b/stream/src/main/java/org/apache/kafka/stream/util/Stamped.java deleted file mode 100644 index ed5d4b50040cc..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/util/Stamped.java +++ /dev/null @@ -1,38 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.util; - -public class Stamped implements Comparable { - - public final V value; - public final long timestamp; - - public Stamped(V value, long timestamp) { - this.value = value; - this.timestamp = timestamp; - } - - public int compareTo(Object other) { - long otherTimestamp = ((Stamped) other).timestamp; - - if (timestamp < otherTimestamp) return -1; - else if (timestamp > otherTimestamp) return 1; - return 0; - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java index c887383d96676..b33437c0b6615 100644 --- a/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java @@ -17,6 +17,8 @@ package org.apache.kafka.stream.util; +import org.apache.kafka.clients.processor.internals.Stamped; + /** * TimestampTracker is a helper class for a sliding window implementation. * It is assumed that elements are added or removed in a FIFO manner. diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java index a4c7d3b8618e2..9dab0d4015883 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Predicate; import org.apache.kafka.stream.topology.internals.KStreamMetadata; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java index ae9bdec644d2d..f6142c97ecb1b 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Predicate; import org.apache.kafka.stream.topology.internals.KStreamMetadata; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java index b227f72a36743..bcc49b8b95e1e 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.KeyValueMapper; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java index b5a9acee45690..58b249344554f 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.ValueMapper; import org.apache.kafka.stream.topology.internals.KStreamMetadata; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java index 9228e01fda4a5..8b66959bd2efb 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java @@ -18,7 +18,7 @@ package org.apache.kafka.stream; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.stream.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KStreamWindowed; import org.apache.kafka.stream.topology.KeyValue; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java index f738e62798f54..7aaf496a6dfd8 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.KeyValueMapper; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java index d0da399c0f38a..7e081f8827622 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.stream; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamContext; @@ -39,6 +40,9 @@ ======= import org.apache.kafka.stream.internals.PartitioningInfo; >>>>>>> compile and test passed +======= +import org.apache.kafka.clients.processor.internals.PartitioningInfo; +>>>>>>> wip import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.ValueMapper; import org.apache.kafka.stream.topology.internals.KStreamMetadata; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java index 45e061d048b1a..7ce80f451b26d 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.internals.KStreamMetadata; import org.apache.kafka.stream.topology.internals.KStreamSource; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java index 8faeeacda5b41..fb4755510731b 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Transformer; import org.apache.kafka.stream.topology.internals.KStreamMetadata; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java index 0d201f2b20b7d..21ce70c16f063 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.PartitioningInfo; import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Window; import org.apache.kafka.stream.topology.internals.KStreamMetadata; diff --git a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java index 8e58407950dea..6336ad076d450 100644 --- a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java @@ -20,7 +20,7 @@ import static org.junit.Assert.assertEquals; import org.apache.kafka.stream.util.MinTimestampTracker; -import org.apache.kafka.stream.util.Stamped; +import org.apache.kafka.clients.processor.internals.Stamped; import org.junit.Test; public class MinTimestampTrackerTest { diff --git a/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java b/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java index 31b5e324a5dcd..e0601914ee6c5 100644 --- a/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.util.ParallelExecutor; +import org.apache.kafka.clients.processor.internals.ParallelExecutor; import org.junit.Test; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java index 92821cc719e2f..3cc89a78d3615 100644 --- a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java @@ -18,13 +18,14 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.processor.TimestampExtractor; 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.stream.internals.StreamGroup; -import org.apache.kafka.stream.internals.TimeBasedChooser; +import org.apache.kafka.clients.processor.internals.StreamGroup; +import org.apache.kafka.clients.processor.internals.TimeBasedChooser; import org.apache.kafka.stream.topology.internals.KStreamSource; import org.apache.kafka.test.MockIngestor; import org.apache.kafka.test.MockKStreamContext; diff --git a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java index 4d1048a90c32a..226520a757514 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java @@ -19,8 +19,8 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.stream.internals.Ingestor; -import org.apache.kafka.stream.internals.StreamGroup; +import org.apache.kafka.clients.processor.internals.Ingestor; +import org.apache.kafka.clients.processor.internals.StreamGroup; import java.util.HashMap; import java.util.HashSet; diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java index 5116f413a8555..b49d3e3c7e7e7 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java @@ -51,14 +51,20 @@ ======= import org.apache.kafka.stream.topology.Processor; import org.apache.kafka.stream.KStreamContext; +<<<<<<< HEAD import org.apache.kafka.stream.RecordCollector; import org.apache.kafka.stream.RestoreFunc; import org.apache.kafka.stream.StateStore; >>>>>>> removing io.confluent imports: wip +======= +import org.apache.kafka.clients.processor.RecordCollector; +import org.apache.kafka.clients.processor.RestoreFunc; +import org.apache.kafka.clients.processor.StateStore; +>>>>>>> wip 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.stream.internals.PunctuationQueue; +import org.apache.kafka.clients.processor.internals.PunctuationQueue; import java.io.File; import java.util.Map; diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java index f11b9df73faab..cf3d66f7266fa 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java @@ -21,7 +21,7 @@ import org.apache.kafka.stream.topology.KeyValue; import org.apache.kafka.stream.topology.Window; import org.apache.kafka.stream.util.FilteredIterator; -import org.apache.kafka.stream.util.Stamped; +import org.apache.kafka.clients.processor.internals.Stamped; import java.util.Iterator; import java.util.LinkedList; diff --git a/temp b/temp index 175295436b35b..efba3bf7f9759 100644 --- a/temp +++ b/temp @@ -1,30 +1,4 @@ -build.gradle: needs merge -stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java: needs merge -stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java: needs merge -stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java: needs merge -stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java: needs merge -stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java: needs merge -stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java: needs merge -stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java: needs merge -stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java: needs merge +stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java: needs merge stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java: needs merge stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java: needs merge diff --git a/temp2 b/temp2 index 1b9255223536b..7eb747f767fc6 100644 --- a/temp2 +++ b/temp2 @@ -1,30 +1,4 @@ -build.gradle -stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java -stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java -stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java -stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java -stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java -stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java -stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java -stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java -stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java -stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java +stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java +stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java -stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java From 58379f39e4e239284f473634c8e4818d703e1640 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 7 Aug 2015 11:17:29 -0700 Subject: [PATCH 144/275] wip --- .../kafka/clients/processor/PTopology.java | 27 +++++---- .../java/org/apache/kafka/stream/Chooser.java | 30 ---------- .../stream/util/MinTimestampTracker.java | 60 ------------------- .../kafka/stream/util/TimestampTracker.java | 58 ------------------ .../kafka/stream/MinTimestampTrackerTest.java | 2 +- 5 files changed, 18 insertions(+), 159 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/Chooser.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java index 7e9fc8b25ecca..410112fb1b312 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -21,32 +21,41 @@ import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; public class PTopology { - private class KeyValueDeserializers { + private class TopicDeserializers { + public KafkaProcessor processor; public Deserializer keyDeserializer; public Deserializer valDeserializer; - public KeyValueDeserializers(Deserializer keyDeserializer, Deserializer valDeserializer) { + public TopicDeserializers(KafkaProcessor processor, + Deserializer keyDeserializer, + Deserializer valDeserializer) { + this.processor = processor; this.keyDeserializer = keyDeserializer; this.valDeserializer = valDeserializer; } } - List sources = new ArrayList<>(); List processors = new ArrayList<>(); - Map topicDesers = new HashMap<>(); + Map topicDesers = new HashMap<>(); boolean built = false; - public List sources() { + public Set sources() { if (!built) throw new IllegalStateException("Topology has not been built."); + Set sources = new HashSet<>(); + for (TopicDeserializers topicDeserializers : topicDesers.values()) { + sources.add(topicDeserializers.processor); + } + return sources; } @@ -61,7 +70,7 @@ public Deserializer keyDeser(String topic) { if (!built) throw new IllegalStateException("Topology has not been built."); - KeyValueDeserializers desers = topicDesers.get(topic); + TopicDeserializers desers = topicDesers.get(topic); if (desers == null) throw new IllegalStateException("The topic " + topic + " is unknown."); @@ -73,7 +82,7 @@ public Deserializer valueDeser(String topic) { if (!built) throw new IllegalStateException("Topology has not been built."); - KeyValueDeserializers desers = topicDesers.get(topic); + TopicDeserializers desers = topicDesers.get(topic); if (desers == null) throw new IllegalStateException("The topic " + topic + " is unknown."); @@ -91,10 +100,8 @@ public final void addProcessor(KafkaProcessor processor, Dese if (topicDesers.containsKey(topic)) throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); - topicDesers.put(topic, new KeyValueDeserializers(keyDeserializer, valDeserializer)); + topicDesers.put(topic, new TopicDeserializers<>(processor, keyDeserializer, valDeserializer)); } - - sources.add(processor); } public final void addProcessor(KafkaProcessor processor, KafkaProcessor... parents) { diff --git a/stream/src/main/java/org/apache/kafka/stream/Chooser.java b/stream/src/main/java/org/apache/kafka/stream/Chooser.java deleted file mode 100644 index 9668027fcf632..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/Chooser.java +++ /dev/null @@ -1,30 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -import org.apache.kafka.clients.processor.internals.RecordQueue; - -public interface Chooser { - - void add(RecordQueue queue); - - RecordQueue next(); - - void close(); - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java deleted file mode 100644 index fb7fa8c0f5041..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/util/MinTimestampTracker.java +++ /dev/null @@ -1,60 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.util; - -import org.apache.kafka.clients.processor.internals.Stamped; - -import java.util.LinkedList; - -/** - * MinTimestampTracker is a helper class for a sliding window implementation. - * It is assumed that elements are added or removed in a FIFO manner. - * It maintains the minimum timestamp of stamped elements that were added but not yet removed. - */ -public class MinTimestampTracker implements TimestampTracker { - - private final LinkedList> descendingSubsequence = new LinkedList>(); - - public void addStampedElement(Stamped elem) { - if (elem == null) throw new NullPointerException(); - - Stamped minElem = descendingSubsequence.peekLast(); - while (minElem != null && minElem.timestamp >= elem.timestamp) { - descendingSubsequence.removeLast(); - minElem = descendingSubsequence.peekLast(); - } - descendingSubsequence.offerLast(elem); - } - - public void removeStampedElement(Stamped elem) { - if (elem != null && descendingSubsequence.peekFirst() == elem) - descendingSubsequence.removeFirst(); - } - - public int size() { - return descendingSubsequence.size(); - } - - public long get() { - Stamped stamped = descendingSubsequence.peekFirst(); - if (stamped == null) return -1L; - - return stamped.timestamp; - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java deleted file mode 100644 index b33437c0b6615..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/util/TimestampTracker.java +++ /dev/null @@ -1,58 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.util; - -import org.apache.kafka.clients.processor.internals.Stamped; - -/** - * TimestampTracker is a helper class for a sliding window implementation. - * It is assumed that elements are added or removed in a FIFO manner. - * It maintains the timestamp, like the min timestamp, the max timestamp, etc. of stamped elements - * that were added but not yet removed. - */ -public interface TimestampTracker { - - /** - * Adds a stamped elements to this tracker. - * - * @param elem the added element - */ - void addStampedElement(Stamped elem); - - /** - * Removed a stamped elements to this tracker. - * - * @param elem the removed element - */ - void removeStampedElement(Stamped elem); - - /** - * Returns the timestamp - * - * @return timestamp, or -1L when empty - */ - long get(); - - /** - * Returns the size of internal structure. The meaning of "size" depends on the implementation. - * - * @return size - */ - int size(); - -} diff --git a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java index 6336ad076d450..88d14ccec4104 100644 --- a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertEquals; -import org.apache.kafka.stream.util.MinTimestampTracker; +import org.apache.kafka.clients.processor.internals.MinTimestampTracker; import org.apache.kafka.clients.processor.internals.Stamped; import org.junit.Test; From 378b88f0992d52c941467f91d2fe63306ce55efd Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 7 Aug 2015 11:45:00 -0700 Subject: [PATCH 145/275] wip --- .../clients/processor/KafkaProcessor.java | 7 ++- .../kafka/clients/processor/PTopology.java | 52 +++++++------------ .../apache/kafka/stream/KStreamException.java | 37 ------------- .../stream/topology/KStreamTopology.java | 9 ++-- .../topology/NotCopartitionedException.java | 2 +- .../topology/internals/KStreamBranch.java | 11 ++-- .../topology/internals/KStreamImpl.java | 3 +- .../topology/internals/KStreamSource.java | 6 +-- .../topology/internals/SourceProcessor.java | 40 -------------- 9 files changed, 38 insertions(+), 129 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/KStreamException.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/SourceProcessor.java diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java b/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java index 332730055fd01..f219420aabbcf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java @@ -26,6 +26,8 @@ public abstract class KafkaProcessor implements Processor(); @@ -34,7 +36,8 @@ public KafkaProcessor(String name) { public String name() { return name; } @Override - public final void receive(K1 key, V1 value) { + public final void receive(K1 key, V1 value, long timestamp) { + this.now = timestamp; this.process(key, value); } @@ -44,7 +47,7 @@ public final void chain(KafkaProcessor child) { public final void forward(K2 key, V2 value) { for (KafkaProcessor child : children) { - child.receive(key, value); + child.receive(key, value, now); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java index 410112fb1b312..7bc805f4dd4b1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.processor; +import org.apache.kafka.clients.processor.internals.KafkaSource; import org.apache.kafka.common.serialization.Deserializer; import java.util.ArrayList; @@ -28,32 +29,18 @@ public class PTopology { - private class TopicDeserializers { - public KafkaProcessor processor; - public Deserializer keyDeserializer; - public Deserializer valDeserializer; - - public TopicDeserializers(KafkaProcessor processor, - Deserializer keyDeserializer, - Deserializer valDeserializer) { - this.processor = processor; - this.keyDeserializer = keyDeserializer; - this.valDeserializer = valDeserializer; - } - } - List processors = new ArrayList<>(); - Map topicDesers = new HashMap<>(); + Map sources = new HashMap<>(); boolean built = false; - public Set sources() { + public Set sources() { if (!built) throw new IllegalStateException("Topology has not been built."); - Set sources = new HashSet<>(); - for (TopicDeserializers topicDeserializers : topicDesers.values()) { - sources.add(topicDeserializers.processor); + Set sources = new HashSet<>(); + for (KafkaSource source : this.sources.values()) { + sources.add(source); } return sources; @@ -63,45 +50,46 @@ public Set topics() { if (!built) throw new IllegalStateException("Topology has not been built."); - return topicDesers.keySet(); + return sources.keySet(); } public Deserializer keyDeser(String topic) { if (!built) throw new IllegalStateException("Topology has not been built."); - TopicDeserializers desers = topicDesers.get(topic); + KafkaSource source = sources.get(topic); - if (desers == null) + if (source == null) throw new IllegalStateException("The topic " + topic + " is unknown."); - return desers.keyDeserializer; + return source.keyDeserializer; } public Deserializer valueDeser(String topic) { if (!built) throw new IllegalStateException("Topology has not been built."); - TopicDeserializers desers = topicDesers.get(topic); + KafkaSource source = sources.get(topic); - if (desers == null) + if (source == null) throw new IllegalStateException("The topic " + topic + " is unknown."); - return desers.valDeserializer; + return source.valDeserializer; } - public final void addProcessor(KafkaProcessor processor, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - if (processors.contains(processor)) - throw new IllegalArgumentException("Processor " + processor.name() + " is already added."); + public final KafkaProcessor addProcessor(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + KafkaSource source = new KafkaSource<>(keyDeserializer, valDeserializer); - processors.add(processor); + processors.add(source); for (String topic : topics) { - if (topicDesers.containsKey(topic)) + if (sources.containsKey(topic)) throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); - topicDesers.put(topic, new TopicDeserializers<>(processor, keyDeserializer, valDeserializer)); + sources.put(topic, source); } + + return source; } public final void addProcessor(KafkaProcessor processor, KafkaProcessor... parents) { diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamException.java b/stream/src/main/java/org/apache/kafka/stream/KStreamException.java deleted file mode 100644 index 66b5930795d3c..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamException.java +++ /dev/null @@ -1,37 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -public class KStreamException extends RuntimeException { - - public KStreamException() { - super(); - } - - public KStreamException(String msg) { - super(msg); - } - - public KStreamException(Throwable t) { - super(t); - } - - public KStreamException(String msg, Throwable t) { - super(msg, t); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java index a9066d8965cf0..5037e49990a85 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java @@ -17,11 +17,12 @@ package org.apache.kafka.stream.topology; +import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.topology.internals.KStreamSource; -import org.apache.kafka.stream.topology.internals.SourceProcessor; +import org.apache.kafka.clients.processor.internals.KafkaSource; /** * KStreamTopology is the class that allows an implementation of {@link KStreamTopology#build()} to create KStream instances. @@ -71,11 +72,7 @@ public abstract class KStreamTopology { * @return KStream */ public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - - SourceProcessor source = new SourceProcessor<>("KAFKA-SOURCE"); - - topology.addProcessor(source, keyDeserializer, valDeserializer, topics); - + KafkaProcessor source = topology.addProcessor(keyDeserializer, valDeserializer, topics); return new KStreamSource<>(topology, source); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java b/stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java index 3b5e253a36426..1045995408d70 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream.topology; -import org.apache.kafka.stream.KStreamException; +import org.apache.kafka.clients.processor.KStreamException; public class NotCopartitionedException extends KStreamException { diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java index 35aa33e01e069..8ba079367c3b7 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java @@ -19,11 +19,8 @@ import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.internals.KafkaSource; import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.internals.Receiver; -import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.Predicate; import java.lang.reflect.Array; @@ -37,7 +34,7 @@ class KStreamBranch extends KafkaProcessor { private final PTopology topology; private final Predicate[] predicates; - private final SourceProcessor[] branches; + private final KafkaSource[] branches; private final KafkaProcessor parent; @SuppressWarnings("unchecked") @@ -47,9 +44,9 @@ public KStreamBranch(Predicate[] predicates, PTopology topology, KafkaProc this.parent = parent; this.topology = topology; this.predicates = Arrays.copyOf(predicates, predicates.length); - this.branches = (SourceProcessor[]) Array.newInstance(SourceProcessor.class, predicates.length); + this.branches = (KafkaSource[]) Array.newInstance(KafkaSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { - branches[i] = new SourceProcessor<>(BRANCH_NAME + BRANCH_INDEX.getAndIncrement()); + branches[i] = new KafkaSource<>(BRANCH_NAME + BRANCH_INDEX.getAndIncrement()); topology.addProcessor(branches[i], parent); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java index 2287231ef78a3..3bb8e11288e15 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.processor.PTopology; import org.apache.kafka.clients.processor.Processor; import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.internals.KafkaSource; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.KStream; import org.apache.kafka.common.serialization.Deserializer; @@ -131,7 +132,7 @@ public KStream through(String topic, Deserializer valDeserializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); - SourceProcessor source = new SourceProcessor<>("KAFKA-SOURCE"); + KafkaSource source = new KafkaSource<>("KAFKA-SOURCE"); topology.addProcessor(source, keyDeserializer, valDeserializer, topic); diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java index 7f6a7ea47481a..48f25c19e8afa 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java @@ -17,13 +17,13 @@ package org.apache.kafka.stream.topology.internals; +import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.clients.processor.internals.KafkaSource; public class KStreamSource extends KStreamImpl { - public KStreamSource(PTopology topology, SourceProcessor source) { + public KStreamSource(PTopology topology, KafkaProcessor source) { super(topology, source); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/SourceProcessor.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/SourceProcessor.java deleted file mode 100644 index ed6713c9bc01a..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/SourceProcessor.java +++ /dev/null @@ -1,40 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.ProcessorContext; - -public class SourceProcessor extends KafkaProcessor { - public SourceProcessor(String name) { - super(name); - } - - @Override - public void init(ProcessorContext context) { - // do nothing - } - - @Override - public void process(K key, V value) { forward(key, value); } - - @Override - public void close() { - // do nothing - } -} From 80ac445a8fee1ebd61c55194b9d6652a19539baa Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 7 Aug 2015 12:38:47 -0700 Subject: [PATCH 146/275] wip --- .../clients/processor/KafkaProcessor.java | 7 ++--- .../kafka/clients/processor/PTopology.java | 30 +++++++------------ .../stream/topology/KStreamTopology.java | 3 +- .../topology/internals/KStreamBranch.java | 2 +- .../topology/internals/KStreamImpl.java | 16 +++++----- 5 files changed, 22 insertions(+), 36 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java b/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java index f219420aabbcf..332730055fd01 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java @@ -26,8 +26,6 @@ public abstract class KafkaProcessor implements Processor(); @@ -36,8 +34,7 @@ public KafkaProcessor(String name) { public String name() { return name; } @Override - public final void receive(K1 key, V1 value, long timestamp) { - this.now = timestamp; + public final void receive(K1 key, V1 value) { this.process(key, value); } @@ -47,7 +44,7 @@ public final void chain(KafkaProcessor child) { public final void forward(K2 key, V2 value) { for (KafkaProcessor child : children) { - child.receive(key, value, now); + child.receive(key, value); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java index 7bc805f4dd4b1..e7386fb3c9623 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -27,17 +27,12 @@ import java.util.Map; import java.util.Set; -public class PTopology { +abstract public class PTopology { List processors = new ArrayList<>(); Map sources = new HashMap<>(); - boolean built = false; - public Set sources() { - if (!built) - throw new IllegalStateException("Topology has not been built."); - Set sources = new HashSet<>(); for (KafkaSource source : this.sources.values()) { sources.add(source); @@ -47,16 +42,16 @@ public Set sources() { } public Set topics() { - if (!built) - throw new IllegalStateException("Topology has not been built."); - return sources.keySet(); } - public Deserializer keyDeser(String topic) { - if (!built) - throw new IllegalStateException("Topology has not been built."); + public KafkaSource source(String topic) { + KafkaSource source = sources.get(topic); + return source; + } + + public Deserializer keyDeser(String topic) { KafkaSource source = sources.get(topic); if (source == null) @@ -66,9 +61,6 @@ public Deserializer keyDeser(String topic) { } public Deserializer valueDeser(String topic) { - if (!built) - throw new IllegalStateException("Topology has not been built."); - KafkaSource source = sources.get(topic); if (source == null) @@ -77,7 +69,7 @@ public Deserializer valueDeser(String topic) { return source.valDeserializer; } - public final KafkaProcessor addProcessor(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + public final KafkaProcessor addSource(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { KafkaSource source = new KafkaSource<>(keyDeserializer, valDeserializer); processors.add(source); @@ -92,7 +84,7 @@ public final KafkaProcessor addProcessor(Deserializer keyD return source; } - public final void addProcessor(KafkaProcessor processor, KafkaProcessor... parents) { + public final void addSource(KafkaProcessor processor, KafkaProcessor... parents) { if (processors.contains(processor)) throw new IllegalArgumentException("Processor " + processor.name() + " is already added."); @@ -108,7 +100,5 @@ public final void addProcessor(KafkaProcessor processor, Kafk } } - public void build() { - built = true; - } + abstract public void build(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java index 5037e49990a85..dc9a607e40ebf 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.topology.internals.KStreamSource; -import org.apache.kafka.clients.processor.internals.KafkaSource; /** * KStreamTopology is the class that allows an implementation of {@link KStreamTopology#build()} to create KStream instances. @@ -72,7 +71,7 @@ public abstract class KStreamTopology { * @return KStream */ public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KafkaProcessor source = topology.addProcessor(keyDeserializer, valDeserializer, topics); + KafkaProcessor source = topology.addSource(keyDeserializer, valDeserializer, topics); return new KStreamSource<>(topology, source); } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java index 8ba079367c3b7..dfa54b77a7c83 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java @@ -47,7 +47,7 @@ public KStreamBranch(Predicate[] predicates, PTopology topology, KafkaProc this.branches = (KafkaSource[]) Array.newInstance(KafkaSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { branches[i] = new KafkaSource<>(BRANCH_NAME + BRANCH_INDEX.getAndIncrement()); - topology.addProcessor(branches[i], parent); + topology.addSource(branches[i], parent); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java index 3bb8e11288e15..75646a559e973 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java @@ -51,7 +51,7 @@ protected KStreamImpl(PTopology topology, KafkaProcessor processor) public KStream filter(Predicate predicate) { KStreamFilter filter = new KStreamFilter<>(predicate); - topology.addProcessor(filter, processor); + topology.addSource(filter, processor); return new KStreamImpl<>(topology, filter); } @@ -61,7 +61,7 @@ public KStream filter(Predicate predicate) { public KStream filterOut(final Predicate predicate) { KStreamFilter filter = new KStreamFilter<>(predicate, true); - topology.addProcessor(filter, processor); + topology.addSource(filter, processor); return new KStreamImpl<>(topology, filter); } @@ -71,7 +71,7 @@ public KStream filterOut(final Predicate predicate) { public KStream map(KeyValueMapper mapper) { KStreamMap map = new KStreamMap<>(mapper); - topology.addProcessor(map, processor); + topology.addSource(map, processor); return new KStreamImpl<>(topology, map); } @@ -81,7 +81,7 @@ public KStream map(KeyValueMapper mapper) { public KStream mapValues(ValueMapper mapper) { KStreamMapValues map = new KStreamMapValues<>(mapper); - topology.addProcessor(map, processor); + topology.addSource(map, processor); return new KStreamImpl<>(topology, map); } @@ -91,7 +91,7 @@ public KStream mapValues(ValueMapper mapper) { public KStream flatMap(KeyValueMapper> mapper) { KStreamFlatMap map = new KStreamFlatMap<>(mapper); - topology.addProcessor(map, processor); + topology.addSource(map, processor); return new KStreamImpl<>(topology, map); } @@ -101,7 +101,7 @@ public KStream flatMap(KeyValueMapper KStream flatMapValues(ValueMapper> mapper) { KStreamFlatMapValues map = new KStreamFlatMapValues<>(mapper); - topology.addProcessor(map, processor); + topology.addSource(map, processor); return new KStreamImpl<>(topology, map); } @@ -173,7 +173,7 @@ public void close() { @SuppressWarnings("unchecked") @Override public KStream process(final KafkaProcessor current) { - topology.addProcessor(current, processor); + topology.addSource(current, processor); return new KStreamImpl(topology, current); } @@ -188,6 +188,6 @@ public void process(K key, V value) { } }; - topology.addProcessor(wrapper, processor); + topology.addSource(wrapper, processor); } } From 9b31888094f2734202a59c59d01b541f50f59662 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 7 Aug 2015 14:35:52 -0700 Subject: [PATCH 147/275] wip --- .../clients/processor/KafkaProcessor.java | 19 +- .../kafka/clients/processor/PTopology.java | 14 +- .../clients/processor/ProcessorContext.java | 2 +- .../stream/state/InMemoryKeyValueStore.java | 10 +- .../stream/state/RocksDBKeyValueStore.java | 9 +- .../state/internals/MeteredKeyValueStore.java | 5 +- .../stream/topology/KStreamTopology.java | 44 +---- .../topology/internals/KStreamBranch.java | 23 +-- .../topology/internals/KStreamImpl.java | 22 ++- .../kafka/stream/util/OffsetCheckpoint.java | 164 ------------------ temp | 7 +- temp2 | 7 +- 12 files changed, 64 insertions(+), 262 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java b/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java index 332730055fd01..afd2581f5592f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java @@ -20,7 +20,7 @@ import java.util.ArrayList; import java.util.List; -public abstract class KafkaProcessor implements Processor, Receiver { +public abstract class KafkaProcessor implements Processor, Receiver, Punctuator { protected final List> children; @@ -33,11 +33,6 @@ public KafkaProcessor(String name) { public String name() { return name; } - @Override - public final void receive(K1 key, V1 value) { - this.process(key, value); - } - public final void chain(KafkaProcessor child) { children.add(child); } @@ -48,6 +43,18 @@ public final void forward(K2 key, V2 value) { } } + @Override + public final void receive(K1 key, V1 value) { + this.process(key, value); + } + + /* Following functions can be overridden by users */ + + @Override + public void punctuate(long streamTime) { + // do nothing + } + @Override public void init(ProcessorContext context) { // do nothing diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java index e7386fb3c9623..f85ad1a3e06dd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -29,8 +29,8 @@ abstract public class PTopology { - List processors = new ArrayList<>(); - Map sources = new HashMap<>(); + private List processors = new ArrayList<>(); + private Map sources = new HashMap<>(); public Set sources() { Set sources = new HashSet<>(); @@ -69,7 +69,13 @@ public Deserializer valueDeser(String topic) { return source.valDeserializer; } - public final KafkaProcessor addSource(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + public final void init(ProcessorContext context) { + for (KafkaProcessor processor : processors) { + processor.init(context); + } + } + + public final KafkaSource addSource(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { KafkaSource source = new KafkaSource<>(keyDeserializer, valDeserializer); processors.add(source); @@ -84,7 +90,7 @@ public final KafkaProcessor addSource(Deserializer keyDese return source; } - public final void addSource(KafkaProcessor processor, KafkaProcessor... parents) { + public final void addProcessor(KafkaProcessor processor, KafkaProcessor... parents) { if (processors.contains(processor)) throw new IllegalArgumentException("Processor " + processor.name() + " is already added."); diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java b/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java index caf6417678550..f2c42ec2ac18e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java +++ b/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java @@ -98,7 +98,7 @@ public interface ProcessorContext { void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); - void schedule(Processor processor, long interval); + void schedule(KafkaProcessor processor, long interval); void commit(); diff --git a/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java index 62323462dadf0..59293d46fb9fa 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream.state; -import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.stream.state.internals.MeteredKeyValueStore; @@ -36,11 +36,11 @@ */ public class InMemoryKeyValueStore extends MeteredKeyValueStore { - public InMemoryKeyValueStore(String name, KStreamContext context) { + public InMemoryKeyValueStore(String name, ProcessorContext context) { this(name, context, new SystemTime()); } - public InMemoryKeyValueStore(String name, KStreamContext context, Time time) { + public InMemoryKeyValueStore(String name, ProcessorContext context, Time time) { super(name, new MemoryStore(name, context), context, "kafka-streams", time); } @@ -48,10 +48,10 @@ private static class MemoryStore implements KeyValueStore { private final String name; private final NavigableMap map; - private final KStreamContext context; + private final ProcessorContext context; @SuppressWarnings("unchecked") - public MemoryStore(String name, KStreamContext context) { + public MemoryStore(String name, ProcessorContext context) { super(); this.name = name; this.map = new TreeMap<>(); diff --git a/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java index 155c39d084d4e..4a8eaca3d9299 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java @@ -17,6 +17,7 @@ package org.apache.kafka.stream.state; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.SystemTime; @@ -40,11 +41,11 @@ public class RocksDBKeyValueStore extends MeteredKeyValueStore { - public RocksDBKeyValueStore(String name, KStreamContext context) { + public RocksDBKeyValueStore(String name, ProcessorContext context) { this(name, context, new SystemTime()); } - public RocksDBKeyValueStore(String name, KStreamContext context, Time time) { + public RocksDBKeyValueStore(String name, ProcessorContext context, Time time) { super(name, new RocksDBStore(name, context), context, "kafka-streams", time); } @@ -64,7 +65,7 @@ private static class RocksDBStore implements KeyValueStore { private final String topic; private final int partition; - private final KStreamContext context; + private final ProcessorContext context; private final Options options; private final WriteOptions wOptions; @@ -76,7 +77,7 @@ private static class RocksDBStore implements KeyValueStore { private RocksDB db; @SuppressWarnings("unchecked") - public RocksDBStore(String name, KStreamContext context) { + public RocksDBStore(String name, ProcessorContext context) { this.topic = name; this.partition = context.id(); this.context = context; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java index 74647d4881825..37f3e4b23174d 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java @@ -17,6 +17,7 @@ package org.apache.kafka.stream.state.internals; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.clients.processor.RecordCollector; import org.apache.kafka.clients.processor.RestoreFunc; @@ -60,10 +61,10 @@ public class MeteredKeyValueStore implements KeyValueStore { private final int partition; private final Set dirty; private final int maxDirty; - private final KStreamContext context; + private final ProcessorContext context; // always wrap the logged store with the metered store - public MeteredKeyValueStore(final String name, final KeyValueStore inner, KStreamContext context, String group, Time time) { + public MeteredKeyValueStore(final String name, final KeyValueStore inner, ProcessorContext context, String group, Time time) { this.inner = inner; this.time = time; diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java index dc9a607e40ebf..004d08f8e6872 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java @@ -26,39 +26,7 @@ /** * KStreamTopology is the class that allows an implementation of {@link KStreamTopology#build()} to create KStream instances. */ -public abstract class KStreamTopology { - - private PTopology topology = new PTopology(); - - /** - * Initializes a stream processing topology. This method may be called multiple times. - * An application constructs a processing logic using KStream API. - *

- * For example, - *

- *
-     *   KStreamTopology topology = new KStreamTopology() {
-     *     public void build() {
-     *       KStream<Integer, PageView> pageViewStream = from("pageView").mapValues(...);
-     *       KStream<Integer, AdClick> adClickStream = from("adClick").join(pageViewStream, ...).process(...);
-     *     }
-     *   }
-     *
-     *   KafkaStreaming streaming = new KafkaStreaming(build, streamingConfig)
-     *   streaming.run();
-     * 
- */ - public abstract void build(); - - /** - * Creates a KStream instance for the specified topics. The stream is added to the default synchronization group. - * - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream - */ - public KStream from(String... topics) { - return from(null, null, topics); - } +public abstract class KStreamTopology extends PTopology { /** * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. @@ -71,13 +39,7 @@ public abstract class KStreamTopology { * @return KStream */ public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KafkaProcessor source = topology.addSource(keyDeserializer, valDeserializer, topics); - return new KStreamSource<>(topology, source); - } - - public PTopology get() { - this.topology.build(); - - return this.topology; + KafkaProcessor source = addSource(keyDeserializer, valDeserializer, topics); + return new KStreamSource<>(this, source); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java index dfa54b77a7c83..fdc1281f3389f 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java @@ -30,43 +30,36 @@ class KStreamBranch extends KafkaProcessor { private static final String BRANCH_NAME = "KAFKA-BRANCH"; - private static final AtomicInteger BRANCH_INDEX = new AtomicInteger(1); private final PTopology topology; private final Predicate[] predicates; - private final KafkaSource[] branches; - private final KafkaProcessor parent; + private final KStreamFilter[] branches; @SuppressWarnings("unchecked") public KStreamBranch(Predicate[] predicates, PTopology topology, KafkaProcessor parent) { super(BRANCH_NAME); - this.parent = parent; this.topology = topology; this.predicates = Arrays.copyOf(predicates, predicates.length); - this.branches = (KafkaSource[]) Array.newInstance(KafkaSource.class, predicates.length); + this.branches = (KStreamFilter[]) Array.newInstance(KStreamFilter.class, predicates.length); + + // NOTE that branches here is just a list of predicates, hence not necessarily mutual exclusive for (int i = 0; i < branches.length; i++) { - branches[i] = new KafkaSource<>(BRANCH_NAME + BRANCH_INDEX.getAndIncrement()); - topology.addSource(branches[i], parent); + branches[i] = new KStreamFilter<>(predicates[i], false); + topology.addProcessor(branches[i], parent); } } @Override public void process(K key, V value) { - for (int i = 0; i < predicates.length; i++) { - Predicate predicate = predicates[i]; - if (predicate.apply( key, value)) { - branches[i].receive(key, value); - return; - } - } + forward(key, value); } @SuppressWarnings("unchecked") public KStream[] branches() { KStream[] streams = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); for (int i = 0; i < branches.length; i++) { - streams[i] = new KStreamSource<>(topology, branches[i]); + streams[i] = new KStreamImpl<>(topology, branches[i]); } return streams; } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java index 75646a559e973..ebc31efe491ab 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java @@ -41,7 +41,7 @@ public class KStreamImpl implements KStream { protected KStreamContext context; protected KStreamMetadata metadata; - protected KStreamImpl(PTopology topology, KafkaProcessor processor) { + public KStreamImpl(PTopology topology, KafkaProcessor processor) { this.topology = topology; this.processor = processor; } @@ -51,7 +51,7 @@ protected KStreamImpl(PTopology topology, KafkaProcessor processor) public KStream filter(Predicate predicate) { KStreamFilter filter = new KStreamFilter<>(predicate); - topology.addSource(filter, processor); + topology.addProcessor(filter, processor); return new KStreamImpl<>(topology, filter); } @@ -61,7 +61,7 @@ public KStream filter(Predicate predicate) { public KStream filterOut(final Predicate predicate) { KStreamFilter filter = new KStreamFilter<>(predicate, true); - topology.addSource(filter, processor); + topology.addProcessor(filter, processor); return new KStreamImpl<>(topology, filter); } @@ -71,7 +71,7 @@ public KStream filterOut(final Predicate predicate) { public KStream map(KeyValueMapper mapper) { KStreamMap map = new KStreamMap<>(mapper); - topology.addSource(map, processor); + topology.addProcessor(map, processor); return new KStreamImpl<>(topology, map); } @@ -81,7 +81,7 @@ public KStream map(KeyValueMapper mapper) { public KStream mapValues(ValueMapper mapper) { KStreamMapValues map = new KStreamMapValues<>(mapper); - topology.addSource(map, processor); + topology.addProcessor(map, processor); return new KStreamImpl<>(topology, map); } @@ -91,7 +91,7 @@ public KStream mapValues(ValueMapper mapper) { public KStream flatMap(KeyValueMapper> mapper) { KStreamFlatMap map = new KStreamFlatMap<>(mapper); - topology.addSource(map, processor); + topology.addProcessor(map, processor); return new KStreamImpl<>(topology, map); } @@ -101,7 +101,7 @@ public KStream flatMap(KeyValueMapper KStream flatMapValues(ValueMapper> mapper) { KStreamFlatMapValues map = new KStreamFlatMapValues<>(mapper); - topology.addSource(map, processor); + topology.addProcessor(map, processor); return new KStreamImpl<>(topology, map); } @@ -132,9 +132,7 @@ public KStream through(String topic, Deserializer valDeserializer) { process(this.getSendProcessor(topic, keySerializer, valSerializer)); - KafkaSource source = new KafkaSource<>("KAFKA-SOURCE"); - - topology.addProcessor(source, keyDeserializer, valDeserializer, topic); + KafkaSource source = topology.addSource(keyDeserializer, valDeserializer, topic); return new KStreamSource<>(topology, source); } @@ -173,7 +171,7 @@ public void close() { @SuppressWarnings("unchecked") @Override public KStream process(final KafkaProcessor current) { - topology.addSource(current, processor); + topology.addProcessor(current, processor); return new KStreamImpl(topology, current); } @@ -188,6 +186,6 @@ public void process(K key, V value) { } }; - topology.addSource(wrapper, processor); + topology.addProcessor(wrapper, processor); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java b/stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java deleted file mode 100644 index 4a68a1868e500..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/util/OffsetCheckpoint.java +++ /dev/null @@ -1,164 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.util; - -import org.apache.kafka.common.TopicPartition; - -import java.io.*; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -/** - * This class saves out a map of topic/partition=>offsets to a file. The format of the file is UTF-8 text containing the following: - *

- *   <version>
- *   <n>
- *   <topic_name_1> <partition_1> <offset_1>
- *   .
- *   .
- *   .
- *   <topic_name_n> <partition_n> <offset_n>
- * 
- * The first line contains a number designating the format version (currently 0), the next line contains - * a number giving the total number of offsets. Each successive line gives a topic/partition/offset triple - * separated by spaces. - */ -public class OffsetCheckpoint { - - private static final int VERSION = 0; - - private final File file; - private final Object lock; - - public OffsetCheckpoint(File file) throws IOException { - new File(file + ".tmp").delete(); // try to delete any existing temp files for cleanliness - this.file = file; - this.lock = new Object(); - } - - public void write(Map offsets) throws IOException { - synchronized (lock) { - // write to temp file and then swap with the existing file - File temp = new File(file.getAbsolutePath() + ".tmp"); - - FileOutputStream fileOutputStream = new FileOutputStream(temp); - BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream)); - try { - writeIntLine(writer, VERSION); - writeIntLine(writer, offsets.size()); - - // write the entries - for (Map.Entry entry : offsets.entrySet()) - writeEntry(writer, entry.getKey(), entry.getValue()); - - // flush the buffer and then fsync the underlying file - writer.flush(); - fileOutputStream.getFD().sync(); - } finally { - writer.close(); - } - - // swap new offset checkpoint file with previous one - if (!temp.renameTo(file)) { - // renameTo() fails on Windows if the destination file exists. - file.delete(); - if (!temp.renameTo(file)) - throw new IOException(String.format("File rename from %s to %s failed.", - temp.getAbsolutePath(), - file.getAbsolutePath())); - } - } - } - - private void writeIntLine(BufferedWriter writer, int number) throws IOException { - writer.write(Integer.toString(number)); - writer.newLine(); - } - - private void writeEntry(BufferedWriter writer, TopicPartition part, long offset) throws IOException { - writer.write(part.topic()); - writer.write(' '); - writer.write(Integer.toString(part.partition())); - writer.write(' '); - writer.write(Long.toString(offset)); - writer.newLine(); - } - - public Map read() throws IOException { - synchronized (lock) { - BufferedReader reader = null; - try { - reader = new BufferedReader(new FileReader(file)); - } catch (FileNotFoundException e) { - return Collections.emptyMap(); - } - - try { - int version = readInt(reader); - switch (version) { - case 0: - int expectedSize = readInt(reader); - Map offsets = new HashMap(); - String line = reader.readLine(); - while (line != null) { - String[] pieces = line.split("\\s+"); - if (pieces.length != 3) - throw new IOException(String.format("Malformed line in offset checkpoint file: '%s'.", - line)); - - String topic = pieces[0]; - int partition = Integer.parseInt(pieces[1]); - long offset = Long.parseLong(pieces[2]); - offsets.put(new TopicPartition(topic, partition), offset); - line = reader.readLine(); - } - if (offsets.size() != expectedSize) - throw new IOException(String.format("Expected %d entries but found only %d", - expectedSize, - offsets.size())); - return offsets; - - default: - throw new IllegalArgumentException("Unknown offset checkpoint version: " + version); - } - } finally { - if (reader != null) - reader.close(); - } - } - } - - private int readInt(BufferedReader reader) throws IOException { - String line = reader.readLine(); - if (line == null) - throw new EOFException("File ended prematurely."); - int val = Integer.parseInt(line); - return val; - } - - public void delete() throws IOException { - file.delete(); - } - - @Override - public String toString() { - return this.file.getAbsolutePath(); - } - -} diff --git a/temp b/temp index efba3bf7f9759..b260c1577d816 100644 --- a/temp +++ b/temp @@ -1,4 +1,3 @@ -stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java: needs merge -stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java: needs merge +stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java: needs merge +stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java: needs merge +stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java: needs merge diff --git a/temp2 b/temp2 index 7eb747f767fc6..d5a8f9c139fbc 100644 --- a/temp2 +++ b/temp2 @@ -1,4 +1,3 @@ -stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java -stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java -stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java -stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java +stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java +stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java +stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java From 23ce16e3554e97de4915f63dfbd6f95a6757b65b Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 7 Aug 2015 16:38:45 -0700 Subject: [PATCH 148/275] wip --- .../kafka/clients/processor/Chooser.java | 30 ++ .../clients/processor/KStreamException.java | 37 +++ .../clients/processor/KafkaProcessor.java | 4 +- .../clients/processor/KafkaStreaming.java | 146 +++++++++ .../kafka/clients/processor/PTopology.java | 13 +- .../kafka/clients/processor/Processor.java | 0 .../clients/processor/ProcessorContext.java | 1 - .../kafka/clients/processor/Punctuator.java | 23 ++ .../kafka/clients/processor/Receiver.java | 23 ++ .../clients/processor/RecordCollector.java | 30 ++ .../kafka/clients/processor/RestoreFunc.java | 24 ++ .../kafka/clients/processor/StateStore.java | 52 +++ .../clients/processor/TimestampExtractor.java | 34 ++ .../clients/processor/TimestampTracker.java | 58 ++++ .../clients/processor/internals/Ingestor.java | 41 +++ .../processor/internals/IngestorImpl.java | 133 ++++++++ .../processor/internals/KStreamThread.java | 304 ++++++++++++++++++ .../processor/internals/KafkaSource.java | 50 +++ .../internals/MinTimestampTracker.java | 60 ++++ .../processor/internals/OffsetCheckpoint.java | 172 ++++++++++ .../processor/internals/ParallelExecutor.java | 152 +++++++++ .../processor/internals/PartitioningInfo.java | 27 ++ .../processor/internals/ProcessorConfig.java | 104 ++++++ .../internals/ProcessorContextImpl.java | 225 +++++++++++++ .../internals/ProcessorStateManager.java | 193 +++++++++++ .../processor/internals/PunctuationQueue.java | 52 +++ .../internals/PunctuationSchedule.java | 39 +++ .../internals/RecordCollectorImpl.java | 92 ++++++ .../processor/internals/RecordQueue.java | 122 +++++++ .../internals/RoundRobinChooser.java | 47 +++ .../clients/processor/internals/Stamped.java | 38 +++ .../processor/internals/StampedRecord.java | 48 +++ .../processor/internals/StreamGroup.java | 260 +++++++++++++++ .../processor/internals/StreamingConfig.java | 144 +++++++++ .../processor/internals/TimeBasedChooser.java | 62 ++++ .../kafka/stream/examples/KStreamJob.java | 77 +++++ .../stream/examples/SimpleProcessJob.java | 74 +++++ .../stream/examples/StatefulProcessJob.java | 95 ++++++ .../stream/state/RocksDBKeyValueStore.java | 1 - .../state/internals/MeteredKeyValueStore.java | 1 - .../kafka/stream/topology/ValueJoiner.java | 3 +- .../topology/internals/KStreamBranch.java | 2 - .../topology/internals/KStreamImpl.java | 2 +- .../topology/internals/KStreamJoin.java | 40 ++- .../topology/internals/KStreamSource.java | 1 - .../topology/internals/KStreamWindow.java | 89 +++++ .../internals/KStreamWindowedImpl.java | 78 ----- 47 files changed, 3195 insertions(+), 108 deletions(-) create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/KStreamException.java rename {clients => stream}/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java (95%) create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/KafkaStreaming.java rename {clients => stream}/src/main/java/org/apache/kafka/clients/processor/PTopology.java (94%) rename {clients => stream}/src/main/java/org/apache/kafka/clients/processor/Processor.java (100%) rename {clients => stream}/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java (99%) create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/Punctuator.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/Receiver.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/RecordCollector.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/RestoreFunc.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/StateStore.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/TimestampExtractor.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/Ingestor.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/IngestorImpl.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/OffsetCheckpoint.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/ParallelExecutor.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/PartitioningInfo.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorStateManager.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationQueue.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordCollectorImpl.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueue.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/Stamped.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/StampedRecord.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindow.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindowedImpl.java diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java b/stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java new file mode 100644 index 0000000000000..935576554ae08 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +import org.apache.kafka.clients.processor.internals.RecordQueue; + +public interface Chooser { + + void add(RecordQueue queue); + + RecordQueue next(); + + void close(); + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/KStreamException.java b/stream/src/main/java/org/apache/kafka/clients/processor/KStreamException.java new file mode 100644 index 0000000000000..5c3b58c9e5531 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/KStreamException.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +public class KStreamException extends RuntimeException { + + public KStreamException() { + super(); + } + + public KStreamException(String msg) { + super(msg); + } + + public KStreamException(Throwable t) { + super(t); + } + + public KStreamException(String msg, Throwable t) { + super(msg, t); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java similarity index 95% rename from clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java rename to stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java index afd2581f5592f..3c750f9a9433a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java @@ -62,8 +62,6 @@ public void init(ProcessorContext context) { @Override public void close() { - for (KafkaProcessor child : children) { - child.close(); - } + // do nothing } } diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaStreaming.java new file mode 100644 index 0000000000000..af3a60c5b4b95 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaStreaming.java @@ -0,0 +1,146 @@ +/** + * 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.clients.processor; + +import org.apache.kafka.clients.processor.internals.KStreamThread; +import org.apache.kafka.clients.processor.internals.ProcessorConfig; +import org.apache.kafka.clients.processor.internals.StreamingConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and + * sends output to zero or more output topics. + *

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

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

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

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

+ *    Properties props = new Properties();
+ *    props.put("bootstrap.servers", "localhost:4242");
+ *    StreamingConfig config = new StreamingConfig(props);
+ *    config.processor(ExampleStreamProcessor.class);
+ *    config.serialization(new StringSerializer(), new StringDeserializer());
+ *    KafkaStreaming container = new KafkaStreaming(new MyKStreamTopology(), config);
+ *    container.run();
+ * 
+ * + */ +public class KafkaStreaming implements Runnable { + + 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 STOPPING = 2; + private static final int STOPPED = 3; + private int state = CREATED; + + private final ProcessorConfig config; + private final Object lock = new Object(); + private final KStreamThread[] threads; + + + public KafkaStreaming(Class topologyClass, StreamingConfig streamingConfig) throws Exception { + if (streamingConfig.timestampExtractor() == null) + throw new NullPointerException("timestamp extractor is missing"); + + this.config = new ProcessorConfig(streamingConfig.config()); + + this.threads = new KStreamThread[this.config.numStreamThreads]; + for (int i = 0; i < this.threads.length; i++) { + this.threads[i] = new KStreamThread(topologyClass, streamingConfig); + } + } + + /** + * Execute the stream processors + */ + public void run() { + synchronized (lock) { + log.info("Starting container"); + if (state == CREATED) { + if (!config.stateDir.exists() && !config.stateDir.mkdirs()) + throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); + + for (KStreamThread thread : threads) thread.start(); + log.info("Start-up complete"); + } else { + throw new IllegalStateException("This container was already started"); + } + + state = RUNNING; + while (state == RUNNING) { + try { + lock.wait(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + + if (state == STOPPING) { + log.info("Shutting down the container"); + + for (KStreamThread thread : threads) + thread.close(); + + for (KStreamThread thread : threads) { + try { + thread.join(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + state = STOPPED; + lock.notifyAll(); + log.info("Shutdown complete"); + } + } + } + + /** + * Shutdown this streaming instance. + */ + public void close() { + synchronized (lock) { + if (state == CREATED || state == RUNNING) { + state = STOPPING; + lock.notifyAll(); + } + while (state == STOPPING) { + try { + lock.wait(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java similarity index 94% rename from clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java rename to stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java index f85ad1a3e06dd..02a9187f8c2c9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -46,9 +46,7 @@ public Set topics() { } public KafkaSource source(String topic) { - KafkaSource source = sources.get(topic); - - return source; + return sources.get(topic); } public Deserializer keyDeser(String topic) { @@ -106,5 +104,14 @@ public final void addProcessor(KafkaProcessor processor, Kafk } } + public final void close() { + for (KafkaProcessor processor : processors) { + processor.close(); + } + + processors.clear(); + sources.clear(); + } + abstract public void build(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/Processor.java b/stream/src/main/java/org/apache/kafka/clients/processor/Processor.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/clients/processor/Processor.java rename to stream/src/main/java/org/apache/kafka/clients/processor/Processor.java diff --git a/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java b/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java similarity index 99% rename from clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java rename to stream/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java index f2c42ec2ac18e..c94b037a1dbbb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.serialization.Serializer; import java.io.File; -import java.util.Map; public interface ProcessorContext { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/Punctuator.java b/stream/src/main/java/org/apache/kafka/clients/processor/Punctuator.java new file mode 100644 index 0000000000000..c52f207ff0f8b --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/Punctuator.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +public interface Punctuator { + + void punctuate(long streamTime); +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/Receiver.java b/stream/src/main/java/org/apache/kafka/clients/processor/Receiver.java new file mode 100644 index 0000000000000..5a949cf75a21f --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/Receiver.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +public interface Receiver { + + void receive(K key, V value); +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/RecordCollector.java b/stream/src/main/java/org/apache/kafka/clients/processor/RecordCollector.java new file mode 100644 index 0000000000000..14471b4c3683a --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/RecordCollector.java @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serializer; + +public interface RecordCollector { + + void send(ProducerRecord record); + + void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer); + + void flush(); +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/RestoreFunc.java b/stream/src/main/java/org/apache/kafka/clients/processor/RestoreFunc.java new file mode 100644 index 0000000000000..4b5a78e1d4523 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/RestoreFunc.java @@ -0,0 +1,24 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +// TODO: this should be removed once we move to Java 8 +public interface RestoreFunc { + + void apply(byte[] key, byte[] value); +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/StateStore.java b/stream/src/main/java/org/apache/kafka/clients/processor/StateStore.java new file mode 100644 index 0000000000000..348ab10425d08 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/StateStore.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +/** + * A storage engine for managing state maintained by a stream processor. + * + *

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

+ */ +public interface StateStore { + + /** + * The name of this store. + * @return the storage name + */ + String name(); + + /** + * Flush any cached data + */ + void flush(); + + /** + * Close the storage engine + */ + void close(); + + /** + * If the storage is persistent + */ + boolean persistent(); +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/TimestampExtractor.java b/stream/src/main/java/org/apache/kafka/clients/processor/TimestampExtractor.java new file mode 100644 index 0000000000000..4f4b6c2f488ce --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/TimestampExtractor.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +/** + * An interface that allows the KStream framework to extract a timestamp from a key-value pair + */ +public interface TimestampExtractor { + + /** + * Extracts a timestamp from a key-value pair from a topic + * + * @param topic the topic name + * @param key the key object + * @param value the value object + * @return timestamp + */ + long extract(String topic, Object key, Object value); +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java new file mode 100644 index 0000000000000..2272fef904c4c --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor; + +import org.apache.kafka.clients.processor.internals.Stamped; + +/** + * TimestampTracker is a helper class for a sliding window implementation. + * It is assumed that elements are added or removed in a FIFO manner. + * It maintains the timestamp, like the min timestamp, the max timestamp, etc. of stamped elements + * that were added but not yet removed. + */ +public interface TimestampTracker { + + /** + * Adds a stamped elements to this tracker. + * + * @param elem the added element + */ + void addStampedElement(Stamped elem); + + /** + * Removed a stamped elements to this tracker. + * + * @param elem the removed element + */ + void removeStampedElement(Stamped elem); + + /** + * Returns the timestamp + * + * @return timestamp, or -1L when empty + */ + long get(); + + /** + * Returns the size of internal structure. The meaning of "size" depends on the implementation. + * + * @return size + */ + int size(); + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/Ingestor.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/Ingestor.java new file mode 100644 index 0000000000000..a59313480e016 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/Ingestor.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor.internals; + +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; +import java.util.Set; + +public interface Ingestor { + + Set topics(); + + void poll(long timeoutMs); + + void pause(TopicPartition partition); + + void unpause(TopicPartition partition, long offset); + + void commit(Map offsets); + + int numPartitions(String topic); + + void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition); + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/IngestorImpl.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/IngestorImpl.java new file mode 100644 index 0000000000000..59758e7798559 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/IngestorImpl.java @@ -0,0 +1,133 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.consumer.CommitType; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +public class IngestorImpl implements Ingestor { + + private static final Logger log = LoggerFactory.getLogger(IngestorImpl.class); + + private final Set topics; + private final Consumer consumer; + private final Set unpaused = new HashSet<>(); + private final Map partitionGroups = new HashMap<>(); + + public IngestorImpl(Consumer consumer, Set topics) { + this.consumer = consumer; + this.topics = Collections.unmodifiableSet(topics); + for (String topic : this.topics) consumer.subscribe(topic); + } + + public void open() { + for (String topic : this.topics) consumer.subscribe(topic); + } + + public void init() { + unpaused.clear(); + unpaused.addAll(consumer.subscriptions()); + } + + @Override + public Set topics() { + return topics; + } + + @Override + public void poll(long timeoutMs) { + synchronized (this) { + ConsumerRecords records = consumer.poll(timeoutMs); + + for (TopicPartition partition : unpaused) { + StreamGroup streamGroup = partitionGroups.get(partition); + + if (streamGroup != null) + streamGroup.addRecords(partition, records.records(partition).iterator()); + else + log.warn("unused topic: " + partition.topic()); + } + } + } + + @Override + public void pause(TopicPartition partition) { + synchronized (this) { + consumer.seek(partition, Long.MAX_VALUE); // hack: stop consuming from this partition by setting a big offset + unpaused.remove(partition); + } + } + + @Override + public void unpause(TopicPartition partition, long lastOffset) { + synchronized (this) { + consumer.seek(partition, lastOffset); + unpaused.add(partition); + } + } + + @Override + public void commit(Map offsets) { + synchronized (this) { + consumer.commit(offsets, CommitType.SYNC); + } + } + + @Override + public int numPartitions(String topic) { + return consumer.partitionsFor(topic).size(); + } + + @SuppressWarnings("unchecked") + @Override + public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition) { + synchronized (this) { + partitionGroups.put(partition, streamGroup); + unpaused.add(partition); + } + } + + public void clear() { + unpaused.clear(); + partitionGroups.clear(); + } + + public boolean commitNeeded(Map offsets) { + for (TopicPartition tp : offsets.keySet()) { + if (consumer.committed(tp) != offsets.get(tp)) { + return true; + } + } + return false; + } + + public void close() { + consumer.close(); + clear(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java new file mode 100644 index 0000000000000..a9c8b047a733e --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java @@ -0,0 +1,304 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.processor.PTopology; +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.serialization.Serializer; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +public class KStreamThread extends Thread { + + private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); + + private final Class topologyClass; + private final ArrayList streamGroups = new ArrayList<>(); + private final Map kstreamContexts = new HashMap<>(); + private final IngestorImpl ingestor; + private final RecordCollectorImpl collector; + private final StreamingConfig streamingConfig; + private final ProcessorConfig config; + private final Metrics metrics; + private final KafkaStreamingMetrics streamingMetrics; + private final Time time; + private volatile boolean running; + private long lastCommit; + private long nextStateCleaning; + private long recordsProcessed; + + protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { + @Override + public void onPartitionsAssigned(Consumer consumer, Collection assignment) { + ingestor.init(); + addPartitions(assignment); + } + + @Override + public void onPartitionsRevoked(Consumer consumer, Collection assignment) { + commitAll(time.milliseconds()); + removePartitions(); + ingestor.clear(); + } + }; + + @SuppressWarnings("unchecked") + public KStreamThread(Class topologyClass, StreamingConfig streamingConfig) throws Exception { + super(); + + if (streamingConfig.timestampExtractor() == null) + throw new NullPointerException("timestamp extractor is missing"); + + this.metrics = new Metrics(); + this.config = new ProcessorConfig(streamingConfig.config()); + this.topologyClass = topologyClass; + + this.streamingConfig = streamingConfig; + this.streamingMetrics = new KafkaStreamingMetrics(); + + // build the topology without initialization to get the topics for consumer + PTopology topology = topologyClass.getConstructor().newInstance(); + topology.build(); + + // create the producer and consumer clients + Producer producer = new KafkaProducer<>(streamingConfig.config(), new ByteArraySerializer(), new ByteArraySerializer()); + this.collector = new RecordCollectorImpl(producer, (Serializer) streamingConfig.keySerializer(), (Serializer) streamingConfig.valueSerializer()); + + Consumer consumer = new KafkaConsumer<>(streamingConfig.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + this.ingestor = new IngestorImpl(consumer, topology.topics()); + + this.running = true; + this.lastCommit = 0; + this.nextStateCleaning = Long.MAX_VALUE; + this.recordsProcessed = 0; + this.time = new SystemTime(); + } + + /** + * Execute the stream processors + */ + @Override + public synchronized void run() { + log.info("Starting a kstream thread"); + try { + ingestor.open(); + runLoop(); + } catch (RuntimeException e) { + log.error("Uncaught error during processing: ", e); + throw e; + } finally { + shutdown(); + } + } + + private void shutdown() { + log.info("Shutting down a kstream thread"); + commitAll(time.milliseconds()); + + collector.close(); + ingestor.close(); + removePartitions(); + log.info("kstream thread shutdown complete"); + } + + /** + * Shutdown this streaming thread. + */ + public synchronized void close() { + running = false; + } + + private void runLoop() { + try { + boolean readyForNextExecution = false; + + while (stillRunning()) { + ingestor.poll(readyForNextExecution ? 0 : this.config.pollTimeMs); + + for (StreamGroup group : this.streamGroups) { + readyForNextExecution = group.process(); + } + + maybeCommit(); + maybeCleanState(); + } + } catch (Exception e) { + throw new KafkaException(e); + } + } + + private boolean stillRunning() { + if (!running) { + log.debug("Shutting down at user request."); + return false; + } + if (config.totalRecordsToProcess >= 0 && recordsProcessed >= config.totalRecordsToProcess) { + log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", config.totalRecordsToProcess); + return false; + } + return true; + } + + private void maybeCommit() { + long now = time.milliseconds(); + if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < now) { + log.trace("Committing processor instances because the commit interval has elapsed."); + commitAll(now); + } + } + + private void commitAll(long now) { + Map commit = new HashMap<>(); + for (ProcessorContextImpl context : kstreamContexts.values()) { + context.flush(); + commit.putAll(context.consumedOffsets()); + } + + // check if commit is really needed, i.e. if all the offsets are already committed + if (ingestor.commitNeeded(commit)) { + // TODO: for exactly-once we need to make sure the flush and commit + // are executed atomically whenever it is triggered by user + collector.flush(); + ingestor.commit(commit); // TODO: can this be async? + streamingMetrics.commitTime.record(now - lastCommit); + } + } + + /* delete any state dirs that aren't for active contexts */ + private void maybeCleanState() { + long now = time.milliseconds(); + if (now > nextStateCleaning) { + File[] stateDirs = config.stateDir.listFiles(); + if (stateDirs != null) { + for (File dir : stateDirs) { + try { + Integer id = Integer.parseInt(dir.getName()); + if (!kstreamContexts.keySet().contains(id)) { + log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); + Utils.rm(dir); + } + } catch (NumberFormatException e) { + log.warn("Deleting unknown directory in state directory {}.", dir.getAbsolutePath()); + Utils.rm(dir); + } + } + } + nextStateCleaning = Long.MAX_VALUE; + } + } + + private void addPartitions(Collection assignment) { + HashSet partitions = new HashSet<>(assignment); + + for (TopicPartition partition : partitions) { + final Integer id = partition.partition(); // TODO: switch this to the group id + ProcessorContextImpl context = kstreamContexts.get(id); + if (context == null) { + try { + // build the topology and initialize with the context + PTopology topology = this.topologyClass.getConstructor().newInstance(); + context = new ProcessorContextImpl(id, ingestor, topology, collector, streamingConfig, config, metrics); + topology.build(); + topology.init(context); + context.initialized(); + kstreamContexts.put(id, context); + } catch (Exception e) { + throw new KafkaException(e); + } + + streamGroups.add(context.streamGroup); + } + } + + nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; + } + + private void removePartitions() { + for (ProcessorContextImpl context : kstreamContexts.values()) { + log.info("Removing task context {}", context.id()); + try { + context.close(); + } catch (Exception e) { + throw new KafkaException(e); + } + streamingMetrics.processorDestruction.record(); + } + streamGroups.clear(); + } + + private class KafkaStreamingMetrics { + final Sensor commitTime; + final Sensor processTime; + final Sensor windowTime; + final Sensor processorCreation; + final Sensor processorDestruction; + + public KafkaStreamingMetrics() { + String group = "kafka-streaming"; + + this.commitTime = metrics.sensor("commit-time"); + this.commitTime.add(new MetricName(group, "commit-time-avg-ms"), new Avg()); + this.commitTime.add(new MetricName(group, "commits-time-max-ms"), new Max()); + this.commitTime.add(new MetricName(group, "commits-per-second"), new Rate(new Count())); + + this.processTime = metrics.sensor("process-time"); + this.commitTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); + this.commitTime.add(new MetricName(group, "process-time-max-ms"), new Max()); + this.commitTime.add(new MetricName(group, "process-calls-per-second"), new Rate(new Count())); + + this.windowTime = metrics.sensor("window-time"); + this.windowTime.add(new MetricName(group, "window-time-avg-ms"), new Avg()); + this.windowTime.add(new MetricName(group, "window-time-max-ms"), new Max()); + this.windowTime.add(new MetricName(group, "window-calls-per-second"), new Rate(new Count())); + + this.processorCreation = metrics.sensor("processor-creation"); + this.processorCreation.add(new MetricName(group, "processor-creation"), new Rate(new Count())); + + this.processorDestruction = metrics.sensor("processor-destruction"); + this.processorDestruction.add(new MetricName(group, "processor-destruction"), new Rate(new Count())); + + } + + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java new file mode 100644 index 0000000000000..be93731e99d65 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java @@ -0,0 +1,50 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.common.serialization.Deserializer; + +public class KafkaSource extends KafkaProcessor { + + private static final String SOURCE_NAME = "KAFKA-SOURCE"; + + public Deserializer keyDeserializer; + public Deserializer valDeserializer; + + public KafkaSource(Deserializer keyDeserializer, Deserializer valDeserializer) { + super(SOURCE_NAME); + + this.keyDeserializer = keyDeserializer; + this.valDeserializer = valDeserializer; + } + + @Override + public void init(ProcessorContext context) { + // do nothing + } + + @Override + public void process(K key, V value) { forward(key, value); } + + @Override + public void close() { + // do nothing + } +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java new file mode 100644 index 0000000000000..dc20d6d138e3a --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java @@ -0,0 +1,60 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.processor.TimestampTracker; + +import java.util.LinkedList; + +/** + * MinTimestampTracker is a helper class for a sliding window implementation. + * It is assumed that elements are added or removed in a FIFO manner. + * It maintains the minimum timestamp of stamped elements that were added but not yet removed. + */ +public class MinTimestampTracker implements TimestampTracker { + + private final LinkedList> descendingSubsequence = new LinkedList>(); + + public void addStampedElement(Stamped elem) { + if (elem == null) throw new NullPointerException(); + + Stamped minElem = descendingSubsequence.peekLast(); + while (minElem != null && minElem.timestamp >= elem.timestamp) { + descendingSubsequence.removeLast(); + minElem = descendingSubsequence.peekLast(); + } + descendingSubsequence.offerLast(elem); + } + + public void removeStampedElement(Stamped elem) { + if (elem != null && descendingSubsequence.peekFirst() == elem) + descendingSubsequence.removeFirst(); + } + + public int size() { + return descendingSubsequence.size(); + } + + public long get() { + Stamped stamped = descendingSubsequence.peekFirst(); + if (stamped == null) return -1L; + + return stamped.timestamp; + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/OffsetCheckpoint.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/OffsetCheckpoint.java new file mode 100644 index 0000000000000..37585373be5e5 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/OffsetCheckpoint.java @@ -0,0 +1,172 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor.internals; + +import org.apache.kafka.common.TopicPartition; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.EOFException; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** + * This class saves out a map of topic/partition=>offsets to a file. The format of the file is UTF-8 text containing the following: + *

+ *   <version>
+ *   <n>
+ *   <topic_name_1> <partition_1> <offset_1>
+ *   .
+ *   .
+ *   .
+ *   <topic_name_n> <partition_n> <offset_n>
+ * 
+ * The first line contains a number designating the format version (currently 0), the next line contains + * a number giving the total number of offsets. Each successive line gives a topic/partition/offset triple + * separated by spaces. + */ +public class OffsetCheckpoint { + + private static final int VERSION = 0; + + private final File file; + private final Object lock; + + public OffsetCheckpoint(File file) throws IOException { + new File(file + ".tmp").delete(); // try to delete any existing temp files for cleanliness + this.file = file; + this.lock = new Object(); + } + + public void write(Map offsets) throws IOException { + synchronized (lock) { + // write to temp file and then swap with the existing file + File temp = new File(file.getAbsolutePath() + ".tmp"); + + FileOutputStream fileOutputStream = new FileOutputStream(temp); + BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream)); + try { + writeIntLine(writer, VERSION); + writeIntLine(writer, offsets.size()); + + // write the entries + for (Map.Entry entry : offsets.entrySet()) + writeEntry(writer, entry.getKey(), entry.getValue()); + + // flush the buffer and then fsync the underlying file + writer.flush(); + fileOutputStream.getFD().sync(); + } finally { + writer.close(); + } + + // swap new offset checkpoint file with previous one + if (!temp.renameTo(file)) { + // renameTo() fails on Windows if the destination file exists. + file.delete(); + if (!temp.renameTo(file)) + throw new IOException(String.format("File rename from %s to %s failed.", + temp.getAbsolutePath(), + file.getAbsolutePath())); + } + } + } + + private void writeIntLine(BufferedWriter writer, int number) throws IOException { + writer.write(Integer.toString(number)); + writer.newLine(); + } + + private void writeEntry(BufferedWriter writer, TopicPartition part, long offset) throws IOException { + writer.write(part.topic()); + writer.write(' '); + writer.write(Integer.toString(part.partition())); + writer.write(' '); + writer.write(Long.toString(offset)); + writer.newLine(); + } + + public Map read() throws IOException { + synchronized (lock) { + BufferedReader reader = null; + try { + reader = new BufferedReader(new FileReader(file)); + } catch (FileNotFoundException e) { + return Collections.emptyMap(); + } + + try { + int version = readInt(reader); + switch (version) { + case 0: + int expectedSize = readInt(reader); + Map offsets = new HashMap(); + String line = reader.readLine(); + while (line != null) { + String[] pieces = line.split("\\s+"); + if (pieces.length != 3) + throw new IOException(String.format("Malformed line in offset checkpoint file: '%s'.", + line)); + + String topic = pieces[0]; + int partition = Integer.parseInt(pieces[1]); + long offset = Long.parseLong(pieces[2]); + offsets.put(new TopicPartition(topic, partition), offset); + line = reader.readLine(); + } + if (offsets.size() != expectedSize) + throw new IOException(String.format("Expected %d entries but found only %d", + expectedSize, + offsets.size())); + return offsets; + + default: + throw new IllegalArgumentException("Unknown offset checkpoint version: " + version); + } + } finally { + if (reader != null) + reader.close(); + } + } + } + + private int readInt(BufferedReader reader) throws IOException { + String line = reader.readLine(); + if (line == null) + throw new EOFException("File ended prematurely."); + int val = Integer.parseInt(line); + return val; + } + + public void delete() throws IOException { + file.delete(); + } + + @Override + public String toString() { + return this.file.getAbsolutePath(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ParallelExecutor.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ParallelExecutor.java new file mode 100644 index 0000000000000..894712bed48f0 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ParallelExecutor.java @@ -0,0 +1,152 @@ +/** + * 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.clients.processor.internals; + +import java.util.ArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.LockSupport; + +/** + * A lightweight parallel executor + */ +public class ParallelExecutor { + + /** + * A parallel task must implement this interface + */ + public interface Task { + /** + * Executes a task + * + * @return boolean true if the task are ready for next execution + */ + boolean process(); + } + + private final WorkerThread[] workerThreads; + private final AtomicInteger taskIndex = new AtomicInteger(0); + private volatile ArrayList tasks = new ArrayList<>(); + private volatile CountDownLatch latch; + private volatile boolean readyForNextExecution = true; + private volatile boolean running = true; + private volatile Exception exception; + + public ParallelExecutor(int parallelDegree) { + parallelDegree = Math.max(parallelDegree, 1); + workerThreads = new WorkerThread[parallelDegree - 1]; + for (int i = 0; i < workerThreads.length; i++) { + workerThreads[i] = new WorkerThread(); + workerThreads[i].start(); + } + } + + /** + * Executes tasks in parallel. While this method is executing, other execute call will be blocked. + * + * @param tasks a list of tasks executed in parallel + * @return boolean true if at least one task is ready for next execution, otherwise false + * @throws Exception an exception thrown by a failed task + */ + public boolean execute(ArrayList tasks) throws Exception { + synchronized (this) { + try { + int numTasks = tasks.size(); + exception = null; + readyForNextExecution = false; + if (numTasks > 0) { + this.tasks = tasks; + this.latch = new CountDownLatch(numTasks); + + taskIndex.set(numTasks); + wakeUpWorkers(Math.min(numTasks - 1, workerThreads.length)); + + // the calling thread also picks up tasks + if (taskIndex.get() > 0) doProcess(); + + while (true) { + try { + latch.await(); + break; + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + } + if (exception != null) throw exception; + } finally { + this.tasks = null; + this.latch = null; + this.exception = null; + } + return readyForNextExecution; + } + } + + /** + * Shuts this parallel executor down + */ + public void shutdown() { + synchronized (this) { + running = false; + // wake up all workers + wakeUpWorkers(workerThreads.length); + } + } + + private void doProcess() { + int index = taskIndex.decrementAndGet(); + if (index >= 0) { + try { + if (tasks.get(index).process()) + this.readyForNextExecution = true; + } catch (Exception ex) { + exception = ex; + } finally { + latch.countDown(); + } + } + } + + private void wakeUpWorkers(int numWorkers) { + for (int i = 0; i < numWorkers; i++) + LockSupport.unpark(workerThreads[i]); + } + + private class WorkerThread extends Thread { + + WorkerThread() { + super(); + setDaemon(true); + } + + @Override + public void run() { + while (running) { + if (taskIndex.get() > 0) { + doProcess(); + } else { + // no more work. park this thread. + LockSupport.park(); + Thread.interrupted(); + } + } + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PartitioningInfo.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/PartitioningInfo.java new file mode 100644 index 0000000000000..fa901327e5133 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/PartitioningInfo.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor.internals; + +public class PartitioningInfo { + + public final int numPartitions; + + public PartitioningInfo(int numPartitions) { + this.numPartitions = numPartitions; + } +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java new file mode 100644 index 0000000000000..2673de48fdfa1 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor.internals; + +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 org.apache.kafka.stream.StreamingConfig; + +import java.io.File; +import java.util.Properties; + +public class ProcessorConfig extends AbstractConfig { + + private static final ConfigDef CONFIG; + + static { + CONFIG = new ConfigDef().define(StreamingConfig.TOPICS_CONFIG, + Type.STRING, + "", + Importance.HIGH, + "All the possible topic names this job need to interact with") + .define(StreamingConfig.STATE_DIR_CONFIG, + Type.STRING, + System.getProperty("java.io.tmpdir"), + Importance.MEDIUM, + "") + .define(StreamingConfig.POLL_TIME_MS_CONFIG, + Type.LONG, + 100, + Importance.LOW, + "The amount of time to block waiting for input.") + .define(StreamingConfig.COMMIT_TIME_MS_CONFIG, + Type.LONG, + 30000, + Importance.HIGH, + "The frequency with which to save the position of the processor.") + .define(StreamingConfig.WINDOW_TIME_MS_CONFIG, + Type.LONG, + -1L, + Importance.MEDIUM, + "Setting this to a non-negative value will cause the processor to get called with this frequency even if there is no message.") + .define(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + 1000, + Importance.LOW, + "The maximum number of records to buffer per partition") + .define(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG, + Type.LONG, + 60000, + Importance.LOW, + "The amount of time to wait before deleting state when a partition has migrated.") + .define(StreamingConfig.TOTAL_RECORDS_TO_PROCESS, + Type.LONG, + -1L, + Importance.LOW, + "Exit after processing this many records.") + .define(StreamingConfig.NUM_STREAM_THREADS, + Type.INT, + 1, + Importance.LOW, + "The number of threads to execute stream processing."); + } + + public final String topics; + public final File stateDir; + public final long pollTimeMs; + public final long commitTimeMs; + public final long windowTimeMs; + public final int bufferedRecordsPerPartition; + public final long stateCleanupDelay; + public final long totalRecordsToProcess; + public final int numStreamThreads; + + public ProcessorConfig(Properties processor) { + super(CONFIG, processor); + this.topics = this.getString(StreamingConfig.TOPICS_CONFIG); + this.stateDir = new File(this.getString(StreamingConfig.STATE_DIR_CONFIG)); + this.pollTimeMs = this.getLong(StreamingConfig.POLL_TIME_MS_CONFIG); + this.commitTimeMs = this.getLong(StreamingConfig.COMMIT_TIME_MS_CONFIG); + this.windowTimeMs = this.getLong(StreamingConfig.WINDOW_TIME_MS_CONFIG); + this.bufferedRecordsPerPartition = this.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + this.stateCleanupDelay = this.getLong(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG); + this.totalRecordsToProcess = this.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); + this.numStreamThreads = this.getInt(StreamingConfig.NUM_STREAM_THREADS); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java new file mode 100644 index 0000000000000..a82387576212c --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java @@ -0,0 +1,225 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.RecordCollector; +import org.apache.kafka.clients.processor.StateStore; +import org.apache.kafka.clients.processor.KStreamException; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.RestoreFunc; +import org.apache.kafka.clients.processor.TimestampExtractor; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + + + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +public class ProcessorContextImpl implements ProcessorContext { + + private static final Logger log = LoggerFactory.getLogger(ProcessorContextImpl.class); + + public final int id; + public final StreamGroup streamGroup; + public final Ingestor ingestor; + + private final Metrics metrics; + private final PTopology topology; + private final RecordCollectorImpl collector; + private final ProcessorStateManager stateMgr; + private final StreamingConfig streamingConfig; + private final ProcessorConfig processorConfig; + private final TimestampExtractor timestampExtractor; + + private boolean initialized; + + @SuppressWarnings("unchecked") + public ProcessorContextImpl(int id, + Ingestor ingestor, + PTopology topology, + RecordCollectorImpl collector, + StreamingConfig streamingConfig, + ProcessorConfig processorConfig, + Metrics metrics) throws IOException { + this.id = id; + this.metrics = metrics; + this.ingestor = ingestor; + this.topology = topology; + this.collector = collector; + this.streamingConfig = streamingConfig; + this.processorConfig = processorConfig; + this.timestampExtractor = this.streamingConfig.timestampExtractor(); + + File stateFile = new File(processorConfig.stateDir, Integer.toString(id)); + Consumer restoreConsumer = new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + + this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); + this.streamGroup = new StreamGroup(this, this.ingestor, new TimeBasedChooser(), this.timestampExtractor, this.processorConfig.bufferedRecordsPerPartition); + + stateMgr.init(); + + // update the partition -> source stream map + for (String topic : this.topology.topics()) { + if (!ingestor.topics().contains(topic)) + throw new IllegalArgumentException("topic not subscribed: " + topic); + + TopicPartition partition = new TopicPartition(topic, id); + KafkaSource source = topology.source(topic); + + this.streamGroup.addPartition(partition, source); + this.ingestor.addPartitionStreamToGroup(this.streamGroup, partition); + } + + initialized = false; + } + + @Override + public int id() { + return id; + } + + @Override + public Serializer keySerializer() { + return streamingConfig.keySerializer(); + } + + @Override + public Serializer valueSerializer() { + return streamingConfig.valueSerializer(); + } + + @Override + public Deserializer keyDeserializer() { + return streamingConfig.keyDeserializer(); + } + + @Override + public Deserializer valueDeserializer() { + return streamingConfig.valueDeserializer(); + } + + @Override + public RecordCollector recordCollector() { + return collector; + } + + @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 KStreamException("Can only create state stores during initialization."); + + stateMgr.register(store, restoreFunc); + } + + @Override + public void flush() { + stateMgr.flush(); + } + + public String topic() { + if (streamGroup.record() == null) + throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); + + return streamGroup.record().topic(); + } + + @Override + public int partition() { + if (streamGroup.record() == null) + throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); + + return streamGroup.record().partition(); + } + + @Override + public long offset() { + if (this.streamGroup.record() == null) + throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); + + return this.streamGroup.record().offset(); + } + + @Override + public long timestamp() { + if (streamGroup.record() == null) + throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); + + return streamGroup.record().timestamp; + } + + @Override + public void send(String topic, Object key, Object value) { + collector.send(new ProducerRecord<>(topic, key, value)); + } + + @Override + public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { + if (keySerializer == null || valSerializer == null) + throw new IllegalStateException("key and value serializers must be specified"); + + collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); + } + + @Override + public void commit() { + streamGroup.commitOffset(); + } + + @Override + public void schedule(KafkaProcessor processor, long interval) { + streamGroup.schedule(processor, interval); + } + + public void initialized() { + initialized = true; + } + + public Map consumedOffsets() { + return streamGroup.consumedOffsets(); + } + + public void close() throws Exception { + topology.close(); + stateMgr.close(collector.offsets()); + streamGroup.close(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorStateManager.java new file mode 100644 index 0000000000000..672dccab2312d --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorStateManager.java @@ -0,0 +1,193 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.processor.RestoreFunc; +import org.apache.kafka.clients.processor.StateStore; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class ProcessorStateManager { + + private static final Logger log = LoggerFactory.getLogger(ProcessorStateManager.class); + private static final String CHECKPOINT_FILE_NAME = ".checkpoint"; + + private final int id; + private final File baseDir; + private final Map stores; + private final Consumer restoreConsumer; + private final Map restoredOffsets; + private final Map checkpointedOffsets; + + public ProcessorStateManager(int id, File baseDir, Consumer restoreConsumer) { + this.id = id; + this.baseDir = baseDir; + this.stores = new HashMap<>(); + this.restoreConsumer = restoreConsumer; + this.restoredOffsets = new HashMap<>(); + this.checkpointedOffsets = new HashMap<>(); + } + + public File baseDir() { + return this.baseDir; + } + + public Consumer restoreConsumer() { + return this.restoreConsumer; + } + + public void init() throws IOException { + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + this.checkpointedOffsets.putAll(checkpoint.read()); + + // delete the checkpoint file after finish loading its stored offsets + checkpoint.delete(); + } + + 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().keySet().contains(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.subscriptions().isEmpty()) { + throw new IllegalStateException("Restore consumer should have not subscribed to any partitions beforehand"); + } + restoreConsumer.subscribe(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 { + restoreConsumer.seekToBeginning(storePartition); + } + + // restore its state from changelog records; while restoring the log end offset + // should not change since it is only written by this thread. + while (true) { + for (ConsumerRecord record : restoreConsumer.poll(100)) { + 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-subscribe the change log partition + restoreConsumer.unsubscribe(storePartition); + } + + 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 engine : this.stores.values()) + engine.flush(); + } + } + + public void close(Map ackedOffsets) throws IOException { + if (!stores.isEmpty()) { + log.debug("Closing stores."); + for (Map.Entry entry : stores.entrySet()) { + log.debug("Closing storage engine {}", entry.getKey()); + entry.getValue().flush(); + entry.getValue().close(); + } + + Map checkpointOffsets = new HashMap(restoredOffsets); + 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()) { + if (ackedOffsets.containsKey(part)) + // store the last ack'd offset + 1 (the log position after restoration) + checkpointOffsets.put(part, ackedOffsets.get(part) + 1); + } else { + checkpointOffsets.remove(part); + } + } + + // write the checkpoint file before closing, to indicate clean shutdown + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + checkpoint.write(checkpointOffsets); + } + + // close the restore consumer + restoreConsumer.close(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationQueue.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationQueue.java new file mode 100644 index 0000000000000..ffc37e405c6db --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationQueue.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor.internals; + +import java.util.PriorityQueue; + +public class PunctuationQueue { + + private PriorityQueue pq = new PriorityQueue<>(); + + public void schedule(PunctuationSchedule sched) { + synchronized (pq) { + pq.add(sched); + } + } + + public void close() { + synchronized (pq) { + pq.clear(); + } + } + + public void mayPunctuate(long streamTime) { + synchronized (pq) { + PunctuationSchedule top = pq.peek(); + while (top != null && top.timestamp <= streamTime) { + PunctuationSchedule sched = top; + pq.poll(); + sched.punctuator().punctuate(streamTime); + pq.add(sched.next()); + + top = pq.peek(); + } + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java new file mode 100644 index 0000000000000..889648c3c2eaf --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java @@ -0,0 +1,39 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.processor.Punctuator; + +public class PunctuationSchedule extends Stamped { + + final long interval; + + public PunctuationSchedule(Punctuator punctuator, long interval) { + super(punctuator, System.currentTimeMillis() + interval); + this.interval = interval; + } + + public Punctuator punctuator() { + return value; + } + + public PunctuationSchedule next() { + return new PunctuationSchedule(value, timestamp + interval); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordCollectorImpl.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordCollectorImpl.java new file mode 100644 index 0000000000000..1e7edddd7969c --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordCollectorImpl.java @@ -0,0 +1,92 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.processor.RecordCollector; +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 RecordCollectorImpl implements RecordCollector { + + private static final Logger log = LoggerFactory.getLogger(RecordCollectorImpl.class); + + private final Producer producer; + private final Map offsets; + private final Callback callback = new Callback() { + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception == null) { + TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); + offsets.put(tp, metadata.offset()); + } else { + log.error("Error sending record: ", exception); + } + } + }; + private final Serializer keySerializer; + private final Serializer valueSerializer; + + + public RecordCollectorImpl(Producer producer, Serializer keySerializer, Serializer valueSerializer) { + this.producer = producer; + this.offsets = new HashMap<>(); + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + } + + @Override + public void send(ProducerRecord record) { + send(record, this.keySerializer, this.valueSerializer); + } + + @Override + public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); + byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); + this.producer.send(new ProducerRecord<>(record.topic(), keyBytes, valBytes), callback); + } + + @Override + public void flush() { + this.producer.flush(); + } + + /** + * Closes this RecordCollector + */ + public void close() { + producer.close(); + } + + /** + * The last ack'd offset from the producer + * + * @return the map from TopicPartition to offset + */ + Map offsets() { + return this.offsets; + } +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueue.java new file mode 100644 index 0000000000000..9dc672f27f82f --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueue.java @@ -0,0 +1,122 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.processor.TimestampTracker; +import org.apache.kafka.common.TopicPartition; + +import java.util.ArrayDeque; + +/** + * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). It is intended to be used in + * {@link StreamGroup}. + */ +public class RecordQueue { + + private final ArrayDeque queue = new ArrayDeque<>(); + public final KafkaSource source; + private final TopicPartition partition; + private TimestampTracker> timestampTracker; + private long offset; + + /** + * Creates a new instance of RecordQueue + * + * @param partition partition + * @param source the instance of KStreamImpl that receives records + * @param timestampTracker TimestampTracker + */ + public RecordQueue(TopicPartition partition, KafkaSource source, TimestampTracker> timestampTracker) { + this.partition = partition; + this.source = source; + this.timestampTracker = timestampTracker; + } + + /** + * Returns the partition with which this queue is associated + * + * @return TopicPartition + */ + public TopicPartition partition() { + return partition; + } + + /** + * Adds a StampedRecord to the queue + * + * @param record StampedRecord + */ + public void add(StampedRecord record) { + queue.addLast(record); + timestampTracker.addStampedElement(record); + offset = record.offset(); + } + + /** + * Returns the next record fro the queue + * + * @return StampedRecord + */ + public StampedRecord next() { + StampedRecord elem = queue.pollFirst(); + + if (elem == null) return null; + + timestampTracker.removeStampedElement(elem); + + return elem; + } + + /** + * Returns the highest offset in the queue + * + * @return offset + */ + public long offset() { + return offset; + } + + /** + * Returns the number of records in the queue + * + * @return the number of records + */ + public int size() { + return queue.size(); + } + + /** + * Tests if the queue is empty + * + * @return true if the queue is empty, otherwise false + */ + public boolean isEmpty() { + return queue.isEmpty(); + } + + /** + * Returns a timestamp tracked by the TimestampTracker + * + * @return timestamp + */ + public long trackedTimestamp() { + return timestampTracker.get(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java new file mode 100644 index 0000000000000..b9019f2614056 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor.internals; + +import org.apache.kafka.clients.processor.Chooser; + +import java.util.ArrayDeque; + +public class RoundRobinChooser implements Chooser { + + private final ArrayDeque deque; + + public RoundRobinChooser() { + deque = new ArrayDeque<>(); + } + + @Override + public void add(RecordQueue queue) { + deque.offer(queue); + } + + @Override + public RecordQueue next() { + return deque.poll(); + } + + @Override + public void close() { + deque.clear(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/Stamped.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/Stamped.java new file mode 100644 index 0000000000000..f1d1bb0018237 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/Stamped.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor.internals; + +public class Stamped implements Comparable { + + public final V value; + public final long timestamp; + + public Stamped(V value, long timestamp) { + this.value = value; + this.timestamp = timestamp; + } + + public int compareTo(Object other) { + long otherTimestamp = ((Stamped) other).timestamp; + + if (timestamp < otherTimestamp) return -1; + else if (timestamp > otherTimestamp) return 1; + return 0; + } + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StampedRecord.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StampedRecord.java new file mode 100644 index 0000000000000..5778873917fa8 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StampedRecord.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +// TODO: making this class exposed to user in the lower-level Processor +public class StampedRecord extends Stamped> { + + StampedRecord(ConsumerRecord record, long timestamp) { + super(record, timestamp); + } + + public String topic() { + return value.topic(); + } + + public int partition() { + return value.partition(); + } + + public Object key() { + return value.key(); + } + + public Object value() { + return value.value(); + } + + public long offset() { + return value.offset(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java new file mode 100644 index 0000000000000..55c8dff0c42b3 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java @@ -0,0 +1,260 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.processor.Chooser; +import org.apache.kafka.clients.processor.Punctuator; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.TimestampExtractor; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; + +import java.util.ArrayDeque; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** + * A StreamGroup is composed of multiple streams from different topics that need to be synchronized. + */ +public class StreamGroup { + + private final ProcessorContext context; + private final Ingestor ingestor; + private final Chooser chooser; + private final TimestampExtractor timestampExtractor; + private final Map stash = new HashMap<>(); + + private final int desiredUnprocessed; + + // TODO: merge stash, consumedOffset, and newRecordBuffer into sth. like partition metadata + private final Map consumedOffsets; + private final PunctuationQueue punctuationQueue = new PunctuationQueue(); + private final ArrayDeque newRecordBuffer = new ArrayDeque<>(); + + private long streamTime = -1; + private boolean commitRequested = false; + private StampedRecord currRecord = null; + private volatile int buffered = 0; + + /** + * Creates StreamGroup + * + * @param context the task context + * @param ingestor the instance of {@link Ingestor} + * @param chooser the instance of {@link Chooser} + * @param timestampExtractor the instance of {@link TimestampExtractor} + * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic + */ + public StreamGroup(ProcessorContext context, + Ingestor ingestor, + Chooser chooser, + TimestampExtractor timestampExtractor, + int desiredUnprocessedPerPartition) { + this.context = context; + this.ingestor = ingestor; + this.chooser = chooser; + this.timestampExtractor = timestampExtractor; + this.desiredUnprocessed = desiredUnprocessedPerPartition; + this.consumedOffsets = new HashMap<>(); + } + + public StampedRecord record() { + return currRecord; + } + + /** + * Adds a partition and its receiver to this stream synchronizer + * + * @param partition the partition + * @param source the instance of KStreamImpl + */ + @SuppressWarnings("unchecked") + public void addPartition(TopicPartition partition, KafkaSource source) { + synchronized (this) { + RecordQueue recordQueue = stash.get(partition); + + if (recordQueue == null) { + stash.put(partition, createRecordQueue(partition, source)); + } else { + throw new IllegalStateException("duplicate partition"); + } + } + } + + /** + * Adds records + * + * @param partition the partition + * @param iterator the iterator of records + */ + @SuppressWarnings("unchecked") + public void addRecords(TopicPartition partition, Iterator> iterator) { + synchronized (this) { + newRecordBuffer.addLast(new NewRecords(partition, iterator)); + } + } + + @SuppressWarnings("unchecked") + private void ingestNewRecords() { + for (NewRecords newRecords : newRecordBuffer) { + TopicPartition partition = newRecords.partition; + Iterator> iterator = newRecords.iterator; + + RecordQueue recordQueue = stash.get(partition); + if (recordQueue != null) { + boolean wasEmpty = recordQueue.isEmpty(); + + while (iterator.hasNext()) { + ConsumerRecord record = iterator.next(); + + // deserialize the raw record, extract the timestamp and put into the queue + Deserializer keyDeserializer = recordQueue.source.keyDeserializer; + Deserializer valDeserializer = recordQueue.source.valDeserializer; + + Object key = keyDeserializer.deserialize(record.topic(), record.key()); + Object value = valDeserializer.deserialize(record.topic(), record.value()); + ConsumerRecord deserializedRecord = new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), key, value); + + long timestamp = timestampExtractor.extract(record.topic(), key, value); + recordQueue.add(new StampedRecord(deserializedRecord, timestamp)); + buffered++; + } + + int queueSize = recordQueue.size(); + if (wasEmpty && queueSize > 0) chooser.add(recordQueue); + + // if we have buffered enough for this partition, pause + if (queueSize >= this.desiredUnprocessed) { + ingestor.pause(partition); + } + } + } + newRecordBuffer.clear(); + } + + /** + * Schedules a punctuation for the processor + * + * @param punctuator the punctuator requesting scheduler + * @param interval the interval in milliseconds + */ + public void schedule(Punctuator punctuator, long interval) { + punctuationQueue.schedule(new PunctuationSchedule(punctuator, interval)); + } + + /** + * Processes one record + */ + @SuppressWarnings("unchecked") + public boolean process() { + synchronized (this) { + boolean readyForNextExecution = false; + ingestNewRecords(); + + RecordQueue recordQueue = chooser.next(); + if (recordQueue == null) { + return false; + } + + if (recordQueue.size() == 0) throw new IllegalStateException("empty record queue"); + + if (recordQueue.size() == this.desiredUnprocessed) { + ingestor.unpause(recordQueue.partition(), recordQueue.offset()); + } + + long trackedTimestamp = recordQueue.trackedTimestamp(); + currRecord = recordQueue.next(); + + if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; + + recordQueue.source.receive(currRecord.key(), currRecord.value()); + consumedOffsets.put(recordQueue.partition(), currRecord.offset()); + + // TODO: local state flush and downstream producer flush + // need to be done altogether with offset commit atomically + if (commitRequested) { + // flush local state + context.flush(); + + // flush produced records in the downstream + context.recordCollector().flush(); + + // commit consumed offsets + ingestor.commit(consumedOffsets()); + } + + if (commitRequested) ingestor.commit(Collections.singletonMap( + new TopicPartition(currRecord.topic(), currRecord.partition()), + currRecord.offset())); + + if (recordQueue.size() > 0) { + readyForNextExecution = true; + chooser.add(recordQueue); + } + + buffered--; + currRecord = null; + + punctuationQueue.mayPunctuate(streamTime); + + return readyForNextExecution; + } + } + + /** + * Returns consumed offsets + * + * @return the map of partition to consumed offset + */ + public Map consumedOffsets() { + return this.consumedOffsets; + } + + /** + * Request committing the current record's offset + */ + public void commitOffset() { + this.commitRequested = true; + } + + public int buffered() { + return buffered; + } + + public void close() { + chooser.close(); + stash.clear(); + } + + protected RecordQueue createRecordQueue(TopicPartition partition, KafkaSource source) { + return new RecordQueue(partition, source, new MinTimestampTracker<>()); + } + + private static class NewRecords { + final TopicPartition partition; + final Iterator> iterator; + + NewRecords(TopicPartition partition, Iterator> iterator) { + this.partition = partition; + this.iterator = iterator; + } + } +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java new file mode 100644 index 0000000000000..66f89418fb925 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java @@ -0,0 +1,144 @@ +/** + * 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.clients.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.processor.KafkaStreaming; +import org.apache.kafka.clients.processor.TimestampExtractor; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.*; + +/** + * Configuration information passed to the {@link KafkaStreaming} instance for configuring the associated + * {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer}, and + * {@link org.apache.kafka.clients.consumer.KafkaConsumer KafkaConsumer}, as + * well as the processor itself. + */ +public class StreamingConfig { + + /** topics */ + public static final String TOPICS_CONFIG = "topics"; + + /** state.dir */ + public static final String STATE_DIR_CONFIG = "state.dir"; + + /** poll.time.ms */ + public static final String POLL_TIME_MS_CONFIG = "poll.time.ms"; + + /** commit.time.ms */ + public static final String COMMIT_TIME_MS_CONFIG = "commit.time.ms"; + + /** window.time.ms */ + public static final String WINDOW_TIME_MS_CONFIG = "window.time.ms"; + + /** buffered.records.per.partition */ + public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; + + /** state.cleanup.delay */ + public static final String STATE_CLEANUP_DELAY_CONFIG = "state.cleanup.delay"; + + /** total.records.to.process */ + public static final String TOTAL_RECORDS_TO_PROCESS = "total.records.to.process"; + + /** num.of.stream.threads */ + public static final String NUM_STREAM_THREADS = "num.of.stream.threads"; + + private final Properties config; + private final Map context = new HashMap(); + private Serializer keySerializer; + private Serializer valSerializer; + private Deserializer keyDeserializer; + private Deserializer valDeserializer; + private TimestampExtractor timestampExtractor; + + public StreamingConfig(Properties config) { + this.config = config; + this.config.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + this.config.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); + this.config.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); + } + + @Override + public StreamingConfig clone() { + return new StreamingConfig(this.config); + } + + public void addContextObject(String key, Object value) { + this.context.put(key, value); + } + + @SuppressWarnings("all") + + public void serialization(Serializer serializer, Deserializer deserializer) { + keySerializer(serializer); + valueSerializer(serializer); + keyDeserializer(deserializer); + valueDeserializer(deserializer); + } + + public void keySerializer(Serializer serializer) { + this.keySerializer = serializer; + } + + public void valueSerializer(Serializer serializer) { + this.valSerializer = serializer; + } + + public void keyDeserializer(Deserializer deserializer) { + this.keyDeserializer = deserializer; + } + + public void valueDeserializer(Deserializer deserializer) { + this.valDeserializer = deserializer; + } + + public Properties config() { + return this.config; + } + + public Map context() { + return this.context; + } + + public Serializer keySerializer() { + return this.keySerializer; + } + + public Serializer valueSerializer() { + return this.valSerializer; + } + + public Deserializer keyDeserializer() { + return this.keyDeserializer; + } + + public Deserializer valueDeserializer() { + return this.valDeserializer; + } + + public void timestampExtractor(TimestampExtractor timestampExtractor) { + this.timestampExtractor = timestampExtractor; + } + + public TimestampExtractor timestampExtractor() { + return this.timestampExtractor; + } +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java new file mode 100644 index 0000000000000..812c3b8852a52 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java @@ -0,0 +1,62 @@ +/** + * 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.clients.processor.internals; + + +import org.apache.kafka.clients.processor.Chooser; + +import java.util.Comparator; +import java.util.PriorityQueue; + +public class TimeBasedChooser implements Chooser { + + private final PriorityQueue pq; + + public TimeBasedChooser() { + this(new Comparator() { + public int compare(RecordQueue queue1, RecordQueue queue2) { + long time1 = queue1.trackedTimestamp(); + long time2 = queue2.trackedTimestamp(); + + if (time1 < time2) return -1; + if (time1 > time2) return 1; + return 0; + } + }); + } + + private TimeBasedChooser(Comparator comparator) { + pq = new PriorityQueue<>(3, comparator); + } + + @Override + public void add(RecordQueue queue) { + pq.offer(queue); + } + + @Override + public RecordQueue next() { + return pq.poll(); + } + + @Override + public void close() { + pq.clear(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java new file mode 100644 index 0000000000000..a6cdf5359301d --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.examples; + +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.clients.processor.KafkaStreaming; +import org.apache.kafka.clients.processor.internals.StreamingConfig; +import org.apache.kafka.stream.KStream; +import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.topology.KeyValue; +import org.apache.kafka.stream.topology.KeyValueMapper; +import org.apache.kafka.stream.topology.Predicate; + +import java.util.Properties; + +public class KStreamJob { + + private static class MyKStreamTopology extends KStreamTopology { + + @Override + public void build() { + // With overridden de-serializer + KStream stream1 = from(new StringDeserializer(), new StringDeserializer(), "topic1"); + + KStream stream2 = + stream1.map(new KeyValueMapper() { + @Override + public KeyValue apply(String key, String value) { + return new KeyValue<>(key, new Integer(value)); + } + }).filter(new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + }); + + KStream[] streams = stream2.branch( + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + }, + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + } + ); + + streams[0].sendTo("topic2"); + streams[1].sendTo("topic3"); + } + } + + public static void main(String[] args) throws Exception { + KafkaStreaming kstream = new KafkaStreaming(MyKStreamTopology.class, new StreamingConfig(new Properties())); + kstream.run(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java new file mode 100644 index 0000000000000..117b2ceade0fb --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java @@ -0,0 +1,74 @@ +/** + * 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.stream.examples; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.internals.KafkaSource; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.clients.processor.KafkaStreaming; +import org.apache.kafka.clients.processor.internals.StreamingConfig; + +import java.util.Properties; + +public class SimpleProcessJob { + + private static class MyProcessor extends KafkaProcessor { + private ProcessorContext context; + + public MyProcessor(String name) { + super(name); + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + @Override + public void process(String key, Integer value) { + System.out.println("[" + key + ", " + value + "]"); + + context.commit(); + + context.send("topic-dest", key, value); + } + + @Override + public void close() { + // do nothing + } + } + + private static class MyPTopology extends PTopology { + + @Override + public void build() { + KafkaSource source = addSource(new StringDeserializer(), new IntegerDeserializer(), "topic-source"); + + addProcessor(new MyProcessor("processor"), source); + } + } + + public static void main(String[] args) throws Exception { + KafkaStreaming streaming = new KafkaStreaming(MyPTopology.class, new StreamingConfig(new Properties())); + streaming.run(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java new file mode 100644 index 0000000000000..f2a73abaf8e98 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.examples; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.internals.KafkaSource; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.clients.processor.KafkaStreaming; +import org.apache.kafka.clients.processor.internals.StreamingConfig; +import org.apache.kafka.stream.state.Entry; +import org.apache.kafka.stream.state.InMemoryKeyValueStore; +import org.apache.kafka.stream.state.KeyValueIterator; +import org.apache.kafka.stream.state.KeyValueStore; + +import java.util.Properties; + +public class StatefulProcessJob { + + private static class MyProcessor extends KafkaProcessor { + private ProcessorContext context; + private KeyValueStore kvStore; + + public MyProcessor(String name) { + super(name); + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + this.context.schedule(this, 1000); + + this.kvStore = new InMemoryKeyValueStore<>("local-state", context); + } + + @Override + public void process(String key, Integer value) { + Integer oldValue = this.kvStore.get(key); + if (oldValue == null) { + this.kvStore.put(key, value); + } else { + int newValue = oldValue + value; + this.kvStore.put(key, newValue); + } + + context.commit(); + } + + @Override + public void punctuate(long streamTime) { + KeyValueIterator iter = this.kvStore.all(); + while (iter.hasNext()) { + Entry entry = iter.next(); + System.out.println("[" + entry.key() + ", " + entry.value() + "]"); + } + } + + @Override + public void close() { + this.kvStore.close(); + } + } + + private static class MyPTopology extends PTopology { + + @Override + public void build() { + KafkaSource source = addSource(new StringDeserializer(), new IntegerDeserializer(), "topic-source"); + + addProcessor(new MyProcessor("processor"), source); + } + } + + public static void main(String[] args) throws Exception { + KafkaStreaming streaming = new KafkaStreaming(MyPTopology.class, new StreamingConfig(new Properties())); + streaming.run(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java index 4a8eaca3d9299..47fcb71222592 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java @@ -18,7 +18,6 @@ package org.apache.kafka.stream.state; import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.SystemTime; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java index 37f3e4b23174d..273e5bd0a01c6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java @@ -18,7 +18,6 @@ package org.apache.kafka.stream.state.internals; import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.clients.processor.RecordCollector; import org.apache.kafka.clients.processor.RestoreFunc; import org.apache.kafka.clients.producer.ProducerRecord; diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java b/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java index 7f8fff0a98e5d..bc021b6d9e0c3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java @@ -17,8 +17,7 @@ package org.apache.kafka.stream.topology; -public interface ValueJoiner { +public interface ValueJoiner { R apply(V1 value1, V2 value2); - } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java index fdc1281f3389f..55d90db535e5c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java @@ -19,13 +19,11 @@ import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.internals.KafkaSource; import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.topology.Predicate; import java.lang.reflect.Array; import java.util.Arrays; -import java.util.concurrent.atomic.AtomicInteger; class KStreamBranch extends KafkaProcessor { diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java index ebc31efe491ab..54c9407159743 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java @@ -108,7 +108,7 @@ public KStream flatMapValues(ValueMapper> @Override public KStreamWindowed with(Window window) { - return (KStreamWindowed) chain(new KStreamWindowedImpl<>(window, topology)); + return (KStreamWindowed) chain(new KStreamWindow<>(window, topology)); } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java index 9aa5d14933192..e5b07ffc89d85 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java @@ -17,33 +17,38 @@ package org.apache.kafka.stream.topology.internals; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.internals.Receiver; -import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.NotCopartitionedException; import org.apache.kafka.stream.topology.ValueJoiner; import org.apache.kafka.stream.topology.Window; import java.util.Iterator; -class KStreamJoin extends KStreamImpl { +class KStreamJoin extends KafkaProcessor { + + private static final String JOIN_NAME = "KAFKA-JOIN"; private static abstract class Finder { abstract Iterator find(K key, long timestamp); } + private KStreamMetadata thisMetadata; + private KStreamMetadata otherMetadata; private final Finder finder1; private final Finder finder2; - private final ValueJoiner joiner; + private final ValueJoiner joiner; final Receiver receiverForOtherStream; - private KStreamMetadata thisMetadata; - private KStreamMetadata otherMetadata; - KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology topology) { - super(topology); + private ProcessorContext context; - final Window window1 = stream1.window; - final Window window2 = stream2.window; + KStreamJoin(KStreamWindow stream1, KStreamWindow stream2, boolean prior, ValueJoiner joiner) { + super(JOIN_NAME); + + final Window window1 = stream1.window(); + final Window window2 = stream2.window(); if (prior) { this.finder1 = new Finder() { @@ -83,13 +88,18 @@ public void bind(KStreamContext context, KStreamMetadata metadata) { throw new NotCopartitionedException(); } - @SuppressWarnings("unchecked") @Override - public void receive(Object key, Object value, long timestamp) { - Iterator iter = finder2.find((K) key, timestamp); + public void init(ProcessorContext context) { + this.context = context; + } + + @Override + public void process(K key, V1 value) { + long timestamp = context.timestamp(); + Iterator iter = finder2.find(key, timestamp); if (iter != null) { while (iter.hasNext()) { - doJoin((K) key, (V1) value, iter.next(), timestamp); + doJoin(key,value, iter.next()); } } } @@ -122,8 +132,8 @@ public void close() { } // TODO: use the "outer-stream" topic as the resulted join stream topic - private void doJoin(K key, V1 value1, V2 value2, long timestamp) { - forward(key, joiner.apply(value1, value2), timestamp); + private void doJoin(K key, V1 value1, V2 value2) { + forward(key, joiner.apply(value1, value2)); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java index 48f25c19e8afa..cfebb3cc0cfb3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.internals.KafkaSource; public class KStreamSource extends KStreamImpl { diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindow.java new file mode 100644 index 0000000000000..38e31f0c041df --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindow.java @@ -0,0 +1,89 @@ +/** + * 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.stream.topology.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KStream; +import org.apache.kafka.stream.topology.KStreamWindowed; +import org.apache.kafka.stream.topology.ValueJoiner; +import org.apache.kafka.stream.topology.Window; + +public class KStreamWindow extends KafkaProcessor { + + private static final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { + + public KStreamWindowedImpl(PTopology topology, KafkaProcessor processor) { + super(topology, processor); + } + + @Override + public KStream join(KStreamWindowed other, ValueJoiner processor) { + return join(other, false, processor); + } + + @Override + public KStream joinPrior(KStreamWindowed other, ValueJoiner processor) { + return join(other, true, processor); + } + + private KStream join(KStreamWindowed other, boolean prior, ValueJoiner processor) { + + KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; + + KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, topology); + otherImpl.registerReceiver(stream.receiverForOtherStream); + + return chain(stream); + } + } + + private static final String WINDOW_NAME = "KAFKA-WINDOW"; + + private final Window window; + private ProcessorContext context; + + KStreamWindow(Window window) { + super(WINDOW_NAME); + this.window = window; + } + + public Window window() { + return window; + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + @SuppressWarnings("unchecked") + @Override + public void process(K key, V value) { + synchronized (this) { + window.put(key, value, context.timestamp()); + forward(key, value); + } + } + + @Override + public void close() { + window.close(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindowedImpl.java deleted file mode 100644 index 6f67f1c06d394..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindowedImpl.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.KStreamWindowed; -import org.apache.kafka.stream.topology.ValueJoiner; -import org.apache.kafka.stream.topology.Window; - -public class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { - - final Window window; - - KStreamWindowedImpl(Window window, KStreamTopology initializer) { - super(initializer); - this.window = window; - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - super.bind(context, metadata); - window.init(context); - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - window.put((K) key, (V) value, timestamp); - // KStreamWindowed needs to forward the topic name since it may receive directly from KStreamSource - forward(key, value, timestamp); - } - } - - @Override - public void close() { - window.close(); - super.close(); - } - - @Override - public KStream join(KStreamWindowed other, ValueJoiner processor) { - return join(other, false, processor); - } - - @Override - public KStream joinPrior(KStreamWindowed other, ValueJoiner processor) { - return join(other, true, processor); - } - - private KStream join(KStreamWindowed other, boolean prior, ValueJoiner processor) { - - KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; - - KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, topology); - otherImpl.registerReceiver(stream.receiverForOtherStream); - - return chain(stream); - } - -} From 6e342209c568dd6673fba45de3dd64dc5f060399 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 7 Aug 2015 23:25:47 -0700 Subject: [PATCH 149/275] wip --- .../kafka/clients/processor/PTopology.java | 2 + .../processor/internals/KStreamThread.java | 2 + .../internals/ProcessorContextImpl.java | 22 +- .../processor/internals/StreamGroup.java | 9 +- .../java/org/apache/kafka/stream/KStream.java | 5 - .../apache/kafka/stream/KStreamContext.java | 3 + .../kafka/stream/examples/KStreamJob.java | 8 +- .../kafka/stream/internals/Receiver.java | 31 --- .../stream/topology/KStreamTopology.java | 45 ----- .../stream/topology/KStreamWindowed.java | 54 ----- .../kafka/stream/topology/KeyValue.java | 34 ---- .../kafka/stream/topology/KeyValueMapper.java | 23 --- .../topology/NotCopartitionedException.java | 28 --- .../kafka/stream/topology/Predicate.java | 24 --- .../kafka/stream/topology/Processor.java | 31 --- .../kafka/stream/topology/Transformer.java | 27 --- .../kafka/stream/topology/ValueJoiner.java | 23 --- .../kafka/stream/topology/ValueMapper.java | 23 --- .../apache/kafka/stream/topology/Window.java | 37 ---- .../topology/internals/KStreamBranch.java | 64 ------ .../topology/internals/KStreamFilter.java | 48 ----- .../topology/internals/KStreamFlatMap.java | 43 ---- .../internals/KStreamFlatMapValues.java | 41 ---- .../topology/internals/KStreamImpl.java | 191 ------------------ .../topology/internals/KStreamJoin.java | 139 ------------- .../stream/topology/internals/KStreamMap.java | 41 ---- .../topology/internals/KStreamMapValues.java | 40 ---- .../topology/internals/KStreamMetadata.java | 67 ------ .../topology/internals/KStreamSource.java | 28 --- .../topology/internals/KStreamWindow.java | 89 -------- .../kafka/stream/KStreamBranchTest.java | 6 +- .../kafka/stream/KStreamFilterTest.java | 6 +- .../kafka/stream/KStreamFlatMapTest.java | 7 +- .../stream/KStreamFlatMapValuesTest.java | 6 +- .../apache/kafka/stream/KStreamJoinTest.java | 11 +- .../apache/kafka/stream/KStreamMapTest.java | 7 +- .../kafka/stream/KStreamMapValuesTest.java | 5 + .../kafka/stream/KStreamSourceTest.java | 5 +- .../kafka/stream/KStreamTransformTest.java | 6 +- .../kafka/stream/KStreamWindowedTest.java | 6 +- .../apache/kafka/stream/StreamGroupTest.java | 2 +- .../apache/kafka/test/MockKStreamContext.java | 4 + .../kafka/test/MockKStreamTopology.java | 2 +- .../org/apache/kafka/test/MockProcessor.java | 2 +- .../apache/kafka/test/UnlimitedWindow.java | 4 +- temp | 7 +- temp2 | 7 +- 47 files changed, 68 insertions(+), 1247 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/Receiver.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/Processor.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/Window.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindow.java diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java index 02a9187f8c2c9..2add7efad4a30 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -67,6 +67,7 @@ public Deserializer valueDeser(String topic) { return source.valDeserializer; } + // TODO: init by the ordering of DAG public final void init(ProcessorContext context) { for (KafkaProcessor processor : processors) { processor.init(context); @@ -104,6 +105,7 @@ public final void addProcessor(KafkaProcessor processor, Kafk } } + // TODO: close by the ordering of DAG public final void close() { for (KafkaProcessor processor : processors) { processor.close(); diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java index a9c8b047a733e..0aba432e6bb92 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java @@ -248,6 +248,8 @@ private void addPartitions(Collection assignment) { streamGroups.add(context.streamGroup); } + + context.addPartition(partition); } nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java index a82387576212c..a019ee0f16e56 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java @@ -78,6 +78,11 @@ public ProcessorContextImpl(int id, this.processorConfig = processorConfig; this.timestampExtractor = this.streamingConfig.timestampExtractor(); + for (String topic : this.topology.topics()) { + if (!ingestor.topics().contains(topic)) + throw new IllegalArgumentException("topic not subscribed: " + topic); + } + File stateFile = new File(processorConfig.stateDir, Integer.toString(id)); Consumer restoreConsumer = new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); @@ -86,19 +91,16 @@ public ProcessorContextImpl(int id, stateMgr.init(); - // update the partition -> source stream map - for (String topic : this.topology.topics()) { - if (!ingestor.topics().contains(topic)) - throw new IllegalArgumentException("topic not subscribed: " + topic); + initialized = false; + } - TopicPartition partition = new TopicPartition(topic, id); - KafkaSource source = topology.source(topic); + public void addPartition(TopicPartition partition) { - this.streamGroup.addPartition(partition, source); - this.ingestor.addPartitionStreamToGroup(this.streamGroup, partition); - } + // update the partition -> source stream map + KafkaSource source = topology.source(partition.topic()); - initialized = false; + this.streamGroup.addPartition(partition, source); + this.ingestor.addPartitionStreamToGroup(this.streamGroup, partition); } @Override diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java index 55c8dff0c42b3..a54c7295af25b 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java @@ -30,6 +30,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.Set; /** * A StreamGroup is composed of multiple streams from different topics that need to be synchronized. @@ -80,6 +81,10 @@ public StampedRecord record() { return currRecord; } + public Set partitions() { + return stash.keySet(); + } + /** * Adds a partition and its receiver to this stream synchronizer * @@ -235,10 +240,6 @@ public void commitOffset() { this.commitRequested = true; } - public int buffered() { - return buffered; - } - public void close() { chooser.close(); stash.clear(); diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java index 8722539329e0f..5c90dccf05a46 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java @@ -21,11 +21,6 @@ import org.apache.kafka.clients.processor.Processor; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.stream.topology.KStreamWindowed; -import org.apache.kafka.stream.topology.KeyValueMapper; -import org.apache.kafka.stream.topology.Predicate; -import org.apache.kafka.stream.topology.ValueMapper; -import org.apache.kafka.stream.topology.Window; /** * KStream is an abstraction of a stream of key-value pairs. diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java b/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java index e11ccdb7c5971..ec5aefff170d4 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java @@ -32,8 +32,11 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +<<<<<<< HEAD import org.apache.kafka.stream.topology.Processor; >>>>>>> added missing files +======= +>>>>>>> wip import java.io.File; import java.util.Map; diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java index a6cdf5359301d..a3ee21c14a6be 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java @@ -21,10 +21,10 @@ import org.apache.kafka.clients.processor.KafkaStreaming; import org.apache.kafka.clients.processor.internals.StreamingConfig; import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.KeyValue; -import org.apache.kafka.stream.topology.KeyValueMapper; -import org.apache.kafka.stream.topology.Predicate; +import org.apache.kafka.stream.KStreamTopology; +import org.apache.kafka.stream.KeyValue; +import org.apache.kafka.stream.KeyValueMapper; +import org.apache.kafka.stream.Predicate; import java.util.Properties; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/Receiver.java b/stream/src/main/java/org/apache/kafka/stream/internals/Receiver.java deleted file mode 100644 index ef933bd5ad4b6..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/Receiver.java +++ /dev/null @@ -1,31 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; - -public interface Receiver { - - void bind(KStreamContext context, KStreamMetadata metadata); - - void receive(Object key, Object value, long timestamp); - - void close(); - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java deleted file mode 100644 index 004d08f8e6872..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamTopology.java +++ /dev/null @@ -1,45 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.topology.internals.KStreamSource; - -/** - * KStreamTopology is the class that allows an implementation of {@link KStreamTopology#build()} to create KStream instances. - */ -public abstract class KStreamTopology extends PTopology { - - /** - * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. - * - * @param keyDeserializer key deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param valDeserializer value deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream - */ - public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KafkaProcessor source = addSource(keyDeserializer, valDeserializer, topics); - return new KStreamSource<>(this, source); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java b/stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java deleted file mode 100644 index d4ddda679ddbb..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KStreamWindowed.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -import org.apache.kafka.stream.KStream; - -/** - * KStreamWindowed is an abstraction of a stream of key-value pairs with a window. - */ -public interface KStreamWindowed extends KStream { - - /** - * Creates a new stream by joining this windowed stream with the other windowed stream. - * Each element arrived from either of the streams is joined with elements in a window of each other. - * The resulting values are computed by applying a joiner. - * - * @param other the other windowed stream - * @param joiner ValueJoiner - * @param the value type of the other stream - * @param the value type of the new stream - * @return KStream - */ - KStream join(KStreamWindowed other, ValueJoiner joiner); - - /** - * 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 if - * the element from the other stream has an older timestamp. - * The resulting values are computed by applying a joiner. - * - * @param other the other windowed stream - * @param joiner the instance ValueJoiner - * @param the value type of the other stream - * @param the value type of the new stream - * @return KStream - */ - KStream joinPrior(KStreamWindowed other, ValueJoiner joiner); - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java deleted file mode 100644 index 37a99d062db69..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValue.java +++ /dev/null @@ -1,34 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -public class KeyValue { - - public final K key; - public final V value; - - public KeyValue(K key, V value) { - this.key = key; - this.value = value; - } - - public static KeyValue pair(K key, V value) { - return new KeyValue(key, value); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java deleted file mode 100644 index ccaf1f0b314f0..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/KeyValueMapper.java +++ /dev/null @@ -1,23 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -public interface KeyValueMapper { - - KeyValue apply(K1 key, V1 value); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java b/stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java deleted file mode 100644 index 1045995408d70..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/NotCopartitionedException.java +++ /dev/null @@ -1,28 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -import org.apache.kafka.clients.processor.KStreamException; - -public class NotCopartitionedException extends KStreamException { - - public NotCopartitionedException() { - super(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java b/stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java deleted file mode 100644 index ae9b1e8072fbe..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Predicate.java +++ /dev/null @@ -1,24 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -public interface Predicate { - - boolean apply(K key, V value); - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Processor.java b/stream/src/main/java/org/apache/kafka/stream/topology/Processor.java deleted file mode 100644 index 01fe371983bb0..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Processor.java +++ /dev/null @@ -1,31 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -import org.apache.kafka.stream.KStreamContext; - -public interface Processor { - - void init(KStreamContext context); - - void process(K key, V value); - - void punctuate(long streamTime); - - void close(); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java b/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java deleted file mode 100644 index 24b01cdc24405..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Transformer.java +++ /dev/null @@ -1,27 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -public interface Transformer extends Processor { - - interface Forwarder { - public void send(K key, V value, long timestamp); - } - - public void forwarder(Forwarder forwarder); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java b/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java deleted file mode 100644 index bc021b6d9e0c3..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/ValueJoiner.java +++ /dev/null @@ -1,23 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -public interface ValueJoiner { - - R apply(V1 value1, V2 value2); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java deleted file mode 100644 index 401c05e004caa..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/ValueMapper.java +++ /dev/null @@ -1,23 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -public interface ValueMapper { - - V2 apply(V1 value); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/Window.java b/stream/src/main/java/org/apache/kafka/stream/topology/Window.java deleted file mode 100644 index 708d5eb86831f..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/Window.java +++ /dev/null @@ -1,37 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.clients.processor.StateStore; - -import java.util.Iterator; - -public interface Window extends StateStore { - - void init(KStreamContext context); - - Iterator find(K key, long timestamp); - - Iterator findAfter(K key, long timestamp); - - Iterator findBefore(K key, long timestamp); - - void put(K key, V value, long timestamp); - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java deleted file mode 100644 index 55d90db535e5c..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamBranch.java +++ /dev/null @@ -1,64 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.topology.Predicate; - -import java.lang.reflect.Array; -import java.util.Arrays; - -class KStreamBranch extends KafkaProcessor { - - private static final String BRANCH_NAME = "KAFKA-BRANCH"; - - private final PTopology topology; - private final Predicate[] predicates; - private final KStreamFilter[] branches; - - @SuppressWarnings("unchecked") - public KStreamBranch(Predicate[] predicates, PTopology topology, KafkaProcessor parent) { - super(BRANCH_NAME); - - this.topology = topology; - this.predicates = Arrays.copyOf(predicates, predicates.length); - this.branches = (KStreamFilter[]) Array.newInstance(KStreamFilter.class, predicates.length); - - // NOTE that branches here is just a list of predicates, hence not necessarily mutual exclusive - for (int i = 0; i < branches.length; i++) { - branches[i] = new KStreamFilter<>(predicates[i], false); - topology.addProcessor(branches[i], parent); - } - } - - @Override - public void process(K key, V value) { - forward(key, value); - } - - @SuppressWarnings("unchecked") - public KStream[] branches() { - KStream[] streams = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); - for (int i = 0; i < branches.length; i++) { - streams[i] = new KStreamImpl<>(topology, branches[i]); - } - return streams; - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java deleted file mode 100644 index b22a87c86d5ed..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFilter.java +++ /dev/null @@ -1,48 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.stream.topology.Predicate; - -class KStreamFilter extends KafkaProcessor { - - private static final String FILTER_NAME = "KAFKA-FILTER"; - - private final Predicate predicate; - private final boolean filterOut; - - public KStreamFilter(Predicate predicate) { - this(predicate, false); - } - - public KStreamFilter(Predicate predicate, boolean filterOut) { - super(FILTER_NAME); - - this.predicate = predicate; - this.filterOut = filterOut; - } - - @Override - public void process(K key, V value) { - if ((!filterOut && predicate.apply(key, value)) - || (filterOut && !predicate.apply(key, value))) { - forward(key, value); - } - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java deleted file mode 100644 index b48e700ebe179..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMap.java +++ /dev/null @@ -1,43 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.stream.topology.KeyValue; -import org.apache.kafka.stream.topology.KeyValueMapper; - -class KStreamFlatMap extends KafkaProcessor { - - private static final String FLATMAP_NAME = "KAFKA-FLATMAP"; - - private final KeyValueMapper> mapper; - - KStreamFlatMap(KeyValueMapper> mapper) { - super(FLATMAP_NAME); - - this.mapper = mapper; - } - - @Override - public void process(K1 key, V1 value) { - KeyValue> newPair = mapper.apply(key, value); - for (V2 v : newPair.value) { - forward(newPair.key, v); - } - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java deleted file mode 100644 index fe496fe44a027..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamFlatMapValues.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.stream.topology.ValueMapper; - -class KStreamFlatMapValues extends KafkaProcessor { - - private static final String FLATMAPVALUES_NAME = "KAFKA-FLATMAPVALUES"; - - private final ValueMapper> mapper; - - KStreamFlatMapValues(ValueMapper> mapper) { - super(FLATMAPVALUES_NAME); - this.mapper = mapper; - } - - @Override - public void process(K1 key, V1 value) { - Iterable newValues = mapper.apply(value); - for (V2 v : newValues) { - forward(key, v); - } - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java deleted file mode 100644 index 54c9407159743..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamImpl.java +++ /dev/null @@ -1,191 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.Processor; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.clients.processor.internals.KafkaSource; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.KStream; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.stream.topology.KStreamWindowed; -import org.apache.kafka.stream.topology.KeyValueMapper; -import org.apache.kafka.stream.topology.Predicate; -import org.apache.kafka.stream.topology.ValueMapper; -import org.apache.kafka.stream.topology.Window; - -public class KStreamImpl implements KStream { - - private static final String PROCESSOR_NAME = "KAFKA-PROCESS"; - - protected PTopology topology; - protected KafkaProcessor processor; - protected KStreamContext context; - protected KStreamMetadata metadata; - - public KStreamImpl(PTopology topology, KafkaProcessor processor) { - this.topology = topology; - this.processor = processor; - } - - @SuppressWarnings("unchecked") - @Override - public KStream filter(Predicate predicate) { - KStreamFilter filter = new KStreamFilter<>(predicate); - - topology.addProcessor(filter, processor); - - return new KStreamImpl<>(topology, filter); - } - - @SuppressWarnings("unchecked") - @Override - public KStream filterOut(final Predicate predicate) { - KStreamFilter filter = new KStreamFilter<>(predicate, true); - - topology.addProcessor(filter, processor); - - return new KStreamImpl<>(topology, filter); - } - - @SuppressWarnings("unchecked") - @Override - public KStream map(KeyValueMapper mapper) { - KStreamMap map = new KStreamMap<>(mapper); - - topology.addProcessor(map, processor); - - return new KStreamImpl<>(topology, map); - } - - @SuppressWarnings("unchecked") - @Override - public KStream mapValues(ValueMapper mapper) { - KStreamMapValues map = new KStreamMapValues<>(mapper); - - topology.addProcessor(map, processor); - - return new KStreamImpl<>(topology, map); - } - - @SuppressWarnings("unchecked") - @Override - public KStream flatMap(KeyValueMapper> mapper) { - KStreamFlatMap map = new KStreamFlatMap<>(mapper); - - topology.addProcessor(map, processor); - - return new KStreamImpl<>(topology, map); - } - - @SuppressWarnings("unchecked") - @Override - public KStream flatMapValues(ValueMapper> mapper) { - KStreamFlatMapValues map = new KStreamFlatMapValues<>(mapper); - - topology.addProcessor(map, processor); - - return new KStreamImpl<>(topology, map); - } - - @Override - public KStreamWindowed with(Window window) { - return (KStreamWindowed) chain(new KStreamWindow<>(window, topology)); - } - - @Override - public KStream[] branch(Predicate... predicates) { - KStreamBranch branch = new KStreamBranch<>(predicates, topology, processor); - return branch.branches(); - } - - @SuppressWarnings("unchecked") - @Override - public KStream through(String topic) { - return through(topic, null, null, null, null); - } - - @SuppressWarnings("unchecked") - @Override - public KStream through(String topic, - Serializer keySerializer, - Serializer valSerializer, - Deserializer keyDeserializer, - Deserializer valDeserializer) { - process(this.getSendProcessor(topic, keySerializer, valSerializer)); - - KafkaSource source = topology.addSource(keyDeserializer, valDeserializer, topic); - - return new KStreamSource<>(topology, source); - } - - @Override - public void sendTo(String topic) { - process(this.getSendProcessor(topic, null, null)); - } - - @Override - public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { - process(this.getSendProcessor(topic, keySerializer, valSerializer)); - } - - @SuppressWarnings("unchecked") - private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { - return new Processor() { - private ProcessorContext context; - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void process(K1 key, V1 value) { - this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); - } - - @Override - public void close() { - } - }; - } - - @SuppressWarnings("unchecked") - @Override - public KStream process(final KafkaProcessor current) { - topology.addProcessor(current, processor); - - return new KStreamImpl(topology, current); - } - - @SuppressWarnings("unchecked") - @Override - public void process(final Processor current) { - KafkaProcessor wrapper = new KafkaProcessor(PROCESSOR_NAME) { - @Override - public void process(K key, V value) { - current.process(key, value); - } - }; - - topology.addProcessor(wrapper, processor); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java deleted file mode 100644 index e5b07ffc89d85..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamJoin.java +++ /dev/null @@ -1,139 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.internals.Receiver; -import org.apache.kafka.stream.topology.NotCopartitionedException; -import org.apache.kafka.stream.topology.ValueJoiner; -import org.apache.kafka.stream.topology.Window; - -import java.util.Iterator; - -class KStreamJoin extends KafkaProcessor { - - private static final String JOIN_NAME = "KAFKA-JOIN"; - - private static abstract class Finder { - abstract Iterator find(K key, long timestamp); - } - - private KStreamMetadata thisMetadata; - private KStreamMetadata otherMetadata; - private final Finder finder1; - private final Finder finder2; - private final ValueJoiner joiner; - final Receiver receiverForOtherStream; - - private ProcessorContext context; - - KStreamJoin(KStreamWindow stream1, KStreamWindow stream2, boolean prior, ValueJoiner joiner) { - super(JOIN_NAME); - - final Window window1 = stream1.window(); - final Window window2 = stream2.window(); - - if (prior) { - this.finder1 = new Finder() { - Iterator find(K key, long timestamp) { - return window1.findAfter(key, timestamp); - } - }; - this.finder2 = new Finder() { - Iterator find(K key, long timestamp) { - return window2.findBefore(key, timestamp); - } - }; - } else { - this.finder1 = new Finder() { - Iterator find(K key, long timestamp) { - return window1.find(key, timestamp); - } - }; - this.finder2 = new Finder() { - Iterator find(K key, long timestamp) { - return window2.find(key, timestamp); - } - }; - } - - this.joiner = joiner; - - this.receiverForOtherStream = getReceiverForOther(); - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - super.bind(context, metadata); - - thisMetadata = metadata; - if (otherMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) - throw new NotCopartitionedException(); - } - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void process(K key, V1 value) { - long timestamp = context.timestamp(); - Iterator iter = finder2.find(key, timestamp); - if (iter != null) { - while (iter.hasNext()) { - doJoin(key,value, iter.next()); - } - } - } - - private Receiver getReceiverForOther() { - return new Receiver() { - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - otherMetadata = metadata; - if (thisMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) - throw new NotCopartitionedException(); - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value2, long timestamp) { - Iterator iter = finder1.find((K) key, timestamp); - if (iter != null) { - while (iter.hasNext()) { - doJoin((K) key, iter.next(), (V2) value2, timestamp); - } - } - } - - @Override - public void close() { - // down stream instances are close when the primary stream is closed - } - }; - } - - // TODO: use the "outer-stream" topic as the resulted join stream topic - private void doJoin(K key, V1 value1, V2 value2) { - forward(key, joiner.apply(value1, value2)); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java deleted file mode 100644 index 1833e1f7391cd..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMap.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.stream.topology.KeyValue; -import org.apache.kafka.stream.topology.KeyValueMapper; - -class KStreamMap extends KafkaProcessor { - - private static final String MAP_NAME = "KAFKA-MAP"; - - private final KeyValueMapper mapper; - - public KStreamMap(KeyValueMapper mapper) { - super(MAP_NAME); - - this.mapper = mapper; - } - - @Override - public void process(K1 key, V1 value) { - KeyValue newPair = mapper.apply(key, value); - forward(newPair.key, newPair.value); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java deleted file mode 100644 index 61e5d466ffb7d..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMapValues.java +++ /dev/null @@ -1,40 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.stream.topology.ValueMapper; - -class KStreamMapValues extends KafkaProcessor { - - private static final String MAPVALUES_NAME = "KAFKA-MAPVALUES"; - - private final ValueMapper mapper; - - public KStreamMapValues(ValueMapper mapper) { - super(MAPVALUES_NAME); - - this.mapper = mapper; - } - - @Override - public void process(K1 key, V1 value) { - V2 newValue = mapper.apply(value); - forward(key, newValue); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java deleted file mode 100644 index 936525d8fa792..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamMetadata.java +++ /dev/null @@ -1,67 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.clients.processor.internals.StreamGroup; - -import java.util.Collections; -import java.util.Map; - -public class KStreamMetadata { - - public static final String UNKNOWN_TOPICNAME = "__UNKNOWN_TOPIC__"; - public static final int UNKNOWN_PARTITION = -1; - - public static KStreamMetadata unjoinable() { - return new KStreamMetadata(Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(UNKNOWN_PARTITION))); - } - - public StreamGroup streamGroup; - public final Map topicPartitionInfos; - - public KStreamMetadata(Map topicPartitionInfos) { - this.topicPartitionInfos = topicPartitionInfos; - } - - boolean isJoinCompatibleWith(KStreamMetadata other) { - // the two streams should only be joinable if they are inside the same sync group - // and their contained streams all have the same number of partitions - if (this.streamGroup != other.streamGroup) - return false; - - int numPartitions = -1; - for (PartitioningInfo partitionInfo : this.topicPartitionInfos.values()) { - if (partitionInfo.numPartitions < 0) { - return false; - } else if (numPartitions >= 0) { - if (partitionInfo.numPartitions != numPartitions) - return false; - } else { - numPartitions = partitionInfo.numPartitions; - } - } - - for (PartitioningInfo partitionInfo : other.topicPartitionInfos.values()) { - if (partitionInfo.numPartitions != numPartitions) - return false; - } - - return true; - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java deleted file mode 100644 index cfebb3cc0cfb3..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamSource.java +++ /dev/null @@ -1,28 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; - -public class KStreamSource extends KStreamImpl { - - public KStreamSource(PTopology topology, KafkaProcessor source) { - super(topology, source); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindow.java deleted file mode 100644 index 38e31f0c041df..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamWindow.java +++ /dev/null @@ -1,89 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.topology.KStreamWindowed; -import org.apache.kafka.stream.topology.ValueJoiner; -import org.apache.kafka.stream.topology.Window; - -public class KStreamWindow extends KafkaProcessor { - - private static final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { - - public KStreamWindowedImpl(PTopology topology, KafkaProcessor processor) { - super(topology, processor); - } - - @Override - public KStream join(KStreamWindowed other, ValueJoiner processor) { - return join(other, false, processor); - } - - @Override - public KStream joinPrior(KStreamWindowed other, ValueJoiner processor) { - return join(other, true, processor); - } - - private KStream join(KStreamWindowed other, boolean prior, ValueJoiner processor) { - - KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; - - KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, topology); - otherImpl.registerReceiver(stream.receiverForOtherStream); - - return chain(stream); - } - } - - private static final String WINDOW_NAME = "KAFKA-WINDOW"; - - private final Window window; - private ProcessorContext context; - - KStreamWindow(Window window) { - super(WINDOW_NAME); - this.window = window; - } - - public Window window() { - return window; - } - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @SuppressWarnings("unchecked") - @Override - public void process(K key, V value) { - synchronized (this) { - window.put(key, value, context.timestamp()); - forward(key, value); - } - } - - @Override - public void close() { - window.close(); - } -} diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java index 9dab0d4015883..4cd2b5db2bf75 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java @@ -18,10 +18,8 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.Predicate; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamContext; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java index f6142c97ecb1b..276ee1fb62cac 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java @@ -18,10 +18,8 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.Predicate; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java index bcc49b8b95e1e..cba15b27ada30 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java @@ -18,11 +18,8 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.KeyValue; -import org.apache.kafka.stream.topology.KeyValueMapper; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java index 58b249344554f..1158b3e154929 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java @@ -18,10 +18,8 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.ValueMapper; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java index 8b66959bd2efb..92ae9adc52226 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java @@ -19,18 +19,11 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.KStreamWindowed; -import org.apache.kafka.stream.topology.KeyValue; -import org.apache.kafka.stream.topology.KeyValueMapper; -import org.apache.kafka.stream.topology.ValueJoiner; -import org.apache.kafka.stream.topology.ValueMapper; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; -import org.apache.kafka.stream.topology.NotCopartitionedException; import org.apache.kafka.test.UnlimitedWindow; import org.junit.Test; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java index 7aaf496a6dfd8..9c36f219c87b7 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java @@ -18,11 +18,8 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.KeyValue; -import org.apache.kafka.stream.topology.KeyValueMapper; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java index 7e081f8827622..79c7e12e4dddb 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java @@ -42,11 +42,16 @@ >>>>>>> compile and test passed ======= import org.apache.kafka.clients.processor.internals.PartitioningInfo; +<<<<<<< HEAD >>>>>>> wip import org.apache.kafka.stream.topology.KStreamTopology; import org.apache.kafka.stream.topology.ValueMapper; import org.apache.kafka.stream.topology.internals.KStreamMetadata; import org.apache.kafka.stream.topology.internals.KStreamSource; +======= +import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.stream.internals.KStreamSource; +>>>>>>> wip import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java index 7ce80f451b26d..ed3c0c6df1ccb 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java @@ -18,9 +18,8 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockKStreamContext; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java index fb4755510731b..667926dda8a51 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java @@ -18,10 +18,8 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.Transformer; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamContext; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java index 21ce70c16f063..2fe37940d1e74 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java @@ -18,10 +18,8 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.Window; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamContext; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.UnlimitedWindow; diff --git a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java index 3cc89a78d3615..ee49cda357182 100644 --- a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java @@ -26,7 +26,7 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.clients.processor.internals.StreamGroup; import org.apache.kafka.clients.processor.internals.TimeBasedChooser; -import org.apache.kafka.stream.topology.internals.KStreamSource; +import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockIngestor; import org.apache.kafka.test.MockKStreamContext; import org.apache.kafka.test.MockKStreamTopology; diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java index b49d3e3c7e7e7..51f6d683937b7 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java @@ -17,6 +17,7 @@ package org.apache.kafka.test; +<<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamContext; import io.confluent.streaming.Processor; @@ -50,6 +51,9 @@ >>>>>>> new api model ======= import org.apache.kafka.stream.topology.Processor; +======= +import org.apache.kafka.stream.Processor; +>>>>>>> wip import org.apache.kafka.stream.KStreamContext; <<<<<<< HEAD import org.apache.kafka.stream.RecordCollector; diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java index bf61a6ecbfe94..413fd349b5240 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java @@ -17,7 +17,7 @@ package org.apache.kafka.test; -import org.apache.kafka.stream.topology.KStreamTopology; +import org.apache.kafka.stream.KStreamTopology; public class MockKStreamTopology extends KStreamTopology { @Override diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java index 3b1e9065aad5a..f06248559e303 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -17,7 +17,7 @@ package org.apache.kafka.test; -import org.apache.kafka.stream.topology.Processor; +import org.apache.kafka.stream.Processor; import org.apache.kafka.stream.KStreamContext; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java index cf3d66f7266fa..4a50fe871cd55 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java @@ -18,8 +18,8 @@ package org.apache.kafka.test; import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.topology.KeyValue; -import org.apache.kafka.stream.topology.Window; +import org.apache.kafka.stream.KeyValue; +import org.apache.kafka.stream.Window; import org.apache.kafka.stream.util.FilteredIterator; import org.apache.kafka.clients.processor.internals.Stamped; diff --git a/temp b/temp index b260c1577d816..966d60b3355fc 100644 --- a/temp +++ b/temp @@ -1,3 +1,4 @@ -stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java: needs merge -stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java: needs merge -stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java: needs merge +stream/src/main/java/org/apache/kafka/stream/KStreamContext.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java: needs merge +stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java: needs merge diff --git a/temp2 b/temp2 index d5a8f9c139fbc..4427ad7768ce6 100644 --- a/temp2 +++ b/temp2 @@ -1,3 +1,4 @@ -stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java -stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java -stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java +stream/src/main/java/org/apache/kafka/stream/KStreamContext.java +stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java From 5eb024001e425d456b83ba001d177ac58902cda0 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 7 Aug 2015 23:26:57 -0700 Subject: [PATCH 150/275] add folder --- .../apache/kafka/stream/KStreamTopology.java | 44 ++++ .../apache/kafka/stream/KStreamWindowed.java | 52 +++++ .../org/apache/kafka/stream/KeyValue.java | 34 +++ .../apache/kafka/stream/KeyValueMapper.java | 23 +++ .../stream/NotCopartitionedException.java | 28 +++ .../org/apache/kafka/stream/Predicate.java | 24 +++ .../org/apache/kafka/stream/Processor.java | 31 +++ .../apache/kafka/stream/SlidingWindow.java | 146 +++++++++++++ .../org/apache/kafka/stream/Transformer.java | 27 +++ .../org/apache/kafka/stream/ValueJoiner.java | 23 +++ .../org/apache/kafka/stream/ValueMapper.java | 23 +++ .../java/org/apache/kafka/stream/Window.java | 37 ++++ .../kafka/stream/internals/KStreamBranch.java | 64 ++++++ .../kafka/stream/internals/KStreamFilter.java | 48 +++++ .../stream/internals/KStreamFlatMap.java | 43 ++++ .../internals/KStreamFlatMapValues.java | 41 ++++ .../kafka/stream/internals/KStreamImpl.java | 194 ++++++++++++++++++ .../kafka/stream/internals/KStreamJoin.java | 166 +++++++++++++++ .../kafka/stream/internals/KStreamMap.java | 41 ++++ .../stream/internals/KStreamMapValues.java | 40 ++++ .../stream/internals/KStreamMetadata.java | 67 ++++++ .../kafka/stream/internals/KStreamSource.java | 28 +++ .../kafka/stream/internals/KStreamWindow.java | 96 +++++++++ 23 files changed, 1320 insertions(+) create mode 100644 stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/KStreamWindowed.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/KeyValue.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/KeyValueMapper.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/Predicate.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/Processor.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/Transformer.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/ValueJoiner.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/ValueMapper.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/Window.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamFilter.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMap.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMapValues.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamMap.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamMapValues.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamMetadata.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java new file mode 100644 index 0000000000000..7745b3168b826 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.internals.KStreamSource; + +/** + * KStreamTopology is the class that allows an implementation of {@link KStreamTopology#build()} to create KStream instances. + */ +public abstract class KStreamTopology extends PTopology { + + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * + * @param keyDeserializer key deserializer used to read this source KStream, + * if not specified the default deserializer defined in the configs will be used + * @param valDeserializer value deserializer used to read this source KStream, + * if not specified the default deserializer defined in the configs will be used + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + KafkaProcessor source = addSource(keyDeserializer, valDeserializer, topics); + return new KStreamSource<>(this, source); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamWindowed.java b/stream/src/main/java/org/apache/kafka/stream/KStreamWindowed.java new file mode 100644 index 0000000000000..e3660b0f6f8ca --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamWindowed.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +/** + * KStreamWindowed is an abstraction of a stream of key-value pairs with a window. + */ +public interface KStreamWindowed extends KStream { + + /** + * Creates a new stream by joining this windowed stream with the other windowed stream. + * Each element arrived from either of the streams is joined with elements in a window of each other. + * The resulting values are computed by applying a joiner. + * + * @param other the other windowed stream + * @param joiner ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream + * @return KStream + */ + KStream join(KStreamWindowed other, ValueJoiner joiner); + + /** + * 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 if + * the element from the other stream has an older timestamp. + * The resulting values are computed by applying a joiner. + * + * @param other the other windowed stream + * @param joiner the instance ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream + * @return KStream + */ + KStream joinPrior(KStreamWindowed other, ValueJoiner joiner); + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/KeyValue.java b/stream/src/main/java/org/apache/kafka/stream/KeyValue.java new file mode 100644 index 0000000000000..f81c68cb3d41e --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/KeyValue.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +public class KeyValue { + + public final K key; + public final V value; + + public KeyValue(K key, V value) { + this.key = key; + this.value = value; + } + + public static KeyValue pair(K key, V value) { + return new KeyValue(key, value); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/KeyValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/KeyValueMapper.java new file mode 100644 index 0000000000000..9f8fbc4e7a458 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/KeyValueMapper.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +public interface KeyValueMapper { + + KeyValue apply(K1 key, V1 value); +} diff --git a/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java b/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java new file mode 100644 index 0000000000000..b1f34ebd15303 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +import org.apache.kafka.clients.processor.KStreamException; + +public class NotCopartitionedException extends KStreamException { + + public NotCopartitionedException() { + super(); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/Predicate.java b/stream/src/main/java/org/apache/kafka/stream/Predicate.java new file mode 100644 index 0000000000000..932f3ea267322 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/Predicate.java @@ -0,0 +1,24 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +public interface Predicate { + + boolean apply(K key, V value); + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/Processor.java b/stream/src/main/java/org/apache/kafka/stream/Processor.java new file mode 100644 index 0000000000000..c97f8dbccb818 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/Processor.java @@ -0,0 +1,31 @@ +/** + * 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.stream; + +import org.apache.kafka.stream.KStreamContext; + +public interface Processor { + + void init(KStreamContext context); + + void process(K key, V value); + + void punctuate(long streamTime); + + void close(); +} diff --git a/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java new file mode 100644 index 0000000000000..04ddb17f576af --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java @@ -0,0 +1,146 @@ +/** + * 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.stream; + +import org.apache.kafka.stream.util.FilteredIterator; +import org.apache.kafka.clients.processor.internals.Stamped; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; + +public class SlidingWindow implements Window { + + private String name; + private final long duration; + private final int maxCount; + private LinkedList list = new LinkedList(); + private HashMap>> map = new HashMap>>(); + + public SlidingWindow(String name, long duration, int maxCount) { + this.name = name; + this.duration = duration; + this.maxCount = maxCount; + } + + @Override + public void init(KStreamContext context) { + } + + @Override + public Iterator findAfter(K key, final long timestamp) { + return find(key, timestamp, timestamp + duration); + } + + @Override + public Iterator findBefore(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp); + } + + @Override + public Iterator find(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp + duration); + } + + /* + * finds items in the window between startTime and endTime (both inclusive) + */ + private Iterator find(K key, final long startTime, final long endTime) { + final LinkedList> values = map.get(key); + + if (values == null) { + return null; + } else { + return new FilteredIterator>(values.iterator()) { + @Override + protected V filter(Stamped item) { + if (startTime <= item.timestamp && item.timestamp <= endTime) + return item.value; + else + return null; + } + }; + } + } + + @Override + public void put(K key, V value, long timestamp) { + list.offerLast(key); + + LinkedList> values = map.get(key); + if (values == null) { + values = new LinkedList>(); + map.put(key, values); + } + + values.offerLast(new Stamped(value, timestamp)); + + evictExcess(); + evictExpired(timestamp - duration); + } + + private void evictExcess() { + while (list.size() > maxCount) { + K oldestKey = list.pollFirst(); + + LinkedList> values = map.get(oldestKey); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } + } + + private void evictExpired(long cutoffTime) { + while (true) { + K oldestKey = list.peekFirst(); + + LinkedList> values = map.get(oldestKey); + Stamped oldestValue = values.peekFirst(); + + 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() { + // TODO + } + + @Override + public void close() { + // TODO + } + + @Override + public boolean persistent() { + // TODO: should not be persistent, right? + return false; + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/Transformer.java b/stream/src/main/java/org/apache/kafka/stream/Transformer.java new file mode 100644 index 0000000000000..565be2637eb4f --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/Transformer.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +public interface Transformer extends Processor { + + interface Forwarder { + public void send(K key, V value, long timestamp); + } + + public void forwarder(Forwarder forwarder); +} diff --git a/stream/src/main/java/org/apache/kafka/stream/ValueJoiner.java b/stream/src/main/java/org/apache/kafka/stream/ValueJoiner.java new file mode 100644 index 0000000000000..b161efb650d6b --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/ValueJoiner.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +public interface ValueJoiner { + + R apply(V1 value1, V2 value2); +} diff --git a/stream/src/main/java/org/apache/kafka/stream/ValueMapper.java b/stream/src/main/java/org/apache/kafka/stream/ValueMapper.java new file mode 100644 index 0000000000000..a83415570064e --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/ValueMapper.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +public interface ValueMapper { + + V2 apply(V1 value); +} diff --git a/stream/src/main/java/org/apache/kafka/stream/Window.java b/stream/src/main/java/org/apache/kafka/stream/Window.java new file mode 100644 index 0000000000000..1eccdb1cf6749 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/Window.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.clients.processor.StateStore; + +import java.util.Iterator; + +public interface Window extends StateStore { + + void init(KStreamContext context); + + Iterator find(K key, long timestamp); + + Iterator findAfter(K key, long timestamp); + + Iterator findBefore(K key, long timestamp); + + void put(K key, V value, long timestamp); + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java new file mode 100644 index 0000000000000..f4ce6bd23029b --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.stream.KStream; +import org.apache.kafka.stream.Predicate; + +import java.lang.reflect.Array; +import java.util.Arrays; + +class KStreamBranch extends KafkaProcessor { + + private static final String BRANCH_NAME = "KAFKA-BRANCH"; + + private final PTopology topology; + private final Predicate[] predicates; + private final KStreamFilter[] branches; + + @SuppressWarnings("unchecked") + public KStreamBranch(Predicate[] predicates, PTopology topology, KafkaProcessor parent) { + super(BRANCH_NAME); + + this.topology = topology; + this.predicates = Arrays.copyOf(predicates, predicates.length); + this.branches = (KStreamFilter[]) Array.newInstance(KStreamFilter.class, predicates.length); + + // NOTE that branches here is just a list of predicates, hence not necessarily mutual exclusive + for (int i = 0; i < branches.length; i++) { + branches[i] = new KStreamFilter<>(predicates[i], false); + topology.addProcessor(branches[i], parent); + } + } + + @Override + public void process(K key, V value) { + forward(key, value); + } + + @SuppressWarnings("unchecked") + public KStream[] branches() { + KStream[] streams = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); + for (int i = 0; i < branches.length; i++) { + streams[i] = new KStreamImpl<>(topology, branches[i]); + } + return streams; + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFilter.java new file mode 100644 index 0000000000000..6030ecea2cc56 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFilter.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.stream.Predicate; + +class KStreamFilter extends KafkaProcessor { + + private static final String FILTER_NAME = "KAFKA-FILTER"; + + private final Predicate predicate; + private final boolean filterOut; + + public KStreamFilter(Predicate predicate) { + this(predicate, false); + } + + public KStreamFilter(Predicate predicate, boolean filterOut) { + super(FILTER_NAME); + + this.predicate = predicate; + this.filterOut = filterOut; + } + + @Override + public void process(K key, V value) { + if ((!filterOut && predicate.apply(key, value)) + || (filterOut && !predicate.apply(key, value))) { + forward(key, value); + } + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMap.java new file mode 100644 index 0000000000000..a78b594689003 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMap.java @@ -0,0 +1,43 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.stream.KeyValue; +import org.apache.kafka.stream.KeyValueMapper; + +class KStreamFlatMap extends KafkaProcessor { + + private static final String FLATMAP_NAME = "KAFKA-FLATMAP"; + + private final KeyValueMapper> mapper; + + KStreamFlatMap(KeyValueMapper> mapper) { + super(FLATMAP_NAME); + + this.mapper = mapper; + } + + @Override + public void process(K1 key, V1 value) { + KeyValue> newPair = mapper.apply(key, value); + for (V2 v : newPair.value) { + forward(newPair.key, v); + } + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMapValues.java new file mode 100644 index 0000000000000..ded053910f121 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMapValues.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.stream.ValueMapper; + +class KStreamFlatMapValues extends KafkaProcessor { + + private static final String FLATMAPVALUES_NAME = "KAFKA-FLATMAPVALUES"; + + private final ValueMapper> mapper; + + KStreamFlatMapValues(ValueMapper> mapper) { + super(FLATMAPVALUES_NAME); + this.mapper = mapper; + } + + @Override + public void process(K1 key, V1 value) { + Iterable newValues = mapper.apply(value); + for (V2 v : newValues) { + forward(key, v); + } + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java new file mode 100644 index 0000000000000..2895384c58e28 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java @@ -0,0 +1,194 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.Processor; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.internals.KafkaSource; +import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.stream.KStream; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.stream.KStreamWindowed; +import org.apache.kafka.stream.KeyValueMapper; +import org.apache.kafka.stream.Predicate; +import org.apache.kafka.stream.ValueMapper; +import org.apache.kafka.stream.Window; + +public class KStreamImpl implements KStream { + + private static final String PROCESSOR_NAME = "KAFKA-PROCESS"; + + protected PTopology topology; + protected KafkaProcessor processor; + protected KStreamContext context; + + public KStreamImpl(PTopology topology, KafkaProcessor processor) { + this.topology = topology; + this.processor = processor; + } + + @SuppressWarnings("unchecked") + @Override + public KStream filter(Predicate predicate) { + KStreamFilter filter = new KStreamFilter<>(predicate); + + topology.addProcessor(filter, processor); + + return new KStreamImpl<>(topology, filter); + } + + @SuppressWarnings("unchecked") + @Override + public KStream filterOut(final Predicate predicate) { + KStreamFilter filter = new KStreamFilter<>(predicate, true); + + topology.addProcessor(filter, processor); + + return new KStreamImpl<>(topology, filter); + } + + @SuppressWarnings("unchecked") + @Override + public KStream map(KeyValueMapper mapper) { + KStreamMap map = new KStreamMap<>(mapper); + + topology.addProcessor(map, processor); + + return new KStreamImpl<>(topology, map); + } + + @SuppressWarnings("unchecked") + @Override + public KStream mapValues(ValueMapper mapper) { + KStreamMapValues map = new KStreamMapValues<>(mapper); + + topology.addProcessor(map, processor); + + return new KStreamImpl<>(topology, map); + } + + @SuppressWarnings("unchecked") + @Override + public KStream flatMap(KeyValueMapper> mapper) { + KStreamFlatMap map = new KStreamFlatMap<>(mapper); + + topology.addProcessor(map, processor); + + return new KStreamImpl<>(topology, map); + } + + @SuppressWarnings("unchecked") + @Override + public KStream flatMapValues(ValueMapper> mapper) { + KStreamFlatMapValues map = new KStreamFlatMapValues<>(mapper); + + topology.addProcessor(map, processor); + + return new KStreamImpl<>(topology, map); + } + + @Override + public KStreamWindowed with(Window window) { + KStreamWindow windowed = new KStreamWindow<>(window); + + topology.addProcessor(windowed, processor); + + return new KStreamWindow.KStreamWindowedImpl<>(topology, windowed); + } + + @Override + public KStream[] branch(Predicate... predicates) { + KStreamBranch branch = new KStreamBranch<>(predicates, topology, processor); + return branch.branches(); + } + + @SuppressWarnings("unchecked") + @Override + public KStream through(String topic) { + return through(topic, null, null, null, null); + } + + @SuppressWarnings("unchecked") + @Override + public KStream through(String topic, + Serializer keySerializer, + Serializer valSerializer, + Deserializer keyDeserializer, + Deserializer valDeserializer) { + process(this.getSendProcessor(topic, keySerializer, valSerializer)); + + KafkaSource source = topology.addSource(keyDeserializer, valDeserializer, topic); + + return new KStreamSource<>(topology, source); + } + + @Override + public void sendTo(String topic) { + process(this.getSendProcessor(topic, null, null)); + } + + @Override + public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { + process(this.getSendProcessor(topic, keySerializer, valSerializer)); + } + + @SuppressWarnings("unchecked") + private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { + return new Processor() { + private ProcessorContext context; + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + @Override + public void process(K1 key, V1 value) { + this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); + } + + @Override + public void close() { + } + }; + } + + @SuppressWarnings("unchecked") + @Override + public KStream process(final KafkaProcessor current) { + topology.addProcessor(current, processor); + + return new KStreamImpl(topology, current); + } + + @SuppressWarnings("unchecked") + @Override + public void process(final Processor current) { + KafkaProcessor wrapper = new KafkaProcessor(PROCESSOR_NAME) { + @Override + public void process(K key, V value) { + current.process(key, value); + } + }; + + topology.addProcessor(wrapper, processor); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java new file mode 100644 index 0000000000000..f80c0d708a08f --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java @@ -0,0 +1,166 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.internals.ProcessorContextImpl; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.stream.ValueJoiner; +import org.apache.kafka.stream.Window; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +class KStreamJoin extends KafkaProcessor { + + private static final String JOIN_NAME = "KAFKA-JOIN"; + private static final String JOIN_OTHER_NAME = "KAFKA-JOIN-OTHER"; + + private static abstract class Finder { + abstract Iterator find(K key, long timestamp); + } + + private final KStreamWindow stream1; + private final KStreamWindow stream2; + private final Finder finder1; + private final Finder finder2; + private final ValueJoiner joiner; + final KafkaProcessor processorForOtherStream; + + private ProcessorContext context; + + KStreamJoin(KStreamWindow stream1, KStreamWindow stream2, boolean prior, ValueJoiner joiner) { + super(JOIN_NAME); + + this.stream1 = stream1; + this.stream2 = stream2; + final Window window1 = stream1.window(); + final Window window2 = stream2.window(); + + if (prior) { + this.finder1 = new Finder() { + Iterator find(K key, long timestamp) { + return window1.findAfter(key, timestamp); + } + }; + this.finder2 = new Finder() { + Iterator find(K key, long timestamp) { + return window2.findBefore(key, timestamp); + } + }; + } else { + this.finder1 = new Finder() { + Iterator find(K key, long timestamp) { + return window1.find(key, timestamp); + } + }; + this.finder2 = new Finder() { + Iterator find(K key, long timestamp) { + return window2.find(key, timestamp); + } + }; + } + + this.joiner = joiner; + + this.processorForOtherStream = processorForOther(); + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + + // the two streams should only be joinable if they are inside the same group + // and that group's topics all have the same partitions + ProcessorContextImpl context1 = (ProcessorContextImpl) stream1.context(); + ProcessorContextImpl context2 = (ProcessorContextImpl) stream2.context(); + + if (context1.streamGroup != context2.streamGroup) + throw new IllegalStateException("Stream " + stream1.name() + " and stream " + stream2.name() + " are not joinable" + + " since they belong to different stream groups."); + + Set partitions = context1.streamGroup.partitions(); + Map> partitionsById = new HashMap<>(); + int firstId = -1; + for (TopicPartition partition : partitions) { + if (!partitionsById.containsKey(partition.partition())) { + partitionsById.put(partition.partition(), new ArrayList<>()); + } + partitionsById.get(partition.partition()).add(partition.topic()); + + if (firstId < 0) + firstId = partition.partition(); + } + + List topics = partitionsById.get(firstId); + for (List topicsPerPartition : partitionsById.values()) { + if (topics.size() != topicsPerPartition.size()) + throw new IllegalStateException("Stream " + stream1.name() + " and stream " + stream2.name() + " are not joinable" + + " since their stream group have different partitions for some topics."); + + for (String topic : topicsPerPartition) { + if (!topics.contains(topic)) + throw new IllegalStateException("Stream " + stream1.name() + " and stream " + stream2.name() + " are not joinable" + + " since their stream group have different partitions for some topics."); + } + } + } + + @Override + public void process(K key, V1 value) { + long timestamp = context.timestamp(); + Iterator iter = finder2.find(key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + doJoin(key, value, iter.next()); + } + } + } + + private KafkaProcessor processorForOther() { + return new KafkaProcessor(JOIN_OTHER_NAME) { + + @SuppressWarnings("unchecked") + @Override + public void process(K key, V2 value) { + long timestamp = context.timestamp(); + Iterator iter = finder1.find(key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + doJoin(key, iter.next(), value); + } + } + } + + @Override + public void close() { + // down stream instances are close when the primary stream is closed + } + }; + } + + // TODO: use the "outer-stream" topic as the resulted join stream topic + private void doJoin(K key, V1 value1, V2 value2) { + forward(key, joiner.apply(value1, value2)); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMap.java new file mode 100644 index 0000000000000..d403e6d061dad --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMap.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.stream.KeyValue; +import org.apache.kafka.stream.KeyValueMapper; + +class KStreamMap extends KafkaProcessor { + + private static final String MAP_NAME = "KAFKA-MAP"; + + private final KeyValueMapper mapper; + + public KStreamMap(KeyValueMapper mapper) { + super(MAP_NAME); + + this.mapper = mapper; + } + + @Override + public void process(K1 key, V1 value) { + KeyValue newPair = mapper.apply(key, value); + forward(newPair.key, newPair.value); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMapValues.java new file mode 100644 index 0000000000000..1692bf743f8a1 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMapValues.java @@ -0,0 +1,40 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.stream.ValueMapper; + +class KStreamMapValues extends KafkaProcessor { + + private static final String MAPVALUES_NAME = "KAFKA-MAPVALUES"; + + private final ValueMapper mapper; + + public KStreamMapValues(ValueMapper mapper) { + super(MAPVALUES_NAME); + + this.mapper = mapper; + } + + @Override + public void process(K1 key, V1 value) { + V2 newValue = mapper.apply(value); + forward(key, newValue); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMetadata.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMetadata.java new file mode 100644 index 0000000000000..40fca15243da4 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMetadata.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import org.apache.kafka.clients.processor.internals.PartitioningInfo; +import org.apache.kafka.clients.processor.internals.StreamGroup; + +import java.util.Collections; +import java.util.Map; + +public class KStreamMetadata { + + public static final String UNKNOWN_TOPICNAME = "__UNKNOWN_TOPIC__"; + public static final int UNKNOWN_PARTITION = -1; + + public static KStreamMetadata unjoinable() { + return new KStreamMetadata(Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(UNKNOWN_PARTITION))); + } + + public StreamGroup streamGroup; + public final Map topicPartitionInfos; + + public KStreamMetadata(Map topicPartitionInfos) { + this.topicPartitionInfos = topicPartitionInfos; + } + + boolean isJoinCompatibleWith(KStreamMetadata other) { + // the two streams should only be joinable if they are inside the same sync group + // and their contained streams all have the same number of partitions + if (this.streamGroup != other.streamGroup) + return false; + + int numPartitions = -1; + for (PartitioningInfo partitionInfo : this.topicPartitionInfos.values()) { + if (partitionInfo.numPartitions < 0) { + return false; + } else if (numPartitions >= 0) { + if (partitionInfo.numPartitions != numPartitions) + return false; + } else { + numPartitions = partitionInfo.numPartitions; + } + } + + for (PartitioningInfo partitionInfo : other.topicPartitionInfos.values()) { + if (partitionInfo.numPartitions != numPartitions) + return false; + } + + return true; + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java new file mode 100644 index 0000000000000..73d81e5309cd6 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; + +public class KStreamSource extends KStreamImpl { + + public KStreamSource(PTopology topology, KafkaProcessor source) { + super(topology, source); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java new file mode 100644 index 0000000000000..5ad3d0c6a3e49 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.internals; + +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.KStream; +import org.apache.kafka.stream.KStreamWindowed; +import org.apache.kafka.stream.ValueJoiner; +import org.apache.kafka.stream.Window; + +public class KStreamWindow extends KafkaProcessor { + + public static final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { + + public KStreamWindow windowed; + + public KStreamWindowedImpl(PTopology topology, KStreamWindow windowed) { + super(topology, windowed); + this.windowed = windowed; + } + + @Override + public KStream join(KStreamWindowed other, ValueJoiner processor) { + return join(other, false, processor); + } + + @Override + public KStream joinPrior(KStreamWindowed other, ValueJoiner processor) { + return join(other, true, processor); + } + + private KStream join(KStreamWindowed other, boolean prior, ValueJoiner processor) { + KStreamWindow thisWindow = this.windowed; + KStreamWindow otherWindow = ((KStreamWindowedImpl) other).windowed; + + KStreamJoin join = new KStreamJoin<>(thisWindow, otherWindow, prior, processor); + + topology.addProcessor(join, thisWindow); + topology.addProcessor(join.processorForOtherStream, otherWindow); + + return new KStreamImpl<>(topology, join); + } + } + + private static final String WINDOW_NAME = "KAFKA-WINDOW"; + + private final Window window; + private ProcessorContext context; + + KStreamWindow(Window window) { + super(WINDOW_NAME); + this.window = window; + } + + public Window window() { + return window; + } + + public ProcessorContext context() { return context; } + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + @SuppressWarnings("unchecked") + @Override + public void process(K key, V value) { + synchronized (this) { + window.put(key, value, context.timestamp()); + forward(key, value); + } + } + + @Override + public void close() { + window.close(); + } +} From 746fec1e0ec45ef93d8480d91252029ed9df6d47 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sat, 8 Aug 2015 11:18:09 -0700 Subject: [PATCH 151/275] wip --- .../clients/processor/KafkaProcessor.java | 15 +- .../kafka/clients/processor/PTopology.java | 69 ++++- .../processor/internals/KafkaSource.java | 6 +- .../apache/kafka/stream/KStreamTopology.java | 20 +- .../org/apache/kafka/stream/KeyValue.java | 2 +- .../org/apache/kafka/stream/Transformer.java | 27 -- .../java/org/apache/kafka/stream/Window.java | 1 - .../kafka/stream/internals/KStreamImpl.java | 1 - .../kafka/stream/internals/KStreamSource.java | 7 + .../stream/state/InMemoryKeyValueStore.java | 1 - .../{internals => }/MeteredKeyValueStore.java | 5 +- .../stream/state/RocksDBKeyValueStore.java | 1 - .../kafka/stream/KStreamBranchTest.java | 25 +- .../kafka/stream/KStreamFilterTest.java | 30 ++- .../kafka/stream/KStreamFlatMapTest.java | 24 +- .../stream/KStreamFlatMapValuesTest.java | 25 +- .../apache/kafka/stream/KStreamJoinTest.java | 235 ++++++++---------- .../apache/kafka/stream/KStreamMapTest.java | 26 +- .../kafka/stream/KStreamMapValuesTest.java | 28 ++- .../kafka/stream/KStreamSourceTest.java | 18 +- .../kafka/stream/KStreamTransformTest.java | 140 ----------- .../kafka/stream/KStreamWindowedTest.java | 25 +- .../kafka/stream/ParallelExecutorTest.java | 137 ---------- .../apache/kafka/stream/StreamGroupTest.java | 63 ++--- .../kafka/test/MockKStreamTopology.java | 4 + .../org/apache/kafka/test/MockProcessor.java | 13 +- ...Context.java => MockProcessorContext.java} | 18 +- .../org/apache/kafka/test/MockSource.java | 62 +++++ temp | 12 +- temp2 | 12 +- 30 files changed, 454 insertions(+), 598 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/Transformer.java rename stream/src/main/java/org/apache/kafka/stream/state/{internals => }/MeteredKeyValueStore.java (97%) delete mode 100644 stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java rename stream/src/test/java/org/apache/kafka/test/{MockKStreamContext.java => MockProcessorContext.java} (94%) create mode 100644 stream/src/test/java/org/apache/kafka/test/MockSource.java diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java index 3c750f9a9433a..18a16bf544634 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java @@ -22,18 +22,31 @@ public abstract class KafkaProcessor implements Processor, Receiver, Punctuator { - protected final List> children; + private final List> children; + private final List> parents; private final String name; + public boolean initialized; + public boolean closed; + public KafkaProcessor(String name) { this.name = name; this.children = new ArrayList<>(); + this.parents = new ArrayList<>(); + + this.initialized = false; + this.closed = false; } public String name() { return name; } + public List> children() { return children; } + + public List> parents() { return parents; } + public final void chain(KafkaProcessor child) { + child.parents.add(this); children.add(child); } diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java index 2add7efad4a30..1c98dae79fcd8 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -18,9 +18,12 @@ package org.apache.kafka.clients.processor; import org.apache.kafka.clients.processor.internals.KafkaSource; +import org.apache.kafka.clients.processor.internals.StreamingConfig; import org.apache.kafka.common.serialization.Deserializer; +import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Deque; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -32,6 +35,12 @@ abstract public class PTopology { private List processors = new ArrayList<>(); private Map sources = new HashMap<>(); + protected final StreamingConfig streamingConfig; + + public PTopology(StreamingConfig streamingConfig) { + this.streamingConfig = streamingConfig; + } + public Set sources() { Set sources = new HashSet<>(); for (KafkaSource source : this.sources.values()) { @@ -67,14 +76,37 @@ public Deserializer valueDeser(String topic) { return source.valDeserializer; } - // TODO: init by the ordering of DAG public final void init(ProcessorContext context) { - for (KafkaProcessor processor : processors) { - processor.init(context); + // init the processors following the DAG ordering + // such that parents are always initialized before children + Deque deque = new ArrayDeque<>(); + for (KafkaProcessor processor : sources.values()) { + deque.addLast(processor); + } + + while(!deque.isEmpty()) { + KafkaProcessor processor = deque.pollFirst(); + + boolean parentsInitialized = true; + for (KafkaProcessor parent : (List) processor.parents()) { + if (!parent.initialized) { + parentsInitialized = false; + break; + } + } + + if (parentsInitialized && !processor.initialized) { + processor.init(context); + processor.initialized = true; + + for (KafkaProcessor child : (List) processor.children()) { + deque.addLast(child); + } + } } } - public final KafkaSource addSource(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + public final KafkaSource addSource(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { KafkaSource source = new KafkaSource<>(keyDeserializer, valDeserializer); processors.add(source); @@ -105,10 +137,33 @@ public final void addProcessor(KafkaProcessor processor, Kafk } } - // TODO: close by the ordering of DAG public final void close() { - for (KafkaProcessor processor : processors) { - processor.close(); + // close the processors following the DAG ordering + // such that parents are always initialized before children + Deque deque = new ArrayDeque<>(); + for (KafkaProcessor processor : sources.values()) { + deque.addLast(processor); + } + + while(!deque.isEmpty()) { + KafkaProcessor processor = deque.pollFirst(); + + boolean parentsClosed = true; + for (KafkaProcessor parent : (List) processor.parents()) { + if (!parent.closed) { + parentsClosed = false; + break; + } + } + + if (parentsClosed && !processor.closed) { + processor.close(); + processor.closed = true; + + for (KafkaProcessor child : (List) processor.children()) { + deque.addLast(child); + } + } } processors.clear(); diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java index be93731e99d65..950eb0d4410b2 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java @@ -25,10 +25,10 @@ public class KafkaSource extends KafkaProcessor { private static final String SOURCE_NAME = "KAFKA-SOURCE"; - public Deserializer keyDeserializer; - public Deserializer valDeserializer; + public Deserializer keyDeserializer; + public Deserializer valDeserializer; - public KafkaSource(Deserializer keyDeserializer, Deserializer valDeserializer) { + public KafkaSource(Deserializer keyDeserializer, Deserializer valDeserializer) { super(SOURCE_NAME); this.keyDeserializer = keyDeserializer; diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java index 7745b3168b826..ac99b8a29c26a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.internals.StreamingConfig; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.internals.KStreamSource; @@ -27,6 +28,23 @@ */ public abstract class KStreamTopology extends PTopology { + public KStreamTopology(StreamingConfig streamingConfig) { + super(streamingConfig); + } + + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(String... topics) { + KafkaProcessor source = addSource( + (Deserializer) streamingConfig.keyDeserializer(), + (Deserializer) streamingConfig.valueDeserializer(), topics); + return new KStreamSource<>(this, source); + } + /** * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. * @@ -37,7 +55,7 @@ public abstract class KStreamTopology extends PTopology { * @param topics the topic names, if empty default to all the topics in the config * @return KStream */ - public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { KafkaProcessor source = addSource(keyDeserializer, valDeserializer, topics); return new KStreamSource<>(this, source); } diff --git a/stream/src/main/java/org/apache/kafka/stream/KeyValue.java b/stream/src/main/java/org/apache/kafka/stream/KeyValue.java index f81c68cb3d41e..835962206a348 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KeyValue.java +++ b/stream/src/main/java/org/apache/kafka/stream/KeyValue.java @@ -28,7 +28,7 @@ public KeyValue(K key, V value) { } public static KeyValue pair(K key, V value) { - return new KeyValue(key, value); + return new KeyValue<>(key, value); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/Transformer.java b/stream/src/main/java/org/apache/kafka/stream/Transformer.java deleted file mode 100644 index 565be2637eb4f..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/Transformer.java +++ /dev/null @@ -1,27 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -public interface Transformer extends Processor { - - interface Forwarder { - public void send(K key, V value, long timestamp); - } - - public void forwarder(Forwarder forwarder); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/Window.java b/stream/src/main/java/org/apache/kafka/stream/Window.java index 1eccdb1cf6749..9ec059b22c810 100644 --- a/stream/src/main/java/org/apache/kafka/stream/Window.java +++ b/stream/src/main/java/org/apache/kafka/stream/Window.java @@ -17,7 +17,6 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.clients.processor.StateStore; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java index 2895384c58e28..e720ca63c3c9f 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java @@ -38,7 +38,6 @@ public class KStreamImpl implements KStream { protected PTopology topology; protected KafkaProcessor processor; - protected KStreamContext context; public KStreamImpl(PTopology topology, KafkaProcessor processor) { this.topology = topology; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java index 73d81e5309cd6..a06fb1f427817 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java @@ -22,7 +22,14 @@ public class KStreamSource extends KStreamImpl { + private KafkaProcessor source; + public KStreamSource(PTopology topology, KafkaProcessor source) { super(topology, source); + this.source = source; + } + + public KafkaProcessor source() { + return source; } } diff --git a/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java index 59293d46fb9fa..23858111170a3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java @@ -20,7 +20,6 @@ import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.stream.state.internals.MeteredKeyValueStore; import java.util.Iterator; import java.util.List; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/MeteredKeyValueStore.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java rename to stream/src/main/java/org/apache/kafka/stream/state/MeteredKeyValueStore.java index 273e5bd0a01c6..d8d12d9ddb280 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/internals/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/MeteredKeyValueStore.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.state.internals; +package org.apache.kafka.stream.state; import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.clients.processor.RecordCollector; @@ -32,9 +32,6 @@ 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.stream.state.Entry; -import org.apache.kafka.stream.state.KeyValueIterator; -import org.apache.kafka.stream.state.KeyValueStore; import java.util.HashSet; import java.util.List; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java index 47fcb71222592..f5db6509f180a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.stream.state.internals.MeteredKeyValueStore; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.CompactionStyle; import org.rocksdb.CompressionType; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java index 4cd2b5db2bf75..b5393344a4e86 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java @@ -17,16 +17,14 @@ package org.apache.kafka.stream; -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamContext; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; import org.junit.Test; import java.lang.reflect.Array; -import java.util.Collections; import static org.junit.Assert.assertEquals; @@ -95,7 +93,9 @@ public boolean apply(Integer key, String value) { ======= private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamTopology topology = new MockKStreamTopology(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); @SuppressWarnings("unchecked") @Test @@ -122,12 +122,11 @@ public boolean apply(Integer key, String value) { final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - KStreamTopology initializer = new MockKStreamTopology(); - KStreamSource stream; + KStream stream; KStream[] branches; MockProcessor[] processors; - stream = new KStreamSource<>(null, initializer); + stream = topology.from(keyDeserializer, valDeserializer, topicName); branches = stream.branch(isEven, isMultipleOfThree, isOdd); assertEquals(3, branches.length); @@ -139,14 +138,16 @@ public boolean apply(Integer key, String value) { } for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(3, processors[0].processed.size()); assertEquals(1, processors[1].processed.size()); assertEquals(3, processors[2].processed.size()); - stream = new KStreamSource<>(null, initializer); + // ----------------------------------------------- + + stream = topology.from(keyDeserializer, valDeserializer, topicName); branches = stream.branch(isEven, isOdd, isMultipleOfThree); assertEquals(3, branches.length); @@ -157,10 +158,8 @@ public boolean apply(Integer key, String value) { branches[i].process(processors[i]); } - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(3, processors[0].processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java index 276ee1fb62cac..9852f17edee5b 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java @@ -17,24 +17,23 @@ package org.apache.kafka.stream; -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockKStreamContext; import org.junit.Test; -import java.util.Collections; - import static org.junit.Assert.assertEquals; public class KStreamFilterTest { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamTopology topology = new MockKStreamTopology(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); private Predicate isMultipleOfThree = new Predicate() { @Override @@ -50,6 +49,7 @@ public void testFilter() { <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -66,17 +66,18 @@ public void testFilter() { ======= KStreamTopology initializer = new MockKStreamTopology(); KStreamSource stream; +======= + KStream stream; +>>>>>>> wip MockProcessor processor; >>>>>>> compile and test passed processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, initializer); + stream = topology.from(keyDeserializer, valDeserializer, topicName); stream.filter(isMultipleOfThree).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(2, processor.processed.size()); @@ -86,8 +87,7 @@ public void testFilter() { public void testFilterOut() { final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - KStreamTopology initializer = new MockKStreamTopology(); - KStreamSource stream; + KStream stream; MockProcessor processor; <<<<<<< HEAD @@ -108,14 +108,12 @@ public void testFilterOut() { MockProcessor processor; ======= processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, initializer); + stream = topology.from(keyDeserializer, valDeserializer, topicName); stream.filterOut(isMultipleOfThree).process(processor); >>>>>>> compile and test passed - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(5, processor.processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java index cba15b27ada30..e18af652d041e 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java @@ -17,30 +17,30 @@ package org.apache.kafka.stream; -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockKStreamContext; import org.junit.Test; import static org.junit.Assert.assertEquals; import java.util.ArrayList; -import java.util.Collections; public class KStreamFlatMapTest { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamTopology topology = new MockKStreamTopology(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testFlatMap() { - KeyValueMapper, Integer, String> mapper = - new KeyValueMapper, Integer, String>() { + KeyValueMapper> mapper = + new KeyValueMapper>() { @Override public KeyValue> apply(Integer key, String value) { ArrayList result = new ArrayList(); @@ -56,6 +56,7 @@ public KeyValue> apply(Integer key, String value) { <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology topology = new MockKStreamTopology(); @@ -91,17 +92,18 @@ public KeyValue> apply(Integer key, String value) { KStreamTopology topology = new MockKStreamTopology(); KStreamSource stream; +======= + KStream stream; +>>>>>>> wip MockProcessor processor; processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, topology); + stream = topology.from(keyDeserializer, valDeserializer, topicName); stream.flatMap(mapper).process(processor); >>>>>>> compile and test passed - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(6, processor.processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java index 1158b3e154929..e49b3500bcba7 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java @@ -17,16 +17,14 @@ package org.apache.kafka.stream; -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockKStreamContext; import org.junit.Test; import java.util.ArrayList; -import java.util.Collections; import static org.junit.Assert.assertEquals; @@ -34,13 +32,15 @@ public class KStreamFlatMapValuesTest { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamTopology topology = new MockKStreamTopology(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testFlatMapValues() { - ValueMapper, String> mapper = - new ValueMapper, String>() { + ValueMapper> mapper = + new ValueMapper>() { @Override public Iterable apply(String value) { ArrayList result = new ArrayList(); @@ -55,6 +55,7 @@ public Iterable apply(String value) { <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -71,17 +72,18 @@ public Iterable apply(String value) { ======= KStreamTopology initializer = new MockKStreamTopology(); KStreamSource stream; +======= + KStream stream; +>>>>>>> wip MockProcessor processor; >>>>>>> compile and test passed processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, initializer); + stream = topology.from(keyDeserializer, valDeserializer, topicName); stream.flatMapValues(mapper).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(8, processor.processed.size()); @@ -92,5 +94,4 @@ public Iterable apply(String value) { assertEquals(expected[i], processor.processed.get(i)); } } - } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java index 92ae9adc52226..4d84fa0641175 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java @@ -17,13 +17,15 @@ package org.apache.kafka.stream; +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.clients.processor.internals.PartitioningInfo; import org.apache.kafka.stream.internals.KStreamMetadata; import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockKStreamContext; +import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.UnlimitedWindow; import org.junit.Test; @@ -37,6 +39,10 @@ public class KStreamJoinTest { private String topicName = "topic"; + private KStreamTopology topology = new MockKStreamTopology(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); private ValueJoiner joiner = new ValueJoiner() { @@ -53,7 +59,7 @@ public String apply(String value) { } }; - private ValueMapper, String> valueMapper2 = new ValueMapper, String>() { + private ValueMapper> valueMapper2 = new ValueMapper>() { @Override public Iterable apply(String value) { return (Iterable) Utils.mkSet(value); @@ -68,14 +74,17 @@ public KeyValue apply(Integer key, String value) { } }; - KeyValueMapper, Integer, String> keyValueMapper2 = - new KeyValueMapper, Integer, String>() { + KeyValueMapper> keyValueMapper2 = + new KeyValueMapper>() { @Override public KeyValue> apply(Integer key, String value) { return KeyValue.pair(key, (Iterable) Utils.mkSet(value)); } }; + + // TODO: initialize with context + @Test public void testJoin() { @@ -106,39 +115,25 @@ public void testJoin() { final int[] expectedKeys = new int[]{0, 1, 2, 3}; >>>>>>> compile and test passed - KStreamSource stream1; - KStreamSource stream2; + KStream stream1; + KStream stream2; KStreamWindowed windowed1; KStreamWindowed windowed2; MockProcessor processor; String[] expected; - KStreamTopology initializer = new MockKStreamTopology(); processor = new MockProcessor<>(); - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); - - boolean exceptionRaised = false; - - try { - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); + windowed1 = stream1.with(new UnlimitedWindow<>()); + windowed2 = stream2.with(new UnlimitedWindow<>()); - assertFalse(exceptionRaised); + windowed1.join(windowed2, joiner).process(processor); // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i]); } assertEquals(0, processor.processed.size()); @@ -146,7 +141,7 @@ public void testJoin() { // push two items to the other stream. the main stream's window has two items for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i]); } assertEquals(2, processor.processed.size()); @@ -162,7 +157,7 @@ public void testJoin() { // push all items to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], 0L); + ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i]); } assertEquals(2, processor.processed.size()); @@ -179,7 +174,7 @@ public void testJoin() { // push all items to the other stream. this should produce 6 items for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i]); } assertEquals(6, processor.processed.size()); @@ -196,8 +191,8 @@ public void testJoinPrior() { final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KStreamSource stream1; - KStreamSource stream2; + KStream stream1; + KStream stream2; KStreamWindowed windowed1; KStreamWindowed windowed2; MockProcessor processor; @@ -206,6 +201,7 @@ public void testJoinPrior() { <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -247,13 +243,20 @@ public void testJoinPrior() { } catch (NotCopartitionedException e) { exceptionRaised = true; } +======= + processor = new MockProcessor<>(); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); + windowed1 = stream1.with(new UnlimitedWindow<>()); + windowed2 = stream2.with(new UnlimitedWindow<>()); +>>>>>>> wip - assertFalse(exceptionRaised); + windowed1.joinPrior(windowed2, joiner).process(processor); // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i); + ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i], i); } assertEquals(0, processor.processed.size()); @@ -262,7 +265,7 @@ public void testJoinPrior() { // no corresponding item in the main window has a newer timestamp for (int i = 0; i < 2; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i + 1); + ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i], i + 1); } assertEquals(0, processor.processed.size()); @@ -272,7 +275,7 @@ public void testJoinPrior() { // push all items with newer timestamps to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - stream1.receive(expectedKeys[i], "X" + expectedKeys[i], i + 2); + ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i], i + 2); } assertEquals(2, processor.processed.size()); @@ -289,7 +292,7 @@ public void testJoinPrior() { // push all items with older timestamps to the other stream. this should produce six items for (int i = 0; i < expectedKeys.length; i++) { - stream2.receive(expectedKeys[i], "Y" + expectedKeys[i], i); + ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i], i); } assertEquals(6, processor.processed.size()); @@ -304,15 +307,14 @@ public void testJoinPrior() { @Test public void testMap() { - KStreamSource stream1; - KStreamSource stream2; + KStream stream1; + KStream stream2; KStream mapped1; KStream mapped2; KStreamWindowed windowed1; KStreamWindowed windowed2; MockProcessor processor; - KStreamTopology initializer = new MockKStreamTopology(); processor = new MockProcessor<>(); <<<<<<< HEAD @@ -345,18 +347,18 @@ public void testMap() { >>>>>>> compile and test passed try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.map(keyValueMapper); mapped2 = stream2.map(keyValueMapper); exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1 = stream1.with(new UnlimitedWindow<>()); + windowed2 = mapped2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); + KStreamContext context = new MockProcessorContext(null, null); stream1.bind(context, streamMetadata); stream2.bind(context, streamMetadata); @@ -367,20 +369,18 @@ public void testMap() { assertTrue(exceptionRaised); try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.map(keyValueMapper); mapped2 = stream2.map(keyValueMapper); exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); + windowed1 = mapped1.with(new UnlimitedWindow<>()); + windowed2 = stream2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; @@ -389,20 +389,18 @@ public void testMap() { assertTrue(exceptionRaised); try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.map(keyValueMapper); mapped2 = stream2.map(keyValueMapper); exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1 = mapped1.with(new UnlimitedWindow<>()); + windowed2 = mapped2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; @@ -413,15 +411,14 @@ public void testMap() { @Test public void testFlatMap() { - KStreamSource stream1; - KStreamSource stream2; + KStream stream1; + KStream stream2; KStream mapped1; KStream mapped2; KStreamWindowed windowed1; KStreamWindowed windowed2; MockProcessor processor; - KStreamTopology initializer = new MockKStreamTopology(); processor = new MockProcessor<>(); <<<<<<< HEAD @@ -454,20 +451,18 @@ public void testFlatMap() { >>>>>>> compile and test passed try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.flatMap(keyValueMapper2); mapped2 = stream2.flatMap(keyValueMapper2); exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1 = stream1.with(new UnlimitedWindow<>()); + windowed2 = mapped2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; @@ -476,20 +471,18 @@ public void testFlatMap() { assertTrue(exceptionRaised); try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.flatMap(keyValueMapper2); mapped2 = stream2.flatMap(keyValueMapper2); exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); + windowed1 = mapped1.with(new UnlimitedWindow<>()); + windowed2 = stream2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; @@ -498,20 +491,18 @@ public void testFlatMap() { assertTrue(exceptionRaised); try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.flatMap(keyValueMapper2); mapped2 = stream2.flatMap(keyValueMapper2); exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1 = mapped1.with(new UnlimitedWindow<>()); + windowed2 = mapped2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; @@ -522,8 +513,8 @@ public void testFlatMap() { @Test public void testMapValues() { - KStreamSource stream1; - KStreamSource stream2; + KStream stream1; + KStream stream2; KStream mapped1; KStream mapped2; KStreamWindowed windowed1; @@ -563,20 +554,18 @@ public void testMapValues() { >>>>>>> compile and test passed try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.mapValues(valueMapper); mapped2 = stream2.mapValues(valueMapper); exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1 = stream1.with(new UnlimitedWindow<>()); + windowed2 = mapped2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; @@ -585,20 +574,18 @@ public void testMapValues() { assertFalse(exceptionRaised); try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.mapValues(valueMapper); mapped2 = stream2.mapValues(valueMapper); exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); + windowed1 = mapped1.with(new UnlimitedWindow<>()); + windowed2 = stream2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; @@ -607,20 +594,18 @@ public void testMapValues() { assertFalse(exceptionRaised); try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.mapValues(valueMapper); mapped2 = stream2.mapValues(valueMapper); exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1 = mapped1.with(new UnlimitedWindow<>()); + windowed2 = mapped2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; @@ -631,8 +616,8 @@ public void testMapValues() { @Test public void testFlatMapValues() { - KStreamSource stream1; - KStreamSource stream2; + KStream stream1; + KStream stream2; KStream mapped1; KStream mapped2; KStreamWindowed windowed1; @@ -672,20 +657,18 @@ public void testFlatMapValues() { >>>>>>> compile and test passed try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.flatMapValues(valueMapper2); mapped2 = stream2.flatMapValues(valueMapper2); exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1 = stream1.with(new UnlimitedWindow<>()); + windowed2 = mapped2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; @@ -694,20 +677,18 @@ public void testFlatMapValues() { assertFalse(exceptionRaised); try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.flatMapValues(valueMapper2); mapped2 = stream2.flatMapValues(valueMapper2); exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); + windowed1 = mapped1.with(new UnlimitedWindow<>()); + windowed2 = stream2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; @@ -716,20 +697,18 @@ public void testFlatMapValues() { assertFalse(exceptionRaised); try { - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); + stream1 = topology.from(keyDeserializer, valDeserializer, topicName); + stream2 = topology.from(keyDeserializer, valDeserializer, topicName); mapped1 = stream1.flatMapValues(valueMapper2); mapped2 = stream2.flatMapValues(valueMapper2); exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow()); - windowed2 = mapped2.with(new UnlimitedWindow()); + windowed1 = mapped1.with(new UnlimitedWindow<>()); + windowed2 = mapped2.with(new UnlimitedWindow<>()); windowed1.join(windowed2, joiner).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); + KStreamContext context = new MockProcessorContext(null, null); } catch (NotCopartitionedException e) { exceptionRaised = true; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java index 9c36f219c87b7..51de88df8497e 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java @@ -17,29 +17,28 @@ package org.apache.kafka.stream; -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockKStreamContext; import org.junit.Test; -import java.util.Collections; - import static org.junit.Assert.assertEquals; public class KStreamMapTest { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamTopology topology = new MockKStreamTopology(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testMap() { - KeyValueMapper mapper = - new KeyValueMapper() { + KeyValueMapper mapper = + new KeyValueMapper() { @Override public KeyValue apply(Integer key, String value) { return KeyValue.pair(value, key); @@ -51,6 +50,7 @@ public KeyValue apply(Integer key, String value) { <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -67,17 +67,18 @@ public KeyValue apply(Integer key, String value) { ======= KStreamTopology initializer = new MockKStreamTopology(); KStreamSource stream; +======= + KStream stream; +>>>>>>> wip MockProcessor processor; >>>>>>> compile and test passed processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, initializer); + stream = topology.from(keyDeserializer, valDeserializer, topicName); stream.map(mapper).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(4, processor.processed.size()); @@ -88,5 +89,4 @@ public KeyValue apply(Integer key, String value) { assertEquals(expected[i], processor.processed.get(i)); } } - } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java index 79c7e12e4dddb..8c0a50795daa9 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java @@ -20,6 +20,7 @@ <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD import io.confluent.streaming.KStreamContext; <<<<<<< HEAD <<<<<<< HEAD @@ -50,29 +51,36 @@ import org.apache.kafka.stream.topology.internals.KStreamSource; ======= import org.apache.kafka.stream.internals.KStreamMetadata; +======= +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +>>>>>>> wip import org.apache.kafka.stream.internals.KStreamSource; >>>>>>> wip import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; +<<<<<<< HEAD import org.apache.kafka.test.MockKStreamContext; >>>>>>> removing io.confluent imports: wip +======= +>>>>>>> wip import org.junit.Test; -import java.util.Collections; - import static org.junit.Assert.assertEquals; public class KStreamMapValuesTest { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamTopology topology = new MockKStreamTopology(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testFlatMapValues() { - ValueMapper mapper = - new ValueMapper() { + ValueMapper mapper = + new ValueMapper() { @Override public Integer apply(String value) { return value.length(); @@ -84,6 +92,7 @@ public Integer apply(String value) { <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -105,12 +114,15 @@ public Integer apply(String value) { processor = new MockProcessor<>(); stream = new KStreamSource<>(null, initializer); +======= + KStream stream; + MockProcessor processor = new MockProcessor<>(); + stream = topology.from(keyDeserializer, valDeserializer, topicName); +>>>>>>> wip stream.mapValues(mapper).process(processor); - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], Integer.toString(expectedKeys[i]), 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], Integer.toString(expectedKeys[i])); } assertEquals(4, processor.processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java index ed3c0c6df1ccb..7adf2a6d94c15 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java @@ -17,12 +17,10 @@ package org.apache.kafka.stream; -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockKStreamContext; import org.junit.Test; import java.util.Collections; @@ -33,7 +31,9 @@ public class KStreamSourceTest { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamTopology topology = new MockKStreamTopology(); + private StringDeserializer keyDeserializer = new StringDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testKStreamSource() { @@ -41,6 +41,7 @@ public void testKStreamSource() { <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); <<<<<<< HEAD @@ -59,19 +60,19 @@ public void testKStreamSource() { >>>>>>> removing io.confluent imports: wip ======= KStreamTopology initializer = new MockKStreamTopology(); +======= +>>>>>>> wip MockProcessor processor = new MockProcessor<>(); >>>>>>> compile and test passed - KStreamSource stream = new KStreamSource<>(null, initializer); + KStream stream = topology.from(keyDeserializer, valDeserializer, topicName); stream.process(processor); final String[] expectedKeys = new String[]{"k1", "k2", "k3"}; final String[] expectedValues = new String[]{"v1", "v2", "v3"}; - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], expectedValues[i], 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], expectedValues[i]); } assertEquals(3, processor.processed.size()); @@ -80,5 +81,4 @@ public void testKStreamSource() { assertEquals(expectedKeys[i] + ":" + expectedValues[i], processor.processed.get(i)); } } - } diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java deleted file mode 100644 index 667926dda8a51..0000000000000 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamTransformTest.java +++ /dev/null @@ -1,140 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.internals.KStreamMetadata; -import org.apache.kafka.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamContext; -import org.apache.kafka.test.MockKStreamTopology; -import org.apache.kafka.test.MockProcessor; -import org.junit.Test; - -import java.util.Collections; - -import static org.junit.Assert.assertEquals; - -public class KStreamTransformTest { - - private String topicName = "topic"; - - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - - @Test - public void testTransform() { - final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - - KStreamTopology topology = new MockKStreamTopology(); - KStreamSource stream; - MockProcessor processor; - Transformer transformer; - - processor = new MockProcessor<>(); - transformer = new Transformer() { - KStreamContext context; - Forwarder forwarder; - - public void init(KStreamContext context) { - this.context = context; - } - - public void forwarder(Forwarder forwarder) { - this.forwarder = forwarder; - } - - public void process(String key, String value) { - forwarder.send(Integer.parseInt(value), 0, 0L); - } - - public void punctuate(long timestamp) { - } - - public void close() { - } - }; - - stream = new KStreamSource<>(null, topology); - - stream.transform(transformer).process(processor); - - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(null, Integer.toString(expectedKeys[i]), 0L); - } - - assertEquals(expectedKeys.length, processor.processed.size()); - for (int i = 0; i < expectedKeys.length; i++) { - assertEquals(expectedKeys[i] + ":" + 0, processor.processed.get(i)); - } - } - - @Test - public void testTransformEmitOnPuncutation() { - final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - - KStreamTopology topology = new MockKStreamTopology(); - KStreamSource stream; - MockProcessor processor; - Transformer transformer; - - processor = new MockProcessor<>(); - transformer = new Transformer() { - KStreamContext context; - Forwarder forwarder; - Integer currentKey; - - public void init(KStreamContext context) { - this.context = context; - } - - public void forwarder(Forwarder forwarder) { - this.forwarder = forwarder; - } - - public void process(Integer key, String value) { - currentKey = Integer.parseInt(value); - } - - public void punctuate(long timestamp) { - forwarder.send(currentKey, 0, 0L); - } - - public void close() { - } - }; - - stream = new KStreamSource<>(null, topology); - - stream.transform(transformer).process(processor); - - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(null, Integer.toString(expectedKeys[i]), 0L); - if (i % 3 == 2) transformer.punctuate(0L); - } - - final int[] expected = new int[]{3, 6}; - assertEquals(2, processor.processed.size()); - for (int i = 0; i < 2; i++) { - assertEquals(expected[i] + ":" + 0, processor.processed.get(i)); - } - } - -} diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java index 2fe37940d1e74..a11d85de4ae9a 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java @@ -17,10 +17,9 @@ package org.apache.kafka.stream; -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.internals.KStreamMetadata; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamContext; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.UnlimitedWindow; import org.junit.Test; @@ -34,13 +33,16 @@ public class KStreamWindowedTest { private String topicName = "topic"; - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); + private KStreamTopology topology = new MockKStreamTopology(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testWindowedStream() { final int[] expectedKeys = new int[]{0, 1, 2, 3}; +<<<<<<< HEAD <<<<<<< HEAD KStreamSource stream; Window window; @@ -63,20 +65,19 @@ public void testWindowedStream() { Window window; KStreamTopology initializer = new MockKStreamTopology(); >>>>>>> compile and test passed +======= + KStream stream; + Window window; +>>>>>>> wip window = new UnlimitedWindow<>(); - stream = new KStreamSource<>(null, initializer); + stream = topology.from(keyDeserializer, valDeserializer, topicName); stream.with(window); - boolean exceptionRaised = false; - - KStreamContext context = new MockKStreamContext(null, null); - stream.bind(context, streamMetadata); - // two items in the window for (int i = 0; i < 2; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(1, countItem(window.find(0, 0L))); @@ -87,7 +88,7 @@ public void testWindowedStream() { // previous two items + all items, thus two are duplicates, in the window for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "Y" + expectedKeys[i], 0L); + ((KStreamSource) stream).source().receive(expectedKeys[i], "Y" + expectedKeys[i]); } assertEquals(2, countItem(window.find(0, 0L))); diff --git a/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java b/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java deleted file mode 100644 index e0601914ee6c5..0000000000000 --- a/stream/src/test/java/org/apache/kafka/stream/ParallelExecutorTest.java +++ /dev/null @@ -1,137 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -import org.apache.kafka.clients.processor.internals.ParallelExecutor; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -public class ParallelExecutorTest { - - @Test - public void testExecutingShortTaskList() throws Exception { - ParallelExecutor parallelExecutor = new ParallelExecutor(10); - ArrayList taskList = new ArrayList<>(); - AtomicInteger counter = new AtomicInteger(0); - - for (int i = 0; i < 5; i++) { - taskList.add(new TestTask(counter)); - } - - parallelExecutor.execute(taskList); - - for (TestTask task : taskList) { - assertEquals(task.executionCount, 1); - } - assertEquals(counter.get(), taskList.size()); - - parallelExecutor.execute(taskList); - - for (TestTask task : taskList) { - assertEquals(task.executionCount, 2); - } - assertEquals(counter.get(), taskList.size() * 2); - } - - @Test - public void testExecutingLongTaskList() throws Exception { - ParallelExecutor parallelExecutor = new ParallelExecutor(10); - ArrayList taskList = new ArrayList<>(); - AtomicInteger counter = new AtomicInteger(0); - - for (int i = 0; i < 20; i++) { - taskList.add(new TestTask(counter)); - } - - parallelExecutor.execute(taskList); - - for (TestTask task : taskList) { - assertEquals(task.executionCount, 1); - } - assertEquals(counter.get(), taskList.size()); - - parallelExecutor.execute(taskList); - - for (TestTask task : taskList) { - assertEquals(task.executionCount, 2); - } - assertEquals(counter.get(), taskList.size() * 2); - } - - @Test - public void testException() { - ParallelExecutor parallelExecutor = new ParallelExecutor(10); - ArrayList taskList = new ArrayList<>(); - AtomicInteger counter = new AtomicInteger(0); - - for (int i = 0; i < 20; i++) { - if (i == 15) { - taskList.add(new TestTask(counter) { - @Override - public boolean process() { - throw new TestException(); - } - }); - } else { - taskList.add(new TestTask(counter)); - } - } - - Exception exception = null; - try { - parallelExecutor.execute(taskList); - } catch (Exception ex) { - exception = ex; - } - - assertEquals(counter.get(), taskList.size() - 1); - assertFalse(exception == null); - assertTrue(exception instanceof TestException); - } - - private static class TestTask implements ParallelExecutor.Task { - public volatile int executionCount = 0; - private AtomicInteger counter; - - TestTask(AtomicInteger counter) { - this.counter = counter; - } - - @Override - public boolean process() { - try { - Thread.sleep(20); - executionCount++; - } catch (Exception ex) { - // ignore - } - counter.incrementAndGet(); - - return true; - } - } - - private static class TestException extends RuntimeException { - } -} diff --git a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java index ee49cda357182..4fc437a29c413 100644 --- a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java @@ -26,13 +26,11 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.clients.processor.internals.StreamGroup; import org.apache.kafka.clients.processor.internals.TimeBasedChooser; -import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockIngestor; -import org.apache.kafka.test.MockKStreamContext; -import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.MockSource; import org.junit.Test; -import java.util.ArrayList; import java.util.Arrays; import static org.junit.Assert.assertEquals; @@ -43,6 +41,7 @@ public class StreamGroupTest { private static Serializer serializer = new IntegerSerializer(); private static Deserializer deserializer = new IntegerDeserializer(); +<<<<<<< HEAD <<<<<<< HEAD public MockKStreamSource() { <<<<<<< HEAD @@ -82,6 +81,8 @@ public void receive(Object key, Object value, long timestamp) { } +======= +>>>>>>> wip @SuppressWarnings("unchecked") @Test public void testAddPartition() { @@ -89,7 +90,7 @@ public void testAddPartition() { MockIngestor mockIngestor = new MockIngestor(); StreamGroup streamGroup = new StreamGroup( - new MockKStreamContext(serializer, deserializer), + new MockProcessorContext(serializer, deserializer), mockIngestor, new TimeBasedChooser(), new TimestampExtractor() { @@ -105,19 +106,19 @@ public long extract(String topic, Object key, Object value) { TopicPartition partition1 = new TopicPartition("topic1", 1); TopicPartition partition2 = new TopicPartition("topic2", 1); - MockKStreamSource stream1 = new MockKStreamSource(); - MockKStreamSource stream2 = new MockKStreamSource(); - MockKStreamSource stream3 = new MockKStreamSource(); + MockSource source1 = new MockSource(deserializer, deserializer); + MockSource source2 = new MockSource(deserializer, deserializer); + MockSource source3 = new MockSource(deserializer, deserializer); - streamGroup.addPartition(partition1, stream1); + streamGroup.addPartition(partition1, source1); mockIngestor.addPartitionStreamToGroup(streamGroup, partition1); - streamGroup.addPartition(partition2, stream2); + streamGroup.addPartition(partition2, source2); mockIngestor.addPartitionStreamToGroup(streamGroup, partition2); Exception exception = null; try { - streamGroup.addPartition(partition1, stream3); + streamGroup.addPartition(partition1, source3); } catch (Exception ex) { exception = ex; } @@ -138,8 +139,8 @@ public long extract(String topic, Object key, Object value) { )); streamGroup.process(); - assertEquals(stream1.numReceived, 1); - assertEquals(stream2.numReceived, 0); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 0); assertEquals(mockIngestor.paused.size(), 1); assertTrue(mockIngestor.paused.contains(partition2)); @@ -151,53 +152,53 @@ public long extract(String topic, Object key, Object value) { )); streamGroup.process(); - assertEquals(stream1.numReceived, 2); - assertEquals(stream2.numReceived, 0); + assertEquals(source1.numReceived, 2); + assertEquals(source2.numReceived, 0); assertEquals(mockIngestor.paused.size(), 2); assertTrue(mockIngestor.paused.contains(partition1)); assertTrue(mockIngestor.paused.contains(partition2)); streamGroup.process(); - assertEquals(stream1.numReceived, 3); - assertEquals(stream2.numReceived, 0); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 0); streamGroup.process(); - assertEquals(stream1.numReceived, 3); - assertEquals(stream2.numReceived, 1); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 1); assertEquals(mockIngestor.paused.size(), 1); assertTrue(mockIngestor.paused.contains(partition2)); streamGroup.process(); - assertEquals(stream1.numReceived, 4); - assertEquals(stream2.numReceived, 1); + assertEquals(source1.numReceived, 4); + assertEquals(source2.numReceived, 1); assertEquals(mockIngestor.paused.size(), 1); streamGroup.process(); - assertEquals(stream1.numReceived, 4); - assertEquals(stream2.numReceived, 2); + assertEquals(source1.numReceived, 4); + assertEquals(source2.numReceived, 2); assertEquals(mockIngestor.paused.size(), 0); streamGroup.process(); - assertEquals(stream1.numReceived, 5); - assertEquals(stream2.numReceived, 2); + assertEquals(source1.numReceived, 5); + assertEquals(source2.numReceived, 2); streamGroup.process(); - assertEquals(stream1.numReceived, 5); - assertEquals(stream2.numReceived, 3); + assertEquals(source1.numReceived, 5); + assertEquals(source2.numReceived, 3); streamGroup.process(); - assertEquals(stream1.numReceived, 5); - assertEquals(stream2.numReceived, 4); + assertEquals(source1.numReceived, 5); + assertEquals(source2.numReceived, 4); assertEquals(mockIngestor.paused.size(), 0); streamGroup.process(); - assertEquals(stream1.numReceived, 5); - assertEquals(stream2.numReceived, 4); + assertEquals(source1.numReceived, 5); + assertEquals(source2.numReceived, 4); } private Iterable> records(ConsumerRecord... recs) { diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java index 413fd349b5240..421d758cc450f 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java @@ -20,6 +20,10 @@ import org.apache.kafka.stream.KStreamTopology; public class MockKStreamTopology extends KStreamTopology { + public MockKStreamTopology() { + super(null); + } + @Override public void build() { } diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java index f06248559e303..4a09b5658fba5 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -17,12 +17,14 @@ package org.apache.kafka.test; -import org.apache.kafka.stream.Processor; -import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.clients.processor.Processor; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.Punctuator; +import org.apache.kafka.clients.processor.Receiver; import java.util.ArrayList; -public class MockProcessor implements Processor { +public class MockProcessor implements Processor, Receiver, Punctuator { public final ArrayList processed = new ArrayList<>(); public final ArrayList punctuated = new ArrayList<>(); @@ -32,7 +34,10 @@ public void process(K key, V value) { } @Override - public void init(KStreamContext context) { + public void receive(K key, V value) { process(key, value); } + + @Override + public void init(ProcessorContext context) { } @Override diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java similarity index 94% rename from stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java rename to stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 51f6d683937b7..d1a9e30483e9b 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -17,6 +17,7 @@ package org.apache.kafka.test; +<<<<<<< HEAD:stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java <<<<<<< HEAD <<<<<<< HEAD import io.confluent.streaming.KStreamContext; @@ -61,6 +62,10 @@ import org.apache.kafka.stream.StateStore; >>>>>>> removing io.confluent imports: wip ======= +======= +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.ProcessorContext; +>>>>>>> wip:stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java import org.apache.kafka.clients.processor.RecordCollector; import org.apache.kafka.clients.processor.RestoreFunc; import org.apache.kafka.clients.processor.StateStore; @@ -71,9 +76,8 @@ import org.apache.kafka.clients.processor.internals.PunctuationQueue; import java.io.File; -import java.util.Map; -public class MockKStreamContext implements KStreamContext { +public class MockProcessorContext implements ProcessorContext { <<<<<<< HEAD Serializer serializer; @@ -175,9 +179,8 @@ public void schedule(Processor processor, long interval) { ======= Serializer serializer; Deserializer deserializer; - private final PunctuationQueue punctuationQueue = new PunctuationQueue(); - public MockKStreamContext(Serializer serializer, Deserializer deserializer) { + public MockProcessorContext(Serializer serializer, Deserializer deserializer) { this.serializer = serializer; this.deserializer = deserializer; } @@ -212,11 +215,6 @@ public RecordCollector recordCollector() { throw new UnsupportedOperationException("recordCollector() not supported."); } - @Override - public Map getContext() { - throw new UnsupportedOperationException("getContext() not supported."); - } - @Override public File stateDir() { throw new UnsupportedOperationException("stateDir() not supported."); @@ -248,7 +246,7 @@ public void send(String topic, Object key, Object value, Serializer keyS } @Override - public void schedule(Processor processor, long interval) { + public void schedule(KafkaProcessor processor, long interval) { throw new UnsupportedOperationException("schedule() not supported"); } diff --git a/stream/src/test/java/org/apache/kafka/test/MockSource.java b/stream/src/test/java/org/apache/kafka/test/MockSource.java new file mode 100644 index 0000000000000..152c62c4daadd --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/test/MockSource.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.test; + +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.internals.KafkaSource; +import org.apache.kafka.common.serialization.Deserializer; + +import java.util.ArrayList; + +public class MockSource extends KafkaSource { + + private ProcessorContext context; + + public Deserializer keyDeserializer; + public Deserializer valDeserializer; + + public int numReceived = 0; + public ArrayList keys = new ArrayList<>(); + public ArrayList values = new ArrayList<>(); + public ArrayList timestamps = new ArrayList<>(); + + public MockSource(Deserializer keyDeserializer, Deserializer valDeserializer) { + super(keyDeserializer, valDeserializer); + + this.keyDeserializer = keyDeserializer; + this.valDeserializer = valDeserializer; + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + @Override + public void process(K key, V value) { + this.numReceived++; + this.keys.add(key); + this.values.add(value); + this.timestamps.add(context.timestamp()); + } + + @Override + public void close() { + // do nothing + } +} diff --git a/temp b/temp index 966d60b3355fc..c4745705db762 100644 --- a/temp +++ b/temp @@ -1,4 +1,10 @@ -stream/src/main/java/org/apache/kafka/stream/KStreamContext.java: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java: needs merge stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java: needs merge -stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java: needs merge +stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java: needs merge +stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java: needs merge diff --git a/temp2 b/temp2 index 4427ad7768ce6..a1493c554cf06 100644 --- a/temp2 +++ b/temp2 @@ -1,4 +1,10 @@ -stream/src/main/java/org/apache/kafka/stream/KStreamContext.java -stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java -stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java +stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java +stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java +stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java From 5502aa78ae9a0cd5ce73a340d7fb56b48eb07913 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sat, 8 Aug 2015 13:42:00 -0700 Subject: [PATCH 152/275] wip --- .../clients/processor/KStreamException.java | 37 ----- .../processor/internals/ParallelExecutor.java | 152 ------------------ .../processor/internals/PartitioningInfo.java | 27 ---- .../internals/ProcessorContextImpl.java | 4 +- .../java/org/apache/kafka/stream/KStream.java | 4 +- .../stream/NotCopartitionedException.java | 28 ---- .../org/apache/kafka/stream/Processor.java | 31 ---- .../apache/kafka/stream/SlidingWindow.java | 5 +- .../java/org/apache/kafka/stream/Window.java | 3 +- .../{util => internals}/FilteredIterator.java | 2 +- .../kafka/stream/internals/KStreamImpl.java | 3 +- .../stream/internals/KStreamMetadata.java | 67 -------- .../kafka/stream/FilteredIteratorTest.java | 2 +- .../apache/kafka/stream/KStreamJoinTest.java | 31 ++-- .../kafka/test/MockProcessorContext.java | 9 +- .../apache/kafka/test/UnlimitedWindow.java | 10 +- temp | 10 +- temp2 | 10 +- 18 files changed, 45 insertions(+), 390 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/KStreamException.java delete mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/ParallelExecutor.java delete mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/PartitioningInfo.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/Processor.java rename stream/src/main/java/org/apache/kafka/stream/{util => internals}/FilteredIterator.java (97%) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamMetadata.java diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/KStreamException.java b/stream/src/main/java/org/apache/kafka/clients/processor/KStreamException.java deleted file mode 100644 index 5c3b58c9e5531..0000000000000 --- a/stream/src/main/java/org/apache/kafka/clients/processor/KStreamException.java +++ /dev/null @@ -1,37 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.clients.processor; - -public class KStreamException extends RuntimeException { - - public KStreamException() { - super(); - } - - public KStreamException(String msg) { - super(msg); - } - - public KStreamException(Throwable t) { - super(t); - } - - public KStreamException(String msg, Throwable t) { - super(msg, t); - } -} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ParallelExecutor.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ParallelExecutor.java deleted file mode 100644 index 894712bed48f0..0000000000000 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ParallelExecutor.java +++ /dev/null @@ -1,152 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.clients.processor.internals; - -import java.util.ArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.LockSupport; - -/** - * A lightweight parallel executor - */ -public class ParallelExecutor { - - /** - * A parallel task must implement this interface - */ - public interface Task { - /** - * Executes a task - * - * @return boolean true if the task are ready for next execution - */ - boolean process(); - } - - private final WorkerThread[] workerThreads; - private final AtomicInteger taskIndex = new AtomicInteger(0); - private volatile ArrayList tasks = new ArrayList<>(); - private volatile CountDownLatch latch; - private volatile boolean readyForNextExecution = true; - private volatile boolean running = true; - private volatile Exception exception; - - public ParallelExecutor(int parallelDegree) { - parallelDegree = Math.max(parallelDegree, 1); - workerThreads = new WorkerThread[parallelDegree - 1]; - for (int i = 0; i < workerThreads.length; i++) { - workerThreads[i] = new WorkerThread(); - workerThreads[i].start(); - } - } - - /** - * Executes tasks in parallel. While this method is executing, other execute call will be blocked. - * - * @param tasks a list of tasks executed in parallel - * @return boolean true if at least one task is ready for next execution, otherwise false - * @throws Exception an exception thrown by a failed task - */ - public boolean execute(ArrayList tasks) throws Exception { - synchronized (this) { - try { - int numTasks = tasks.size(); - exception = null; - readyForNextExecution = false; - if (numTasks > 0) { - this.tasks = tasks; - this.latch = new CountDownLatch(numTasks); - - taskIndex.set(numTasks); - wakeUpWorkers(Math.min(numTasks - 1, workerThreads.length)); - - // the calling thread also picks up tasks - if (taskIndex.get() > 0) doProcess(); - - while (true) { - try { - latch.await(); - break; - } catch (InterruptedException ex) { - Thread.interrupted(); - } - } - } - if (exception != null) throw exception; - } finally { - this.tasks = null; - this.latch = null; - this.exception = null; - } - return readyForNextExecution; - } - } - - /** - * Shuts this parallel executor down - */ - public void shutdown() { - synchronized (this) { - running = false; - // wake up all workers - wakeUpWorkers(workerThreads.length); - } - } - - private void doProcess() { - int index = taskIndex.decrementAndGet(); - if (index >= 0) { - try { - if (tasks.get(index).process()) - this.readyForNextExecution = true; - } catch (Exception ex) { - exception = ex; - } finally { - latch.countDown(); - } - } - } - - private void wakeUpWorkers(int numWorkers) { - for (int i = 0; i < numWorkers; i++) - LockSupport.unpark(workerThreads[i]); - } - - private class WorkerThread extends Thread { - - WorkerThread() { - super(); - setDaemon(true); - } - - @Override - public void run() { - while (running) { - if (taskIndex.get() > 0) { - doProcess(); - } else { - // no more work. park this thread. - LockSupport.park(); - Thread.interrupted(); - } - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PartitioningInfo.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/PartitioningInfo.java deleted file mode 100644 index fa901327e5133..0000000000000 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PartitioningInfo.java +++ /dev/null @@ -1,27 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.clients.processor.internals; - -public class PartitioningInfo { - - public final int numPartitions; - - public PartitioningInfo(int numPartitions) { - this.numPartitions = numPartitions; - } -} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java index a019ee0f16e56..6a4091286f130 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java @@ -23,11 +23,11 @@ import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.clients.processor.RecordCollector; import org.apache.kafka.clients.processor.StateStore; -import org.apache.kafka.clients.processor.KStreamException; import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.RestoreFunc; import org.apache.kafka.clients.processor.TimestampExtractor; import org.apache.kafka.clients.producer.ProducerRecord; +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.ByteArrayDeserializer; @@ -146,7 +146,7 @@ public Metrics metrics() { @Override public void register(StateStore store, RestoreFunc restoreFunc) { if (initialized) - throw new KStreamException("Can only create state stores during initialization."); + throw new KafkaException("Can only create state stores during initialization."); stateMgr.register(store, restoreFunc); } diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java index 5c90dccf05a46..af09d6d78e8d2 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java @@ -103,7 +103,7 @@ public interface KStream { /** * Sends key-value to a topic, also creates a new stream from the topic. * The created stream is added to the default synchronization group. - * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). + * This is equivalent to calling sendTo(topic) and from(topic). * * @param topic the topic name * @return KStream @@ -113,7 +113,7 @@ public interface KStream { /** * Sends key-value to a topic, also creates a new stream from the topic. * The created stream is added to the default synchronization group. - * This is equivalent to calling sendTo(topic) and KStreamContext.from(topic). + * This is equivalent to calling sendTo(topic) and from(topic). * * @param topic the topic name * @param keySerializer key serializer used to send key-value pairs, diff --git a/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java b/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java deleted file mode 100644 index b1f34ebd15303..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/NotCopartitionedException.java +++ /dev/null @@ -1,28 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -import org.apache.kafka.clients.processor.KStreamException; - -public class NotCopartitionedException extends KStreamException { - - public NotCopartitionedException() { - super(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/Processor.java b/stream/src/main/java/org/apache/kafka/stream/Processor.java deleted file mode 100644 index c97f8dbccb818..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/Processor.java +++ /dev/null @@ -1,31 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -import org.apache.kafka.stream.KStreamContext; - -public interface Processor { - - void init(KStreamContext context); - - void process(K key, V value); - - void punctuate(long streamTime); - - void close(); -} diff --git a/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java index 04ddb17f576af..fd0bb5393716a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java @@ -17,7 +17,8 @@ package org.apache.kafka.stream; -import org.apache.kafka.stream.util.FilteredIterator; +import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.internals.FilteredIterator; import org.apache.kafka.clients.processor.internals.Stamped; import java.util.HashMap; @@ -39,7 +40,7 @@ public SlidingWindow(String name, long duration, int maxCount) { } @Override - public void init(KStreamContext context) { + public void init(ProcessorContext context) { } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/Window.java b/stream/src/main/java/org/apache/kafka/stream/Window.java index 9ec059b22c810..3707b838b7958 100644 --- a/stream/src/main/java/org/apache/kafka/stream/Window.java +++ b/stream/src/main/java/org/apache/kafka/stream/Window.java @@ -17,13 +17,14 @@ package org.apache.kafka.stream; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.clients.processor.StateStore; import java.util.Iterator; public interface Window extends StateStore { - void init(KStreamContext context); + void init(ProcessorContext context); Iterator find(K key, long timestamp); diff --git a/stream/src/main/java/org/apache/kafka/stream/util/FilteredIterator.java b/stream/src/main/java/org/apache/kafka/stream/internals/FilteredIterator.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/util/FilteredIterator.java rename to stream/src/main/java/org/apache/kafka/stream/internals/FilteredIterator.java index daff513d328cd..179039710c355 100644 --- a/stream/src/main/java/org/apache/kafka/stream/util/FilteredIterator.java +++ b/stream/src/main/java/org/apache/kafka/stream/internals/FilteredIterator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.util; +package org.apache.kafka.stream.internals; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java index e720ca63c3c9f..1a7a833a0c5c3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java @@ -22,13 +22,12 @@ import org.apache.kafka.clients.processor.Processor; import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.clients.processor.internals.KafkaSource; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.KStream; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.stream.KStreamWindowed; import org.apache.kafka.stream.KeyValueMapper; import org.apache.kafka.stream.Predicate; +import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.ValueMapper; import org.apache.kafka.stream.Window; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMetadata.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMetadata.java deleted file mode 100644 index 40fca15243da4..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMetadata.java +++ /dev/null @@ -1,67 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.clients.processor.internals.StreamGroup; - -import java.util.Collections; -import java.util.Map; - -public class KStreamMetadata { - - public static final String UNKNOWN_TOPICNAME = "__UNKNOWN_TOPIC__"; - public static final int UNKNOWN_PARTITION = -1; - - public static KStreamMetadata unjoinable() { - return new KStreamMetadata(Collections.singletonMap(UNKNOWN_TOPICNAME, new PartitioningInfo(UNKNOWN_PARTITION))); - } - - public StreamGroup streamGroup; - public final Map topicPartitionInfos; - - public KStreamMetadata(Map topicPartitionInfos) { - this.topicPartitionInfos = topicPartitionInfos; - } - - boolean isJoinCompatibleWith(KStreamMetadata other) { - // the two streams should only be joinable if they are inside the same sync group - // and their contained streams all have the same number of partitions - if (this.streamGroup != other.streamGroup) - return false; - - int numPartitions = -1; - for (PartitioningInfo partitionInfo : this.topicPartitionInfos.values()) { - if (partitionInfo.numPartitions < 0) { - return false; - } else if (numPartitions >= 0) { - if (partitionInfo.numPartitions != numPartitions) - return false; - } else { - numPartitions = partitionInfo.numPartitions; - } - } - - for (PartitioningInfo partitionInfo : other.topicPartitionInfos.values()) { - if (partitionInfo.numPartitions != numPartitions) - return false; - } - - return true; - } -} diff --git a/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java b/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java index 6b0c679279b7b..66efc1476e02d 100644 --- a/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertEquals; -import org.apache.kafka.stream.util.FilteredIterator; +import org.apache.kafka.stream.internals.FilteredIterator; import org.junit.Test; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java index 4d84fa0641175..2722d1a32379d 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java @@ -20,8 +20,6 @@ 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.clients.processor.internals.PartitioningInfo; -import org.apache.kafka.stream.internals.KStreamMetadata; import org.apache.kafka.stream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopology; import org.apache.kafka.test.MockProcessor; @@ -32,8 +30,6 @@ import java.util.Collections; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; public class KStreamJoinTest { @@ -43,8 +39,6 @@ public class KStreamJoinTest { private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - private ValueJoiner joiner = new ValueJoiner() { @Override public String apply(String value1, String value2) { @@ -83,8 +77,6 @@ public KeyValue> apply(Integer key, String value) { }; - // TODO: initialize with context - @Test public void testJoin() { @@ -253,10 +245,15 @@ public void testJoinPrior() { windowed1.joinPrior(windowed2, joiner).process(processor); + MockProcessorContext context = new MockProcessorContext(null, null); + topology.init(context); + // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i], i); + context.setTime(i); + + ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i]); } assertEquals(0, processor.processed.size()); @@ -265,7 +262,9 @@ public void testJoinPrior() { // no corresponding item in the main window has a newer timestamp for (int i = 0; i < 2; i++) { - ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i], i + 1); + context.setTime(i + 1); + + ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i]); } assertEquals(0, processor.processed.size()); @@ -275,7 +274,9 @@ public void testJoinPrior() { // push all items with newer timestamps to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i], i + 2); + context.setTime(i + 2); + + ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i]); } assertEquals(2, processor.processed.size()); @@ -292,7 +293,9 @@ public void testJoinPrior() { // push all items with older timestamps to the other stream. this should produce six items for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i], i); + context.setTime(i); + + ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i]); } assertEquals(6, processor.processed.size()); @@ -302,6 +305,7 @@ public void testJoinPrior() { for (int i = 0; i < expected.length; i++) { assertEquals(expected[i], processor.processed.get(i)); } +<<<<<<< HEAD } @@ -715,6 +719,9 @@ public void testFlatMapValues() { } assertFalse(exceptionRaised); +======= +>>>>>>> wip } + // TODO: test for joinability } diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java index d1a9e30483e9b..d73531fc34097 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -73,7 +73,6 @@ 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.clients.processor.internals.PunctuationQueue; import java.io.File; @@ -180,11 +179,17 @@ public void schedule(Processor processor, long interval) { Serializer serializer; Deserializer deserializer; + long timestamp = -1L; + public MockProcessorContext(Serializer serializer, Deserializer deserializer) { this.serializer = serializer; this.deserializer = deserializer; } + public void setTime(long timestamp) { + this.timestamp = timestamp; + } + @Override public int id() { return -1; @@ -272,7 +277,7 @@ public long offset() { @Override public long timestamp() { - throw new UnsupportedOperationException("timestamp() not supported."); + return this.timestamp; } >>>>>>> compile and test passed diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java index 4a50fe871cd55..e60c60b0f77f4 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java @@ -17,10 +17,10 @@ package org.apache.kafka.test; -import org.apache.kafka.stream.KStreamContext; +import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.stream.KeyValue; import org.apache.kafka.stream.Window; -import org.apache.kafka.stream.util.FilteredIterator; +import org.apache.kafka.stream.internals.FilteredIterator; import org.apache.kafka.clients.processor.internals.Stamped; import java.util.Iterator; @@ -31,8 +31,8 @@ public class UnlimitedWindow implements Window { private LinkedList>> list = new LinkedList<>(); @Override - public void init(KStreamContext context) { - + public void init(ProcessorContext context) { + // do nothing } @Override @@ -63,7 +63,7 @@ protected V filter(Stamped> item) { @Override public void put(K key, V value, long timestamp) { - list.add(new Stamped>(KeyValue.pair(key, value), timestamp)); + list.add(new Stamped<>(KeyValue.pair(key, value), timestamp)); } @Override diff --git a/temp b/temp index c4745705db762..2fc7736c10c37 100644 --- a/temp +++ b/temp @@ -1,10 +1,2 @@ -stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java: needs merge +stream/src/main/java/org/apache/kafka/stream/KStreamContext.java: needs merge stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java: needs merge -stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java: needs merge diff --git a/temp2 b/temp2 index a1493c554cf06..c7f46c46812fc 100644 --- a/temp2 +++ b/temp2 @@ -1,10 +1,2 @@ -stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java +stream/src/main/java/org/apache/kafka/stream/KStreamContext.java stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java -stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java -stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java From 701987472a00dad801e42e5ab82001fc93f45b3d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sat, 8 Aug 2015 13:56:41 -0700 Subject: [PATCH 153/275] compiles --- .../java/org/apache/kafka/clients/processor/PTopology.java | 2 ++ .../kafka/clients/processor/internals/ProcessorConfig.java | 1 - .../kafka/clients/processor/internals/StreamGroup.java | 2 +- .../kafka/clients/processor/internals/StreamingConfig.java | 5 ++++- .../main/java/org/apache/kafka/stream/KStreamTopology.java | 6 ++++++ .../java/org/apache/kafka/stream/internals/KStreamImpl.java | 2 +- .../java/org/apache/kafka/stream/internals/KStreamJoin.java | 2 +- 7 files changed, 15 insertions(+), 5 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java index 1c98dae79fcd8..b511578e98a8e 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -37,6 +37,8 @@ abstract public class PTopology { protected final StreamingConfig streamingConfig; + public PTopology() { this(null); } + public PTopology(StreamingConfig streamingConfig) { this.streamingConfig = streamingConfig; } diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java index 2673de48fdfa1..d429994724360 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; -import org.apache.kafka.stream.StreamingConfig; import java.io.File; import java.util.Properties; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java index a54c7295af25b..b169962f3d82c 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java @@ -246,7 +246,7 @@ public void close() { } protected RecordQueue createRecordQueue(TopicPartition partition, KafkaSource source) { - return new RecordQueue(partition, source, new MinTimestampTracker<>()); + return new RecordQueue(partition, source, new MinTimestampTracker>()); } private static class NewRecords { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java index 66f89418fb925..972eba78796f4 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java @@ -24,7 +24,10 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import java.util.*; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + /** * Configuration information passed to the {@link KafkaStreaming} instance for configuring the associated diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java index ac99b8a29c26a..a55478e2e9a2e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; import org.apache.kafka.clients.processor.internals.StreamingConfig; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.internals.KStreamSource; @@ -28,6 +29,8 @@ */ public abstract class KStreamTopology extends PTopology { + public KStreamTopology() { super(); } + public KStreamTopology(StreamingConfig streamingConfig) { super(streamingConfig); } @@ -39,6 +42,9 @@ public KStreamTopology(StreamingConfig streamingConfig) { * @return KStream */ public KStream from(String... topics) { + if (streamingConfig == null) + throw new KafkaException("No default deserializers specified in the config."); + KafkaProcessor source = addSource( (Deserializer) streamingConfig.keyDeserializer(), (Deserializer) streamingConfig.valueDeserializer(), topics); diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java index 1a7a833a0c5c3..b120e36b070ce 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java @@ -149,7 +149,7 @@ public void sendTo(String topic, Serializer keySerializer, Serializer valS } @SuppressWarnings("unchecked") - private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { + private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { return new Processor() { private ProcessorContext context; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java index f80c0d708a08f..1b60a80ec8b14 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java @@ -104,7 +104,7 @@ public void init(ProcessorContext context) { int firstId = -1; for (TopicPartition partition : partitions) { if (!partitionsById.containsKey(partition.partition())) { - partitionsById.put(partition.partition(), new ArrayList<>()); + partitionsById.put(partition.partition(), new ArrayList()); } partitionsById.get(partition.partition()).add(partition.topic()); From e7859fffb67ee9d98ed54bdcd29e9f55471f6a3c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sat, 8 Aug 2015 14:03:28 -0700 Subject: [PATCH 154/275] minor renaming --- .../clients/processor/KafkaStreaming.java | 146 --------- .../kafka/clients/processor/PTopology.java | 8 +- .../processor/internals/KStreamThread.java | 306 ------------------ .../processor/internals/ProcessorConfig.java | 37 +-- .../internals/ProcessorContextImpl.java | 20 +- .../processor/internals/StreamGroup.java | 7 + .../processor/internals/StreamingConfig.java | 147 --------- .../apache/kafka/stream/KStreamTopology.java | 12 +- .../kafka/stream/examples/KStreamJob.java | 6 +- .../stream/examples/SimpleProcessJob.java | 6 +- .../stream/examples/StatefulProcessJob.java | 6 +- 11 files changed, 55 insertions(+), 646 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/KafkaStreaming.java delete mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java delete mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaStreaming.java deleted file mode 100644 index af3a60c5b4b95..0000000000000 --- a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaStreaming.java +++ /dev/null @@ -1,146 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.clients.processor; - -import org.apache.kafka.clients.processor.internals.KStreamThread; -import org.apache.kafka.clients.processor.internals.ProcessorConfig; -import org.apache.kafka.clients.processor.internals.StreamingConfig; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and - * sends output to zero or more output topics. - *

- * This processing is defined by extending the {@link PTopology} abstract class to specify the transformation operator build. The - * {@link KafkaStreaming} instance will be responsible for the lifecycle of these processors. It will instantiate and - * start one or more of these processors to process the Kafka partitions assigned to this particular instance. - *

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

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

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

- *    Properties props = new Properties();
- *    props.put("bootstrap.servers", "localhost:4242");
- *    StreamingConfig config = new StreamingConfig(props);
- *    config.processor(ExampleStreamProcessor.class);
- *    config.serialization(new StringSerializer(), new StringDeserializer());
- *    KafkaStreaming container = new KafkaStreaming(new MyKStreamTopology(), config);
- *    container.run();
- * 
- * - */ -public class KafkaStreaming implements Runnable { - - 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 STOPPING = 2; - private static final int STOPPED = 3; - private int state = CREATED; - - private final ProcessorConfig config; - private final Object lock = new Object(); - private final KStreamThread[] threads; - - - public KafkaStreaming(Class topologyClass, StreamingConfig streamingConfig) throws Exception { - if (streamingConfig.timestampExtractor() == null) - throw new NullPointerException("timestamp extractor is missing"); - - this.config = new ProcessorConfig(streamingConfig.config()); - - this.threads = new KStreamThread[this.config.numStreamThreads]; - for (int i = 0; i < this.threads.length; i++) { - this.threads[i] = new KStreamThread(topologyClass, streamingConfig); - } - } - - /** - * Execute the stream processors - */ - public void run() { - synchronized (lock) { - log.info("Starting container"); - if (state == CREATED) { - if (!config.stateDir.exists() && !config.stateDir.mkdirs()) - throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); - - for (KStreamThread thread : threads) thread.start(); - log.info("Start-up complete"); - } else { - throw new IllegalStateException("This container was already started"); - } - - state = RUNNING; - while (state == RUNNING) { - try { - lock.wait(); - } catch (InterruptedException ex) { - Thread.interrupted(); - } - } - - if (state == STOPPING) { - log.info("Shutting down the container"); - - for (KStreamThread thread : threads) - thread.close(); - - for (KStreamThread thread : threads) { - try { - thread.join(); - } catch (InterruptedException ex) { - Thread.interrupted(); - } - } - state = STOPPED; - lock.notifyAll(); - log.info("Shutdown complete"); - } - } - } - - /** - * Shutdown this streaming instance. - */ - public void close() { - synchronized (lock) { - if (state == CREATED || state == RUNNING) { - state = STOPPING; - lock.notifyAll(); - } - while (state == STOPPING) { - try { - lock.wait(); - } catch (InterruptedException ex) { - Thread.interrupted(); - } - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java index b511578e98a8e..54aba0b9313d9 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -17,8 +17,8 @@ package org.apache.kafka.clients.processor; +import org.apache.kafka.stream.internals.KStreamConfig; import org.apache.kafka.clients.processor.internals.KafkaSource; -import org.apache.kafka.clients.processor.internals.StreamingConfig; import org.apache.kafka.common.serialization.Deserializer; import java.util.ArrayDeque; @@ -35,12 +35,12 @@ abstract public class PTopology { private List processors = new ArrayList<>(); private Map sources = new HashMap<>(); - protected final StreamingConfig streamingConfig; + protected final KStreamConfig KStreamConfig; public PTopology() { this(null); } - public PTopology(StreamingConfig streamingConfig) { - this.streamingConfig = streamingConfig; + public PTopology(KStreamConfig KStreamConfig) { + this.KStreamConfig = KStreamConfig; } public Set sources() { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java deleted file mode 100644 index 0aba432e6bb92..0000000000000 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java +++ /dev/null @@ -1,306 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.clients.processor.internals; - -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.processor.PTopology; -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.serialization.Serializer; -import org.apache.kafka.common.utils.SystemTime; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; - -public class KStreamThread extends Thread { - - private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); - - private final Class topologyClass; - private final ArrayList streamGroups = new ArrayList<>(); - private final Map kstreamContexts = new HashMap<>(); - private final IngestorImpl ingestor; - private final RecordCollectorImpl collector; - private final StreamingConfig streamingConfig; - private final ProcessorConfig config; - private final Metrics metrics; - private final KafkaStreamingMetrics streamingMetrics; - private final Time time; - private volatile boolean running; - private long lastCommit; - private long nextStateCleaning; - private long recordsProcessed; - - protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { - @Override - public void onPartitionsAssigned(Consumer consumer, Collection assignment) { - ingestor.init(); - addPartitions(assignment); - } - - @Override - public void onPartitionsRevoked(Consumer consumer, Collection assignment) { - commitAll(time.milliseconds()); - removePartitions(); - ingestor.clear(); - } - }; - - @SuppressWarnings("unchecked") - public KStreamThread(Class topologyClass, StreamingConfig streamingConfig) throws Exception { - super(); - - if (streamingConfig.timestampExtractor() == null) - throw new NullPointerException("timestamp extractor is missing"); - - this.metrics = new Metrics(); - this.config = new ProcessorConfig(streamingConfig.config()); - this.topologyClass = topologyClass; - - this.streamingConfig = streamingConfig; - this.streamingMetrics = new KafkaStreamingMetrics(); - - // build the topology without initialization to get the topics for consumer - PTopology topology = topologyClass.getConstructor().newInstance(); - topology.build(); - - // create the producer and consumer clients - Producer producer = new KafkaProducer<>(streamingConfig.config(), new ByteArraySerializer(), new ByteArraySerializer()); - this.collector = new RecordCollectorImpl(producer, (Serializer) streamingConfig.keySerializer(), (Serializer) streamingConfig.valueSerializer()); - - Consumer consumer = new KafkaConsumer<>(streamingConfig.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); - this.ingestor = new IngestorImpl(consumer, topology.topics()); - - this.running = true; - this.lastCommit = 0; - this.nextStateCleaning = Long.MAX_VALUE; - this.recordsProcessed = 0; - this.time = new SystemTime(); - } - - /** - * Execute the stream processors - */ - @Override - public synchronized void run() { - log.info("Starting a kstream thread"); - try { - ingestor.open(); - runLoop(); - } catch (RuntimeException e) { - log.error("Uncaught error during processing: ", e); - throw e; - } finally { - shutdown(); - } - } - - private void shutdown() { - log.info("Shutting down a kstream thread"); - commitAll(time.milliseconds()); - - collector.close(); - ingestor.close(); - removePartitions(); - log.info("kstream thread shutdown complete"); - } - - /** - * Shutdown this streaming thread. - */ - public synchronized void close() { - running = false; - } - - private void runLoop() { - try { - boolean readyForNextExecution = false; - - while (stillRunning()) { - ingestor.poll(readyForNextExecution ? 0 : this.config.pollTimeMs); - - for (StreamGroup group : this.streamGroups) { - readyForNextExecution = group.process(); - } - - maybeCommit(); - maybeCleanState(); - } - } catch (Exception e) { - throw new KafkaException(e); - } - } - - private boolean stillRunning() { - if (!running) { - log.debug("Shutting down at user request."); - return false; - } - if (config.totalRecordsToProcess >= 0 && recordsProcessed >= config.totalRecordsToProcess) { - log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", config.totalRecordsToProcess); - return false; - } - return true; - } - - private void maybeCommit() { - long now = time.milliseconds(); - if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < now) { - log.trace("Committing processor instances because the commit interval has elapsed."); - commitAll(now); - } - } - - private void commitAll(long now) { - Map commit = new HashMap<>(); - for (ProcessorContextImpl context : kstreamContexts.values()) { - context.flush(); - commit.putAll(context.consumedOffsets()); - } - - // check if commit is really needed, i.e. if all the offsets are already committed - if (ingestor.commitNeeded(commit)) { - // TODO: for exactly-once we need to make sure the flush and commit - // are executed atomically whenever it is triggered by user - collector.flush(); - ingestor.commit(commit); // TODO: can this be async? - streamingMetrics.commitTime.record(now - lastCommit); - } - } - - /* delete any state dirs that aren't for active contexts */ - private void maybeCleanState() { - long now = time.milliseconds(); - if (now > nextStateCleaning) { - File[] stateDirs = config.stateDir.listFiles(); - if (stateDirs != null) { - for (File dir : stateDirs) { - try { - Integer id = Integer.parseInt(dir.getName()); - if (!kstreamContexts.keySet().contains(id)) { - log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); - Utils.rm(dir); - } - } catch (NumberFormatException e) { - log.warn("Deleting unknown directory in state directory {}.", dir.getAbsolutePath()); - Utils.rm(dir); - } - } - } - nextStateCleaning = Long.MAX_VALUE; - } - } - - private void addPartitions(Collection assignment) { - HashSet partitions = new HashSet<>(assignment); - - for (TopicPartition partition : partitions) { - final Integer id = partition.partition(); // TODO: switch this to the group id - ProcessorContextImpl context = kstreamContexts.get(id); - if (context == null) { - try { - // build the topology and initialize with the context - PTopology topology = this.topologyClass.getConstructor().newInstance(); - context = new ProcessorContextImpl(id, ingestor, topology, collector, streamingConfig, config, metrics); - topology.build(); - topology.init(context); - context.initialized(); - kstreamContexts.put(id, context); - } catch (Exception e) { - throw new KafkaException(e); - } - - streamGroups.add(context.streamGroup); - } - - context.addPartition(partition); - } - - nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; - } - - private void removePartitions() { - for (ProcessorContextImpl context : kstreamContexts.values()) { - log.info("Removing task context {}", context.id()); - try { - context.close(); - } catch (Exception e) { - throw new KafkaException(e); - } - streamingMetrics.processorDestruction.record(); - } - streamGroups.clear(); - } - - private class KafkaStreamingMetrics { - final Sensor commitTime; - final Sensor processTime; - final Sensor windowTime; - final Sensor processorCreation; - final Sensor processorDestruction; - - public KafkaStreamingMetrics() { - String group = "kafka-streaming"; - - this.commitTime = metrics.sensor("commit-time"); - this.commitTime.add(new MetricName(group, "commit-time-avg-ms"), new Avg()); - this.commitTime.add(new MetricName(group, "commits-time-max-ms"), new Max()); - this.commitTime.add(new MetricName(group, "commits-per-second"), new Rate(new Count())); - - this.processTime = metrics.sensor("process-time"); - this.commitTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); - this.commitTime.add(new MetricName(group, "process-time-max-ms"), new Max()); - this.commitTime.add(new MetricName(group, "process-calls-per-second"), new Rate(new Count())); - - this.windowTime = metrics.sensor("window-time"); - this.windowTime.add(new MetricName(group, "window-time-avg-ms"), new Avg()); - this.windowTime.add(new MetricName(group, "window-time-max-ms"), new Max()); - this.windowTime.add(new MetricName(group, "window-calls-per-second"), new Rate(new Count())); - - this.processorCreation = metrics.sensor("processor-creation"); - this.processorCreation.add(new MetricName(group, "processor-creation"), new Rate(new Count())); - - this.processorDestruction = metrics.sensor("processor-destruction"); - this.processorDestruction.add(new MetricName(group, "processor-destruction"), new Rate(new Count())); - - } - - } - -} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java index d429994724360..9f2f870bcd9cc 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.stream.internals.KStreamConfig; import java.io.File; import java.util.Properties; @@ -30,47 +31,47 @@ public class ProcessorConfig extends AbstractConfig { private static final ConfigDef CONFIG; static { - CONFIG = new ConfigDef().define(StreamingConfig.TOPICS_CONFIG, + CONFIG = new ConfigDef().define(KStreamConfig.TOPICS_CONFIG, Type.STRING, "", Importance.HIGH, "All the possible topic names this job need to interact with") - .define(StreamingConfig.STATE_DIR_CONFIG, + .define(KStreamConfig.STATE_DIR_CONFIG, Type.STRING, System.getProperty("java.io.tmpdir"), Importance.MEDIUM, "") - .define(StreamingConfig.POLL_TIME_MS_CONFIG, + .define(KStreamConfig.POLL_TIME_MS_CONFIG, Type.LONG, 100, Importance.LOW, "The amount of time to block waiting for input.") - .define(StreamingConfig.COMMIT_TIME_MS_CONFIG, + .define(KStreamConfig.COMMIT_TIME_MS_CONFIG, Type.LONG, 30000, Importance.HIGH, "The frequency with which to save the position of the processor.") - .define(StreamingConfig.WINDOW_TIME_MS_CONFIG, + .define(KStreamConfig.WINDOW_TIME_MS_CONFIG, Type.LONG, -1L, Importance.MEDIUM, "Setting this to a non-negative value will cause the processor to get called with this frequency even if there is no message.") - .define(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, + .define(KStreamConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, Type.INT, 1000, Importance.LOW, "The maximum number of records to buffer per partition") - .define(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG, + .define(KStreamConfig.STATE_CLEANUP_DELAY_CONFIG, Type.LONG, 60000, Importance.LOW, "The amount of time to wait before deleting state when a partition has migrated.") - .define(StreamingConfig.TOTAL_RECORDS_TO_PROCESS, + .define(KStreamConfig.TOTAL_RECORDS_TO_PROCESS, Type.LONG, -1L, Importance.LOW, "Exit after processing this many records.") - .define(StreamingConfig.NUM_STREAM_THREADS, + .define(KStreamConfig.NUM_STREAM_THREADS, Type.INT, 1, Importance.LOW, @@ -89,15 +90,15 @@ public class ProcessorConfig extends AbstractConfig { public ProcessorConfig(Properties processor) { super(CONFIG, processor); - this.topics = this.getString(StreamingConfig.TOPICS_CONFIG); - this.stateDir = new File(this.getString(StreamingConfig.STATE_DIR_CONFIG)); - this.pollTimeMs = this.getLong(StreamingConfig.POLL_TIME_MS_CONFIG); - this.commitTimeMs = this.getLong(StreamingConfig.COMMIT_TIME_MS_CONFIG); - this.windowTimeMs = this.getLong(StreamingConfig.WINDOW_TIME_MS_CONFIG); - this.bufferedRecordsPerPartition = this.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); - this.stateCleanupDelay = this.getLong(StreamingConfig.STATE_CLEANUP_DELAY_CONFIG); - this.totalRecordsToProcess = this.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); - this.numStreamThreads = this.getInt(StreamingConfig.NUM_STREAM_THREADS); + this.topics = this.getString(KStreamConfig.TOPICS_CONFIG); + this.stateDir = new File(this.getString(KStreamConfig.STATE_DIR_CONFIG)); + this.pollTimeMs = this.getLong(KStreamConfig.POLL_TIME_MS_CONFIG); + this.commitTimeMs = this.getLong(KStreamConfig.COMMIT_TIME_MS_CONFIG); + this.windowTimeMs = this.getLong(KStreamConfig.WINDOW_TIME_MS_CONFIG); + this.bufferedRecordsPerPartition = this.getInt(KStreamConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + this.stateCleanupDelay = this.getLong(KStreamConfig.STATE_CLEANUP_DELAY_CONFIG); + this.totalRecordsToProcess = this.getLong(KStreamConfig.TOTAL_RECORDS_TO_PROCESS); + this.numStreamThreads = this.getInt(KStreamConfig.NUM_STREAM_THREADS); } } diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java index 6a4091286f130..1ded993f9eee7 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java @@ -35,7 +35,7 @@ import org.apache.kafka.common.serialization.Serializer; - +import org.apache.kafka.stream.internals.KStreamConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,7 +55,7 @@ public class ProcessorContextImpl implements ProcessorContext { private final PTopology topology; private final RecordCollectorImpl collector; private final ProcessorStateManager stateMgr; - private final StreamingConfig streamingConfig; + private final org.apache.kafka.stream.internals.KStreamConfig KStreamConfig; private final ProcessorConfig processorConfig; private final TimestampExtractor timestampExtractor; @@ -66,7 +66,7 @@ public ProcessorContextImpl(int id, Ingestor ingestor, PTopology topology, RecordCollectorImpl collector, - StreamingConfig streamingConfig, + KStreamConfig KStreamConfig, ProcessorConfig processorConfig, Metrics metrics) throws IOException { this.id = id; @@ -74,9 +74,9 @@ public ProcessorContextImpl(int id, this.ingestor = ingestor; this.topology = topology; this.collector = collector; - this.streamingConfig = streamingConfig; + this.KStreamConfig = KStreamConfig; this.processorConfig = processorConfig; - this.timestampExtractor = this.streamingConfig.timestampExtractor(); + this.timestampExtractor = this.KStreamConfig.timestampExtractor(); for (String topic : this.topology.topics()) { if (!ingestor.topics().contains(topic)) @@ -84,7 +84,7 @@ public ProcessorContextImpl(int id, } File stateFile = new File(processorConfig.stateDir, Integer.toString(id)); - Consumer restoreConsumer = new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + Consumer restoreConsumer = new KafkaConsumer<>(KStreamConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); this.streamGroup = new StreamGroup(this, this.ingestor, new TimeBasedChooser(), this.timestampExtractor, this.processorConfig.bufferedRecordsPerPartition); @@ -110,22 +110,22 @@ public int id() { @Override public Serializer keySerializer() { - return streamingConfig.keySerializer(); + return KStreamConfig.keySerializer(); } @Override public Serializer valueSerializer() { - return streamingConfig.valueSerializer(); + return KStreamConfig.valueSerializer(); } @Override public Deserializer keyDeserializer() { - return streamingConfig.keyDeserializer(); + return KStreamConfig.keyDeserializer(); } @Override public Deserializer valueDeserializer() { - return streamingConfig.valueDeserializer(); + return KStreamConfig.valueDeserializer(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java index b169962f3d82c..544127cca0df3 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java @@ -22,6 +22,13 @@ import org.apache.kafka.clients.processor.Punctuator; import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.clients.processor.TimestampExtractor; +import org.apache.kafka.clients.processor.internals.Ingestor; +import org.apache.kafka.clients.processor.internals.KafkaSource; +import org.apache.kafka.clients.processor.internals.MinTimestampTracker; +import org.apache.kafka.clients.processor.internals.PunctuationQueue; +import org.apache.kafka.clients.processor.internals.PunctuationSchedule; +import org.apache.kafka.clients.processor.internals.RecordQueue; +import org.apache.kafka.clients.processor.internals.StampedRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java deleted file mode 100644 index 972eba78796f4..0000000000000 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamingConfig.java +++ /dev/null @@ -1,147 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.clients.processor.internals; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.processor.KafkaStreaming; -import org.apache.kafka.clients.processor.TimestampExtractor; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; - -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - - -/** - * Configuration information passed to the {@link KafkaStreaming} instance for configuring the associated - * {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer}, and - * {@link org.apache.kafka.clients.consumer.KafkaConsumer KafkaConsumer}, as - * well as the processor itself. - */ -public class StreamingConfig { - - /** topics */ - public static final String TOPICS_CONFIG = "topics"; - - /** state.dir */ - public static final String STATE_DIR_CONFIG = "state.dir"; - - /** poll.time.ms */ - public static final String POLL_TIME_MS_CONFIG = "poll.time.ms"; - - /** commit.time.ms */ - public static final String COMMIT_TIME_MS_CONFIG = "commit.time.ms"; - - /** window.time.ms */ - public static final String WINDOW_TIME_MS_CONFIG = "window.time.ms"; - - /** buffered.records.per.partition */ - public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; - - /** state.cleanup.delay */ - public static final String STATE_CLEANUP_DELAY_CONFIG = "state.cleanup.delay"; - - /** total.records.to.process */ - public static final String TOTAL_RECORDS_TO_PROCESS = "total.records.to.process"; - - /** num.of.stream.threads */ - public static final String NUM_STREAM_THREADS = "num.of.stream.threads"; - - private final Properties config; - private final Map context = new HashMap(); - private Serializer keySerializer; - private Serializer valSerializer; - private Deserializer keyDeserializer; - private Deserializer valDeserializer; - private TimestampExtractor timestampExtractor; - - public StreamingConfig(Properties config) { - this.config = config; - this.config.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - this.config.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); - this.config.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); - } - - @Override - public StreamingConfig clone() { - return new StreamingConfig(this.config); - } - - public void addContextObject(String key, Object value) { - this.context.put(key, value); - } - - @SuppressWarnings("all") - - public void serialization(Serializer serializer, Deserializer deserializer) { - keySerializer(serializer); - valueSerializer(serializer); - keyDeserializer(deserializer); - valueDeserializer(deserializer); - } - - public void keySerializer(Serializer serializer) { - this.keySerializer = serializer; - } - - public void valueSerializer(Serializer serializer) { - this.valSerializer = serializer; - } - - public void keyDeserializer(Deserializer deserializer) { - this.keyDeserializer = deserializer; - } - - public void valueDeserializer(Deserializer deserializer) { - this.valDeserializer = deserializer; - } - - public Properties config() { - return this.config; - } - - public Map context() { - return this.context; - } - - public Serializer keySerializer() { - return this.keySerializer; - } - - public Serializer valueSerializer() { - return this.valSerializer; - } - - public Deserializer keyDeserializer() { - return this.keyDeserializer; - } - - public Deserializer valueDeserializer() { - return this.valDeserializer; - } - - public void timestampExtractor(TimestampExtractor timestampExtractor) { - this.timestampExtractor = timestampExtractor; - } - - public TimestampExtractor timestampExtractor() { - return this.timestampExtractor; - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java index a55478e2e9a2e..119675122de2c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java @@ -19,7 +19,7 @@ import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.internals.StreamingConfig; +import org.apache.kafka.stream.internals.KStreamConfig; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.internals.KStreamSource; @@ -31,8 +31,8 @@ public abstract class KStreamTopology extends PTopology { public KStreamTopology() { super(); } - public KStreamTopology(StreamingConfig streamingConfig) { - super(streamingConfig); + public KStreamTopology(KStreamConfig KStreamConfig) { + super(KStreamConfig); } /** @@ -42,12 +42,12 @@ public KStreamTopology(StreamingConfig streamingConfig) { * @return KStream */ public KStream from(String... topics) { - if (streamingConfig == null) + if (KStreamConfig == null) throw new KafkaException("No default deserializers specified in the config."); KafkaProcessor source = addSource( - (Deserializer) streamingConfig.keyDeserializer(), - (Deserializer) streamingConfig.valueDeserializer(), topics); + (Deserializer) KStreamConfig.keyDeserializer(), + (Deserializer) KStreamConfig.valueDeserializer(), topics); return new KStreamSource<>(this, source); } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java index a3ee21c14a6be..25c9fc66e6e60 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java @@ -17,9 +17,9 @@ package org.apache.kafka.stream.examples; +import org.apache.kafka.stream.internals.KStreamConfig; +import org.apache.kafka.stream.KStreamProcess; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.clients.processor.KafkaStreaming; -import org.apache.kafka.clients.processor.internals.StreamingConfig; import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.KStreamTopology; import org.apache.kafka.stream.KeyValue; @@ -71,7 +71,7 @@ public boolean apply(String key, Integer value) { } public static void main(String[] args) throws Exception { - KafkaStreaming kstream = new KafkaStreaming(MyKStreamTopology.class, new StreamingConfig(new Properties())); + KStreamProcess kstream = new KStreamProcess(MyKStreamTopology.class, new KStreamConfig(new Properties())); kstream.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java index 117b2ceade0fb..5fd134b0d1823 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java @@ -20,11 +20,11 @@ import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.internals.KStreamConfig; import org.apache.kafka.clients.processor.internals.KafkaSource; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.clients.processor.KafkaStreaming; -import org.apache.kafka.clients.processor.internals.StreamingConfig; +import org.apache.kafka.stream.KStreamProcess; import java.util.Properties; @@ -68,7 +68,7 @@ public void build() { } public static void main(String[] args) throws Exception { - KafkaStreaming streaming = new KafkaStreaming(MyPTopology.class, new StreamingConfig(new Properties())); + KStreamProcess streaming = new KStreamProcess(MyPTopology.class, new KStreamConfig(new Properties())); streaming.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java index f2a73abaf8e98..e371161e431c7 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java @@ -17,14 +17,14 @@ package org.apache.kafka.stream.examples; +import org.apache.kafka.stream.KStreamProcess; import org.apache.kafka.clients.processor.KafkaProcessor; import org.apache.kafka.clients.processor.PTopology; import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.clients.processor.internals.KafkaSource; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.clients.processor.KafkaStreaming; -import org.apache.kafka.clients.processor.internals.StreamingConfig; +import org.apache.kafka.stream.internals.KStreamConfig; import org.apache.kafka.stream.state.Entry; import org.apache.kafka.stream.state.InMemoryKeyValueStore; import org.apache.kafka.stream.state.KeyValueIterator; @@ -89,7 +89,7 @@ public void build() { } public static void main(String[] args) throws Exception { - KafkaStreaming streaming = new KafkaStreaming(MyPTopology.class, new StreamingConfig(new Properties())); + KStreamProcess streaming = new KStreamProcess(MyPTopology.class, new KStreamConfig(new Properties())); streaming.run(); } } From 6ed0104cadb906b8760880f0a9628a75faa17fef Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sat, 8 Aug 2015 14:03:57 -0700 Subject: [PATCH 155/275] adding files --- .../apache/kafka/stream/KStreamProcess.java | 147 +++++++++ .../kafka/stream/internals/KStreamConfig.java | 147 +++++++++ .../kafka/stream/internals/KStreamThread.java | 311 ++++++++++++++++++ 3 files changed, 605 insertions(+) create mode 100644 stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamConfig.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamThread.java diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java b/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java new file mode 100644 index 0000000000000..cea6342aa7430 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java @@ -0,0 +1,147 @@ +/** + * 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.stream; + +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.stream.internals.KStreamThread; +import org.apache.kafka.clients.processor.internals.ProcessorConfig; +import org.apache.kafka.stream.internals.KStreamConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and + * sends output to zero or more output topics. + *

+ * This processing is defined by extending the {@link PTopology} abstract class to specify the transformation operator build. The + * {@link KStreamProcess} instance will be responsible for the lifecycle of these processors. It will instantiate and + * start one or more of these processors to process the Kafka partitions assigned to this particular instance. + *

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

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

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

+ *    Properties props = new Properties();
+ *    props.put("bootstrap.servers", "localhost:4242");
+ *    KStreamConfig config = new KStreamConfig(props);
+ *    config.processor(ExampleStreamProcessor.class);
+ *    config.serialization(new StringSerializer(), new StringDeserializer());
+ *    KStreamProcess container = new KStreamProcess(new MyKStreamTopology(), config);
+ *    container.run();
+ * 
+ * + */ +public class KStreamProcess implements Runnable { + + private static final Logger log = LoggerFactory.getLogger(KStreamProcess.class); + + // Container States + private static final int CREATED = 0; + private static final int RUNNING = 1; + private static final int STOPPING = 2; + private static final int STOPPED = 3; + private int state = CREATED; + + private final ProcessorConfig config; + private final Object lock = new Object(); + private final KStreamThread[] threads; + + + public KStreamProcess(Class topologyClass, KStreamConfig KStreamConfig) throws Exception { + if (KStreamConfig.timestampExtractor() == null) + throw new NullPointerException("timestamp extractor is missing"); + + this.config = new ProcessorConfig(KStreamConfig.config()); + + this.threads = new KStreamThread[this.config.numStreamThreads]; + for (int i = 0; i < this.threads.length; i++) { + this.threads[i] = new KStreamThread(topologyClass, KStreamConfig); + } + } + + /** + * Execute the stream processors + */ + public void run() { + synchronized (lock) { + log.info("Starting container"); + if (state == CREATED) { + if (!config.stateDir.exists() && !config.stateDir.mkdirs()) + throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); + + for (KStreamThread thread : threads) thread.start(); + log.info("Start-up complete"); + } else { + throw new IllegalStateException("This container was already started"); + } + + state = RUNNING; + while (state == RUNNING) { + try { + lock.wait(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + + if (state == STOPPING) { + log.info("Shutting down the container"); + + for (KStreamThread thread : threads) + thread.close(); + + for (KStreamThread thread : threads) { + try { + thread.join(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + state = STOPPED; + lock.notifyAll(); + log.info("Shutdown complete"); + } + } + } + + /** + * Shutdown this streaming instance. + */ + public void close() { + synchronized (lock) { + if (state == CREATED || state == RUNNING) { + state = STOPPING; + lock.notifyAll(); + } + while (state == STOPPING) { + try { + lock.wait(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamConfig.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamConfig.java new file mode 100644 index 0000000000000..0e52975fac867 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamConfig.java @@ -0,0 +1,147 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.stream.KStreamProcess; +import org.apache.kafka.clients.processor.TimestampExtractor; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + + +/** + * Configuration information passed to the {@link KStreamProcess} instance for configuring the associated + * {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer}, and + * {@link org.apache.kafka.clients.consumer.KafkaConsumer KafkaConsumer}, as + * well as the processor itself. + */ +public class KStreamConfig { + + /** topics */ + public static final String TOPICS_CONFIG = "topics"; + + /** state.dir */ + public static final String STATE_DIR_CONFIG = "state.dir"; + + /** poll.time.ms */ + public static final String POLL_TIME_MS_CONFIG = "poll.time.ms"; + + /** commit.time.ms */ + public static final String COMMIT_TIME_MS_CONFIG = "commit.time.ms"; + + /** window.time.ms */ + public static final String WINDOW_TIME_MS_CONFIG = "window.time.ms"; + + /** buffered.records.per.partition */ + public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; + + /** state.cleanup.delay */ + public static final String STATE_CLEANUP_DELAY_CONFIG = "state.cleanup.delay"; + + /** total.records.to.process */ + public static final String TOTAL_RECORDS_TO_PROCESS = "total.records.to.process"; + + /** num.of.stream.threads */ + public static final String NUM_STREAM_THREADS = "num.of.stream.threads"; + + private final Properties config; + private final Map context = new HashMap(); + private Serializer keySerializer; + private Serializer valSerializer; + private Deserializer keyDeserializer; + private Deserializer valDeserializer; + private TimestampExtractor timestampExtractor; + + public KStreamConfig(Properties config) { + this.config = config; + this.config.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + this.config.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); + this.config.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); + } + + @Override + public KStreamConfig clone() { + return new KStreamConfig(this.config); + } + + public void addContextObject(String key, Object value) { + this.context.put(key, value); + } + + @SuppressWarnings("all") + + public void serialization(Serializer serializer, Deserializer deserializer) { + keySerializer(serializer); + valueSerializer(serializer); + keyDeserializer(deserializer); + valueDeserializer(deserializer); + } + + public void keySerializer(Serializer serializer) { + this.keySerializer = serializer; + } + + public void valueSerializer(Serializer serializer) { + this.valSerializer = serializer; + } + + public void keyDeserializer(Deserializer deserializer) { + this.keyDeserializer = deserializer; + } + + public void valueDeserializer(Deserializer deserializer) { + this.valDeserializer = deserializer; + } + + public Properties config() { + return this.config; + } + + public Map context() { + return this.context; + } + + public Serializer keySerializer() { + return this.keySerializer; + } + + public Serializer valueSerializer() { + return this.valSerializer; + } + + public Deserializer keyDeserializer() { + return this.keyDeserializer; + } + + public Deserializer valueDeserializer() { + return this.valDeserializer; + } + + public void timestampExtractor(TimestampExtractor timestampExtractor) { + this.timestampExtractor = timestampExtractor; + } + + public TimestampExtractor timestampExtractor() { + return this.timestampExtractor; + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamThread.java new file mode 100644 index 0000000000000..8cd8f1b349263 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamThread.java @@ -0,0 +1,311 @@ +/** + * 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.stream.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.internals.IngestorImpl; +import org.apache.kafka.clients.processor.internals.ProcessorConfig; +import org.apache.kafka.clients.processor.internals.ProcessorContextImpl; +import org.apache.kafka.clients.processor.internals.RecordCollectorImpl; +import org.apache.kafka.clients.processor.internals.StreamGroup; +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.serialization.Serializer; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +public class KStreamThread extends Thread { + + private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); + + private final Class topologyClass; + private final ArrayList streamGroups = new ArrayList<>(); + private final Map kstreamContexts = new HashMap<>(); + private final IngestorImpl ingestor; + private final RecordCollectorImpl collector; + private final KStreamConfig KStreamConfig; + private final ProcessorConfig config; + private final Metrics metrics; + private final KafkaStreamingMetrics streamingMetrics; + private final Time time; + private volatile boolean running; + private long lastCommit; + private long nextStateCleaning; + private long recordsProcessed; + + protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { + @Override + public void onPartitionsAssigned(Consumer consumer, Collection assignment) { + ingestor.init(); + addPartitions(assignment); + } + + @Override + public void onPartitionsRevoked(Consumer consumer, Collection assignment) { + commitAll(time.milliseconds()); + removePartitions(); + ingestor.clear(); + } + }; + + @SuppressWarnings("unchecked") + public KStreamThread(Class topologyClass, KStreamConfig KStreamConfig) throws Exception { + super(); + + if (KStreamConfig.timestampExtractor() == null) + throw new NullPointerException("timestamp extractor is missing"); + + this.metrics = new Metrics(); + this.config = new ProcessorConfig(KStreamConfig.config()); + this.topologyClass = topologyClass; + + this.KStreamConfig = KStreamConfig; + this.streamingMetrics = new KafkaStreamingMetrics(); + + // build the topology without initialization to get the topics for consumer + PTopology topology = topologyClass.getConstructor().newInstance(); + topology.build(); + + // create the producer and consumer clients + Producer producer = new KafkaProducer<>(KStreamConfig.config(), new ByteArraySerializer(), new ByteArraySerializer()); + this.collector = new RecordCollectorImpl(producer, (Serializer) KStreamConfig.keySerializer(), (Serializer) KStreamConfig.valueSerializer()); + + Consumer consumer = new KafkaConsumer<>(KStreamConfig.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + this.ingestor = new IngestorImpl(consumer, topology.topics()); + + this.running = true; + this.lastCommit = 0; + this.nextStateCleaning = Long.MAX_VALUE; + this.recordsProcessed = 0; + this.time = new SystemTime(); + } + + /** + * Execute the stream processors + */ + @Override + public synchronized void run() { + log.info("Starting a kstream thread"); + try { + ingestor.open(); + runLoop(); + } catch (RuntimeException e) { + log.error("Uncaught error during processing: ", e); + throw e; + } finally { + shutdown(); + } + } + + private void shutdown() { + log.info("Shutting down a kstream thread"); + commitAll(time.milliseconds()); + + collector.close(); + ingestor.close(); + removePartitions(); + log.info("kstream thread shutdown complete"); + } + + /** + * Shutdown this streaming thread. + */ + public synchronized void close() { + running = false; + } + + private void runLoop() { + try { + boolean readyForNextExecution = false; + + while (stillRunning()) { + ingestor.poll(readyForNextExecution ? 0 : this.config.pollTimeMs); + + for (StreamGroup group : this.streamGroups) { + readyForNextExecution = group.process(); + } + + maybeCommit(); + maybeCleanState(); + } + } catch (Exception e) { + throw new KafkaException(e); + } + } + + private boolean stillRunning() { + if (!running) { + log.debug("Shutting down at user request."); + return false; + } + if (config.totalRecordsToProcess >= 0 && recordsProcessed >= config.totalRecordsToProcess) { + log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", config.totalRecordsToProcess); + return false; + } + return true; + } + + private void maybeCommit() { + long now = time.milliseconds(); + if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < now) { + log.trace("Committing processor instances because the commit interval has elapsed."); + commitAll(now); + } + } + + private void commitAll(long now) { + Map commit = new HashMap<>(); + for (ProcessorContextImpl context : kstreamContexts.values()) { + context.flush(); + commit.putAll(context.consumedOffsets()); + } + + // check if commit is really needed, i.e. if all the offsets are already committed + if (ingestor.commitNeeded(commit)) { + // TODO: for exactly-once we need to make sure the flush and commit + // are executed atomically whenever it is triggered by user + collector.flush(); + ingestor.commit(commit); // TODO: can this be async? + streamingMetrics.commitTime.record(now - lastCommit); + } + } + + /* delete any state dirs that aren't for active contexts */ + private void maybeCleanState() { + long now = time.milliseconds(); + if (now > nextStateCleaning) { + File[] stateDirs = config.stateDir.listFiles(); + if (stateDirs != null) { + for (File dir : stateDirs) { + try { + Integer id = Integer.parseInt(dir.getName()); + if (!kstreamContexts.keySet().contains(id)) { + log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); + Utils.rm(dir); + } + } catch (NumberFormatException e) { + log.warn("Deleting unknown directory in state directory {}.", dir.getAbsolutePath()); + Utils.rm(dir); + } + } + } + nextStateCleaning = Long.MAX_VALUE; + } + } + + private void addPartitions(Collection assignment) { + HashSet partitions = new HashSet<>(assignment); + + for (TopicPartition partition : partitions) { + final Integer id = partition.partition(); // TODO: switch this to the group id + ProcessorContextImpl context = kstreamContexts.get(id); + if (context == null) { + try { + // build the topology and initialize with the context + PTopology topology = this.topologyClass.getConstructor().newInstance(); + context = new ProcessorContextImpl(id, ingestor, topology, collector, KStreamConfig, config, metrics); + topology.build(); + topology.init(context); + context.initialized(); + kstreamContexts.put(id, context); + } catch (Exception e) { + throw new KafkaException(e); + } + + streamGroups.add(context.streamGroup); + } + + context.addPartition(partition); + } + + nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; + } + + private void removePartitions() { + for (ProcessorContextImpl context : kstreamContexts.values()) { + log.info("Removing task context {}", context.id()); + try { + context.close(); + } catch (Exception e) { + throw new KafkaException(e); + } + streamingMetrics.processorDestruction.record(); + } + streamGroups.clear(); + } + + private class KafkaStreamingMetrics { + final Sensor commitTime; + final Sensor processTime; + final Sensor windowTime; + final Sensor processorCreation; + final Sensor processorDestruction; + + public KafkaStreamingMetrics() { + String group = "kafka-streaming"; + + this.commitTime = metrics.sensor("commit-time"); + this.commitTime.add(new MetricName(group, "commit-time-avg-ms"), new Avg()); + this.commitTime.add(new MetricName(group, "commits-time-max-ms"), new Max()); + this.commitTime.add(new MetricName(group, "commits-per-second"), new Rate(new Count())); + + this.processTime = metrics.sensor("process-time"); + this.commitTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); + this.commitTime.add(new MetricName(group, "process-time-max-ms"), new Max()); + this.commitTime.add(new MetricName(group, "process-calls-per-second"), new Rate(new Count())); + + this.windowTime = metrics.sensor("window-time"); + this.windowTime.add(new MetricName(group, "window-time-avg-ms"), new Avg()); + this.windowTime.add(new MetricName(group, "window-time-max-ms"), new Max()); + this.windowTime.add(new MetricName(group, "window-calls-per-second"), new Rate(new Count())); + + this.processorCreation = metrics.sensor("processor-creation"); + this.processorCreation.add(new MetricName(group, "processor-creation"), new Rate(new Count())); + + this.processorDestruction = metrics.sensor("processor-destruction"); + this.processorDestruction.add(new MetricName(group, "processor-destruction"), new Rate(new Count())); + + } + + } + +} From e2934810aab63684c6fcf773a9fb4fcadbdd4d61 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sat, 8 Aug 2015 21:19:36 -0700 Subject: [PATCH 156/275] adding files --- checkstyle/import-control.xml | 1 + kafka-work.iws | 199 +++++++----------- .../kafka/clients/processor/Chooser.java | 2 - .../clients/processor/KafkaProcessor.java | 12 +- .../{internals => }/KafkaSource.java | 8 +- .../kafka/clients/processor/PTopology.java | 16 +- .../{internals => }/ProcessorConfig.java | 39 ++-- .../clients/processor/ProcessorContext.java | 3 + .../processor/ProcessorProperties.java} | 14 +- .../kafka/clients/processor/RecordQueue.java | 83 ++++++++ .../processor/{internals => }/Stamped.java | 2 +- .../{internals => }/StampedRecord.java | 4 +- .../clients/processor/TimestampTracker.java | 2 - .../processor}/internals/KStreamThread.java | 27 ++- .../internals/MinTimestampTracker.java | 1 + .../internals/ProcessorContextImpl.java | 62 +++++- .../internals/PunctuationSchedule.java | 1 + ...{RecordQueue.java => RecordQueueImpl.java} | 34 ++- .../internals/RoundRobinChooser.java | 1 + .../processor/internals/StreamGroup.java | 18 +- .../processor/internals/TimeBasedChooser.java | 1 + .../apache/kafka/stream/KStreamProcess.java | 16 +- .../apache/kafka/stream/KStreamTopology.java | 16 +- .../apache/kafka/stream/SlidingWindow.java | 2 +- .../kafka/stream/examples/KStreamJob.java | 4 +- .../stream/examples/SimpleProcessJob.java | 6 +- .../stream/examples/StatefulProcessJob.java | 6 +- .../kafka/stream/internals/KStreamBranch.java | 4 +- .../kafka/stream/internals/KStreamImpl.java | 2 +- .../kafka/stream/internals/KStreamJoin.java | 45 +--- .../kafka/stream/internals/KStreamWindow.java | 4 +- .../kafka/stream/KStreamBranchTest.java | 12 +- .../apache/kafka/stream/KStreamJoinTest.java | 24 ++- .../kafka/stream/KStreamSourceTest.java | 2 - .../kafka/stream/KStreamWindowedTest.java | 6 +- .../kafka/stream/MinTimestampTrackerTest.java | 2 +- .../org/apache/kafka/test/MockProcessor.java | 6 +- .../kafka/test/MockProcessorContext.java | 6 + .../org/apache/kafka/test/MockSource.java | 17 +- .../apache/kafka/test/UnlimitedWindow.java | 2 +- temp | 3 +- temp2 | 3 +- 42 files changed, 392 insertions(+), 326 deletions(-) rename stream/src/main/java/org/apache/kafka/clients/processor/{internals => }/KafkaSource.java (86%) rename stream/src/main/java/org/apache/kafka/clients/processor/{internals => }/ProcessorConfig.java (68%) rename stream/src/main/java/org/apache/kafka/{stream/internals/KStreamConfig.java => clients/processor/ProcessorProperties.java} (91%) create mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/RecordQueue.java rename stream/src/main/java/org/apache/kafka/clients/processor/{internals => }/Stamped.java (95%) rename stream/src/main/java/org/apache/kafka/clients/processor/{internals => }/StampedRecord.java (91%) rename stream/src/main/java/org/apache/kafka/{stream => clients/processor}/internals/KStreamThread.java (92%) rename stream/src/main/java/org/apache/kafka/clients/processor/internals/{RecordQueue.java => RecordQueueImpl.java} (82%) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 624ef38bcc244..96a79588ed727 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -113,6 +113,7 @@ + diff --git a/kafka-work.iws b/kafka-work.iws index 46c931c0f0708..7c61e8a0889e2 100644 --- a/kafka-work.iws +++ b/kafka-work.iws @@ -2,117 +2,46 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + - - - - - - - - - + + + - - - - - + - - - - + + - - - - - + + - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + - - - - - + @@ -229,22 +158,42 @@ - + - + - + + + + + + + + + + + + + + + + + + + + + @@ -314,7 +263,6 @@ @@ -984,22 +933,6 @@ - - - - - - - - - - - - - - - - @@ -1064,19 +997,35 @@ - + - + + + + + + + + + + + + + + + + + diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java b/stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java index 935576554ae08..0d7aac763bc51 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.processor; -import org.apache.kafka.clients.processor.internals.RecordQueue; - public interface Chooser { void add(RecordQueue queue); diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java index 18a16bf544634..2000a4dcfaa08 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java @@ -39,11 +39,17 @@ public KafkaProcessor(String name) { this.closed = false; } - public String name() { return name; } + public String name() { + return name; + } - public List> children() { return children; } + public List> children() { + return children; + } - public List> parents() { return parents; } + public List> parents() { + return parents; + } public final void chain(KafkaProcessor child) { child.parents.add(this); diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaSource.java similarity index 86% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java rename to stream/src/main/java/org/apache/kafka/clients/processor/KafkaSource.java index 950eb0d4410b2..15c60636d50ea 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KafkaSource.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaSource.java @@ -15,10 +15,8 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.clients.processor; -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.ProcessorContext; import org.apache.kafka.common.serialization.Deserializer; public class KafkaSource extends KafkaProcessor { @@ -41,7 +39,9 @@ public void init(ProcessorContext context) { } @Override - public void process(K key, V value) { forward(key, value); } + public void process(K key, V value) { + forward(key, value); + } @Override public void close() { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java index 54aba0b9313d9..67e63fe53d84f 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.processor; -import org.apache.kafka.stream.internals.KStreamConfig; -import org.apache.kafka.clients.processor.internals.KafkaSource; import org.apache.kafka.common.serialization.Deserializer; import java.util.ArrayDeque; @@ -35,12 +33,14 @@ abstract public class PTopology { private List processors = new ArrayList<>(); private Map sources = new HashMap<>(); - protected final KStreamConfig KStreamConfig; + protected final ProcessorProperties properties; - public PTopology() { this(null); } + public PTopology() { + this(null); + } - public PTopology(KStreamConfig KStreamConfig) { - this.KStreamConfig = KStreamConfig; + public PTopology(ProcessorProperties properties) { + this.properties = properties; } public Set sources() { @@ -86,7 +86,7 @@ public final void init(ProcessorContext context) { deque.addLast(processor); } - while(!deque.isEmpty()) { + while (!deque.isEmpty()) { KafkaProcessor processor = deque.pollFirst(); boolean parentsInitialized = true; @@ -147,7 +147,7 @@ public final void close() { deque.addLast(processor); } - while(!deque.isEmpty()) { + while (!deque.isEmpty()) { KafkaProcessor processor = deque.pollFirst(); boolean parentsClosed = true; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorConfig.java similarity index 68% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java rename to stream/src/main/java/org/apache/kafka/clients/processor/ProcessorConfig.java index 9f2f870bcd9cc..2c4b2f980fad4 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorConfig.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorConfig.java @@ -15,13 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.clients.processor; 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 org.apache.kafka.stream.internals.KStreamConfig; import java.io.File; import java.util.Properties; @@ -31,47 +30,47 @@ public class ProcessorConfig extends AbstractConfig { private static final ConfigDef CONFIG; static { - CONFIG = new ConfigDef().define(KStreamConfig.TOPICS_CONFIG, + CONFIG = new ConfigDef().define(ProcessorProperties.TOPICS_CONFIG, Type.STRING, "", Importance.HIGH, "All the possible topic names this job need to interact with") - .define(KStreamConfig.STATE_DIR_CONFIG, + .define(ProcessorProperties.STATE_DIR_CONFIG, Type.STRING, System.getProperty("java.io.tmpdir"), Importance.MEDIUM, "") - .define(KStreamConfig.POLL_TIME_MS_CONFIG, + .define(ProcessorProperties.POLL_TIME_MS_CONFIG, Type.LONG, 100, Importance.LOW, "The amount of time to block waiting for input.") - .define(KStreamConfig.COMMIT_TIME_MS_CONFIG, + .define(ProcessorProperties.COMMIT_TIME_MS_CONFIG, Type.LONG, 30000, Importance.HIGH, "The frequency with which to save the position of the processor.") - .define(KStreamConfig.WINDOW_TIME_MS_CONFIG, + .define(ProcessorProperties.WINDOW_TIME_MS_CONFIG, Type.LONG, -1L, Importance.MEDIUM, "Setting this to a non-negative value will cause the processor to get called with this frequency even if there is no message.") - .define(KStreamConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, + .define(ProcessorProperties.BUFFERED_RECORDS_PER_PARTITION_CONFIG, Type.INT, 1000, Importance.LOW, "The maximum number of records to buffer per partition") - .define(KStreamConfig.STATE_CLEANUP_DELAY_CONFIG, + .define(ProcessorProperties.STATE_CLEANUP_DELAY_CONFIG, Type.LONG, 60000, Importance.LOW, "The amount of time to wait before deleting state when a partition has migrated.") - .define(KStreamConfig.TOTAL_RECORDS_TO_PROCESS, + .define(ProcessorProperties.TOTAL_RECORDS_TO_PROCESS, Type.LONG, -1L, Importance.LOW, "Exit after processing this many records.") - .define(KStreamConfig.NUM_STREAM_THREADS, + .define(ProcessorProperties.NUM_STREAM_THREADS, Type.INT, 1, Importance.LOW, @@ -90,15 +89,15 @@ public class ProcessorConfig extends AbstractConfig { public ProcessorConfig(Properties processor) { super(CONFIG, processor); - this.topics = this.getString(KStreamConfig.TOPICS_CONFIG); - this.stateDir = new File(this.getString(KStreamConfig.STATE_DIR_CONFIG)); - this.pollTimeMs = this.getLong(KStreamConfig.POLL_TIME_MS_CONFIG); - this.commitTimeMs = this.getLong(KStreamConfig.COMMIT_TIME_MS_CONFIG); - this.windowTimeMs = this.getLong(KStreamConfig.WINDOW_TIME_MS_CONFIG); - this.bufferedRecordsPerPartition = this.getInt(KStreamConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); - this.stateCleanupDelay = this.getLong(KStreamConfig.STATE_CLEANUP_DELAY_CONFIG); - this.totalRecordsToProcess = this.getLong(KStreamConfig.TOTAL_RECORDS_TO_PROCESS); - this.numStreamThreads = this.getInt(KStreamConfig.NUM_STREAM_THREADS); + this.topics = this.getString(ProcessorProperties.TOPICS_CONFIG); + this.stateDir = new File(this.getString(ProcessorProperties.STATE_DIR_CONFIG)); + this.pollTimeMs = this.getLong(ProcessorProperties.POLL_TIME_MS_CONFIG); + this.commitTimeMs = this.getLong(ProcessorProperties.COMMIT_TIME_MS_CONFIG); + this.windowTimeMs = this.getLong(ProcessorProperties.WINDOW_TIME_MS_CONFIG); + this.bufferedRecordsPerPartition = this.getInt(ProcessorProperties.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + this.stateCleanupDelay = this.getLong(ProcessorProperties.STATE_CLEANUP_DELAY_CONFIG); + this.totalRecordsToProcess = this.getLong(ProcessorProperties.TOTAL_RECORDS_TO_PROCESS); + this.numStreamThreads = this.getInt(ProcessorProperties.NUM_STREAM_THREADS); } } diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java b/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java index c94b037a1dbbb..514ff6945f6f4 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java @@ -25,6 +25,9 @@ public interface ProcessorContext { + // TODO: this is better moved to a KStreamContext + boolean joinable(ProcessorContext other); + /** * Returns the partition group id * diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamConfig.java b/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorProperties.java similarity index 91% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamConfig.java rename to stream/src/main/java/org/apache/kafka/clients/processor/ProcessorProperties.java index 0e52975fac867..21f11fa4c9f1c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamConfig.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorProperties.java @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.clients.processor; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.stream.KStreamProcess; -import org.apache.kafka.clients.processor.TimestampExtractor; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; @@ -30,12 +28,12 @@ /** - * Configuration information passed to the {@link KStreamProcess} instance for configuring the associated + * Configuration information passed to the process instance for configuring the associated * {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer}, and * {@link org.apache.kafka.clients.consumer.KafkaConsumer KafkaConsumer}, as * well as the processor itself. */ -public class KStreamConfig { +public class ProcessorProperties { /** topics */ public static final String TOPICS_CONFIG = "topics"; @@ -72,7 +70,7 @@ public class KStreamConfig { private Deserializer valDeserializer; private TimestampExtractor timestampExtractor; - public KStreamConfig(Properties config) { + public ProcessorProperties(Properties config) { this.config = config; this.config.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); this.config.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); @@ -80,8 +78,8 @@ public KStreamConfig(Properties config) { } @Override - public KStreamConfig clone() { - return new KStreamConfig(this.config); + public ProcessorProperties clone() { + return new ProcessorProperties(this.config); } public void addContextObject(String key, Object value) { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/RecordQueue.java b/stream/src/main/java/org/apache/kafka/clients/processor/RecordQueue.java new file mode 100644 index 0000000000000..99495d9736ff3 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/clients/processor/RecordQueue.java @@ -0,0 +1,83 @@ +/** + * 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.clients.processor; + +import org.apache.kafka.common.TopicPartition; + +/** + * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). + */ +public interface RecordQueue { + + /** + * Returns the partition with which this queue is associated + * + * @return TopicPartition + */ + TopicPartition partition(); + + /** + * Returns the corresponding source processor with this queue + * + * @return KafkaProcessor + */ + KafkaProcessor source(); + + /** + * Adds a StampedRecord to the queue + * + * @param record StampedRecord + */ + void add(StampedRecord record); + + /** + * Returns the next record fro the queue + * + * @return StampedRecord + */ + StampedRecord next(); + + /** + * Returns the highest offset in the queue + * + * @return offset + */ + long offset(); + + /** + * Returns the number of records in the queue + * + * @return the number of records + */ + int size(); + + /** + * Tests if the queue is empty + * + * @return true if the queue is empty, otherwise false + */ + boolean isEmpty(); + + /** + * Returns a timestamp tracked by the TimestampTracker + * + * @return timestamp + */ + long trackedTimestamp(); + +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/Stamped.java b/stream/src/main/java/org/apache/kafka/clients/processor/Stamped.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/Stamped.java rename to stream/src/main/java/org/apache/kafka/clients/processor/Stamped.java index f1d1bb0018237..9654e77f8d964 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/Stamped.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/Stamped.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.clients.processor; public class Stamped implements Comparable { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StampedRecord.java b/stream/src/main/java/org/apache/kafka/clients/processor/StampedRecord.java similarity index 91% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/StampedRecord.java rename to stream/src/main/java/org/apache/kafka/clients/processor/StampedRecord.java index 5778873917fa8..6936eeaeca7db 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StampedRecord.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/StampedRecord.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.clients.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; // TODO: making this class exposed to user in the lower-level Processor public class StampedRecord extends Stamped> { - StampedRecord(ConsumerRecord record, long timestamp) { + public StampedRecord(ConsumerRecord record, long timestamp) { super(record, timestamp); } diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java index 2272fef904c4c..5625eb16d79eb 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.processor; -import org.apache.kafka.clients.processor.internals.Stamped; - /** * TimestampTracker is a helper class for a sliding window implementation. * It is assumed that elements are added or removed in a FIFO manner. diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java similarity index 92% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamThread.java rename to stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java index 8cd8f1b349263..6bedafccdb365 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java @@ -15,17 +15,14 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.clients.processor.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.internals.IngestorImpl; -import org.apache.kafka.clients.processor.internals.ProcessorConfig; -import org.apache.kafka.clients.processor.internals.ProcessorContextImpl; -import org.apache.kafka.clients.processor.internals.RecordCollectorImpl; -import org.apache.kafka.clients.processor.internals.StreamGroup; +import org.apache.kafka.clients.processor.ProcessorConfig; +import org.apache.kafka.clients.processor.ProcessorProperties; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.KafkaException; @@ -62,7 +59,7 @@ public class KStreamThread extends Thread { private final Map kstreamContexts = new HashMap<>(); private final IngestorImpl ingestor; private final RecordCollectorImpl collector; - private final KStreamConfig KStreamConfig; + private final ProcessorProperties properties; private final ProcessorConfig config; private final Metrics metrics; private final KafkaStreamingMetrics streamingMetrics; @@ -88,17 +85,17 @@ public void onPartitionsRevoked(Consumer consumer, Collection topologyClass, KStreamConfig KStreamConfig) throws Exception { + public KStreamThread(Class topologyClass, ProcessorProperties properties) throws Exception { super(); - if (KStreamConfig.timestampExtractor() == null) + if (properties.timestampExtractor() == null) throw new NullPointerException("timestamp extractor is missing"); this.metrics = new Metrics(); - this.config = new ProcessorConfig(KStreamConfig.config()); + this.config = new ProcessorConfig(properties.config()); this.topologyClass = topologyClass; - this.KStreamConfig = KStreamConfig; + this.properties = properties; this.streamingMetrics = new KafkaStreamingMetrics(); // build the topology without initialization to get the topics for consumer @@ -106,10 +103,10 @@ public KStreamThread(Class topologyClass, KStreamConfig KSt topology.build(); // create the producer and consumer clients - Producer producer = new KafkaProducer<>(KStreamConfig.config(), new ByteArraySerializer(), new ByteArraySerializer()); - this.collector = new RecordCollectorImpl(producer, (Serializer) KStreamConfig.keySerializer(), (Serializer) KStreamConfig.valueSerializer()); + Producer producer = new KafkaProducer<>(properties.config(), new ByteArraySerializer(), new ByteArraySerializer()); + this.collector = new RecordCollectorImpl(producer, (Serializer) properties.keySerializer(), (Serializer) properties.valueSerializer()); - Consumer consumer = new KafkaConsumer<>(KStreamConfig.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + Consumer consumer = new KafkaConsumer<>(properties.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); this.ingestor = new IngestorImpl(consumer, topology.topics()); this.running = true; @@ -242,7 +239,7 @@ private void addPartitions(Collection assignment) { try { // build the topology and initialize with the context PTopology topology = this.topologyClass.getConstructor().newInstance(); - context = new ProcessorContextImpl(id, ingestor, topology, collector, KStreamConfig, config, metrics); + context = new ProcessorContextImpl(id, ingestor, topology, collector, properties, config, metrics); topology.build(); topology.init(context); context.initialized(); diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java index dc20d6d138e3a..ec190cf367aa6 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.processor.internals; +import org.apache.kafka.clients.processor.Stamped; import org.apache.kafka.clients.processor.TimestampTracker; import java.util.LinkedList; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java index 1ded993f9eee7..1d38cfd85850a 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java @@ -19,8 +19,11 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.processor.KafkaSource; import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.clients.processor.ProcessorConfig; import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.ProcessorProperties; import org.apache.kafka.clients.processor.RecordCollector; import org.apache.kafka.clients.processor.StateStore; import org.apache.kafka.clients.processor.KafkaProcessor; @@ -35,13 +38,16 @@ import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.stream.internals.KStreamConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; +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 { @@ -55,7 +61,7 @@ public class ProcessorContextImpl implements ProcessorContext { private final PTopology topology; private final RecordCollectorImpl collector; private final ProcessorStateManager stateMgr; - private final org.apache.kafka.stream.internals.KStreamConfig KStreamConfig; + private final ProcessorProperties processorProperties; private final ProcessorConfig processorConfig; private final TimestampExtractor timestampExtractor; @@ -66,7 +72,7 @@ public ProcessorContextImpl(int id, Ingestor ingestor, PTopology topology, RecordCollectorImpl collector, - KStreamConfig KStreamConfig, + ProcessorProperties processorProperties, ProcessorConfig processorConfig, Metrics metrics) throws IOException { this.id = id; @@ -74,9 +80,9 @@ public ProcessorContextImpl(int id, this.ingestor = ingestor; this.topology = topology; this.collector = collector; - this.KStreamConfig = KStreamConfig; + this.processorProperties = processorProperties; this.processorConfig = processorConfig; - this.timestampExtractor = this.KStreamConfig.timestampExtractor(); + this.timestampExtractor = this.processorProperties.timestampExtractor(); for (String topic : this.topology.topics()) { if (!ingestor.topics().contains(topic)) @@ -84,7 +90,7 @@ public ProcessorContextImpl(int id, } File stateFile = new File(processorConfig.stateDir, Integer.toString(id)); - Consumer restoreConsumer = new KafkaConsumer<>(KStreamConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + Consumer restoreConsumer = new KafkaConsumer<>(processorProperties.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); this.streamGroup = new StreamGroup(this, this.ingestor, new TimeBasedChooser(), this.timestampExtractor, this.processorConfig.bufferedRecordsPerPartition); @@ -95,7 +101,6 @@ public ProcessorContextImpl(int id, } public void addPartition(TopicPartition partition) { - // update the partition -> source stream map KafkaSource source = topology.source(partition.topic()); @@ -103,6 +108,41 @@ public void addPartition(TopicPartition partition) { this.ingestor.addPartitionStreamToGroup(this.streamGroup, partition); } + @Override + public boolean joinable(ProcessorContext o) { + + ProcessorContextImpl other = (ProcessorContextImpl) o; + + if (this.streamGroup != other.streamGroup) + return false; + + Set partitions = this.streamGroup.partitions(); + Map> partitionsById = new HashMap<>(); + int firstId = -1; + for (TopicPartition partition : partitions) { + if (!partitionsById.containsKey(partition.partition())) { + partitionsById.put(partition.partition(), new ArrayList()); + } + partitionsById.get(partition.partition()).add(partition.topic()); + + if (firstId < 0) + firstId = partition.partition(); + } + + List topics = partitionsById.get(firstId); + for (List topicsPerPartition : partitionsById.values()) { + if (topics.size() != topicsPerPartition.size()) + return false; + + for (String topic : topicsPerPartition) { + if (!topics.contains(topic)) + return false; + } + } + + return true; + } + @Override public int id() { return id; @@ -110,22 +150,22 @@ public int id() { @Override public Serializer keySerializer() { - return KStreamConfig.keySerializer(); + return processorProperties.keySerializer(); } @Override public Serializer valueSerializer() { - return KStreamConfig.valueSerializer(); + return processorProperties.valueSerializer(); } @Override public Deserializer keyDeserializer() { - return KStreamConfig.keyDeserializer(); + return processorProperties.keyDeserializer(); } @Override public Deserializer valueDeserializer() { - return KStreamConfig.valueDeserializer(); + return processorProperties.valueDeserializer(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java index 889648c3c2eaf..fdbe49a36000e 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java @@ -18,6 +18,7 @@ package org.apache.kafka.clients.processor.internals; import org.apache.kafka.clients.processor.Punctuator; +import org.apache.kafka.clients.processor.Stamped; public class PunctuationSchedule extends Stamped { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueueImpl.java similarity index 82% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueue.java rename to stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueueImpl.java index 9dc672f27f82f..6fa5554ebe9f2 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueueImpl.java @@ -18,20 +18,20 @@ package org.apache.kafka.clients.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.clients.processor.KafkaSource; +import org.apache.kafka.clients.processor.RecordQueue; +import org.apache.kafka.clients.processor.StampedRecord; import org.apache.kafka.clients.processor.TimestampTracker; import org.apache.kafka.common.TopicPartition; import java.util.ArrayDeque; -/** - * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). It is intended to be used in - * {@link StreamGroup}. - */ -public class RecordQueue { - - private final ArrayDeque queue = new ArrayDeque<>(); - public final KafkaSource source; +public class RecordQueueImpl implements RecordQueue { + private final KafkaSource source; private final TopicPartition partition; + private final ArrayDeque queue = new ArrayDeque<>(); + private TimestampTracker> timestampTracker; private long offset; @@ -42,17 +42,23 @@ public class RecordQueue { * @param source the instance of KStreamImpl that receives records * @param timestampTracker TimestampTracker */ - public RecordQueue(TopicPartition partition, KafkaSource source, TimestampTracker> timestampTracker) { + public RecordQueueImpl(TopicPartition partition, KafkaSource source, TimestampTracker> timestampTracker) { this.partition = partition; this.source = source; this.timestampTracker = timestampTracker; } + @Override + public KafkaProcessor source() { + return source; + } + /** * Returns the partition with which this queue is associated * * @return TopicPartition */ + @Override public TopicPartition partition() { return partition; } @@ -62,10 +68,12 @@ public TopicPartition partition() { * * @param record StampedRecord */ + @Override public void add(StampedRecord record) { queue.addLast(record); - timestampTracker.addStampedElement(record); + offset = record.offset(); + timestampTracker.addStampedElement(record); } /** @@ -73,6 +81,7 @@ public void add(StampedRecord record) { * * @return StampedRecord */ + @Override public StampedRecord next() { StampedRecord elem = queue.pollFirst(); @@ -88,6 +97,7 @@ public StampedRecord next() { * * @return offset */ + @Override public long offset() { return offset; } @@ -97,6 +107,7 @@ public long offset() { * * @return the number of records */ + @Override public int size() { return queue.size(); } @@ -106,6 +117,7 @@ public int size() { * * @return true if the queue is empty, otherwise false */ + @Override public boolean isEmpty() { return queue.isEmpty(); } @@ -115,8 +127,8 @@ public boolean isEmpty() { * * @return timestamp */ + @Override public long trackedTimestamp() { return timestampTracker.get(); } - } diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java index b9019f2614056..f67010e18f5eb 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java @@ -18,6 +18,7 @@ package org.apache.kafka.clients.processor.internals; import org.apache.kafka.clients.processor.Chooser; +import org.apache.kafka.clients.processor.RecordQueue; import java.util.ArrayDeque; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java index 544127cca0df3..6ad71b5756190 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java @@ -19,16 +19,12 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.processor.Chooser; +import org.apache.kafka.clients.processor.KafkaSource; import org.apache.kafka.clients.processor.Punctuator; import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.clients.processor.RecordQueue; +import org.apache.kafka.clients.processor.StampedRecord; import org.apache.kafka.clients.processor.TimestampExtractor; -import org.apache.kafka.clients.processor.internals.Ingestor; -import org.apache.kafka.clients.processor.internals.KafkaSource; -import org.apache.kafka.clients.processor.internals.MinTimestampTracker; -import org.apache.kafka.clients.processor.internals.PunctuationQueue; -import org.apache.kafka.clients.processor.internals.PunctuationSchedule; -import org.apache.kafka.clients.processor.internals.RecordQueue; -import org.apache.kafka.clients.processor.internals.StampedRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; @@ -138,8 +134,8 @@ private void ingestNewRecords() { ConsumerRecord record = iterator.next(); // deserialize the raw record, extract the timestamp and put into the queue - Deserializer keyDeserializer = recordQueue.source.keyDeserializer; - Deserializer valDeserializer = recordQueue.source.valDeserializer; + Deserializer keyDeserializer = ((KafkaSource) recordQueue.source()).keyDeserializer; + Deserializer valDeserializer = ((KafkaSource) recordQueue.source()).valDeserializer; Object key = keyDeserializer.deserialize(record.topic(), record.key()); Object value = valDeserializer.deserialize(record.topic(), record.value()); @@ -197,7 +193,7 @@ public boolean process() { if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - recordQueue.source.receive(currRecord.key(), currRecord.value()); + recordQueue.source().receive(currRecord.key(), currRecord.value()); consumedOffsets.put(recordQueue.partition(), currRecord.offset()); // TODO: local state flush and downstream producer flush @@ -253,7 +249,7 @@ public void close() { } protected RecordQueue createRecordQueue(TopicPartition partition, KafkaSource source) { - return new RecordQueue(partition, source, new MinTimestampTracker>()); + return new RecordQueueImpl(partition, source, new MinTimestampTracker>()); } private static class NewRecords { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java index 812c3b8852a52..0e9a7e93cdb1e 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java +++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.processor.Chooser; +import org.apache.kafka.clients.processor.RecordQueue; import java.util.Comparator; import java.util.PriorityQueue; diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java b/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java index cea6342aa7430..4eeb7ead5f661 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java @@ -18,9 +18,9 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.stream.internals.KStreamThread; -import org.apache.kafka.clients.processor.internals.ProcessorConfig; -import org.apache.kafka.stream.internals.KStreamConfig; +import org.apache.kafka.clients.processor.internals.KStreamThread; +import org.apache.kafka.clients.processor.ProcessorConfig; +import org.apache.kafka.clients.processor.ProcessorProperties; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,7 +44,7 @@ *
  *    Properties props = new Properties();
  *    props.put("bootstrap.servers", "localhost:4242");
- *    KStreamConfig config = new KStreamConfig(props);
+ *    properties config = new properties(props);
  *    config.processor(ExampleStreamProcessor.class);
  *    config.serialization(new StringSerializer(), new StringDeserializer());
  *    KStreamProcess container = new KStreamProcess(new MyKStreamTopology(), config);
@@ -68,15 +68,15 @@ public class KStreamProcess implements Runnable {
     private final KStreamThread[] threads;
 
 
-    public KStreamProcess(Class topologyClass, KStreamConfig KStreamConfig) throws Exception {
-        if (KStreamConfig.timestampExtractor() == null)
+    public KStreamProcess(Class topologyClass, ProcessorProperties processorProperties) throws Exception {
+        if (processorProperties.timestampExtractor() == null)
             throw new NullPointerException("timestamp extractor is missing");
 
-        this.config = new ProcessorConfig(KStreamConfig.config());
+        this.config = new ProcessorConfig(processorProperties.config());
 
         this.threads = new KStreamThread[this.config.numStreamThreads];
         for (int i = 0; i < this.threads.length; i++) {
-            this.threads[i] = new KStreamThread(topologyClass, KStreamConfig);
+            this.threads[i] = new KStreamThread(topologyClass, processorProperties);
         }
     }
 
diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java
index 119675122de2c..3b4201f603502 100644
--- a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java
+++ b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java
@@ -19,7 +19,7 @@
 
 import org.apache.kafka.clients.processor.KafkaProcessor;
 import org.apache.kafka.clients.processor.PTopology;
-import org.apache.kafka.stream.internals.KStreamConfig;
+import org.apache.kafka.clients.processor.ProcessorProperties;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.stream.internals.KStreamSource;
@@ -29,10 +29,12 @@
  */
 public abstract class KStreamTopology extends PTopology {
 
-    public KStreamTopology() { super(); }
+    public KStreamTopology() {
+        super();
+    }
 
-    public KStreamTopology(KStreamConfig KStreamConfig) {
-        super(KStreamConfig);
+    public KStreamTopology(ProcessorProperties properties) {
+        super(properties);
     }
 
     /**
@@ -42,12 +44,12 @@ public KStreamTopology(KStreamConfig KStreamConfig) {
      * @return KStream
      */
     public  KStream from(String... topics) {
-        if (KStreamConfig == null)
+        if (properties == null)
             throw new KafkaException("No default deserializers specified in the config.");
 
         KafkaProcessor source = addSource(
-            (Deserializer) KStreamConfig.keyDeserializer(),
-            (Deserializer) KStreamConfig.valueDeserializer(), topics);
+            (Deserializer) properties.keyDeserializer(),
+            (Deserializer) properties.valueDeserializer(), topics);
         return new KStreamSource<>(this, source);
     }
 
diff --git a/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java
index fd0bb5393716a..c7f4c98e4dfb7 100644
--- a/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java
+++ b/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java
@@ -19,7 +19,7 @@
 
 import org.apache.kafka.clients.processor.ProcessorContext;
 import org.apache.kafka.stream.internals.FilteredIterator;
-import org.apache.kafka.clients.processor.internals.Stamped;
+import org.apache.kafka.clients.processor.Stamped;
 
 import java.util.HashMap;
 import java.util.Iterator;
diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java
index 25c9fc66e6e60..ba3698b8e668c 100644
--- a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java
+++ b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java
@@ -17,7 +17,7 @@
 
 package org.apache.kafka.stream.examples;
 
-import org.apache.kafka.stream.internals.KStreamConfig;
+import org.apache.kafka.clients.processor.ProcessorProperties;
 import org.apache.kafka.stream.KStreamProcess;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.stream.KStream;
@@ -71,7 +71,7 @@ public boolean apply(String key, Integer value) {
     }
 
     public static void main(String[] args) throws Exception {
-        KStreamProcess kstream = new KStreamProcess(MyKStreamTopology.class, new KStreamConfig(new Properties()));
+        KStreamProcess kstream = new KStreamProcess(MyKStreamTopology.class, new ProcessorProperties(new Properties()));
         kstream.run();
     }
 }
diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java
index 5fd134b0d1823..d05c6b81505d1 100644
--- a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java
+++ b/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java
@@ -20,8 +20,8 @@
 import org.apache.kafka.clients.processor.KafkaProcessor;
 import org.apache.kafka.clients.processor.PTopology;
 import org.apache.kafka.clients.processor.ProcessorContext;
-import org.apache.kafka.stream.internals.KStreamConfig;
-import org.apache.kafka.clients.processor.internals.KafkaSource;
+import org.apache.kafka.clients.processor.ProcessorProperties;
+import org.apache.kafka.clients.processor.KafkaSource;
 import org.apache.kafka.common.serialization.IntegerDeserializer;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.stream.KStreamProcess;
@@ -68,7 +68,7 @@ public void build() {
     }
 
     public static void main(String[] args) throws Exception {
-        KStreamProcess streaming = new KStreamProcess(MyPTopology.class, new KStreamConfig(new Properties()));
+        KStreamProcess streaming = new KStreamProcess(MyPTopology.class, new ProcessorProperties(new Properties()));
         streaming.run();
     }
 }
diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java
index e371161e431c7..9ca11dadb7732 100644
--- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java
+++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java
@@ -21,10 +21,10 @@
 import org.apache.kafka.clients.processor.KafkaProcessor;
 import org.apache.kafka.clients.processor.PTopology;
 import org.apache.kafka.clients.processor.ProcessorContext;
-import org.apache.kafka.clients.processor.internals.KafkaSource;
+import org.apache.kafka.clients.processor.KafkaSource;
 import org.apache.kafka.common.serialization.IntegerDeserializer;
 import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.stream.internals.KStreamConfig;
+import org.apache.kafka.clients.processor.ProcessorProperties;
 import org.apache.kafka.stream.state.Entry;
 import org.apache.kafka.stream.state.InMemoryKeyValueStore;
 import org.apache.kafka.stream.state.KeyValueIterator;
@@ -89,7 +89,7 @@ public void build() {
     }
 
     public static void main(String[] args) throws Exception {
-        KStreamProcess streaming = new KStreamProcess(MyPTopology.class, new KStreamConfig(new Properties()));
+        KStreamProcess streaming = new KStreamProcess(MyPTopology.class, new ProcessorProperties(new Properties()));
         streaming.run();
     }
 }
diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java
index f4ce6bd23029b..661ae16ae6e75 100644
--- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java
+++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java
@@ -55,9 +55,9 @@ public void process(K key, V value) {
 
     @SuppressWarnings("unchecked")
     public KStream[] branches() {
-        KStream[] streams = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length);
+        KStream[] streams = (KStreamSource[]) Array.newInstance(KStreamSource.class, branches.length);
         for (int i = 0; i < branches.length; i++) {
-            streams[i] = new KStreamImpl<>(topology, branches[i]);
+            streams[i] = new KStreamSource<>(topology, branches[i]);
         }
         return streams;
     }
diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java
index b120e36b070ce..1cdeddf81eaaf 100644
--- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java
+++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java
@@ -21,7 +21,7 @@
 import org.apache.kafka.clients.processor.PTopology;
 import org.apache.kafka.clients.processor.Processor;
 import org.apache.kafka.clients.processor.ProcessorContext;
-import org.apache.kafka.clients.processor.internals.KafkaSource;
+import org.apache.kafka.clients.processor.KafkaSource;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.stream.KStreamWindowed;
diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java
index 1b60a80ec8b14..0ef3b09f23209 100644
--- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java
+++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java
@@ -19,17 +19,10 @@
 
 import org.apache.kafka.clients.processor.KafkaProcessor;
 import org.apache.kafka.clients.processor.ProcessorContext;
-import org.apache.kafka.clients.processor.internals.ProcessorContextImpl;
-import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.stream.ValueJoiner;
 import org.apache.kafka.stream.Window;
 
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 class KStreamJoin extends KafkaProcessor {
 
@@ -90,40 +83,10 @@ Iterator find(K key, long timestamp) {
     public void init(ProcessorContext context) {
         this.context = context;
 
-        // the two streams should only be joinable if they are inside the same group
-        // and that group's topics all have the same partitions
-        ProcessorContextImpl context1 = (ProcessorContextImpl) stream1.context();
-        ProcessorContextImpl context2 = (ProcessorContextImpl) stream2.context();
-
-        if (context1.streamGroup != context2.streamGroup)
-            throw new IllegalStateException("Stream " + stream1.name() + " and stream " + stream2.name() + " are not joinable" +
-                " since they belong to different stream groups.");
-
-        Set partitions = context1.streamGroup.partitions();
-        Map> partitionsById = new HashMap<>();
-        int firstId = -1;
-        for (TopicPartition partition : partitions) {
-            if (!partitionsById.containsKey(partition.partition())) {
-                partitionsById.put(partition.partition(), new ArrayList());
-            }
-            partitionsById.get(partition.partition()).add(partition.topic());
-
-            if (firstId < 0)
-                firstId = partition.partition();
-        }
-
-        List topics = partitionsById.get(firstId);
-        for (List topicsPerPartition : partitionsById.values()) {
-            if (topics.size() != topicsPerPartition.size())
-                throw new IllegalStateException("Stream " + stream1.name() + " and stream " + stream2.name() + " are not joinable" +
-                    " since their stream group have different partitions for some topics.");
-
-            for (String topic : topicsPerPartition) {
-                if (!topics.contains(topic))
-                    throw new IllegalStateException("Stream " + stream1.name() + " and stream " + stream2.name() + " are not joinable" +
-                        " since their stream group have different partitions for some topics.");
-            }
-        }
+        // check if these two streams are joinable
+        if (!stream1.context().joinable(stream2.context()))
+            throw new IllegalStateException("Stream " + stream1.name() + " and stream " +
+                stream2.name() + " are not joinable.");
     }
 
     @Override
diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java
index 5ad3d0c6a3e49..412335fc6ca33 100644
--- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java
+++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java
@@ -73,7 +73,9 @@ public Window window() {
         return window;
     }
 
-    public ProcessorContext context() { return context; }
+    public ProcessorContext context() {
+        return context;
+    }
 
     @Override
     public void init(ProcessorContext context) {
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java
index b5393344a4e86..8f6e115cc63fe 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java
@@ -30,6 +30,7 @@
 
 public class KStreamBranchTest {
 
+<<<<<<< HEAD
 <<<<<<< HEAD
   private String topicName = "topic";
 
@@ -92,6 +93,9 @@ public boolean apply(Integer key, String value) {
       stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L);
 =======
     private String topicName = "topic";
+=======
+    private String topic1 = "topic";
+>>>>>>> adding files
 
     private KStreamTopology topology = new MockKStreamTopology();
     private IntegerDeserializer keyDeserializer = new IntegerDeserializer();
@@ -126,7 +130,7 @@ public boolean apply(Integer key, String value) {
         KStream[] branches;
         MockProcessor[] processors;
 
-        stream = topology.from(keyDeserializer, valDeserializer, topicName);
+        stream = topology.from(keyDeserializer, valDeserializer, topic1);
         branches = stream.branch(isEven, isMultipleOfThree, isOdd);
 
         assertEquals(3, branches.length);
@@ -142,6 +146,7 @@ public boolean apply(Integer key, String value) {
         }
 
         assertEquals(3, processors[0].processed.size());
+<<<<<<< HEAD
         assertEquals(1, processors[1].processed.size());
         assertEquals(3, processors[2].processed.size());
 
@@ -166,6 +171,9 @@ public boolean apply(Integer key, String value) {
         assertEquals(4, processors[1].processed.size());
         assertEquals(0, processors[2].processed.size());
 >>>>>>> compile and test passed
+=======
+        assertEquals(2, processors[1].processed.size());
+        assertEquals(4, processors[2].processed.size());
+>>>>>>> adding files
     }
-
 }
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java
index 2722d1a32379d..59b7daf1e9fcc 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java
@@ -27,13 +27,12 @@
 import org.apache.kafka.test.UnlimitedWindow;
 import org.junit.Test;
 
-import java.util.Collections;
-
 import static org.junit.Assert.assertEquals;
 
 public class KStreamJoinTest {
 
-    private String topicName = "topic";
+    private String topic1 = "topic1";
+    private String topic2 = "topic2";
 
     private KStreamTopology topology = new MockKStreamTopology();
     private IntegerDeserializer keyDeserializer = new IntegerDeserializer();
@@ -115,13 +114,17 @@ public void testJoin() {
         String[] expected;
 
         processor = new MockProcessor<>();
-        stream1 = topology.from(keyDeserializer, valDeserializer, topicName);
-        stream2 = topology.from(keyDeserializer, valDeserializer, topicName);
-        windowed1 = stream1.with(new UnlimitedWindow<>());
-        windowed2 = stream2.with(new UnlimitedWindow<>());
+        stream1 = topology.from(keyDeserializer, valDeserializer, topic1);
+        stream2 = topology.from(keyDeserializer, valDeserializer, topic2);
+        windowed1 = stream1.with(new UnlimitedWindow());
+        windowed2 = stream2.with(new UnlimitedWindow());
 
         windowed1.join(windowed2, joiner).process(processor);
 
+        MockProcessorContext context = new MockProcessorContext(null, null);
+        topology.init(context);
+        context.setTime(0L);
+
         // push two items to the main stream. the other stream's window is empty
 
         for (int i = 0; i < 2; i++) {
@@ -237,11 +240,18 @@ public void testJoinPrior() {
         }
 =======
         processor = new MockProcessor<>();
+<<<<<<< HEAD
         stream1 = topology.from(keyDeserializer, valDeserializer, topicName);
         stream2 = topology.from(keyDeserializer, valDeserializer, topicName);
         windowed1 = stream1.with(new UnlimitedWindow<>());
         windowed2 = stream2.with(new UnlimitedWindow<>());
 >>>>>>> wip
+=======
+        stream1 = topology.from(keyDeserializer, valDeserializer, topic1);
+        stream2 = topology.from(keyDeserializer, valDeserializer, topic2);
+        windowed1 = stream1.with(new UnlimitedWindow());
+        windowed2 = stream2.with(new UnlimitedWindow());
+>>>>>>> adding files
 
         windowed1.joinPrior(windowed2, joiner).process(processor);
 
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java
index 7adf2a6d94c15..b3ceddfac1867 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java
@@ -23,8 +23,6 @@
 import org.apache.kafka.test.MockProcessor;
 import org.junit.Test;
 
-import java.util.Collections;
-
 import static org.junit.Assert.assertEquals;
 
 public class KStreamSourceTest {
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java
index a11d85de4ae9a..082dfc0cd1eaa 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java
@@ -21,10 +21,10 @@
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.stream.internals.KStreamSource;
 import org.apache.kafka.test.MockKStreamTopology;
+import org.apache.kafka.test.MockProcessorContext;
 import org.apache.kafka.test.UnlimitedWindow;
 import org.junit.Test;
 
-import java.util.Collections;
 import java.util.Iterator;
 
 import static org.junit.Assert.assertEquals;
@@ -74,6 +74,10 @@ public void testWindowedStream() {
         stream = topology.from(keyDeserializer, valDeserializer, topicName);
         stream.with(window);
 
+        MockProcessorContext context = new MockProcessorContext(null, null);
+        topology.init(context);
+        context.setTime(0L);
+
         // two items in the window
 
         for (int i = 0; i < 2; i++) {
diff --git a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java
index 88d14ccec4104..44c78377998c2 100644
--- a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java
@@ -20,7 +20,7 @@
 import static org.junit.Assert.assertEquals;
 
 import org.apache.kafka.clients.processor.internals.MinTimestampTracker;
-import org.apache.kafka.clients.processor.internals.Stamped;
+import org.apache.kafka.clients.processor.Stamped;
 import org.junit.Test;
 
 public class MinTimestampTrackerTest {
diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java
index 4a09b5658fba5..52ccdf81635d1 100644
--- a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java
+++ b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java
@@ -34,7 +34,9 @@ public void process(K key, V value) {
     }
 
     @Override
-    public void receive(K key, V value) { process(key, value); }
+    public void receive(K key, V value) {
+        process(key, value);
+    }
 
     @Override
     public void init(ProcessorContext context) {
@@ -47,6 +49,6 @@ public void punctuate(long streamTime) {
 
     @Override
     public void close() {
-    }
 
+    }
 }
diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java
index d73531fc34097..283a802ee3024 100644
--- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java
+++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java
@@ -190,6 +190,12 @@ public void setTime(long timestamp) {
         this.timestamp = timestamp;
     }
 
+    @Override
+    public boolean joinable(ProcessorContext other) {
+        // TODO
+        return true;
+    }
+
     @Override
     public int id() {
         return -1;
diff --git a/stream/src/test/java/org/apache/kafka/test/MockSource.java b/stream/src/test/java/org/apache/kafka/test/MockSource.java
index 152c62c4daadd..efc3c0dfd3cc1 100644
--- a/stream/src/test/java/org/apache/kafka/test/MockSource.java
+++ b/stream/src/test/java/org/apache/kafka/test/MockSource.java
@@ -17,23 +17,19 @@
 
 package org.apache.kafka.test;
 
-import org.apache.kafka.clients.processor.ProcessorContext;
-import org.apache.kafka.clients.processor.internals.KafkaSource;
+import org.apache.kafka.clients.processor.KafkaSource;
 import org.apache.kafka.common.serialization.Deserializer;
 
 import java.util.ArrayList;
 
 public class MockSource extends KafkaSource {
 
-    private ProcessorContext context;
-
     public Deserializer keyDeserializer;
     public Deserializer valDeserializer;
 
     public int numReceived = 0;
     public ArrayList keys = new ArrayList<>();
     public ArrayList values = new ArrayList<>();
-    public ArrayList timestamps = new ArrayList<>();
 
     public MockSource(Deserializer keyDeserializer, Deserializer valDeserializer) {
         super(keyDeserializer, valDeserializer);
@@ -42,21 +38,10 @@ public MockSource(Deserializer keyDeserializer, Deserializer
Date: Mon, 10 Aug 2015 17:45:47 -0700
Subject: [PATCH 157/275] minor API changes

---
 .../clients/processor/KafkaProcessor.java     | 10 +++-----
 .../kafka/clients/processor/Processor.java    |  6 +++--
 .../kafka/clients/processor/Receiver.java     | 23 -----------------
 .../processor/internals/StreamGroup.java      |  2 +-
 .../java/org/apache/kafka/stream/KStream.java |  8 ------
 .../kafka/stream/internals/KStreamImpl.java   | 18 ++-----------
 .../kafka/stream/KStreamBranchTest.java       |  2 +-
 .../kafka/stream/KStreamFilterTest.java       |  4 +--
 .../kafka/stream/KStreamFlatMapTest.java      |  2 +-
 .../stream/KStreamFlatMapValuesTest.java      |  2 +-
 .../apache/kafka/stream/KStreamJoinTest.java  | 16 ++++++------
 .../apache/kafka/stream/KStreamMapTest.java   |  2 +-
 .../kafka/stream/KStreamMapValuesTest.java    |  2 +-
 .../kafka/stream/KStreamSourceTest.java       |  2 +-
 .../kafka/stream/KStreamWindowedTest.java     |  4 +--
 .../org/apache/kafka/test/MockProcessor.java  | 25 +++++--------------
 16 files changed, 34 insertions(+), 94 deletions(-)
 delete mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/Receiver.java

diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java
index 2000a4dcfaa08..7b5a351e74a5b 100644
--- a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java
+++ b/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java
@@ -20,7 +20,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
-public abstract class KafkaProcessor implements Processor, Receiver, Punctuator {
+public abstract class KafkaProcessor implements Processor, Punctuator {
 
     private final List> children;
     private final List> parents;
@@ -56,17 +56,13 @@ public final void chain(KafkaProcessor child) {
         children.add(child);
     }
 
+    @Override
     public final void forward(K2 key, V2 value) {
         for (KafkaProcessor child : children) {
-            child.receive(key, value);
+            child.process(key, value);
         }
     }
 
-    @Override
-    public final void receive(K1 key, V1 value) {
-        this.process(key, value);
-    }
-
     /* Following functions can be overridden by users */
 
     @Override
diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/Processor.java b/stream/src/main/java/org/apache/kafka/clients/processor/Processor.java
index 677c5c9ed4ff1..a8d25b76080f0 100644
--- a/stream/src/main/java/org/apache/kafka/clients/processor/Processor.java
+++ b/stream/src/main/java/org/apache/kafka/clients/processor/Processor.java
@@ -67,11 +67,13 @@ public interface ProcessorContext {
 
 package org.apache.kafka.clients.processor;
 
-public interface Processor {
+public interface Processor {
 
     void init(ProcessorContext context);
 
-    void process(K key, V value);
+    void process(K1 key, V1 value);
+
+    void forward(K2 key, V2 value);
 
     void close();
 >>>>>>> wip
diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/Receiver.java b/stream/src/main/java/org/apache/kafka/clients/processor/Receiver.java
deleted file mode 100644
index 5a949cf75a21f..0000000000000
--- a/stream/src/main/java/org/apache/kafka/clients/processor/Receiver.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.clients.processor;
-
-public interface Receiver {
-
-    void receive(K key, V value);
-}
diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java
index 6ad71b5756190..840a89fade8fc 100644
--- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java
+++ b/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java
@@ -193,7 +193,7 @@ public boolean process() {
 
             if (streamTime < trackedTimestamp) streamTime = trackedTimestamp;
 
-            recordQueue.source().receive(currRecord.key(), currRecord.value());
+            recordQueue.source().process(currRecord.key(), currRecord.value());
             consumedOffsets.put(recordQueue.partition(), currRecord.offset());
 
             // TODO: local state flush and downstream producer flush
diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java
index af09d6d78e8d2..aa8496e4f59d9 100644
--- a/stream/src/main/java/org/apache/kafka/stream/KStream.java
+++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java
@@ -18,7 +18,6 @@
 package org.apache.kafka.stream;
 
 import org.apache.kafka.clients.processor.KafkaProcessor;
-import org.apache.kafka.clients.processor.Processor;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.Serializer;
 
@@ -154,11 +153,4 @@ public interface KStream {
      * @param processor the instance of Processor
      */
      KStream process(KafkaProcessor processor);
-
-    /**
-     * Processes all elements in this stream by applying a processor.
-     *
-     * @param processor the instance of Processor
-     */
-    void process(Processor processor);
 }
diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java
index 1cdeddf81eaaf..885a99e41532f 100644
--- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java
+++ b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java
@@ -19,7 +19,6 @@
 
 import org.apache.kafka.clients.processor.KafkaProcessor;
 import org.apache.kafka.clients.processor.PTopology;
-import org.apache.kafka.clients.processor.Processor;
 import org.apache.kafka.clients.processor.ProcessorContext;
 import org.apache.kafka.clients.processor.KafkaSource;
 import org.apache.kafka.common.serialization.Deserializer;
@@ -149,8 +148,8 @@ public void sendTo(String topic, Serializer keySerializer, Serializer valS
     }
 
     @SuppressWarnings("unchecked")
-    private  Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) {
-        return new Processor() {
+    private  KafkaProcessor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) {
+        return new KafkaProcessor("KAFKA-SEND") {
             private ProcessorContext context;
 
             @Override
@@ -176,17 +175,4 @@ public  KStream process(final KafkaProcessor curre
 
         return new KStreamImpl(topology, current);
     }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public void process(final Processor current) {
-        KafkaProcessor wrapper = new KafkaProcessor(PROCESSOR_NAME) {
-            @Override
-            public void process(K key, V value) {
-                current.process(key, value);
-            }
-        };
-
-        topology.addProcessor(wrapper, processor);
-    }
 }
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java
index 8f6e115cc63fe..bfbfccad153fe 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java
@@ -142,7 +142,7 @@ public boolean apply(Integer key, String value) {
         }
 
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]);
+            ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]);
         }
 
         assertEquals(3, processors[0].processed.size());
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java
index 9852f17edee5b..e8447707b38d0 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java
@@ -77,7 +77,7 @@ public void testFilter() {
         stream.filter(isMultipleOfThree).process(processor);
 
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]);
+            ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]);
         }
 
         assertEquals(2, processor.processed.size());
@@ -113,7 +113,7 @@ public void testFilterOut() {
 >>>>>>> compile and test passed
 
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]);
+            ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]);
         }
 
         assertEquals(5, processor.processed.size());
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java
index e18af652d041e..89330d27c16f3 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java
@@ -103,7 +103,7 @@ public KeyValue> apply(Integer key, String value) {
 >>>>>>> compile and test passed
 
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]);
+            ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]);
         }
 
         assertEquals(6, processor.processed.size());
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java
index e49b3500bcba7..d2e6c3db89135 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java
@@ -83,7 +83,7 @@ public Iterable apply(String value) {
         stream.flatMapValues(mapper).process(processor);
 
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]);
+            ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]);
         }
 
         assertEquals(8, processor.processed.size());
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java
index 59b7daf1e9fcc..3082053264703 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java
@@ -128,7 +128,7 @@ public void testJoin() {
         // push two items to the main stream. the other stream's window is empty
 
         for (int i = 0; i < 2; i++) {
-            ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i]);
+            ((KStreamSource) stream1).source().process(expectedKeys[i], "X" + expectedKeys[i]);
         }
 
         assertEquals(0, processor.processed.size());
@@ -136,7 +136,7 @@ public void testJoin() {
         // push two items to the other stream. the main stream's window has two items
 
         for (int i = 0; i < 2; i++) {
-            ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i]);
+            ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]);
         }
 
         assertEquals(2, processor.processed.size());
@@ -152,7 +152,7 @@ public void testJoin() {
         // push all items to the main stream. this should produce two items.
 
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i]);
+            ((KStreamSource) stream1).source().process(expectedKeys[i], "X" + expectedKeys[i]);
         }
 
         assertEquals(2, processor.processed.size());
@@ -169,7 +169,7 @@ public void testJoin() {
 
         // push all items to the other stream. this should produce 6 items
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i]);
+            ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]);
         }
 
         assertEquals(6, processor.processed.size());
@@ -263,7 +263,7 @@ public void testJoinPrior() {
         for (int i = 0; i < 2; i++) {
             context.setTime(i);
 
-            ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i]);
+            ((KStreamSource) stream1).source().process(expectedKeys[i], "X" + expectedKeys[i]);
         }
 
         assertEquals(0, processor.processed.size());
@@ -274,7 +274,7 @@ public void testJoinPrior() {
         for (int i = 0; i < 2; i++) {
             context.setTime(i + 1);
 
-            ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i]);
+            ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]);
         }
 
         assertEquals(0, processor.processed.size());
@@ -286,7 +286,7 @@ public void testJoinPrior() {
         for (int i = 0; i < expectedKeys.length; i++) {
             context.setTime(i + 2);
 
-            ((KStreamSource) stream1).source().receive(expectedKeys[i], "X" + expectedKeys[i]);
+            ((KStreamSource) stream1).source().process(expectedKeys[i], "X" + expectedKeys[i]);
         }
 
         assertEquals(2, processor.processed.size());
@@ -305,7 +305,7 @@ public void testJoinPrior() {
         for (int i = 0; i < expectedKeys.length; i++) {
             context.setTime(i);
 
-            ((KStreamSource) stream2).source().receive(expectedKeys[i], "Y" + expectedKeys[i]);
+            ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]);
         }
 
         assertEquals(6, processor.processed.size());
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java
index 51de88df8497e..03727f431298b 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java
@@ -78,7 +78,7 @@ public KeyValue apply(Integer key, String value) {
         stream.map(mapper).process(processor);
 
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]);
+            ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]);
         }
 
         assertEquals(4, processor.processed.size());
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java
index 8c0a50795daa9..97b91fd671b56 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java
@@ -122,7 +122,7 @@ public Integer apply(String value) {
         stream.mapValues(mapper).process(processor);
 
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream).source().receive(expectedKeys[i], Integer.toString(expectedKeys[i]));
+            ((KStreamSource) stream).source().process(expectedKeys[i], Integer.toString(expectedKeys[i]));
         }
 
         assertEquals(4, processor.processed.size());
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java
index b3ceddfac1867..7dd814262916e 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java
@@ -70,7 +70,7 @@ public void testKStreamSource() {
         final String[] expectedValues = new String[]{"v1", "v2", "v3"};
 
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream).source().receive(expectedKeys[i], expectedValues[i]);
+            ((KStreamSource) stream).source().process(expectedKeys[i], expectedValues[i]);
         }
 
         assertEquals(3, processor.processed.size());
diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java
index 082dfc0cd1eaa..2529acb1c0b61 100644
--- a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java
+++ b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java
@@ -81,7 +81,7 @@ public void testWindowedStream() {
         // two items in the window
 
         for (int i = 0; i < 2; i++) {
-            ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]);
+            ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]);
         }
 
         assertEquals(1, countItem(window.find(0, 0L)));
@@ -92,7 +92,7 @@ public void testWindowedStream() {
         // previous two items + all items, thus two are duplicates, in the window
 
         for (int i = 0; i < expectedKeys.length; i++) {
-            ((KStreamSource) stream).source().receive(expectedKeys[i], "Y" + expectedKeys[i]);
+            ((KStreamSource) stream).source().process(expectedKeys[i], "Y" + expectedKeys[i]);
         }
 
         assertEquals(2, countItem(window.find(0, 0L)));
diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java
index 52ccdf81635d1..4030007c0c2cc 100644
--- a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java
+++ b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java
@@ -17,38 +17,25 @@
 
 package org.apache.kafka.test;
 
-import org.apache.kafka.clients.processor.Processor;
-import org.apache.kafka.clients.processor.ProcessorContext;
-import org.apache.kafka.clients.processor.Punctuator;
-import org.apache.kafka.clients.processor.Receiver;
+import org.apache.kafka.clients.processor.KafkaProcessor;
 
 import java.util.ArrayList;
 
-public class MockProcessor implements Processor, Receiver, Punctuator {
+public class MockProcessor extends KafkaProcessor {
     public final ArrayList processed = new ArrayList<>();
     public final ArrayList punctuated = new ArrayList<>();
 
-    @Override
-    public void process(K key, V value) {
-        processed.add(key + ":" + value);
-    }
-
-    @Override
-    public void receive(K key, V value) {
-        process(key, value);
+    public MockProcessor() {
+        super("MOCK");
     }
 
     @Override
-    public void init(ProcessorContext context) {
+    public void process(K1 key, V1 value) {
+        processed.add(key + ":" + value);
     }
 
     @Override
     public void punctuate(long streamTime) {
         punctuated.add(streamTime);
     }
-
-    @Override
-    public void close() {
-
-    }
 }

From 0952e655150b2dee4eab40abbb3071b5a3feb240 Mon Sep 17 00:00:00 2001
From: Guozhang Wang 
Date: Thu, 13 Aug 2015 15:28:40 -0700
Subject: [PATCH 158/275] Refactor Processor and KStream APIs

---
 .../org/apache/kafka/common/utils/Utils.java  |  14 ++
 .../kafka/clients/processor/PTopology.java    | 176 ------------------
 .../java/org/apache/kafka/stream/KStream.java |   2 +-
 .../apache/kafka/stream/KStreamProcess.java   |  14 +-
 .../apache/kafka/stream/KStreamTopology.java  |  70 -------
 .../apache/kafka/stream/SlidingWindow.java    |   6 +-
 .../java/org/apache/kafka/stream/Window.java  |   4 +-
 .../kafka/stream/examples/KStreamJob.java     |  72 ++++---
 .../stream/examples/SimpleProcessJob.java     |  26 ++-
 .../stream/examples/StatefulProcessJob.java   |  26 ++-
 .../kafka/stream/internals/KStreamFilter.java |  48 -----
 .../internals/FilteredIterator.java           |   2 +-
 .../internals/KStreamBranch.java              |  16 +-
 .../kstream/internals/KStreamFilter.java      |  65 +++++++
 .../internals/KStreamFlatMap.java             |   4 +-
 .../internals/KStreamFlatMapValues.java       |   4 +-
 .../{ => kstream}/internals/KStreamImpl.java  |  31 +--
 .../{ => kstream}/internals/KStreamJoin.java  |   6 +-
 .../{ => kstream}/internals/KStreamMap.java   |   4 +-
 .../internals/KStreamMapValues.java           |   4 +-
 .../internals/KStreamSource.java              |  10 +-
 .../internals/KStreamWindow.java              |  10 +-
 .../processor/Chooser.java                    |   2 +-
 .../processor/KafkaProcessor.java             |  16 +-
 .../processor/KafkaSource.java                |   6 +-
 .../kafka/stream/processor/PConfig.java       |  33 ++++
 .../stream/processor/PTopologyBuilder.java    | 131 +++++++++++++
 .../processor/Processor.java                  |   2 +-
 .../processor/ProcessorConfig.java            |   2 +-
 .../processor/ProcessorContext.java           |   2 +-
 .../processor/ProcessorProperties.java        |   2 +-
 .../processor/Punctuator.java                 |   2 +-
 .../processor/RecordCollector.java            |   2 +-
 .../processor/RecordQueue.java                |   2 +-
 .../processor/RestoreFunc.java                |   2 +-
 .../processor/Stamped.java                    |   2 +-
 .../processor/StampedRecord.java              |   2 +-
 .../processor/StateStore.java                 |   2 +-
 .../processor/TimestampExtractor.java         |   2 +-
 .../processor/TimestampTracker.java           |   2 +-
 .../processor/internals/Ingestor.java         |   2 +-
 .../processor/internals/IngestorImpl.java     |   2 +-
 .../processor/internals/KStreamThread.java    |  24 +--
 .../internals/MinTimestampTracker.java        |   6 +-
 .../processor/internals/OffsetCheckpoint.java |   2 +-
 .../stream/processor/internals/PTopology.java | 138 ++++++++++++++
 .../internals/ProcessorContextImpl.java       |  22 +--
 .../internals/ProcessorStateManager.java      |   6 +-
 .../processor/internals/PunctuationQueue.java |   2 +-
 .../internals/PunctuationSchedule.java        |   6 +-
 .../internals/RecordCollectorImpl.java        |   4 +-
 .../processor/internals/RecordQueueImpl.java  |  12 +-
 .../internals/RoundRobinChooser.java          |   6 +-
 .../processor/internals/StreamGroup.java      |  16 +-
 .../processor/internals/TimeBasedChooser.java |   6 +-
 .../stream/state/InMemoryKeyValueStore.java   |   2 +-
 .../kafka/stream/state/KeyValueStore.java     |   2 +-
 .../stream/state/MeteredKeyValueStore.java    |   6 +-
 .../stream/state/RocksDBKeyValueStore.java    |   2 +-
 .../kafka/stream/FilteredIteratorTest.java    |   2 +-
 .../kafka/stream/KStreamBranchTest.java       |   6 +-
 .../kafka/stream/KStreamFilterTest.java       |   6 +-
 .../kafka/stream/KStreamFlatMapTest.java      |   6 +-
 .../stream/KStreamFlatMapValuesTest.java      |   6 +-
 .../apache/kafka/stream/KStreamJoinTest.java  |   6 +-
 .../apache/kafka/stream/KStreamMapTest.java   |   6 +-
 .../kafka/stream/KStreamMapValuesTest.java    |   7 +-
 .../kafka/stream/KStreamSourceTest.java       |   6 +-
 .../kafka/stream/KStreamWindowedTest.java     |   6 +-
 .../kafka/stream/MinTimestampTrackerTest.java |   4 +-
 .../apache/kafka/stream/StreamGroupTest.java  |   6 +-
 .../org/apache/kafka/test/MockIngestor.java   |   4 +-
 ...y.java => MockKStreamTopologyBuilder.java} |   6 +-
 .../org/apache/kafka/test/MockProcessor.java  |   2 +-
 .../kafka/test/MockProcessorContext.java      |   8 +
 .../org/apache/kafka/test/MockSource.java     |   2 +-
 .../apache/kafka/test/UnlimitedWindow.java    |   6 +-
 temp                                          |   6 +-
 temp2                                         |   6 +-
 79 files changed, 652 insertions(+), 549 deletions(-)
 delete mode 100644 stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java
 delete mode 100644 stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java
 delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/KStreamFilter.java
 rename stream/src/main/java/org/apache/kafka/stream/{ => kstream}/internals/FilteredIterator.java (97%)
 rename stream/src/main/java/org/apache/kafka/stream/{ => kstream}/internals/KStreamBranch.java (79%)
 create mode 100644 stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java
 rename stream/src/main/java/org/apache/kafka/stream/{ => kstream}/internals/KStreamFlatMap.java (93%)
 rename stream/src/main/java/org/apache/kafka/stream/{ => kstream}/internals/KStreamFlatMapValues.java (92%)
 rename stream/src/main/java/org/apache/kafka/stream/{ => kstream}/internals/KStreamImpl.java (83%)
 rename stream/src/main/java/org/apache/kafka/stream/{ => kstream}/internals/KStreamJoin.java (96%)
 rename stream/src/main/java/org/apache/kafka/stream/{ => kstream}/internals/KStreamMap.java (92%)
 rename stream/src/main/java/org/apache/kafka/stream/{ => kstream}/internals/KStreamMapValues.java (92%)
 rename stream/src/main/java/org/apache/kafka/stream/{ => kstream}/internals/KStreamSource.java (77%)
 rename stream/src/main/java/org/apache/kafka/stream/{ => kstream}/internals/KStreamWindow.java (90%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/Chooser.java (95%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/KafkaProcessor.java (89%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/KafkaSource.java (88%)
 create mode 100644 stream/src/main/java/org/apache/kafka/stream/processor/PConfig.java
 create mode 100644 stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/Processor.java (97%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/ProcessorConfig.java (99%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/ProcessorContext.java (98%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/ProcessorProperties.java (99%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/Punctuator.java (95%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/RecordCollector.java (96%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/RecordQueue.java (97%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/RestoreFunc.java (95%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/Stamped.java (96%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/StampedRecord.java (97%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/StateStore.java (97%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/TimestampExtractor.java (96%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/TimestampTracker.java (97%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/Ingestor.java (95%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/IngestorImpl.java (98%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/KStreamThread.java (94%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/MinTimestampTracker.java (92%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/OffsetCheckpoint.java (99%)
 create mode 100644 stream/src/main/java/org/apache/kafka/stream/processor/internals/PTopology.java
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/ProcessorContextImpl.java (93%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/ProcessorStateManager.java (97%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/PunctuationQueue.java (96%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/PunctuationSchedule.java (88%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/RecordCollectorImpl.java (96%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/RecordQueueImpl.java (90%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/RoundRobinChooser.java (88%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/StreamGroup.java (95%)
 rename stream/src/main/java/org/apache/kafka/{clients => stream}/processor/internals/TimeBasedChooser.java (91%)
 rename stream/src/test/java/org/apache/kafka/test/{MockKStreamTopology.java => MockKStreamTopologyBuilder.java} (84%)

diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
index c02458fd84e84..9893e42ecac34 100755
--- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
@@ -25,8 +25,10 @@
 import java.nio.MappedByteBuffer;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.Properties;
@@ -574,5 +576,17 @@ public static void rm(File file) {
             file.delete();
         }
     }
+<<<<<<< HEAD
 >>>>>>> removing io.confluent imports: wip
+=======
+
+    /**
+     * Returns an empty collection if this list is empty
+     * @param other
+     * @return
+     */
+    public static  List safe(List other) {
+        return other == null ? Collections.EMPTY_LIST : other;
+    }
+>>>>>>> Refactor Processor and KStream APIs
 }
diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java b/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java
deleted file mode 100644
index 67e63fe53d84f..0000000000000
--- a/stream/src/main/java/org/apache/kafka/clients/processor/PTopology.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.clients.processor;
-
-import org.apache.kafka.common.serialization.Deserializer;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-abstract public class PTopology {
-
-    private List processors = new ArrayList<>();
-    private Map sources = new HashMap<>();
-
-    protected final ProcessorProperties properties;
-
-    public PTopology() {
-        this(null);
-    }
-
-    public PTopology(ProcessorProperties properties) {
-        this.properties = properties;
-    }
-
-    public Set sources() {
-        Set sources = new HashSet<>();
-        for (KafkaSource source : this.sources.values()) {
-            sources.add(source);
-        }
-
-        return sources;
-    }
-
-    public Set topics() {
-        return sources.keySet();
-    }
-
-    public KafkaSource source(String topic) {
-        return sources.get(topic);
-    }
-
-    public Deserializer keyDeser(String topic) {
-        KafkaSource source = sources.get(topic);
-
-        if (source == null)
-            throw new IllegalStateException("The topic " + topic + " is unknown.");
-
-        return source.keyDeserializer;
-    }
-
-    public Deserializer valueDeser(String topic) {
-        KafkaSource source = sources.get(topic);
-
-        if (source == null)
-            throw new IllegalStateException("The topic " + topic + " is unknown.");
-
-        return source.valDeserializer;
-    }
-
-    public final void init(ProcessorContext context) {
-        // init the processors following the DAG ordering
-        // such that parents are always initialized before children
-        Deque deque = new ArrayDeque<>();
-        for (KafkaProcessor processor : sources.values()) {
-            deque.addLast(processor);
-        }
-
-        while (!deque.isEmpty()) {
-            KafkaProcessor processor = deque.pollFirst();
-
-            boolean parentsInitialized = true;
-            for (KafkaProcessor parent : (List) processor.parents()) {
-                if (!parent.initialized) {
-                    parentsInitialized = false;
-                    break;
-                }
-            }
-
-            if (parentsInitialized && !processor.initialized) {
-                processor.init(context);
-                processor.initialized = true;
-
-                for (KafkaProcessor child : (List) processor.children()) {
-                    deque.addLast(child);
-                }
-            }
-        }
-    }
-
-    public final  KafkaSource addSource(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) {
-        KafkaSource source = new KafkaSource<>(keyDeserializer, valDeserializer);
-
-        processors.add(source);
-
-        for (String topic : topics) {
-            if (sources.containsKey(topic))
-                throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor.");
-
-            sources.put(topic, source);
-        }
-
-        return source;
-    }
-
-    public final  void addProcessor(KafkaProcessor processor, KafkaProcessor... parents) {
-        if (processors.contains(processor))
-            throw new IllegalArgumentException("Processor " + processor.name() + " is already added.");
-
-        processors.add(processor);
-
-        if (parents != null) {
-            for (KafkaProcessor parent : parents) {
-                if (!processors.contains(parent))
-                    throw new IllegalArgumentException("Parent processor " + parent.name() + " is not added yet.");
-
-                parent.chain(processor);
-            }
-        }
-    }
-
-    public final void close() {
-        // close the processors following the DAG ordering
-        // such that parents are always initialized before children
-        Deque deque = new ArrayDeque<>();
-        for (KafkaProcessor processor : sources.values()) {
-            deque.addLast(processor);
-        }
-
-        while (!deque.isEmpty()) {
-            KafkaProcessor processor = deque.pollFirst();
-
-            boolean parentsClosed = true;
-            for (KafkaProcessor parent : (List) processor.parents()) {
-                if (!parent.closed) {
-                    parentsClosed = false;
-                    break;
-                }
-            }
-
-            if (parentsClosed && !processor.closed) {
-                processor.close();
-                processor.closed = true;
-
-                for (KafkaProcessor child : (List) processor.children()) {
-                    deque.addLast(child);
-                }
-            }
-        }
-
-        processors.clear();
-        sources.clear();
-    }
-
-    abstract public void build();
-}
diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java
index aa8496e4f59d9..592359296cf18 100644
--- a/stream/src/main/java/org/apache/kafka/stream/KStream.java
+++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java
@@ -17,7 +17,7 @@
 
 package org.apache.kafka.stream;
 
-import org.apache.kafka.clients.processor.KafkaProcessor;
+import org.apache.kafka.stream.processor.KafkaProcessor;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.Serializer;
 
diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java b/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java
index 4eeb7ead5f661..506ba009e26ee 100644
--- a/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java
+++ b/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java
@@ -17,10 +17,10 @@
 
 package org.apache.kafka.stream;
 
-import org.apache.kafka.clients.processor.PTopology;
-import org.apache.kafka.clients.processor.internals.KStreamThread;
-import org.apache.kafka.clients.processor.ProcessorConfig;
-import org.apache.kafka.clients.processor.ProcessorProperties;
+import org.apache.kafka.stream.processor.PTopologyBuilder;
+import org.apache.kafka.stream.processor.internals.KStreamThread;
+import org.apache.kafka.stream.processor.ProcessorConfig;
+import org.apache.kafka.stream.processor.ProcessorProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -28,7 +28,7 @@
  * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and
  * sends output to zero or more output topics.
  * 

- * This processing is defined by extending the {@link PTopology} abstract class to specify the transformation operator build. The + * This processing is defined by extending the {@link PTopologyBuilder} abstract class to specify the transformation operator build. The * {@link KStreamProcess} 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. *

@@ -68,7 +68,7 @@ public class KStreamProcess implements Runnable { private final KStreamThread[] threads; - public KStreamProcess(Class topologyClass, ProcessorProperties processorProperties) throws Exception { + public KStreamProcess(PTopologyBuilder builder, ProcessorProperties processorProperties) throws Exception { if (processorProperties.timestampExtractor() == null) throw new NullPointerException("timestamp extractor is missing"); @@ -76,7 +76,7 @@ public KStreamProcess(Class topologyClass, ProcessorPropert this.threads = new KStreamThread[this.config.numStreamThreads]; for (int i = 0; i < this.threads.length; i++) { - this.threads[i] = new KStreamThread(topologyClass, processorProperties); + this.threads[i] = new KStreamThread(builder, processorProperties); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java b/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java deleted file mode 100644 index 3b4201f603502..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamTopology.java +++ /dev/null @@ -1,70 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorProperties; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.stream.internals.KStreamSource; - -/** - * KStreamTopology is the class that allows an implementation of {@link KStreamTopology#build()} to create KStream instances. - */ -public abstract class KStreamTopology extends PTopology { - - public KStreamTopology() { - super(); - } - - public KStreamTopology(ProcessorProperties properties) { - super(properties); - } - - /** - * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. - * - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream - */ - public KStream from(String... topics) { - if (properties == null) - throw new KafkaException("No default deserializers specified in the config."); - - KafkaProcessor source = addSource( - (Deserializer) properties.keyDeserializer(), - (Deserializer) properties.valueDeserializer(), topics); - return new KStreamSource<>(this, source); - } - - /** - * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. - * - * @param keyDeserializer key deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param valDeserializer value deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream - */ - public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KafkaProcessor source = addSource(keyDeserializer, valDeserializer, topics); - return new KStreamSource<>(this, source); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java index c7f4c98e4dfb7..5e996c700c029 100644 --- a/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java @@ -17,9 +17,9 @@ package org.apache.kafka.stream; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.stream.internals.FilteredIterator; -import org.apache.kafka.clients.processor.Stamped; +import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.stream.kstream.internals.FilteredIterator; +import org.apache.kafka.stream.processor.Stamped; import java.util.HashMap; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/Window.java b/stream/src/main/java/org/apache/kafka/stream/Window.java index 3707b838b7958..8fb752fc8af54 100644 --- a/stream/src/main/java/org/apache/kafka/stream/Window.java +++ b/stream/src/main/java/org/apache/kafka/stream/Window.java @@ -17,8 +17,8 @@ package org.apache.kafka.stream; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.clients.processor.StateStore; +import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.stream.processor.StateStore; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java index ba3698b8e668c..c648b05940385 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java @@ -17,11 +17,11 @@ package org.apache.kafka.stream.examples; -import org.apache.kafka.clients.processor.ProcessorProperties; +import org.apache.kafka.stream.KStreamTopologyBuilder; +import org.apache.kafka.stream.processor.ProcessorProperties; import org.apache.kafka.stream.KStreamProcess; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.KStreamTopology; import org.apache.kafka.stream.KeyValue; import org.apache.kafka.stream.KeyValueMapper; import org.apache.kafka.stream.Predicate; @@ -30,48 +30,44 @@ public class KStreamJob { - private static class MyKStreamTopology extends KStreamTopology { + public static void main(String[] args) throws Exception { + ProcessorProperties properties = new ProcessorProperties(new Properties()); + KStreamTopologyBuilder builder = new KStreamTopologyBuilder(); - @Override - public void build() { - // With overridden de-serializer - KStream stream1 = from(new StringDeserializer(), new StringDeserializer(), "topic1"); + KStream stream1 = builder.from(new StringDeserializer(), new StringDeserializer(), "topic1"); - KStream stream2 = - stream1.map(new KeyValueMapper() { - @Override - public KeyValue apply(String key, String value) { - return new KeyValue<>(key, new Integer(value)); - } - }).filter(new Predicate() { - @Override - public boolean apply(String key, Integer value) { - return true; - } - }); + KStream stream2 = + stream1.map(new KeyValueMapper() { + @Override + public KeyValue apply(String key, String value) { + return new KeyValue<>(key, new Integer(value)); + } + }).filter(new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + }); - KStream[] streams = stream2.branch( - new Predicate() { - @Override - public boolean apply(String key, Integer value) { - return true; - } - }, - new Predicate() { - @Override - public boolean apply(String key, Integer value) { - return true; - } + KStream[] streams = stream2.branch( + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; } - ); + }, + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + } + ); - streams[0].sendTo("topic2"); - streams[1].sendTo("topic3"); - } - } + streams[0].sendTo("topic2"); + streams[1].sendTo("topic3"); - public static void main(String[] args) throws Exception { - KStreamProcess kstream = new KStreamProcess(MyKStreamTopology.class, new ProcessorProperties(new Properties())); + KStreamProcess kstream = new KStreamProcess(builder, properties); kstream.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java index d05c6b81505d1..e902a3fd3260c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java @@ -17,11 +17,11 @@ package org.apache.kafka.stream.examples; -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.clients.processor.ProcessorProperties; -import org.apache.kafka.clients.processor.KafkaSource; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.PTopologyBuilder; +import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.stream.processor.ProcessorProperties; +import org.apache.kafka.stream.processor.KafkaSource; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.KStreamProcess; @@ -57,18 +57,14 @@ public void close() { } } - private static class MyPTopology extends PTopology { - - @Override - public void build() { - KafkaSource source = addSource(new StringDeserializer(), new IntegerDeserializer(), "topic-source"); + public static void main(String[] args) throws Exception { + ProcessorProperties properties = new ProcessorProperties(new Properties()); + PTopologyBuilder builder = new PTopologyBuilder(); - addProcessor(new MyProcessor("processor"), source); - } - } + builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); + builder.addProcessor("PROCESS", MyProcessor.class, null, "SOURCE"); - public static void main(String[] args) throws Exception { - KStreamProcess streaming = new KStreamProcess(MyPTopology.class, new ProcessorProperties(new Properties())); + KStreamProcess streaming = new KStreamProcess(builder, properties); streaming.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java index 9ca11dadb7732..ce44e9ca0dac2 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java @@ -18,13 +18,13 @@ package org.apache.kafka.stream.examples; import org.apache.kafka.stream.KStreamProcess; -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.clients.processor.KafkaSource; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.PTopologyBuilder; +import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.stream.processor.KafkaSource; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.clients.processor.ProcessorProperties; +import org.apache.kafka.stream.processor.ProcessorProperties; import org.apache.kafka.stream.state.Entry; import org.apache.kafka.stream.state.InMemoryKeyValueStore; import org.apache.kafka.stream.state.KeyValueIterator; @@ -78,18 +78,14 @@ public void close() { } } - private static class MyPTopology extends PTopology { - - @Override - public void build() { - KafkaSource source = addSource(new StringDeserializer(), new IntegerDeserializer(), "topic-source"); + public static void main(String[] args) throws Exception { + ProcessorProperties properties = new ProcessorProperties(new Properties()); + PTopologyBuilder builder = new PTopologyBuilder(); - addProcessor(new MyProcessor("processor"), source); - } - } + builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); + builder.addProcessor("PROCESS", MyProcessor.class, null, "SOURCE"); - public static void main(String[] args) throws Exception { - KStreamProcess streaming = new KStreamProcess(MyPTopology.class, new ProcessorProperties(new Properties())); + KStreamProcess streaming = new KStreamProcess(builder, properties); streaming.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFilter.java deleted file mode 100644 index 6030ecea2cc56..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFilter.java +++ /dev/null @@ -1,48 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.stream.Predicate; - -class KStreamFilter extends KafkaProcessor { - - private static final String FILTER_NAME = "KAFKA-FILTER"; - - private final Predicate predicate; - private final boolean filterOut; - - public KStreamFilter(Predicate predicate) { - this(predicate, false); - } - - public KStreamFilter(Predicate predicate, boolean filterOut) { - super(FILTER_NAME); - - this.predicate = predicate; - this.filterOut = filterOut; - } - - @Override - public void process(K key, V value) { - if ((!filterOut && predicate.apply(key, value)) - || (filterOut && !predicate.apply(key, value))) { - forward(key, value); - } - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/FilteredIterator.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/FilteredIterator.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/internals/FilteredIterator.java rename to stream/src/main/java/org/apache/kafka/stream/kstream/internals/FilteredIterator.java index 179039710c355..4b9fed9421309 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/FilteredIterator.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/FilteredIterator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.stream.kstream.internals; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java similarity index 79% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java rename to stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java index 661ae16ae6e75..4b1fd5f73d517 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java @@ -15,26 +15,28 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.stream.kstream.internals; -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.PTopologyBuilder; import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.Predicate; import java.lang.reflect.Array; import java.util.Arrays; +import java.util.concurrent.atomic.AtomicInteger; class KStreamBranch extends KafkaProcessor { - private static final String BRANCH_NAME = "KAFKA-BRANCH"; + private static final String BRANCH_NAME = "KAFKA-BRANCH-"; + private static final AtomicInteger BRANCH_INDEX = new AtomicInteger(1); - private final PTopology topology; + private final PTopologyBuilder topology; private final Predicate[] predicates; private final KStreamFilter[] branches; @SuppressWarnings("unchecked") - public KStreamBranch(Predicate[] predicates, PTopology topology, KafkaProcessor parent) { + public KStreamBranch(Predicate[] predicates, PTopologyBuilder topology, String parent) { super(BRANCH_NAME); this.topology = topology; @@ -44,7 +46,7 @@ public KStreamBranch(Predicate[] predicates, PTopology topology, KafkaProc // NOTE that branches here is just a list of predicates, hence not necessarily mutual exclusive for (int i = 0; i < branches.length; i++) { branches[i] = new KStreamFilter<>(predicates[i], false); - topology.addProcessor(branches[i], parent); + topology.addProcessor(BRANCH_NAME + BRANCH_INDEX.getAndIncrement(), branches[i], parent); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java new file mode 100644 index 0000000000000..06bac4efc3fd6 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.kstream.internals; + +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.Predicate; +import org.apache.kafka.stream.processor.PConfig; + +import java.util.concurrent.atomic.AtomicInteger; + +class KStreamFilter extends KafkaProcessor { + + public static final String FILTER_NAME = "KAFKA-FILTER-"; + public static final AtomicInteger FILTER_INDEX = new AtomicInteger(1); + + private final PredicateOut predicateOut; + + public static final class PredicateOut { + + public final Predicate predicate; + public final boolean filterOut; + + public PredicateOut(Predicate predicate) { + this(predicate, false); + } + + public PredicateOut(Predicate predicate, boolean filterOut) { + this.predicate = predicate; + this.filterOut = filterOut; + } + } + + @SuppressWarnings("unchecked") + public KStreamFilter(String name, PConfig config) { + super(name, config); + + if (this.config() == null) + throw new IllegalStateException("PConfig should be specified."); + + this.predicateOut = (PredicateOut) config.value(); + } + + @Override + public void process(K key, V value) { + if ((!predicateOut.filterOut && predicateOut.predicate.apply(key, value)) + || (predicateOut.filterOut && !predicateOut.predicate.apply(key, value))) { + forward(key, value); + } + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java similarity index 93% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMap.java rename to stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java index a78b594689003..f371170620a4e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.stream.kstream.internals; -import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.KeyValue; import org.apache.kafka.stream.KeyValueMapper; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java similarity index 92% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMapValues.java rename to stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java index ded053910f121..772874bd8b5c5 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.stream.kstream.internals; -import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.ValueMapper; class KStreamFlatMapValues extends KafkaProcessor { diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java similarity index 83% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java rename to stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java index 885a99e41532f..6a7c7fdc1b31e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.stream.kstream.internals; -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.clients.processor.KafkaSource; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.PConfig; +import org.apache.kafka.stream.processor.PTopologyBuilder; +import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.stream.processor.KafkaSource; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.stream.KStreamWindowed; @@ -34,32 +35,32 @@ public class KStreamImpl implements KStream { private static final String PROCESSOR_NAME = "KAFKA-PROCESS"; - protected PTopology topology; - protected KafkaProcessor processor; + protected PTopologyBuilder topology; + protected String name; - public KStreamImpl(PTopology topology, KafkaProcessor processor) { + public KStreamImpl(PTopologyBuilder topology, String name) { this.topology = topology; - this.processor = processor; + this.name = name; } @SuppressWarnings("unchecked") @Override public KStream filter(Predicate predicate) { - KStreamFilter filter = new KStreamFilter<>(predicate); + String name = KStreamFilter.FILTER_NAME + KStreamFilter.FILTER_INDEX.getAndIncrement(); - topology.addProcessor(filter, processor); + topology.addProcessor(name, KStreamFilter.class, new PConfig("Predicate", new KStreamFilter.PredicateOut<>(predicate)), name); - return new KStreamImpl<>(topology, filter); + return new KStreamImpl<>(topology, name); } @SuppressWarnings("unchecked") @Override public KStream filterOut(final Predicate predicate) { - KStreamFilter filter = new KStreamFilter<>(predicate, true); + String name = KStreamFilter.FILTER_NAME + KStreamFilter.FILTER_INDEX.getAndIncrement(); - topology.addProcessor(filter, processor); + topology.addProcessor(name, KStreamFilter.class, new PConfig("Predicate", new KStreamFilter.PredicateOut<>(predicate, true)), name); - return new KStreamImpl<>(topology, filter); + return new KStreamImpl<>(topology, name); } @SuppressWarnings("unchecked") diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamJoin.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java rename to stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamJoin.java index 0ef3b09f23209..370eda766856b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamJoin.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.stream.kstream.internals; -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.ProcessorContext; import org.apache.kafka.stream.ValueJoiner; import org.apache.kafka.stream.Window; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java similarity index 92% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamMap.java rename to stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java index d403e6d061dad..9797c2c2d6e13 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.stream.kstream.internals; -import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.KeyValue; import org.apache.kafka.stream.KeyValueMapper; diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java similarity index 92% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamMapValues.java rename to stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java index 1692bf743f8a1..707652ce3c8c3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.stream.kstream.internals; -import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.ValueMapper; class KStreamMapValues extends KafkaProcessor { diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSource.java similarity index 77% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java rename to stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSource.java index a06fb1f427817..ae358337991bd 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSource.java @@ -15,17 +15,17 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.stream.kstream.internals; -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.PTopologyBuilder; public class KStreamSource extends KStreamImpl { private KafkaProcessor source; - public KStreamSource(PTopology topology, KafkaProcessor source) { - super(topology, source); + public KStreamSource(PTopologyBuilder topology, KafkaProcessor source) { + super(topology, source.name()); this.source = source; } diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamWindow.java similarity index 90% rename from stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java rename to stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamWindow.java index 412335fc6ca33..28f849eb709ec 100644 --- a/stream/src/main/java/org/apache/kafka/stream/internals/KStreamWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamWindow.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.stream.internals; +package org.apache.kafka.stream.kstream.internals; -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.PTopologyBuilder; +import org.apache.kafka.stream.processor.ProcessorContext; import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.KStreamWindowed; import org.apache.kafka.stream.ValueJoiner; @@ -31,7 +31,7 @@ public static final class KStreamWindowedImpl extends KStreamImpl im public KStreamWindow windowed; - public KStreamWindowedImpl(PTopology topology, KStreamWindow windowed) { + public KStreamWindowedImpl(PTopologyBuilder topology, KStreamWindow windowed) { super(topology, windowed); this.windowed = windowed; } diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java b/stream/src/main/java/org/apache/kafka/stream/processor/Chooser.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java rename to stream/src/main/java/org/apache/kafka/stream/processor/Chooser.java index 0d7aac763bc51..434db8d23d1d7 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/Chooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/Chooser.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; public interface Chooser { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java b/stream/src/main/java/org/apache/kafka/stream/processor/KafkaProcessor.java similarity index 89% rename from stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java rename to stream/src/main/java/org/apache/kafka/stream/processor/KafkaProcessor.java index 7b5a351e74a5b..a46c61732e246 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaProcessor.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/KafkaProcessor.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; import java.util.ArrayList; import java.util.List; @@ -26,23 +26,33 @@ public abstract class KafkaProcessor implements Processor> parents; private final String name; + private final PConfig config; public boolean initialized; - public boolean closed; public KafkaProcessor(String name) { + this(name, null); + } + + public KafkaProcessor(String name, PConfig config) { this.name = name; + this.config = config; + this.children = new ArrayList<>(); this.parents = new ArrayList<>(); this.initialized = false; - this.closed = false; } public String name() { return name; } + public PConfig config() { + return config; + } + + public List> children() { return children; } diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaSource.java b/stream/src/main/java/org/apache/kafka/stream/processor/KafkaSource.java similarity index 88% rename from stream/src/main/java/org/apache/kafka/clients/processor/KafkaSource.java rename to stream/src/main/java/org/apache/kafka/stream/processor/KafkaSource.java index 15c60636d50ea..cea72920cb767 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/KafkaSource.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/KafkaSource.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; import org.apache.kafka.common.serialization.Deserializer; @@ -26,8 +26,8 @@ public class KafkaSource extends KafkaProcessor { public Deserializer keyDeserializer; public Deserializer valDeserializer; - public KafkaSource(Deserializer keyDeserializer, Deserializer valDeserializer) { - super(SOURCE_NAME); + public KafkaSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer) { + super(name); this.keyDeserializer = keyDeserializer; this.valDeserializer = valDeserializer; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/PConfig.java b/stream/src/main/java/org/apache/kafka/stream/processor/PConfig.java new file mode 100644 index 0000000000000..2e867a13cab50 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/processor/PConfig.java @@ -0,0 +1,33 @@ +/** + * 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.stream.processor; + +public class PConfig { + + private String name; + private Object value; + + public PConfig(String name, Object value) { + this.name = name; + this.value = value; + } + + public Object value() { + return value; + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java b/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java new file mode 100644 index 0000000000000..c6797b0875332 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java @@ -0,0 +1,131 @@ +/** + * 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.stream.processor; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.processor.internals.PTopology; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Deque; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class PTopologyBuilder { + + private Map processorClasses = new HashMap<>(); + private Map sources = new HashMap<>(); + + private Map> parents = new HashMap<>(); + private Map> children = new HashMap<>(); + + private class ProcessorClazz { + public Class clazz; + public PConfig config; + + public ProcessorClazz(Class clazz, PConfig config) { + this.clazz = clazz; + this.config = config; + } + } + + public PTopologyBuilder() {} + + @SuppressWarnings("unchecked") + public final KafkaSource addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + KafkaSource source = new KafkaSource(name, keyDeserializer, valDeserializer); + + for (String topic : topics) { + if (sources.containsKey(topic)) + throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); + + sources.put(name, source); + } + + return source; + } + + public final void addProcessor(String name, Class processorClass, PConfig config, String... parentNames) { + if (processorClasses.containsKey(name)) + throw new IllegalArgumentException("Processor " + name + " is already added."); + + processorClasses.put(name, new ProcessorClazz(processorClass, config)); + + if (parentNames != null) { + for (String parent : parentNames) { + if (!processorClasses.containsKey(parent)) + throw new IllegalArgumentException("Parent processor " + parent + " is not added yet."); + + // add to parent list + if (!parents.containsKey(name)) + parents.put(name, new ArrayList<>()); + parents.get(name).add(parent); + + // add to parent list + if (!children.containsKey(parent)) + children.put(parent, new ArrayList<>()); + children.get(parent).add(name); + } + } + } + + /** + * Build the topology by creating the processors + */ + @SuppressWarnings("unchecked") + public PTopology build() { + Map processorMap = new HashMap<>(); + + // create processors + try { + for (String name : processorClasses.keySet()) { + PConfig config = processorClasses.get(name).config; + Class processorClass = processorClasses.get(name).clazz; + KafkaProcessor processor = processorClass.getConstructor(String.class, PConfig.class).newInstance(name, config); + processorMap.put(name, processor); + } + } catch (Exception e) { + throw new KafkaException("KafkaProcessor(String) constructor failed: this should not happen."); + } + + // construct processor parent-child relationships + for (String topic : sources.keySet()) { + KafkaSource source = sources.get(topic); + processorMap.put(source.name(), source); + + // chain children to this processor + for (String child : children.get(source.name())) { + KafkaProcessor childProcessor = processorMap.get(child); + source.chain(childProcessor); + } + } + + for (KafkaProcessor processor : processorMap.values()) { + // chain children to this processor + for (String child : children.get(processor.name())) { + KafkaProcessor childProcessor = processorMap.get(child); + processor.chain(childProcessor); + } + + } + + return new PTopology(processorMap, sources); + } +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/Processor.java b/stream/src/main/java/org/apache/kafka/stream/processor/Processor.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/clients/processor/Processor.java rename to stream/src/main/java/org/apache/kafka/stream/processor/Processor.java index a8d25b76080f0..7ddce341eb72a 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/Processor.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/Processor.java @@ -65,7 +65,7 @@ public interface ProcessorContext { * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; public interface Processor { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorConfig.java similarity index 99% rename from stream/src/main/java/org/apache/kafka/clients/processor/ProcessorConfig.java rename to stream/src/main/java/org/apache/kafka/stream/processor/ProcessorConfig.java index 2c4b2f980fad4..407476710df7d 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorConfig.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorConfig.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorContext.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java rename to stream/src/main/java/org/apache/kafka/stream/processor/ProcessorContext.java index 514ff6945f6f4..b26458b5e62e3 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorContext.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorProperties.java b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorProperties.java similarity index 99% rename from stream/src/main/java/org/apache/kafka/clients/processor/ProcessorProperties.java rename to stream/src/main/java/org/apache/kafka/stream/processor/ProcessorProperties.java index 21f11fa4c9f1c..c2f274a97cc44 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/ProcessorProperties.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorProperties.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/Punctuator.java b/stream/src/main/java/org/apache/kafka/stream/processor/Punctuator.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/clients/processor/Punctuator.java rename to stream/src/main/java/org/apache/kafka/stream/processor/Punctuator.java index c52f207ff0f8b..2fece68c4ef2a 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/Punctuator.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/Punctuator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; public interface Punctuator { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/RecordCollector.java b/stream/src/main/java/org/apache/kafka/stream/processor/RecordCollector.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/clients/processor/RecordCollector.java rename to stream/src/main/java/org/apache/kafka/stream/processor/RecordCollector.java index 14471b4c3683a..2db7f2c7f6314 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/RecordCollector.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/RecordCollector.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/RecordQueue.java b/stream/src/main/java/org/apache/kafka/stream/processor/RecordQueue.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/clients/processor/RecordQueue.java rename to stream/src/main/java/org/apache/kafka/stream/processor/RecordQueue.java index 99495d9736ff3..5e3a345e8600b 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/RecordQueue.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; import org.apache.kafka.common.TopicPartition; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/RestoreFunc.java b/stream/src/main/java/org/apache/kafka/stream/processor/RestoreFunc.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/clients/processor/RestoreFunc.java rename to stream/src/main/java/org/apache/kafka/stream/processor/RestoreFunc.java index 4b5a78e1d4523..ab6e7b25ae495 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/RestoreFunc.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/RestoreFunc.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; // TODO: this should be removed once we move to Java 8 public interface RestoreFunc { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/Stamped.java b/stream/src/main/java/org/apache/kafka/stream/processor/Stamped.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/clients/processor/Stamped.java rename to stream/src/main/java/org/apache/kafka/stream/processor/Stamped.java index 9654e77f8d964..79c8eeb1d9859 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/Stamped.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/Stamped.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; public class Stamped implements Comparable { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/StampedRecord.java b/stream/src/main/java/org/apache/kafka/stream/processor/StampedRecord.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/clients/processor/StampedRecord.java rename to stream/src/main/java/org/apache/kafka/stream/processor/StampedRecord.java index 6936eeaeca7db..2a93f1ef5ecef 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/StampedRecord.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/StampedRecord.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/StateStore.java b/stream/src/main/java/org/apache/kafka/stream/processor/StateStore.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/clients/processor/StateStore.java rename to stream/src/main/java/org/apache/kafka/stream/processor/StateStore.java index 348ab10425d08..71cf5a1cbd648 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/StateStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/StateStore.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; /** * A storage engine for managing state maintained by a stream processor. diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/TimestampExtractor.java b/stream/src/main/java/org/apache/kafka/stream/processor/TimestampExtractor.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/clients/processor/TimestampExtractor.java rename to stream/src/main/java/org/apache/kafka/stream/processor/TimestampExtractor.java index 4f4b6c2f488ce..f135fd8dfaacc 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/TimestampExtractor.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/TimestampExtractor.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; /** * An interface that allows the KStream framework to extract a timestamp from a key-value pair diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/stream/processor/TimestampTracker.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java rename to stream/src/main/java/org/apache/kafka/stream/processor/TimestampTracker.java index 5625eb16d79eb..13acca140a842 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/TimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/TimestampTracker.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor; +package org.apache.kafka.stream.processor; /** * TimestampTracker is a helper class for a sliding window implementation. diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/Ingestor.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/Ingestor.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/Ingestor.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/Ingestor.java index a59313480e016..1d1e233f54f1b 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/Ingestor.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/Ingestor.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; import org.apache.kafka.common.TopicPartition; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/IngestorImpl.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/IngestorImpl.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/IngestorImpl.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/IngestorImpl.java index 59758e7798559..6ba190898ca09 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/IngestorImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/IngestorImpl.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; import org.apache.kafka.clients.consumer.CommitType; import org.apache.kafka.clients.consumer.Consumer; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/KStreamThread.java similarity index 94% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/KStreamThread.java index 6bedafccdb365..0a760cc85da20 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/KStreamThread.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorConfig; -import org.apache.kafka.clients.processor.ProcessorProperties; +import org.apache.kafka.stream.processor.PTopologyBuilder; +import org.apache.kafka.stream.processor.ProcessorConfig; +import org.apache.kafka.stream.processor.ProcessorProperties; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.KafkaException; @@ -54,7 +54,7 @@ public class KStreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); - private final Class topologyClass; + private final PTopologyBuilder builder; private final ArrayList streamGroups = new ArrayList<>(); private final Map kstreamContexts = new HashMap<>(); private final IngestorImpl ingestor; @@ -85,7 +85,7 @@ public void onPartitionsRevoked(Consumer consumer, Collection topologyClass, ProcessorProperties properties) throws Exception { + public KStreamThread(PTopologyBuilder builder, ProcessorProperties properties) throws Exception { super(); if (properties.timestampExtractor() == null) @@ -93,14 +93,13 @@ public KStreamThread(Class topologyClass, ProcessorProperti this.metrics = new Metrics(); this.config = new ProcessorConfig(properties.config()); - this.topologyClass = topologyClass; + this.builder = builder; this.properties = properties; this.streamingMetrics = new KafkaStreamingMetrics(); // build the topology without initialization to get the topics for consumer - PTopology topology = topologyClass.getConstructor().newInstance(); - topology.build(); + PTopology topology = builder.build(); // create the producer and consumer clients Producer producer = new KafkaProducer<>(properties.config(), new ByteArraySerializer(), new ByteArraySerializer()); @@ -238,11 +237,14 @@ private void addPartitions(Collection assignment) { if (context == null) { try { // build the topology and initialize with the context - PTopology topology = this.topologyClass.getConstructor().newInstance(); + PTopology topology = builder.build(); + context = new ProcessorContextImpl(id, ingestor, topology, collector, properties, config, metrics); - topology.build(); + topology.init(context); + context.initialized(); + kstreamContexts.put(id, context); } catch (Exception e) { throw new KafkaException(e); diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/MinTimestampTracker.java similarity index 92% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/MinTimestampTracker.java index ec190cf367aa6..d529e59af8024 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/MinTimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/MinTimestampTracker.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; -import org.apache.kafka.clients.processor.Stamped; -import org.apache.kafka.clients.processor.TimestampTracker; +import org.apache.kafka.stream.processor.Stamped; +import org.apache.kafka.stream.processor.TimestampTracker; import java.util.LinkedList; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/OffsetCheckpoint.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/OffsetCheckpoint.java similarity index 99% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/OffsetCheckpoint.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/OffsetCheckpoint.java index 37585373be5e5..dc143ec244327 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/OffsetCheckpoint.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/OffsetCheckpoint.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; import org.apache.kafka.common.TopicPartition; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/PTopology.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/PTopology.java new file mode 100644 index 0000000000000..8d515d867e34d --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/PTopology.java @@ -0,0 +1,138 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream.processor.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.KafkaSource; +import org.apache.kafka.stream.processor.ProcessorContext; + +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class PTopology { + + private Map processors = new HashMap<>(); + private Map topicSources = new HashMap<>(); + + public PTopology(Map processors, Map topicSources) { + this.processors = processors; + this.topicSources = topicSources; + } + + public Set sources() { + Set sources = new HashSet<>(); + for (KafkaSource source : this.topicSources.values()) { + sources.add(source); + } + + return sources; + } + + public Set topics() { + return topicSources.keySet(); + } + + public KafkaSource source(String topic) { + return topicSources.get(topic); + } + + public Deserializer keyDeser(String topic) { + KafkaSource source = topicSources.get(topic); + + if (source == null) + throw new IllegalStateException("The topic " + topic + " is unknown."); + + return source.keyDeserializer; + } + + public Deserializer valueDeser(String topic) { + KafkaSource source = topicSources.get(topic); + + if (source == null) + throw new IllegalStateException("The topic " + topic + " is unknown."); + + return source.valDeserializer; + } + + /** + * initialize the processors following the DAG ordering + * such that parents are always created and initialized before children + */ + @SuppressWarnings("unchecked") + public final void init(ProcessorContext context) { + Deque deque = new ArrayDeque<>(); + + // initialize sources + for (String topic : topicSources.keySet()) { + KafkaSource source = topicSources.get(topic); + + source.init(context); + source.initialized = true; + + // put source children to be traversed first + for (KafkaProcessor childProcessor : (List) source.children()) { + deque.addLast(childProcessor); + } + } + + // traverse starting at the sources' children, and initialize the processor + // if 1) it has not been initialized, 2) all its parents are initialized + while (!deque.isEmpty()) { + KafkaProcessor processor = deque.pollFirst(); + + if (processor.initialized) + continue; + + boolean parentsInitialized = true; + + for (KafkaProcessor parentProcessor : (List) processor.parents()) { + if (!parentProcessor.initialized) { + parentsInitialized = false; + break; + } + } + + if (parentsInitialized) { + processor.init(context); + processor.initialized = true; + + // put source children to be traversed first + for (KafkaProcessor childProcessor : (List) processor.children()) { + deque.addLast(childProcessor); + } + } + } + } + + public final void close() { + // close the processors + // TODO: do we need to follow the DAG ordering + for (KafkaProcessor processor : processors.values()) { + processor.close(); + } + + processors.clear(); + topicSources.clear(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorContextImpl.java similarity index 93% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorContextImpl.java index 1d38cfd85850a..9f795ab7353fd 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorContextImpl.java @@ -15,20 +15,20 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.processor.KafkaSource; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorConfig; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.clients.processor.ProcessorProperties; -import org.apache.kafka.clients.processor.RecordCollector; -import org.apache.kafka.clients.processor.StateStore; -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.RestoreFunc; -import org.apache.kafka.clients.processor.TimestampExtractor; +import org.apache.kafka.stream.processor.KafkaSource; +import org.apache.kafka.stream.processor.PTopologyBuilder; +import org.apache.kafka.stream.processor.ProcessorConfig; +import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.stream.processor.ProcessorProperties; +import org.apache.kafka.stream.processor.RecordCollector; +import org.apache.kafka.stream.processor.StateStore; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.RestoreFunc; +import org.apache.kafka.stream.processor.TimestampExtractor; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorStateManager.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorStateManager.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorStateManager.java index 672dccab2312d..aeff8e6264595 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/ProcessorStateManager.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorStateManager.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.processor.RestoreFunc; -import org.apache.kafka.clients.processor.StateStore; +import org.apache.kafka.stream.processor.RestoreFunc; +import org.apache.kafka.stream.processor.StateStore; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationQueue.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationQueue.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationQueue.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationQueue.java index ffc37e405c6db..b53095b4c37ec 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationQueue.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationQueue.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; import java.util.PriorityQueue; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationSchedule.java similarity index 88% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationSchedule.java index fdbe49a36000e..7b47820c038b2 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/PunctuationSchedule.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationSchedule.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; -import org.apache.kafka.clients.processor.Punctuator; -import org.apache.kafka.clients.processor.Stamped; +import org.apache.kafka.stream.processor.Punctuator; +import org.apache.kafka.stream.processor.Stamped; public class PunctuationSchedule extends Stamped { diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordCollectorImpl.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordCollectorImpl.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordCollectorImpl.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordCollectorImpl.java index 1e7edddd7969c..b8dc7f4c86ced 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordCollectorImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordCollectorImpl.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; -import org.apache.kafka.clients.processor.RecordCollector; +import org.apache.kafka.stream.processor.RecordCollector; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueueImpl.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordQueueImpl.java similarity index 90% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueueImpl.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordQueueImpl.java index 6fa5554ebe9f2..c0c99014bf3f8 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RecordQueueImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordQueueImpl.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.KafkaSource; -import org.apache.kafka.clients.processor.RecordQueue; -import org.apache.kafka.clients.processor.StampedRecord; -import org.apache.kafka.clients.processor.TimestampTracker; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.KafkaSource; +import org.apache.kafka.stream.processor.RecordQueue; +import org.apache.kafka.stream.processor.StampedRecord; +import org.apache.kafka.stream.processor.TimestampTracker; import org.apache.kafka.common.TopicPartition; import java.util.ArrayDeque; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/RoundRobinChooser.java similarity index 88% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/RoundRobinChooser.java index f67010e18f5eb..33f0fe84f938d 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/RoundRobinChooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/RoundRobinChooser.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; -import org.apache.kafka.clients.processor.Chooser; -import org.apache.kafka.clients.processor.RecordQueue; +import org.apache.kafka.stream.processor.Chooser; +import org.apache.kafka.stream.processor.RecordQueue; import java.util.ArrayDeque; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/StreamGroup.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/StreamGroup.java index 840a89fade8fc..22f3c53d7253a 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/StreamGroup.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.processor.Chooser; -import org.apache.kafka.clients.processor.KafkaSource; -import org.apache.kafka.clients.processor.Punctuator; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.clients.processor.RecordQueue; -import org.apache.kafka.clients.processor.StampedRecord; -import org.apache.kafka.clients.processor.TimestampExtractor; +import org.apache.kafka.stream.processor.Chooser; +import org.apache.kafka.stream.processor.KafkaSource; +import org.apache.kafka.stream.processor.Punctuator; +import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.stream.processor.RecordQueue; +import org.apache.kafka.stream.processor.StampedRecord; +import org.apache.kafka.stream.processor.TimestampExtractor; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; diff --git a/stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/TimeBasedChooser.java similarity index 91% rename from stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java rename to stream/src/main/java/org/apache/kafka/stream/processor/internals/TimeBasedChooser.java index 0e9a7e93cdb1e..7aa4f0afede3d 100644 --- a/stream/src/main/java/org/apache/kafka/clients/processor/internals/TimeBasedChooser.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/TimeBasedChooser.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.clients.processor.internals; +package org.apache.kafka.stream.processor.internals; -import org.apache.kafka.clients.processor.Chooser; -import org.apache.kafka.clients.processor.RecordQueue; +import org.apache.kafka.stream.processor.Chooser; +import org.apache.kafka.stream.processor.RecordQueue; import java.util.Comparator; import java.util.PriorityQueue; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java index 23858111170a3..25e76d1a12eb5 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream.state; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.processor.ProcessorContext; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java index 1e3954a1b3b1a..18cf1b26afab3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java @@ -19,7 +19,7 @@ package org.apache.kafka.stream.state; -import org.apache.kafka.clients.processor.StateStore; +import org.apache.kafka.stream.processor.StateStore; import java.util.List; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/MeteredKeyValueStore.java index d8d12d9ddb280..a2fd7583d0e58 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/MeteredKeyValueStore.java @@ -17,9 +17,9 @@ package org.apache.kafka.stream.state; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.clients.processor.RecordCollector; -import org.apache.kafka.clients.processor.RestoreFunc; +import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.stream.processor.RecordCollector; +import org.apache.kafka.stream.processor.RestoreFunc; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.MeasurableStat; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java b/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java index f5db6509f180a..505582591fe8f 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java @@ -17,7 +17,7 @@ package org.apache.kafka.stream.state; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.processor.ProcessorContext; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.SystemTime; diff --git a/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java b/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java index 66efc1476e02d..21fd738c6a50f 100644 --- a/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertEquals; -import org.apache.kafka.stream.internals.FilteredIterator; +import org.apache.kafka.stream.kstream.internals.FilteredIterator; import org.junit.Test; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java index bfbfccad153fe..05c7924dc335d 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java @@ -19,8 +19,8 @@ import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.stream.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamTopologyBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -97,7 +97,7 @@ public boolean apply(Integer key, String value) { private String topic1 = "topic"; >>>>>>> adding files - private KStreamTopology topology = new MockKStreamTopology(); + private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java index e8447707b38d0..d705e1d5c58a1 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java @@ -19,8 +19,8 @@ import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.stream.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamTopologyBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -31,7 +31,7 @@ public class KStreamFilterTest { private String topicName = "topic"; - private KStreamTopology topology = new MockKStreamTopology(); + private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java index 89330d27c16f3..ed4a6093585ce 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java @@ -19,8 +19,8 @@ import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.stream.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamTopologyBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -32,7 +32,7 @@ public class KStreamFlatMapTest { private String topicName = "topic"; - private KStreamTopology topology = new MockKStreamTopology(); + private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java index d2e6c3db89135..554471fc03810 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java @@ -19,8 +19,8 @@ import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.stream.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamTopologyBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -32,7 +32,7 @@ public class KStreamFlatMapValuesTest { private String topicName = "topic"; - private KStreamTopology topology = new MockKStreamTopology(); + private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java index 3082053264703..c0a8c85a35116 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java @@ -20,8 +20,8 @@ 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.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.stream.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamTopologyBuilder; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.UnlimitedWindow; @@ -34,7 +34,7 @@ public class KStreamJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private KStreamTopology topology = new MockKStreamTopology(); + private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java index 03727f431298b..061b1a459f7e0 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java @@ -19,8 +19,8 @@ import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.stream.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamTopologyBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -30,7 +30,7 @@ public class KStreamMapTest { private String topicName = "topic"; - private KStreamTopology topology = new MockKStreamTopology(); + private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java index 97b91fd671b56..790484008b843 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java @@ -54,10 +54,15 @@ ======= import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; +<<<<<<< HEAD >>>>>>> wip import org.apache.kafka.stream.internals.KStreamSource; >>>>>>> wip import org.apache.kafka.test.MockKStreamTopology; +======= +import org.apache.kafka.stream.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamTopologyBuilder; +>>>>>>> Refactor Processor and KStream APIs import org.apache.kafka.test.MockProcessor; <<<<<<< HEAD import org.apache.kafka.test.MockKStreamContext; @@ -72,7 +77,7 @@ public class KStreamMapValuesTest { private String topicName = "topic"; - private KStreamTopology topology = new MockKStreamTopology(); + private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java index 7dd814262916e..7a5c86d4ec6ea 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java @@ -18,8 +18,8 @@ package org.apache.kafka.stream; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.stream.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamTopologyBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -29,7 +29,7 @@ public class KStreamSourceTest { private String topicName = "topic"; - private KStreamTopology topology = new MockKStreamTopology(); + private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); private StringDeserializer keyDeserializer = new StringDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java index 2529acb1c0b61..39f8430b63bce 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java @@ -19,8 +19,8 @@ import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopology; +import org.apache.kafka.stream.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamTopologyBuilder; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.UnlimitedWindow; import org.junit.Test; @@ -33,7 +33,7 @@ public class KStreamWindowedTest { private String topicName = "topic"; - private KStreamTopology topology = new MockKStreamTopology(); + private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java index 44c78377998c2..984f170695715 100644 --- a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java @@ -19,8 +19,8 @@ import static org.junit.Assert.assertEquals; -import org.apache.kafka.clients.processor.internals.MinTimestampTracker; -import org.apache.kafka.clients.processor.Stamped; +import org.apache.kafka.stream.processor.internals.MinTimestampTracker; +import org.apache.kafka.stream.processor.Stamped; import org.junit.Test; public class MinTimestampTrackerTest { diff --git a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java index 4fc437a29c413..f5bad4ba024ad 100644 --- a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java @@ -18,14 +18,14 @@ package org.apache.kafka.stream; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.processor.TimestampExtractor; +import org.apache.kafka.stream.processor.TimestampExtractor; 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.clients.processor.internals.StreamGroup; -import org.apache.kafka.clients.processor.internals.TimeBasedChooser; +import org.apache.kafka.stream.processor.internals.StreamGroup; +import org.apache.kafka.stream.processor.internals.TimeBasedChooser; import org.apache.kafka.test.MockIngestor; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.MockSource; diff --git a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java index 226520a757514..62c6af02dcd93 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java @@ -19,8 +19,8 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.clients.processor.internals.Ingestor; -import org.apache.kafka.clients.processor.internals.StreamGroup; +import org.apache.kafka.stream.processor.internals.Ingestor; +import org.apache.kafka.stream.processor.internals.StreamGroup; import java.util.HashMap; import java.util.HashSet; diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopologyBuilder.java similarity index 84% rename from stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java rename to stream/src/test/java/org/apache/kafka/test/MockKStreamTopologyBuilder.java index 421d758cc450f..43eda5f1eb942 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopology.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamTopologyBuilder.java @@ -17,10 +17,10 @@ package org.apache.kafka.test; -import org.apache.kafka.stream.KStreamTopology; +import org.apache.kafka.stream.KStreamTopologyBuilder; -public class MockKStreamTopology extends KStreamTopology { - public MockKStreamTopology() { +public class MockKStreamTopologyBuilder extends KStreamTopologyBuilder { + public MockKStreamTopologyBuilder() { super(null); } diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java index 4030007c0c2cc..1b22440625305 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -17,7 +17,7 @@ package org.apache.kafka.test; -import org.apache.kafka.clients.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.KafkaProcessor; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 283a802ee3024..cd6f7de1fc8f3 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -17,6 +17,7 @@ package org.apache.kafka.test; +<<<<<<< HEAD <<<<<<< HEAD:stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java <<<<<<< HEAD <<<<<<< HEAD @@ -70,6 +71,13 @@ import org.apache.kafka.clients.processor.RestoreFunc; import org.apache.kafka.clients.processor.StateStore; >>>>>>> wip +======= +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.stream.processor.RecordCollector; +import org.apache.kafka.stream.processor.RestoreFunc; +import org.apache.kafka.stream.processor.StateStore; +>>>>>>> Refactor Processor and KStream APIs import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; diff --git a/stream/src/test/java/org/apache/kafka/test/MockSource.java b/stream/src/test/java/org/apache/kafka/test/MockSource.java index efc3c0dfd3cc1..cf1ace900d17c 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockSource.java +++ b/stream/src/test/java/org/apache/kafka/test/MockSource.java @@ -17,7 +17,7 @@ package org.apache.kafka.test; -import org.apache.kafka.clients.processor.KafkaSource; +import org.apache.kafka.stream.processor.KafkaSource; import org.apache.kafka.common.serialization.Deserializer; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java index bd468cacedba7..197932a08ddca 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java @@ -17,11 +17,11 @@ package org.apache.kafka.test; -import org.apache.kafka.clients.processor.ProcessorContext; +import org.apache.kafka.stream.processor.ProcessorContext; import org.apache.kafka.stream.KeyValue; import org.apache.kafka.stream.Window; -import org.apache.kafka.stream.internals.FilteredIterator; -import org.apache.kafka.clients.processor.Stamped; +import org.apache.kafka.stream.kstream.internals.FilteredIterator; +import org.apache.kafka.stream.processor.Stamped; import java.util.Iterator; import java.util.LinkedList; diff --git a/temp b/temp index 1644812af6928..abe38868a0a47 100644 --- a/temp +++ b/temp @@ -1,3 +1,3 @@ -checkstyle/import-control.xml: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java: needs merge +clients/src/main/java/org/apache/kafka/common/utils/Utils.java: needs merge +stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java: needs merge +stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java: needs merge diff --git a/temp2 b/temp2 index 0abf132f4df58..c5ac26d2fffd7 100644 --- a/temp2 +++ b/temp2 @@ -1,3 +1,3 @@ -checkstyle/import-control.xml -stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java -stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +clients/src/main/java/org/apache/kafka/common/utils/Utils.java +stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java From b1844bd5897a27ea7d497cbe3718d4c52ac23916 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 13 Aug 2015 16:57:55 -0700 Subject: [PATCH 159/275] Further working on KStreamImpl --- .../java/org/apache/kafka/stream/KStream.java | 22 +--- .../kafka/stream/examples/KStreamJob.java | 9 +- .../kstream/internals/KStreamBranch.java | 42 +++---- .../kstream/internals/KStreamFilter.java | 3 - .../kstream/internals/KStreamFlatMap.java | 13 +- .../internals/KStreamFlatMapValues.java | 14 ++- .../stream/kstream/internals/KStreamImpl.java | 112 +++++++++--------- .../stream/kstream/internals/KStreamMap.java | 15 ++- .../kstream/internals/KStreamMapValues.java | 14 ++- .../stream/processor/PTopologyBuilder.java | 2 - 10 files changed, 121 insertions(+), 125 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java index 592359296cf18..7ee22f74dc7e2 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java @@ -20,6 +20,7 @@ import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.stream.processor.PConfig; /** * KStream is an abstraction of a stream of key-value pairs. @@ -99,16 +100,6 @@ public interface KStream { */ KStream[] branch(Predicate... predicates); - /** - * Sends key-value to a topic, also creates a new stream from the topic. - * The created stream is added to the default synchronization group. - * This is equivalent to calling sendTo(topic) and from(topic). - * - * @param topic the topic name - * @return KStream - */ - KStream through(String topic); - /** * Sends key-value to a topic, also creates a new stream from the topic. * The created stream is added to the default synchronization group. @@ -129,13 +120,6 @@ public interface KStream { */ KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); - /** - * Sends key-value to a topic. - * - * @param topic the topic name - */ - void sendTo(String topic); - /** * Sends key-value to a topic. * @@ -150,7 +134,7 @@ public interface KStream { /** * Processes all elements in this stream by applying a processor. * - * @param processor the instance of Processor + * @param processorClass the class of Processor */ - KStream process(KafkaProcessor processor); + KStream process(Class> processorClass, PConfig config); } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java index c648b05940385..21791d2b7c3da 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java @@ -17,6 +17,8 @@ package org.apache.kafka.stream.examples; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.stream.KStreamTopologyBuilder; import org.apache.kafka.stream.processor.ProcessorProperties; import org.apache.kafka.stream.KStreamProcess; @@ -34,6 +36,9 @@ public static void main(String[] args) throws Exception { ProcessorProperties properties = new ProcessorProperties(new Properties()); KStreamTopologyBuilder builder = new KStreamTopologyBuilder(); + StringSerializer stringSerializer = new StringSerializer(); + IntegerSerializer intSerializer = new IntegerSerializer(); + KStream stream1 = builder.from(new StringDeserializer(), new StringDeserializer(), "topic1"); KStream stream2 = @@ -64,8 +69,8 @@ public boolean apply(String key, Integer value) { } ); - streams[0].sendTo("topic2"); - streams[1].sendTo("topic3"); + streams[0].sendTo("topic2", stringSerializer, intSerializer); + streams[1].sendTo("topic3", stringSerializer, intSerializer); KStreamProcess kstream = new KStreamProcess(builder, properties); kstream.run(); diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java index 4b1fd5f73d517..c9476264a4ac5 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java @@ -17,7 +17,10 @@ package org.apache.kafka.stream.kstream.internals; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.stream.KeyValueMapper; import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.PConfig; import org.apache.kafka.stream.processor.PTopologyBuilder; import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.Predicate; @@ -28,39 +31,38 @@ class KStreamBranch extends KafkaProcessor { - private static final String BRANCH_NAME = "KAFKA-BRANCH-"; - private static final AtomicInteger BRANCH_INDEX = new AtomicInteger(1); - private final PTopologyBuilder topology; private final Predicate[] predicates; - private final KStreamFilter[] branches; @SuppressWarnings("unchecked") - public KStreamBranch(Predicate[] predicates, PTopologyBuilder topology, String parent) { + public KStreamBranch(String name, PConfig config) { + super(name, config); + + if (this.config() == null) + throw new IllegalStateException("PConfig should be specified."); + + Predicate[] predicates = (Predicate[]) config.value(); + } + + Predicate[] predicates, PTopologyBuilder topology, String parent) { super(BRANCH_NAME); this.topology = topology; this.predicates = Arrays.copyOf(predicates, predicates.length); - this.branches = (KStreamFilter[]) Array.newInstance(KStreamFilter.class, predicates.length); - - // NOTE that branches here is just a list of predicates, hence not necessarily mutual exclusive - for (int i = 0; i < branches.length; i++) { - branches[i] = new KStreamFilter<>(predicates[i], false); - topology.addProcessor(BRANCH_NAME + BRANCH_INDEX.getAndIncrement(), branches[i], parent); - } } @Override public void process(K key, V value) { - forward(key, value); - } + if (this.children().size() != this.predicates.length) + throw new KafkaException("Number of branched streams does not match the length of predicates: this should not happen."); - @SuppressWarnings("unchecked") - public KStream[] branches() { - KStream[] streams = (KStreamSource[]) Array.newInstance(KStreamSource.class, branches.length); - for (int i = 0; i < branches.length; i++) { - streams[i] = new KStreamSource<>(topology, branches[i]); + for (int i = 0; i < predicates.length; i++) { + if (predicates[i].apply(key, value)) { + // do not use forward here bu directly call process() and then break the loop + // so that no record is going to be piped to multiple streams + this.children().get(i).process(key, value); + break; + } } - return streams; } } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java index 06bac4efc3fd6..ab7e99f466152 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java @@ -25,9 +25,6 @@ class KStreamFilter extends KafkaProcessor { - public static final String FILTER_NAME = "KAFKA-FILTER-"; - public static final AtomicInteger FILTER_INDEX = new AtomicInteger(1); - private final PredicateOut predicateOut; public static final class PredicateOut { diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java index f371170620a4e..60d3408370aed 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java @@ -20,17 +20,20 @@ import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.KeyValue; import org.apache.kafka.stream.KeyValueMapper; +import org.apache.kafka.stream.processor.PConfig; class KStreamFlatMap extends KafkaProcessor { - private static final String FLATMAP_NAME = "KAFKA-FLATMAP"; - private final KeyValueMapper> mapper; - KStreamFlatMap(KeyValueMapper> mapper) { - super(FLATMAP_NAME); + @SuppressWarnings("unchecked") + KStreamFlatMap(String name, PConfig config) { + super(name, config); + + if (this.config() == null) + throw new IllegalStateException("PConfig should be specified."); - this.mapper = mapper; + this.mapper = (KeyValueMapper>) config.value(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java index 772874bd8b5c5..6e046d0bf25ba 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java @@ -19,16 +19,20 @@ import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.ValueMapper; +import org.apache.kafka.stream.processor.PConfig; class KStreamFlatMapValues extends KafkaProcessor { - private static final String FLATMAPVALUES_NAME = "KAFKA-FLATMAPVALUES"; - private final ValueMapper> mapper; - KStreamFlatMapValues(ValueMapper> mapper) { - super(FLATMAPVALUES_NAME); - this.mapper = mapper; + @SuppressWarnings("unchecked") + KStreamFlatMapValues(String name, PConfig config) { + super(name, config); + + if (this.config() == null) + throw new IllegalStateException("PConfig should be specified."); + + this.mapper = (ValueMapper>) config.value(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java index 6a7c7fdc1b31e..b9b416ff4c3c0 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java @@ -31,9 +31,28 @@ import org.apache.kafka.stream.ValueMapper; import org.apache.kafka.stream.Window; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicInteger; + public class KStreamImpl implements KStream { - private static final String PROCESSOR_NAME = "KAFKA-PROCESS"; + 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-"; + + public static final String SOURCE_NAME = "KAFKA-SOURCE-"; + + public static final AtomicInteger INDEX = new AtomicInteger(1); protected PTopologyBuilder topology; protected String name; @@ -43,64 +62,60 @@ public KStreamImpl(PTopologyBuilder topology, String name) { this.name = name; } - @SuppressWarnings("unchecked") @Override public KStream filter(Predicate predicate) { - String name = KStreamFilter.FILTER_NAME + KStreamFilter.FILTER_INDEX.getAndIncrement(); + String name = FILTER_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFilter.class, new PConfig("Predicate", new KStreamFilter.PredicateOut<>(predicate)), name); + topology.addProcessor(name, KStreamFilter.class, new PConfig("Predicate", new KStreamFilter.PredicateOut<>(predicate)), this.name); return new KStreamImpl<>(topology, name); } - @SuppressWarnings("unchecked") @Override public KStream filterOut(final Predicate predicate) { - String name = KStreamFilter.FILTER_NAME + KStreamFilter.FILTER_INDEX.getAndIncrement(); + String name = FILTER_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFilter.class, new PConfig("Predicate", new KStreamFilter.PredicateOut<>(predicate, true)), name); + topology.addProcessor(name, KStreamFilter.class, new PConfig("Predicate", new KStreamFilter.PredicateOut<>(predicate, true)), this.name); return new KStreamImpl<>(topology, name); } - @SuppressWarnings("unchecked") @Override public KStream map(KeyValueMapper mapper) { - KStreamMap map = new KStreamMap<>(mapper); + String name = MAP_NAME + INDEX.getAndIncrement(); - topology.addProcessor(map, processor); + topology.addProcessor(name, KStreamMap.class, new PConfig("Mapper", mapper), this.name); - return new KStreamImpl<>(topology, map); + return new KStreamImpl<>(topology, name); } - @SuppressWarnings("unchecked") @Override public KStream mapValues(ValueMapper mapper) { - KStreamMapValues map = new KStreamMapValues<>(mapper); + String name = MAPVALUES_NAME + INDEX.getAndIncrement(); - topology.addProcessor(map, processor); + topology.addProcessor(name, KStreamMapValues.class, new PConfig("ValueMapper", mapper), this.name); - return new KStreamImpl<>(topology, map); + return new KStreamImpl<>(topology, name); } @SuppressWarnings("unchecked") @Override public KStream flatMap(KeyValueMapper> mapper) { - KStreamFlatMap map = new KStreamFlatMap<>(mapper); + String name = FLATMAP_NAME + INDEX.getAndIncrement(); - topology.addProcessor(map, processor); + topology.addProcessor(name, KStreamFlatMap.class, new PConfig("Mapper", mapper), this.name); - return new KStreamImpl<>(topology, map); + return new KStreamImpl<>(topology, name); } @SuppressWarnings("unchecked") @Override public KStream flatMapValues(ValueMapper> mapper) { - KStreamFlatMapValues map = new KStreamFlatMapValues<>(mapper); + String name = FLATMAPVALUES_NAME + INDEX.getAndIncrement(); - topology.addProcessor(map, processor); + topology.addProcessor(name, KStreamFlatMapValues.class, new PConfig("ValueMapper", mapper), this.name); - return new KStreamImpl<>(topology, map); + return new KStreamImpl<>(topology, name); } @Override @@ -114,16 +129,16 @@ public KStreamWindowed with(Window window) { @Override public KStream[] branch(Predicate... predicates) { + String name = BRANCH_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, KStreamFlatMapValues.class, new PConfig("ValueMapper", mapper), this.name); + + Arrays.copyOf(predicates, predicates.length) + KStreamBranch branch = new KStreamBranch<>(predicates, topology, processor); return branch.branches(); } - @SuppressWarnings("unchecked") - @Override - public KStream through(String topic) { - return through(topic, null, null, null, null); - } - @SuppressWarnings("unchecked") @Override public KStream through(String topic, @@ -131,49 +146,28 @@ public KStream through(String topic, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { - process(this.getSendProcessor(topic, keySerializer, valSerializer)); + process(KStreamSend.class, new PConfig("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer))); - KafkaSource source = topology.addSource(keyDeserializer, valDeserializer, topic); + String name = SOURCE_NAME + INDEX.getAndIncrement(); - return new KStreamSource<>(topology, source); - } + KafkaSource source = topology.addSource(name, keyDeserializer, valDeserializer, topic); - @Override - public void sendTo(String topic) { - process(this.getSendProcessor(topic, null, null)); + return new KStreamSource<>(topology, source); } @Override - public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { - process(this.getSendProcessor(topic, keySerializer, valSerializer)); - } - @SuppressWarnings("unchecked") - private KafkaProcessor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { - return new KafkaProcessor("KAFKA-SEND") { - private ProcessorContext context; - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void process(K1 key, V1 value) { - this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); - } - - @Override - public void close() { - } - }; + public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { + process(KStreamSend.class, new PConfig("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer))); } @SuppressWarnings("unchecked") @Override - public KStream process(final KafkaProcessor current) { - topology.addProcessor(current, processor); + public KStream process(final Class> clazz, PConfig config) { + String name = PROCESSOR_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, clazz, config, this.name); - return new KStreamImpl(topology, current); + return new KStreamImpl(topology, name); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java index 9797c2c2d6e13..ee0f5f6270826 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java @@ -20,17 +20,22 @@ import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.KeyValue; import org.apache.kafka.stream.KeyValueMapper; +import org.apache.kafka.stream.processor.PConfig; -class KStreamMap extends KafkaProcessor { +import java.util.concurrent.atomic.AtomicInteger; - private static final String MAP_NAME = "KAFKA-MAP"; +class KStreamMap extends KafkaProcessor { private final KeyValueMapper mapper; - public KStreamMap(KeyValueMapper mapper) { - super(MAP_NAME); + @SuppressWarnings("unchecked") + public KStreamMap(String name, PConfig config) { + super(name, config); + + if (this.config() == null) + throw new IllegalStateException("PConfig should be specified."); - this.mapper = mapper; + this.mapper = (KeyValueMapper) config.value(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java index 707652ce3c8c3..2c98ad128bb8b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java @@ -17,19 +17,23 @@ package org.apache.kafka.stream.kstream.internals; +import org.apache.kafka.stream.KeyValueMapper; import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.ValueMapper; +import org.apache.kafka.stream.processor.PConfig; class KStreamMapValues extends KafkaProcessor { - private static final String MAPVALUES_NAME = "KAFKA-MAPVALUES"; - private final ValueMapper mapper; - public KStreamMapValues(ValueMapper mapper) { - super(MAPVALUES_NAME); + @SuppressWarnings("unchecked") + public KStreamMapValues(String name, PConfig config) { + super(name, config); + + if (this.config() == null) + throw new IllegalStateException("PConfig should be specified."); - this.mapper = mapper; + this.mapper = (ValueMapper) config.value(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java b/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java index c6797b0875332..b6d298598b4eb 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java @@ -21,9 +21,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.stream.processor.internals.PTopology; -import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Deque; import java.util.HashMap; import java.util.List; import java.util.Map; From 77c1ba24b384fa46d2045cacf8c58e86c4139980 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 13 Aug 2015 16:58:25 -0700 Subject: [PATCH 160/275] Add two files --- .../kafka/stream/KStreamTopologyBuilder.java | 52 ++++++++++++++++ .../stream/kstream/internals/KStreamSend.java | 62 +++++++++++++++++++ 2 files changed, 114 insertions(+) create mode 100644 stream/src/main/java/org/apache/kafka/stream/KStreamTopologyBuilder.java create mode 100644 stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamTopologyBuilder.java b/stream/src/main/java/org/apache/kafka/stream/KStreamTopologyBuilder.java new file mode 100644 index 0000000000000..77423c016cbb6 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamTopologyBuilder.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.stream; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.stream.kstream.internals.KStreamImpl; +import org.apache.kafka.stream.processor.KafkaSource; +import org.apache.kafka.stream.processor.PTopologyBuilder; +import org.apache.kafka.stream.kstream.internals.KStreamSource; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * KStreamTopologyBuilder is the class to create KStream instances. + */ +public class KStreamTopologyBuilder extends PTopologyBuilder { + + public KStreamTopologyBuilder() { + super(); + } + + /** + * 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 + */ + @SuppressWarnings("unchecked") + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + KafkaSource source = addSource(KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.getAndIncrement(), keyDeserializer, valDeserializer, topics); + return new KStreamSource<>(this, source); + } +} diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java new file mode 100644 index 0000000000000..08729ff656ef3 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java @@ -0,0 +1,62 @@ +/** + * 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.stream.kstream.internals; + +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.stream.processor.PConfig; +import org.apache.kafka.stream.processor.ProcessorContext; + +class KStreamSend extends KafkaProcessor { + + private ProcessorContext context; + + private TopicSer topicSerializers; + + public static final class TopicSer { + public String topic; + public Serializer keySerializer; + public Serializer valSerializer; + + public TopicSer(String topic, Serializer keySerializer, Serializer valSerializer) { + this.topic = topic; + this.keySerializer = keySerializer; + this.valSerializer = valSerializer; + } + } + + @SuppressWarnings("unchecked") + public KStreamSend(String name, PConfig config) { + super(name, config); + + if (this.config() == null) + throw new IllegalStateException("PConfig should be specified."); + + this.topicSerializers = (TopicSer) config.value(); + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + @Override + public void process(K key, V value) { + this.context.send(topicSerializers.topic, key, value, topicSerializers.keySerializer, topicSerializers.valSerializer); + } +} \ No newline at end of file From 36a4eda0626aae971b2480d1aafee8c4b09c5079 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 13 Aug 2015 17:58:26 -0700 Subject: [PATCH 161/275] Continue on KStreamImpl --- .../stream/kstream/internals/KStreamBranch.java | 10 +--------- .../stream/kstream/internals/KStreamImpl.java | 14 ++++++++++---- 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java index c9476264a4ac5..8a7d4f612800f 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java @@ -31,7 +31,6 @@ class KStreamBranch extends KafkaProcessor { - private final PTopologyBuilder topology; private final Predicate[] predicates; @SuppressWarnings("unchecked") @@ -41,14 +40,7 @@ public KStreamBranch(String name, PConfig config) { if (this.config() == null) throw new IllegalStateException("PConfig should be specified."); - Predicate[] predicates = (Predicate[]) config.value(); - } - - Predicate[] predicates, PTopologyBuilder topology, String parent) { - super(BRANCH_NAME); - - this.topology = topology; - this.predicates = Arrays.copyOf(predicates, predicates.length); + this.predicates = (Predicate[]) config.value(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java index b9b416ff4c3c0..0a334b8b43575 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java @@ -31,7 +31,9 @@ import org.apache.kafka.stream.ValueMapper; import org.apache.kafka.stream.Window; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.concurrent.atomic.AtomicInteger; public class KStreamImpl implements KStream { @@ -128,15 +130,19 @@ public KStreamWindowed with(Window window) { } @Override + @SuppressWarnings("unchecked") public KStream[] branch(Predicate... predicates) { String name = BRANCH_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFlatMapValues.class, new PConfig("ValueMapper", mapper), this.name); + topology.addProcessor(name, KStreamBranch.class, new PConfig("Predicates", Arrays.copyOf(predicates, predicates.length)), this.name); - Arrays.copyOf(predicates, predicates.length) + KStreamImpl branch = new KStreamImpl<>(topology, name); + List> avatars = new ArrayList<>(); + for (int i = 0; i < predicates.length; i++) { + avatars.add(branch); + } - KStreamBranch branch = new KStreamBranch<>(predicates, topology, processor); - return branch.branches(); + return (KStream[]) avatars.toArray(); } @SuppressWarnings("unchecked") From 4c5f7249afe69f541c6dbed1f28a355821464696 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 17 Aug 2015 15:54:34 -0700 Subject: [PATCH 162/275] Remove ProcessorProperties --- .../clients/consumer/ConsumerConfig.java | 4 +- .../clients/producer/ProducerConfig.java | 4 +- .../java/org/apache/kafka/stream/KStream.java | 4 +- .../apache/kafka/stream/KStreamProcess.java | 21 +- .../kafka/stream/examples/KStreamJob.java | 8 +- .../stream/examples/SimpleProcessJob.java | 7 +- .../stream/examples/StatefulProcessJob.java | 11 +- .../kstream/internals/KStreamBranch.java | 13 +- .../kstream/internals/KStreamFilter.java | 8 +- .../kstream/internals/KStreamFlatMap.java | 6 +- .../internals/KStreamFlatMapValues.java | 6 +- .../stream/kstream/internals/KStreamImpl.java | 23 +- .../stream/kstream/internals/KStreamMap.java | 8 +- .../kstream/internals/KStreamMapValues.java | 7 +- .../stream/kstream/internals/KStreamSend.java | 6 +- .../stream/processor/KafkaProcessor.java | 6 +- .../stream/processor/PTopologyBuilder.java | 10 +- .../stream/processor/ProcessorConfig.java | 208 ++++++++++++------ .../{PConfig.java => ProcessorMetadata.java} | 4 +- .../stream/processor/ProcessorProperties.java | 145 ------------ .../processor/internals/KStreamThread.java | 62 ++++-- .../internals/ProcessorContextImpl.java | 40 ++-- 22 files changed, 272 insertions(+), 339 deletions(-) rename stream/src/main/java/org/apache/kafka/stream/processor/{PConfig.java => ProcessorMetadata.java} (91%) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/processor/ProcessorProperties.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index b9a2d4e2bc565..347a5bc00c78d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -150,11 +150,11 @@ public class ConsumerConfig extends AbstractConfig { /** key.deserializer */ public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; - private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; + public static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; /** value.deserializer */ public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; - private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; + public static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; /** connections.max.idle.ms */ public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 06f00a99a73a2..6969f61a130bc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -164,11 +164,11 @@ public class ProducerConfig extends AbstractConfig { /** key.serializer */ public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; - private static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; + public static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; /** value.serializer */ public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; - private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; + public static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; /** connections.max.idle.ms */ public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/stream/KStream.java index 7ee22f74dc7e2..8130c5e0ea7f7 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStream.java @@ -20,7 +20,7 @@ import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.stream.processor.PConfig; +import org.apache.kafka.stream.processor.ProcessorMetadata; /** * KStream is an abstraction of a stream of key-value pairs. @@ -136,5 +136,5 @@ public interface KStream { * * @param processorClass the class of Processor */ - KStream process(Class> processorClass, PConfig config); + KStream process(Class> processorClass, ProcessorMetadata config); } diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java b/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java index 506ba009e26ee..3af5033782e42 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java +++ b/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java @@ -20,10 +20,11 @@ import org.apache.kafka.stream.processor.PTopologyBuilder; import org.apache.kafka.stream.processor.internals.KStreamThread; import org.apache.kafka.stream.processor.ProcessorConfig; -import org.apache.kafka.stream.processor.ProcessorProperties; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; + /** * 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. @@ -63,21 +64,21 @@ public class KStreamProcess implements Runnable { private static final int STOPPED = 3; private int state = CREATED; - private final ProcessorConfig config; private final Object lock = new Object(); private final KStreamThread[] threads; + private final File stateDir; - public KStreamProcess(PTopologyBuilder builder, ProcessorProperties processorProperties) throws Exception { - if (processorProperties.timestampExtractor() == null) + public KStreamProcess(PTopologyBuilder builder, ProcessorConfig config) throws Exception { + if (config.getClass(ProcessorConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG) == null) throw new NullPointerException("timestamp extractor is missing"); - this.config = new ProcessorConfig(processorProperties.config()); - - this.threads = new KStreamThread[this.config.numStreamThreads]; + this.threads = new KStreamThread[config.getInt(ProcessorConfig.NUM_STREAM_THREADS_CONFIG)]; for (int i = 0; i < this.threads.length; i++) { - this.threads[i] = new KStreamThread(builder, processorProperties); + this.threads[i] = new KStreamThread(builder, config); } + + this.stateDir = new File(config.getString(ProcessorConfig.STATE_DIR_CONFIG)); } /** @@ -87,8 +88,8 @@ public void run() { synchronized (lock) { log.info("Starting container"); if (state == CREATED) { - if (!config.stateDir.exists() && !config.stateDir.mkdirs()) - throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); + if (!stateDir.exists() && !stateDir.mkdirs()) + throw new IllegalArgumentException("Failed to create state directory: " + stateDir.getAbsolutePath()); for (KStreamThread thread : threads) thread.start(); log.info("Start-up complete"); diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java index 21791d2b7c3da..8fe2e81277c0b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java @@ -19,10 +19,10 @@ 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.stream.KStreamTopologyBuilder; -import org.apache.kafka.stream.processor.ProcessorProperties; +import org.apache.kafka.stream.processor.ProcessorConfig; import org.apache.kafka.stream.KStreamProcess; -import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.KStream; import org.apache.kafka.stream.KeyValue; import org.apache.kafka.stream.KeyValueMapper; @@ -33,7 +33,7 @@ public class KStreamJob { public static void main(String[] args) throws Exception { - ProcessorProperties properties = new ProcessorProperties(new Properties()); + ProcessorConfig config = new ProcessorConfig(new Properties()); KStreamTopologyBuilder builder = new KStreamTopologyBuilder(); StringSerializer stringSerializer = new StringSerializer(); @@ -72,7 +72,7 @@ public boolean apply(String key, Integer value) { streams[0].sendTo("topic2", stringSerializer, intSerializer); streams[1].sendTo("topic3", stringSerializer, intSerializer); - KStreamProcess kstream = new KStreamProcess(builder, properties); + KStreamProcess kstream = new KStreamProcess(builder, config); kstream.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java index e902a3fd3260c..c18d668de0df3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java @@ -19,9 +19,8 @@ import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.processor.PTopologyBuilder; +import org.apache.kafka.stream.processor.ProcessorConfig; import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.processor.ProcessorProperties; -import org.apache.kafka.stream.processor.KafkaSource; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.KStreamProcess; @@ -58,13 +57,13 @@ public void close() { } public static void main(String[] args) throws Exception { - ProcessorProperties properties = new ProcessorProperties(new Properties()); + ProcessorConfig config = new ProcessorConfig(new Properties()); PTopologyBuilder builder = new PTopologyBuilder(); builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); builder.addProcessor("PROCESS", MyProcessor.class, null, "SOURCE"); - KStreamProcess streaming = new KStreamProcess(builder, properties); + KStreamProcess streaming = new KStreamProcess(builder, config); streaming.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java index ce44e9ca0dac2..3be00280d3f35 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java @@ -17,14 +17,13 @@ package org.apache.kafka.stream.examples; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.stream.KStreamProcess; import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.processor.PTopologyBuilder; +import org.apache.kafka.stream.processor.ProcessorConfig; import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.processor.KafkaSource; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.processor.ProcessorProperties; import org.apache.kafka.stream.state.Entry; import org.apache.kafka.stream.state.InMemoryKeyValueStore; import org.apache.kafka.stream.state.KeyValueIterator; @@ -79,13 +78,13 @@ public void close() { } public static void main(String[] args) throws Exception { - ProcessorProperties properties = new ProcessorProperties(new Properties()); + ProcessorConfig config = new ProcessorConfig(new Properties()); PTopologyBuilder builder = new PTopologyBuilder(); builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); builder.addProcessor("PROCESS", MyProcessor.class, null, "SOURCE"); - KStreamProcess streaming = new KStreamProcess(builder, properties); + KStreamProcess streaming = new KStreamProcess(builder, config); streaming.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java index 8a7d4f612800f..491a49a9f783c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java @@ -18,27 +18,20 @@ package org.apache.kafka.stream.kstream.internals; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.stream.KeyValueMapper; import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.PConfig; -import org.apache.kafka.stream.processor.PTopologyBuilder; -import org.apache.kafka.stream.KStream; +import org.apache.kafka.stream.processor.ProcessorMetadata; import org.apache.kafka.stream.Predicate; -import java.lang.reflect.Array; -import java.util.Arrays; -import java.util.concurrent.atomic.AtomicInteger; - class KStreamBranch extends KafkaProcessor { private final Predicate[] predicates; @SuppressWarnings("unchecked") - public KStreamBranch(String name, PConfig config) { + public KStreamBranch(String name, ProcessorMetadata config) { super(name, config); if (this.config() == null) - throw new IllegalStateException("PConfig should be specified."); + throw new IllegalStateException("ProcessorMetadata should be specified."); this.predicates = (Predicate[]) config.value(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java index ab7e99f466152..3b397ad13ff54 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java @@ -19,9 +19,7 @@ import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.Predicate; -import org.apache.kafka.stream.processor.PConfig; - -import java.util.concurrent.atomic.AtomicInteger; +import org.apache.kafka.stream.processor.ProcessorMetadata; class KStreamFilter extends KafkaProcessor { @@ -43,11 +41,11 @@ public PredicateOut(Predicate predicate, boolean filterOut) { } @SuppressWarnings("unchecked") - public KStreamFilter(String name, PConfig config) { + public KStreamFilter(String name, ProcessorMetadata config) { super(name, config); if (this.config() == null) - throw new IllegalStateException("PConfig should be specified."); + throw new IllegalStateException("ProcessorMetadata should be specified."); this.predicateOut = (PredicateOut) config.value(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java index 60d3408370aed..be5b6b70a9e12 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java @@ -20,18 +20,18 @@ import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.KeyValue; import org.apache.kafka.stream.KeyValueMapper; -import org.apache.kafka.stream.processor.PConfig; +import org.apache.kafka.stream.processor.ProcessorMetadata; class KStreamFlatMap extends KafkaProcessor { private final KeyValueMapper> mapper; @SuppressWarnings("unchecked") - KStreamFlatMap(String name, PConfig config) { + KStreamFlatMap(String name, ProcessorMetadata config) { super(name, config); if (this.config() == null) - throw new IllegalStateException("PConfig should be specified."); + throw new IllegalStateException("ProcessorMetadata should be specified."); this.mapper = (KeyValueMapper>) config.value(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java index 6e046d0bf25ba..800bb59338b8e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java @@ -19,18 +19,18 @@ import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.ValueMapper; -import org.apache.kafka.stream.processor.PConfig; +import org.apache.kafka.stream.processor.ProcessorMetadata; class KStreamFlatMapValues extends KafkaProcessor { private final ValueMapper> mapper; @SuppressWarnings("unchecked") - KStreamFlatMapValues(String name, PConfig config) { + KStreamFlatMapValues(String name, ProcessorMetadata config) { super(name, config); if (this.config() == null) - throw new IllegalStateException("PConfig should be specified."); + throw new IllegalStateException("ProcessorMetadata should be specified."); this.mapper = (ValueMapper>) config.value(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java index 0a334b8b43575..22ec8fa2536b3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java @@ -18,9 +18,8 @@ package org.apache.kafka.stream.kstream.internals; import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.PConfig; +import org.apache.kafka.stream.processor.ProcessorMetadata; import org.apache.kafka.stream.processor.PTopologyBuilder; -import org.apache.kafka.stream.processor.ProcessorContext; import org.apache.kafka.stream.processor.KafkaSource; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; @@ -68,7 +67,7 @@ public KStreamImpl(PTopologyBuilder topology, String name) { public KStream filter(Predicate predicate) { String name = FILTER_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFilter.class, new PConfig("Predicate", new KStreamFilter.PredicateOut<>(predicate)), this.name); + topology.addProcessor(name, KStreamFilter.class, new ProcessorMetadata("Predicate", new KStreamFilter.PredicateOut<>(predicate)), this.name); return new KStreamImpl<>(topology, name); } @@ -77,7 +76,7 @@ public KStream filter(Predicate predicate) { public KStream filterOut(final Predicate predicate) { String name = FILTER_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFilter.class, new PConfig("Predicate", new KStreamFilter.PredicateOut<>(predicate, true)), this.name); + topology.addProcessor(name, KStreamFilter.class, new ProcessorMetadata("Predicate", new KStreamFilter.PredicateOut<>(predicate, true)), this.name); return new KStreamImpl<>(topology, name); } @@ -86,7 +85,7 @@ public KStream filterOut(final Predicate predicate) { public KStream map(KeyValueMapper mapper) { String name = MAP_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamMap.class, new PConfig("Mapper", mapper), this.name); + topology.addProcessor(name, KStreamMap.class, new ProcessorMetadata("Mapper", mapper), this.name); return new KStreamImpl<>(topology, name); } @@ -95,7 +94,7 @@ public KStream map(KeyValueMapper mapper) { public KStream mapValues(ValueMapper mapper) { String name = MAPVALUES_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamMapValues.class, new PConfig("ValueMapper", mapper), this.name); + topology.addProcessor(name, KStreamMapValues.class, new ProcessorMetadata("ValueMapper", mapper), this.name); return new KStreamImpl<>(topology, name); } @@ -105,7 +104,7 @@ public KStream mapValues(ValueMapper mapper) { public KStream flatMap(KeyValueMapper> mapper) { String name = FLATMAP_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFlatMap.class, new PConfig("Mapper", mapper), this.name); + topology.addProcessor(name, KStreamFlatMap.class, new ProcessorMetadata("Mapper", mapper), this.name); return new KStreamImpl<>(topology, name); } @@ -115,7 +114,7 @@ public KStream flatMap(KeyValueMapper KStream flatMapValues(ValueMapper> mapper) { String name = FLATMAPVALUES_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFlatMapValues.class, new PConfig("ValueMapper", mapper), this.name); + topology.addProcessor(name, KStreamFlatMapValues.class, new ProcessorMetadata("ValueMapper", mapper), this.name); return new KStreamImpl<>(topology, name); } @@ -134,7 +133,7 @@ public KStreamWindowed with(Window window) { public KStream[] branch(Predicate... predicates) { String name = BRANCH_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamBranch.class, new PConfig("Predicates", Arrays.copyOf(predicates, predicates.length)), this.name); + topology.addProcessor(name, KStreamBranch.class, new ProcessorMetadata("Predicates", Arrays.copyOf(predicates, predicates.length)), this.name); KStreamImpl branch = new KStreamImpl<>(topology, name); List> avatars = new ArrayList<>(); @@ -152,7 +151,7 @@ public KStream through(String topic, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { - process(KStreamSend.class, new PConfig("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer))); + process(KStreamSend.class, new ProcessorMetadata("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer))); String name = SOURCE_NAME + INDEX.getAndIncrement(); @@ -164,12 +163,12 @@ public KStream through(String topic, @Override @SuppressWarnings("unchecked") public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { - process(KStreamSend.class, new PConfig("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer))); + process(KStreamSend.class, new ProcessorMetadata("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer))); } @SuppressWarnings("unchecked") @Override - public KStream process(final Class> clazz, PConfig config) { + public KStream process(final Class> clazz, ProcessorMetadata config) { String name = PROCESSOR_NAME + INDEX.getAndIncrement(); topology.addProcessor(name, clazz, config, this.name); diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java index ee0f5f6270826..32bdb02f425a6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java @@ -20,20 +20,18 @@ import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.KeyValue; import org.apache.kafka.stream.KeyValueMapper; -import org.apache.kafka.stream.processor.PConfig; - -import java.util.concurrent.atomic.AtomicInteger; +import org.apache.kafka.stream.processor.ProcessorMetadata; class KStreamMap extends KafkaProcessor { private final KeyValueMapper mapper; @SuppressWarnings("unchecked") - public KStreamMap(String name, PConfig config) { + public KStreamMap(String name, ProcessorMetadata config) { super(name, config); if (this.config() == null) - throw new IllegalStateException("PConfig should be specified."); + throw new IllegalStateException("ProcessorMetadata should be specified."); this.mapper = (KeyValueMapper) config.value(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java index 2c98ad128bb8b..32649f8f8eba8 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java @@ -17,21 +17,20 @@ package org.apache.kafka.stream.kstream.internals; -import org.apache.kafka.stream.KeyValueMapper; import org.apache.kafka.stream.processor.KafkaProcessor; import org.apache.kafka.stream.ValueMapper; -import org.apache.kafka.stream.processor.PConfig; +import org.apache.kafka.stream.processor.ProcessorMetadata; class KStreamMapValues extends KafkaProcessor { private final ValueMapper mapper; @SuppressWarnings("unchecked") - public KStreamMapValues(String name, PConfig config) { + public KStreamMapValues(String name, ProcessorMetadata config) { super(name, config); if (this.config() == null) - throw new IllegalStateException("PConfig should be specified."); + throw new IllegalStateException("ProcessorMetadata should be specified."); this.mapper = (ValueMapper) config.value(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java index 08729ff656ef3..2229cd862646b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java +++ b/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.PConfig; +import org.apache.kafka.stream.processor.ProcessorMetadata; import org.apache.kafka.stream.processor.ProcessorContext; class KStreamSend extends KafkaProcessor { @@ -41,11 +41,11 @@ public TopicSer(String topic, Serializer keySerializer, Serializer implements Processor> parents; private final String name; - private final PConfig config; + private final ProcessorMetadata config; public boolean initialized; @@ -34,7 +34,7 @@ public KafkaProcessor(String name) { this(name, null); } - public KafkaProcessor(String name, PConfig config) { + public KafkaProcessor(String name, ProcessorMetadata config) { this.name = name; this.config = config; @@ -48,7 +48,7 @@ public String name() { return name; } - public PConfig config() { + public ProcessorMetadata config() { return config; } diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java b/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java index b6d298598b4eb..8e3ea80f4915d 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java @@ -36,9 +36,9 @@ public class PTopologyBuilder { private class ProcessorClazz { public Class clazz; - public PConfig config; + public ProcessorMetadata config; - public ProcessorClazz(Class clazz, PConfig config) { + public ProcessorClazz(Class clazz, ProcessorMetadata config) { this.clazz = clazz; this.config = config; } @@ -60,7 +60,7 @@ public final KafkaSource addSource(String name, Deserializer keyDeserializer, De return source; } - public final void addProcessor(String name, Class processorClass, PConfig config, String... parentNames) { + public final void addProcessor(String name, Class processorClass, ProcessorMetadata config, String... parentNames) { if (processorClasses.containsKey(name)) throw new IllegalArgumentException("Processor " + name + " is already added."); @@ -94,9 +94,9 @@ public PTopology build() { // create processors try { for (String name : processorClasses.keySet()) { - PConfig config = processorClasses.get(name).config; + ProcessorMetadata config = processorClasses.get(name).config; Class processorClass = processorClasses.get(name).clazz; - KafkaProcessor processor = processorClass.getConstructor(String.class, PConfig.class).newInstance(name, config); + KafkaProcessor processor = processorClass.getConstructor(String.class, ProcessorMetadata.class).newInstance(name, config); processorMap.put(name, processor); } } catch (Exception e) { diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorConfig.java index 407476710df7d..f7f87e1e852f1 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorConfig.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorConfig.java @@ -17,87 +17,161 @@ package org.apache.kafka.stream.processor; +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.io.File; +import java.util.Map; import java.util.Properties; public class ProcessorConfig extends AbstractConfig { private static final ConfigDef CONFIG; + /** state.dir */ + public static final String STATE_DIR_CONFIG = "state.dir"; + private static final String STATE_DIR_DOC = "Directory location for state store."; + + /** commit.interval.ms */ + public static final String COMMIT_INTERVAL_MS_CONFIG = "commit.interval.ms"; + private static final String COMMIT_INTERVAL_MS_DOC = "The frequency with which to save the position of the processor."; + + /** poll.ms */ + public static final String POLL_MS_CONFIG = "poll.ms"; + private static final String POLL_MS_DOC = "The amount of time in milliseconds to block waiting for input."; + + /** num.stream.threads */ + public static final String NUM_STREAM_THREADS_CONFIG = "num.stream.threads"; + private static final String NUM_STREAM_THREADS_DOC = "The number of threads to execute stream processing."; + + /** buffered.records.per.partition */ + public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; + private static final String BUFFERED_RECORDS_PER_PARTITION_DOC = "The maximum number of records to buffer per partition."; + + /** state.cleanup.delay */ + public static final String STATE_CLEANUP_DELAY_MS_CONFIG = "state.cleanup.delay.ms"; + private static final String STATE_CLEANUP_DELAY_MS_DOC = "The amount of time in milliseconds to wait before deleting state when a partition has migrated."; + + /** total.records.to.process */ + public static final String TOTAL_RECORDS_TO_PROCESS = "total.records.to.process"; + private static final String TOTAL_RECORDS_TO_DOC = "Exit after processing this many records."; + + /** window.time.ms */ + public static final String WINDOW_TIME_MS_CONFIG = "window.time.ms"; + private static final String WINDOW_TIME_MS_DOC = "Setting this to a non-negative value will cause the processor to get called " + + "with this frequency even if there is no message."; + + /** timestamp.extractor */ + public static final String TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "timestamp.extractor"; + private static final String TIMESTAMP_EXTRACTOR_CLASS_DOC = "Timestamp extractor class that implements the TimestampExtractor interface."; + + /** key.serializer */ + public static final String KEY_SERIALIZER_CLASS_CONFIG = ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; + + /** value.serializer */ + public static final String VALUE_SERIALIZER_CLASS_CONFIG = ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; + + /** key.deserializer */ + public static final String KEY_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; + + /** value.deserializer */ + public static final String VALUE_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; + + + + + private static final String SYSTEM_TEMP_DIRECTORY = System.getProperty("java.io.tmpdir"); + static { - CONFIG = new ConfigDef().define(ProcessorProperties.TOPICS_CONFIG, - Type.STRING, - "", - Importance.HIGH, - "All the possible topic names this job need to interact with") - .define(ProcessorProperties.STATE_DIR_CONFIG, - Type.STRING, - System.getProperty("java.io.tmpdir"), - Importance.MEDIUM, - "") - .define(ProcessorProperties.POLL_TIME_MS_CONFIG, - Type.LONG, - 100, - Importance.LOW, - "The amount of time to block waiting for input.") - .define(ProcessorProperties.COMMIT_TIME_MS_CONFIG, - Type.LONG, - 30000, - Importance.HIGH, - "The frequency with which to save the position of the processor.") - .define(ProcessorProperties.WINDOW_TIME_MS_CONFIG, - Type.LONG, - -1L, - Importance.MEDIUM, - "Setting this to a non-negative value will cause the processor to get called with this frequency even if there is no message.") - .define(ProcessorProperties.BUFFERED_RECORDS_PER_PARTITION_CONFIG, - Type.INT, - 1000, - Importance.LOW, - "The maximum number of records to buffer per partition") - .define(ProcessorProperties.STATE_CLEANUP_DELAY_CONFIG, - Type.LONG, - 60000, - Importance.LOW, - "The amount of time to wait before deleting state when a partition has migrated.") - .define(ProcessorProperties.TOTAL_RECORDS_TO_PROCESS, - Type.LONG, - -1L, - Importance.LOW, - "Exit after processing this many records.") - .define(ProcessorProperties.NUM_STREAM_THREADS, - Type.INT, - 1, - Importance.LOW, - "The number of threads to execute stream processing."); + CONFIG = new ConfigDef().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); } - public final String topics; - public final File stateDir; - public final long pollTimeMs; - public final long commitTimeMs; - public final long windowTimeMs; - public final int bufferedRecordsPerPartition; - public final long stateCleanupDelay; - public final long totalRecordsToProcess; - public final int numStreamThreads; - - public ProcessorConfig(Properties processor) { - super(CONFIG, processor); - this.topics = this.getString(ProcessorProperties.TOPICS_CONFIG); - this.stateDir = new File(this.getString(ProcessorProperties.STATE_DIR_CONFIG)); - this.pollTimeMs = this.getLong(ProcessorProperties.POLL_TIME_MS_CONFIG); - this.commitTimeMs = this.getLong(ProcessorProperties.COMMIT_TIME_MS_CONFIG); - this.windowTimeMs = this.getLong(ProcessorProperties.WINDOW_TIME_MS_CONFIG); - this.bufferedRecordsPerPartition = this.getInt(ProcessorProperties.BUFFERED_RECORDS_PER_PARTITION_CONFIG); - this.stateCleanupDelay = this.getLong(ProcessorProperties.STATE_CLEANUP_DELAY_CONFIG); - this.totalRecordsToProcess = this.getLong(ProcessorProperties.TOTAL_RECORDS_TO_PROCESS); - this.numStreamThreads = this.getInt(ProcessorProperties.NUM_STREAM_THREADS); + public ProcessorConfig(Map props) { + super(CONFIG, props); } + public Properties getConsumerProperties() { + Properties props = new Properties(); + + // set consumer default property values + props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); + + return props; + } + + public Properties getProducerProperties() { + Properties props = new Properties(); + + // set producer default property values + props.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); + + return props; + } + + public static void main(String[] args) { + System.out.println(CONFIG.toHtmlTable()); + } } diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/PConfig.java b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorMetadata.java similarity index 91% rename from stream/src/main/java/org/apache/kafka/stream/processor/PConfig.java rename to stream/src/main/java/org/apache/kafka/stream/processor/ProcessorMetadata.java index 2e867a13cab50..cef30bbfb4c94 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/PConfig.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorMetadata.java @@ -17,12 +17,12 @@ package org.apache.kafka.stream.processor; -public class PConfig { +public class ProcessorMetadata { private String name; private Object value; - public PConfig(String name, Object value) { + public ProcessorMetadata(String name, Object value) { this.name = name; this.value = value; } diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorProperties.java b/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorProperties.java deleted file mode 100644 index c2f274a97cc44..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorProperties.java +++ /dev/null @@ -1,145 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.processor; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; - -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - - -/** - * Configuration information passed to the process instance for configuring the associated - * {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer}, and - * {@link org.apache.kafka.clients.consumer.KafkaConsumer KafkaConsumer}, as - * well as the processor itself. - */ -public class ProcessorProperties { - - /** topics */ - public static final String TOPICS_CONFIG = "topics"; - - /** state.dir */ - public static final String STATE_DIR_CONFIG = "state.dir"; - - /** poll.time.ms */ - public static final String POLL_TIME_MS_CONFIG = "poll.time.ms"; - - /** commit.time.ms */ - public static final String COMMIT_TIME_MS_CONFIG = "commit.time.ms"; - - /** window.time.ms */ - public static final String WINDOW_TIME_MS_CONFIG = "window.time.ms"; - - /** buffered.records.per.partition */ - public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; - - /** state.cleanup.delay */ - public static final String STATE_CLEANUP_DELAY_CONFIG = "state.cleanup.delay"; - - /** total.records.to.process */ - public static final String TOTAL_RECORDS_TO_PROCESS = "total.records.to.process"; - - /** num.of.stream.threads */ - public static final String NUM_STREAM_THREADS = "num.of.stream.threads"; - - private final Properties config; - private final Map context = new HashMap(); - private Serializer keySerializer; - private Serializer valSerializer; - private Deserializer keyDeserializer; - private Deserializer valDeserializer; - private TimestampExtractor timestampExtractor; - - public ProcessorProperties(Properties config) { - this.config = config; - this.config.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - this.config.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); - this.config.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); - } - - @Override - public ProcessorProperties clone() { - return new ProcessorProperties(this.config); - } - - public void addContextObject(String key, Object value) { - this.context.put(key, value); - } - - @SuppressWarnings("all") - - public void serialization(Serializer serializer, Deserializer deserializer) { - keySerializer(serializer); - valueSerializer(serializer); - keyDeserializer(deserializer); - valueDeserializer(deserializer); - } - - public void keySerializer(Serializer serializer) { - this.keySerializer = serializer; - } - - public void valueSerializer(Serializer serializer) { - this.valSerializer = serializer; - } - - public void keyDeserializer(Deserializer deserializer) { - this.keyDeserializer = deserializer; - } - - public void valueDeserializer(Deserializer deserializer) { - this.valDeserializer = deserializer; - } - - public Properties config() { - return this.config; - } - - public Map context() { - return this.context; - } - - public Serializer keySerializer() { - return this.keySerializer; - } - - public Serializer valueSerializer() { - return this.valSerializer; - } - - public Deserializer keyDeserializer() { - return this.keyDeserializer; - } - - public Deserializer valueDeserializer() { - return this.valDeserializer; - } - - public void timestampExtractor(TimestampExtractor timestampExtractor) { - this.timestampExtractor = timestampExtractor; - } - - public TimestampExtractor timestampExtractor() { - return this.timestampExtractor; - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/KStreamThread.java index 0a760cc85da20..a0018a1f7efee 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/KStreamThread.java @@ -22,7 +22,6 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.stream.processor.PTopologyBuilder; import org.apache.kafka.stream.processor.ProcessorConfig; -import org.apache.kafka.stream.processor.ProcessorProperties; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.KafkaException; @@ -55,15 +54,21 @@ public class KStreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); private final PTopologyBuilder builder; - private final ArrayList streamGroups = new ArrayList<>(); - private final Map kstreamContexts = new HashMap<>(); private final IngestorImpl ingestor; private final RecordCollectorImpl collector; - private final ProcessorProperties properties; - private final ProcessorConfig config; + private final ArrayList streamGroups = new ArrayList<>(); + private final Map kstreamContexts = new HashMap<>(); private final Metrics metrics; - private final KafkaStreamingMetrics streamingMetrics; private final Time time; + + private final ProcessorConfig config; + private final File stateDir; + private final long pollTimeMs; + private final long commitTimeMs; + private final long stateCleanupDelayMs; + private final long totalRecordsToProcess; + private final KafkaStreamingMetrics streamingMetrics; + private volatile boolean running; private long lastCommit; private long nextStateCleaning; @@ -85,29 +90,38 @@ public void onPartitionsRevoked(Consumer consumer, Collection producer = new KafkaProducer<>(properties.config(), new ByteArraySerializer(), new ByteArraySerializer()); - this.collector = new RecordCollectorImpl(producer, (Serializer) properties.keySerializer(), (Serializer) properties.valueSerializer()); - - Consumer consumer = new KafkaConsumer<>(properties.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + Producer producer = new KafkaProducer<>(config.getProducerProperties(), + new ByteArraySerializer(), + new ByteArraySerializer()); + this.collector = new RecordCollectorImpl(producer, + (Serializer) config.getConfiguredInstance(ProcessorConfig.KEY_DESERIALIZER_CLASS_CONFIG, Serializer.class), + (Serializer) config.getConfiguredInstance(ProcessorConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Serializer.class)); + + Consumer consumer = new KafkaConsumer<>(config.getConsumerProperties(), + rebalanceCallback, + new ByteArrayDeserializer(), + new ByteArrayDeserializer()); this.ingestor = new IngestorImpl(consumer, topology.topics()); + this.stateDir = new File(this.config.getString(ProcessorConfig.STATE_DIR_CONFIG)); + this.pollTimeMs = config.getLong(ProcessorConfig.POLL_MS_CONFIG); + this.commitTimeMs = config.getLong(ProcessorConfig.COMMIT_INTERVAL_MS_CONFIG); + this.stateCleanupDelayMs = config.getLong(ProcessorConfig.STATE_CLEANUP_DELAY_MS_CONFIG); + this.totalRecordsToProcess = config.getLong(ProcessorConfig.TOTAL_RECORDS_TO_PROCESS); + this.running = true; this.lastCommit = 0; this.nextStateCleaning = Long.MAX_VALUE; @@ -154,7 +168,7 @@ private void runLoop() { boolean readyForNextExecution = false; while (stillRunning()) { - ingestor.poll(readyForNextExecution ? 0 : this.config.pollTimeMs); + ingestor.poll(readyForNextExecution ? 0 : this.pollTimeMs); for (StreamGroup group : this.streamGroups) { readyForNextExecution = group.process(); @@ -173,8 +187,8 @@ private boolean stillRunning() { log.debug("Shutting down at user request."); return false; } - if (config.totalRecordsToProcess >= 0 && recordsProcessed >= config.totalRecordsToProcess) { - log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", config.totalRecordsToProcess); + 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; @@ -182,7 +196,7 @@ private boolean stillRunning() { private void maybeCommit() { long now = time.milliseconds(); - if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < now) { + if (commitTimeMs >= 0 && lastCommit + commitTimeMs < now) { log.trace("Committing processor instances because the commit interval has elapsed."); commitAll(now); } @@ -209,13 +223,13 @@ private void commitAll(long now) { private void maybeCleanState() { long now = time.milliseconds(); if (now > nextStateCleaning) { - File[] stateDirs = config.stateDir.listFiles(); + File[] stateDirs = stateDir.listFiles(); if (stateDirs != null) { for (File dir : stateDirs) { try { Integer id = Integer.parseInt(dir.getName()); if (!kstreamContexts.keySet().contains(id)) { - log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); + log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), stateCleanupDelayMs); Utils.rm(dir); } } catch (NumberFormatException e) { @@ -239,7 +253,7 @@ private void addPartitions(Collection assignment) { // build the topology and initialize with the context PTopology topology = builder.build(); - context = new ProcessorContextImpl(id, ingestor, topology, collector, properties, config, metrics); + context = new ProcessorContextImpl(id, ingestor, topology, collector, config, metrics); topology.init(context); @@ -256,7 +270,7 @@ private void addPartitions(Collection assignment) { context.addPartition(partition); } - nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; + nextStateCleaning = time.milliseconds() + stateCleanupDelayMs; } private void removePartitions() { diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorContextImpl.java index 9f795ab7353fd..2c71b33664c8b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorContextImpl.java @@ -20,10 +20,8 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.stream.processor.KafkaSource; -import org.apache.kafka.stream.processor.PTopologyBuilder; import org.apache.kafka.stream.processor.ProcessorConfig; import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.processor.ProcessorProperties; import org.apache.kafka.stream.processor.RecordCollector; import org.apache.kafka.stream.processor.StateStore; import org.apache.kafka.stream.processor.KafkaProcessor; @@ -54,16 +52,18 @@ public class ProcessorContextImpl implements ProcessorContext { private static final Logger log = LoggerFactory.getLogger(ProcessorContextImpl.class); public final int id; - public final StreamGroup streamGroup; public final Ingestor ingestor; + public final StreamGroup streamGroup; private final Metrics metrics; private final PTopology topology; private final RecordCollectorImpl collector; private final ProcessorStateManager stateMgr; - private final ProcessorProperties processorProperties; - private final ProcessorConfig processorConfig; - private final TimestampExtractor timestampExtractor; + + private final Serializer keySerializer; + private final Serializer valSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valDeserializer; private boolean initialized; @@ -72,28 +72,32 @@ public ProcessorContextImpl(int id, Ingestor ingestor, PTopology topology, RecordCollectorImpl collector, - ProcessorProperties processorProperties, - ProcessorConfig processorConfig, + ProcessorConfig config, Metrics metrics) throws IOException { this.id = id; this.metrics = metrics; this.ingestor = ingestor; this.topology = topology; this.collector = collector; - this.processorProperties = processorProperties; - this.processorConfig = processorConfig; - this.timestampExtractor = this.processorProperties.timestampExtractor(); for (String topic : this.topology.topics()) { if (!ingestor.topics().contains(topic)) throw new IllegalArgumentException("topic not subscribed: " + topic); } - File stateFile = new File(processorConfig.stateDir, Integer.toString(id)); - Consumer restoreConsumer = new KafkaConsumer<>(processorProperties.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + this.keySerializer = config.getConfiguredInstance(ProcessorConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class); + this.valSerializer = config.getConfiguredInstance(ProcessorConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class); + this.keyDeserializer = config.getConfiguredInstance(ProcessorConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + this.valDeserializer = config.getConfiguredInstance(ProcessorConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + + TimestampExtractor extractor = config.getConfiguredInstance(ProcessorConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); + int bufferedRecordsPerPartition = config.getInt(ProcessorConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + + File stateFile = new File(config.getString(ProcessorConfig.STATE_DIR_CONFIG), Integer.toString(id)); + Consumer restoreConsumer = new KafkaConsumer<>(config.getConsumerProperties(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); - this.streamGroup = new StreamGroup(this, this.ingestor, new TimeBasedChooser(), this.timestampExtractor, this.processorConfig.bufferedRecordsPerPartition); + this.streamGroup = new StreamGroup(this, this.ingestor, new TimeBasedChooser(), extractor, bufferedRecordsPerPartition); stateMgr.init(); @@ -150,22 +154,22 @@ public int id() { @Override public Serializer keySerializer() { - return processorProperties.keySerializer(); + return this.keySerializer; } @Override public Serializer valueSerializer() { - return processorProperties.valueSerializer(); + return this.valSerializer; } @Override public Deserializer keyDeserializer() { - return processorProperties.keyDeserializer(); + return this.keyDeserializer; } @Override public Deserializer valueDeserializer() { - return processorProperties.valueDeserializer(); + return this.valDeserializer; } @Override From e13b2d091b6e03eb8132532310a0cb89abae0900 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 17 Aug 2015 16:33:11 -0700 Subject: [PATCH 163/275] Some package and class renaming, fix KafkaSource constrcution at builder --- checkstyle/import-control.xml | 2 +- .../{stream => streaming}/KStreamProcess.java | 12 ++-- .../{stream => streaming}/SlidingWindow.java | 9 +-- .../examples/KStreamJob.java | 18 ++--- .../examples/SimpleProcessJob.java | 14 ++-- .../examples/StatefulProcessJob.java | 22 +++--- .../kstream}/KStream.java | 6 +- .../kstream/KStreamBuilder.java} | 19 +++-- .../kstream}/KStreamWindowed.java | 2 +- .../kstream}/KeyValue.java | 2 +- .../kstream}/KeyValueMapper.java | 2 +- .../kstream}/Predicate.java | 2 +- .../kstream}/ValueJoiner.java | 2 +- .../kstream}/ValueMapper.java | 2 +- .../{stream => streaming/kstream}/Window.java | 6 +- .../kstream/internals/FilteredIterator.java | 2 +- .../kstream/internals/KStreamBranch.java | 8 +-- .../kstream/internals/KStreamFilter.java | 8 +-- .../kstream/internals/KStreamFlatMap.java | 10 +-- .../internals/KStreamFlatMapValues.java | 8 +-- .../kstream/internals/KStreamImpl.java | 26 +++---- .../kstream/internals/KStreamJoin.java | 10 +-- .../kstream/internals/KStreamMap.java | 10 +-- .../kstream/internals/KStreamMapValues.java | 8 +-- .../kstream/internals/KStreamSend.java | 8 +-- .../kstream/internals/KStreamSource.java | 8 +-- .../kstream/internals/KStreamWindow.java | 18 ++--- .../processor/Chooser.java | 2 +- .../processor/KafkaProcessor.java | 2 +- .../processor/KafkaSource.java | 4 +- .../processor/Processor.java | 2 +- .../processor/ProcessorConfig.java | 2 +- .../processor/ProcessorContext.java | 2 +- .../processor/ProcessorMetadata.java | 2 +- .../processor/Punctuator.java | 2 +- .../processor/RecordCollector.java | 2 +- .../processor/RecordQueue.java | 2 +- .../processor/RestoreFunc.java | 2 +- .../processor/Stamped.java | 2 +- .../processor/StampedRecord.java | 2 +- .../processor/StateStore.java | 2 +- .../processor/TimestampExtractor.java | 2 +- .../processor/TimestampTracker.java | 2 +- .../processor/TopologyBuilder.java} | 71 ++++++++++++------- .../processor/internals/Ingestor.java | 2 +- .../processor/internals/IngestorImpl.java | 2 +- .../processor/internals/KStreamThread.java | 14 ++-- .../internals/MinTimestampTracker.java | 6 +- .../processor/internals/OffsetCheckpoint.java | 2 +- .../internals/ProcessorContextImpl.java | 22 +++--- .../internals/ProcessorStateManager.java | 6 +- .../internals/ProcessorTopology.java} | 12 ++-- .../processor/internals/PunctuationQueue.java | 2 +- .../internals/PunctuationSchedule.java | 6 +- .../internals/RecordCollectorImpl.java | 4 +- .../processor/internals/RecordQueueImpl.java | 12 ++-- .../internals/RoundRobinChooser.java | 6 +- .../processor/internals/StreamGroup.java | 16 ++--- .../processor/internals/TimeBasedChooser.java | 6 +- .../{stream => streaming}/state/Entry.java | 2 +- .../state/InMemoryKeyValueStore.java | 4 +- .../state/KeyValueIterator.java | 2 +- .../state/KeyValueStore.java | 4 +- .../state/MeteredKeyValueStore.java | 8 +-- .../state/RocksDBKeyValueStore.java | 4 +- .../FilteredIteratorTest.java | 4 +- .../KStreamBranchTest.java | 11 +-- .../KStreamFilterTest.java | 11 +-- .../KStreamFlatMapTest.java | 12 ++-- .../KStreamFlatMapValuesTest.java | 11 +-- .../KStreamJoinTest.java | 15 ++-- .../{stream => streaming}/KStreamMapTest.java | 12 ++-- .../KStreamMapValuesTest.java | 12 +++- .../KStreamSourceTest.java | 10 +-- .../KStreamWindowedTest.java | 11 +-- .../MinTimestampTrackerTest.java | 6 +- .../StreamGroupTest.java | 8 +-- .../org/apache/kafka/test/MockIngestor.java | 4 +- ...gyBuilder.java => MockKStreamBuilder.java} | 12 ++-- .../org/apache/kafka/test/MockProcessor.java | 2 +- .../kafka/test/MockProcessorContext.java | 8 +++ .../org/apache/kafka/test/MockSource.java | 2 +- .../apache/kafka/test/UnlimitedWindow.java | 10 +-- 83 files changed, 355 insertions(+), 297 deletions(-) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/KStreamProcess.java (92%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/SlidingWindow.java (93%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/examples/KStreamJob.java (84%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/examples/SimpleProcessJob.java (84%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/examples/StatefulProcessJob.java (81%) rename stream/src/main/java/org/apache/kafka/{stream => streaming/kstream}/KStream.java (97%) rename stream/src/main/java/org/apache/kafka/{stream/KStreamTopologyBuilder.java => streaming/kstream/KStreamBuilder.java} (79%) rename stream/src/main/java/org/apache/kafka/{stream => streaming/kstream}/KStreamWindowed.java (98%) rename stream/src/main/java/org/apache/kafka/{stream => streaming/kstream}/KeyValue.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming/kstream}/KeyValueMapper.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming/kstream}/Predicate.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming/kstream}/ValueJoiner.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming/kstream}/ValueMapper.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming/kstream}/Window.java (87%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/FilteredIterator.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamBranch.java (89%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamFilter.java (89%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamFlatMap.java (84%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamFlatMapValues.java (86%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamImpl.java (89%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamJoin.java (93%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamMap.java (83%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamMapValues.java (85%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamSend.java (89%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamSource.java (80%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/kstream/internals/KStreamWindow.java (84%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/Chooser.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/KafkaProcessor.java (98%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/KafkaSource.java (93%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/Processor.java (97%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/ProcessorConfig.java (99%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/ProcessorContext.java (98%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/ProcessorMetadata.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/Punctuator.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/RecordCollector.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/RecordQueue.java (97%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/RestoreFunc.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/Stamped.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/StampedRecord.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/StateStore.java (97%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/TimestampExtractor.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/TimestampTracker.java (97%) rename stream/src/main/java/org/apache/kafka/{stream/processor/PTopologyBuilder.java => streaming/processor/TopologyBuilder.java} (65%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/Ingestor.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/IngestorImpl.java (98%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/KStreamThread.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/MinTimestampTracker.java (92%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/OffsetCheckpoint.java (99%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/ProcessorContextImpl.java (93%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/ProcessorStateManager.java (97%) rename stream/src/main/java/org/apache/kafka/{stream/processor/internals/PTopology.java => streaming/processor/internals/ProcessorTopology.java} (91%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/PunctuationQueue.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/PunctuationSchedule.java (88%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/RecordCollectorImpl.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/RecordQueueImpl.java (90%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/RoundRobinChooser.java (88%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/StreamGroup.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/processor/internals/TimeBasedChooser.java (91%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/state/Entry.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/state/InMemoryKeyValueStore.java (97%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/state/KeyValueIterator.java (95%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/state/KeyValueStore.java (96%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/state/MeteredKeyValueStore.java (97%) rename stream/src/main/java/org/apache/kafka/{stream => streaming}/state/RocksDBKeyValueStore.java (98%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/FilteredIteratorTest.java (96%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/KStreamBranchTest.java (94%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/KStreamFilterTest.java (91%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/KStreamFlatMapTest.java (89%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/KStreamFlatMapValuesTest.java (89%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/KStreamJoinTest.java (97%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/KStreamMapTest.java (87%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/KStreamMapValuesTest.java (88%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/KStreamSourceTest.java (89%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/KStreamWindowedTest.java (90%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/MinTimestampTrackerTest.java (95%) rename stream/src/test/java/org/apache/kafka/{stream => streaming}/StreamGroupTest.java (97%) rename stream/src/test/java/org/apache/kafka/test/{MockKStreamTopologyBuilder.java => MockKStreamBuilder.java} (78%) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 96a79588ed727..16036d6e0a32e 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -115,7 +115,7 @@ - + diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java b/stream/src/main/java/org/apache/kafka/streaming/KStreamProcess.java similarity index 92% rename from stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java rename to stream/src/main/java/org/apache/kafka/streaming/KStreamProcess.java index 3af5033782e42..dd53ac51aed50 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamProcess.java +++ b/stream/src/main/java/org/apache/kafka/streaming/KStreamProcess.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; -import org.apache.kafka.stream.processor.PTopologyBuilder; -import org.apache.kafka.stream.processor.internals.KStreamThread; -import org.apache.kafka.stream.processor.ProcessorConfig; +import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streaming.processor.internals.KStreamThread; +import org.apache.kafka.streaming.processor.ProcessorConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,7 +29,7 @@ * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and * sends output to zero or more output topics. *

- * This processing is defined by extending the {@link PTopologyBuilder} abstract class to specify the transformation operator build. The + * This processing is defined by extending the {@link TopologyBuilder} abstract class to specify the transformation operator build. The * {@link KStreamProcess} 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. *

@@ -69,7 +69,7 @@ public class KStreamProcess implements Runnable { private final File stateDir; - public KStreamProcess(PTopologyBuilder builder, ProcessorConfig config) throws Exception { + public KStreamProcess(TopologyBuilder builder, ProcessorConfig config) throws Exception { if (config.getClass(ProcessorConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG) == null) throw new NullPointerException("timestamp extractor is missing"); diff --git a/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/streaming/SlidingWindow.java similarity index 93% rename from stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java rename to stream/src/main/java/org/apache/kafka/streaming/SlidingWindow.java index 5e996c700c029..2098262cf40e3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/streaming/SlidingWindow.java @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; -import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.kstream.internals.FilteredIterator; -import org.apache.kafka.stream.processor.Stamped; +import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.kstream.internals.FilteredIterator; +import org.apache.kafka.streaming.processor.Stamped; import java.util.HashMap; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java similarity index 84% rename from stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java rename to stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java index 8fe2e81277c0b..d552fec139635 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.kafka.stream.examples; +package org.apache.kafka.streaming.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.stream.KStreamTopologyBuilder; -import org.apache.kafka.stream.processor.ProcessorConfig; -import org.apache.kafka.stream.KStreamProcess; -import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.KeyValue; -import org.apache.kafka.stream.KeyValueMapper; -import org.apache.kafka.stream.Predicate; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.processor.ProcessorConfig; +import org.apache.kafka.streaming.KStreamProcess; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.kstream.Predicate; import java.util.Properties; @@ -34,7 +34,7 @@ public class KStreamJob { public static void main(String[] args) throws Exception { ProcessorConfig config = new ProcessorConfig(new Properties()); - KStreamTopologyBuilder builder = new KStreamTopologyBuilder(); + KStreamBuilder builder = new KStreamBuilder(); StringSerializer stringSerializer = new StringSerializer(); IntegerSerializer intSerializer = new IntegerSerializer(); diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java similarity index 84% rename from stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java rename to stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java index c18d668de0df3..d9ac62e02e438 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/SimpleProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package org.apache.kafka.stream.examples; +package org.apache.kafka.streaming.examples; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.PTopologyBuilder; -import org.apache.kafka.stream.processor.ProcessorConfig; -import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streaming.processor.ProcessorConfig; +import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.KStreamProcess; +import org.apache.kafka.streaming.KStreamProcess; import java.util.Properties; @@ -58,7 +58,7 @@ public void close() { public static void main(String[] args) throws Exception { ProcessorConfig config = new ProcessorConfig(new Properties()); - PTopologyBuilder builder = new PTopologyBuilder(); + TopologyBuilder builder = new TopologyBuilder(); builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); builder.addProcessor("PROCESS", MyProcessor.class, null, "SOURCE"); diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java similarity index 81% rename from stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java rename to stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java index 3be00280d3f35..80286be3eff5b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java @@ -15,19 +15,19 @@ * limitations under the License. */ -package org.apache.kafka.stream.examples; +package org.apache.kafka.streaming.examples; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.KStreamProcess; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.PTopologyBuilder; -import org.apache.kafka.stream.processor.ProcessorConfig; -import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.state.Entry; -import org.apache.kafka.stream.state.InMemoryKeyValueStore; -import org.apache.kafka.stream.state.KeyValueIterator; -import org.apache.kafka.stream.state.KeyValueStore; +import org.apache.kafka.streaming.KStreamProcess; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streaming.processor.ProcessorConfig; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.state.Entry; +import org.apache.kafka.streaming.state.InMemoryKeyValueStore; +import org.apache.kafka.streaming.state.KeyValueIterator; +import org.apache.kafka.streaming.state.KeyValueStore; import java.util.Properties; @@ -79,7 +79,7 @@ public void close() { public static void main(String[] args) throws Exception { ProcessorConfig config = new ProcessorConfig(new Properties()); - PTopologyBuilder builder = new PTopologyBuilder(); + TopologyBuilder builder = new TopologyBuilder(); builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); builder.addProcessor("PROCESS", MyProcessor.class, null, "SOURCE"); diff --git a/stream/src/main/java/org/apache/kafka/stream/KStream.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/KStream.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java index 8130c5e0ea7f7..9256745325de3 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming.kstream; -import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.KafkaProcessor; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.stream.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.ProcessorMetadata; /** * KStream is an abstraction of a stream of key-value pairs. diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamTopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java similarity index 79% rename from stream/src/main/java/org/apache/kafka/stream/KStreamTopologyBuilder.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java index 77423c016cbb6..30b85c6f9a18d 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamTopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java @@ -15,22 +15,21 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming.kstream; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.stream.kstream.internals.KStreamImpl; -import org.apache.kafka.stream.processor.KafkaSource; -import org.apache.kafka.stream.processor.PTopologyBuilder; -import org.apache.kafka.stream.kstream.internals.KStreamSource; - -import java.util.concurrent.atomic.AtomicInteger; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.internals.KStreamImpl; +import org.apache.kafka.streaming.processor.KafkaSource; +import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; /** - * KStreamTopologyBuilder is the class to create KStream instances. + * KStreamBuilder is the class to create KStream instances. */ -public class KStreamTopologyBuilder extends PTopologyBuilder { +public class KStreamBuilder extends TopologyBuilder { - public KStreamTopologyBuilder() { + public KStreamBuilder() { super(); } diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamWindowed.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamWindowed.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/stream/KStreamWindowed.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamWindowed.java index e3660b0f6f8ca..517d63bc45ad2 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamWindowed.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamWindowed.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming.kstream; /** * KStreamWindowed is an abstraction of a stream of key-value pairs with a window. diff --git a/stream/src/main/java/org/apache/kafka/stream/KeyValue.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValue.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/KeyValue.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValue.java index 835962206a348..bd67064bec914 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KeyValue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValue.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming.kstream; public class KeyValue { diff --git a/stream/src/main/java/org/apache/kafka/stream/KeyValueMapper.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueMapper.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/KeyValueMapper.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueMapper.java index 9f8fbc4e7a458..10e05d69e114b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/KeyValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueMapper.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming.kstream; public interface KeyValueMapper { diff --git a/stream/src/main/java/org/apache/kafka/stream/Predicate.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/Predicate.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/Predicate.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/Predicate.java index 932f3ea267322..8c3b6db39c364 100644 --- a/stream/src/main/java/org/apache/kafka/stream/Predicate.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/Predicate.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming.kstream; public interface Predicate { diff --git a/stream/src/main/java/org/apache/kafka/stream/ValueJoiner.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/ValueJoiner.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/ValueJoiner.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/ValueJoiner.java index b161efb650d6b..cc89062369529 100644 --- a/stream/src/main/java/org/apache/kafka/stream/ValueJoiner.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/ValueJoiner.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming.kstream; public interface ValueJoiner { diff --git a/stream/src/main/java/org/apache/kafka/stream/ValueMapper.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/ValueMapper.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/ValueMapper.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/ValueMapper.java index a83415570064e..1592d4a09ae62 100644 --- a/stream/src/main/java/org/apache/kafka/stream/ValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/ValueMapper.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming.kstream; public interface ValueMapper { diff --git a/stream/src/main/java/org/apache/kafka/stream/Window.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java similarity index 87% rename from stream/src/main/java/org/apache/kafka/stream/Window.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java index 8fb752fc8af54..d805466144d4c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/Window.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming.kstream; -import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.processor.StateStore; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.StateStore; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/FilteredIterator.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/FilteredIterator.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/FilteredIterator.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/FilteredIterator.java index 4b9fed9421309..9ffa04a40d0c0 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/FilteredIterator.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/FilteredIterator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java similarity index 89% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java index 491a49a9f783c..1b8aba3b6729e 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.ProcessorMetadata; -import org.apache.kafka.stream.Predicate; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.kstream.Predicate; class KStreamBranch extends KafkaProcessor { diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java similarity index 89% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java index 3b397ad13ff54..58daa2c1c62ab 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.Predicate; -import org.apache.kafka.stream.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.kstream.Predicate; +import org.apache.kafka.streaming.processor.ProcessorMetadata; class KStreamFilter extends KafkaProcessor { diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java similarity index 84% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java index be5b6b70a9e12..7348bd69d6a27 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.KeyValue; -import org.apache.kafka.stream.KeyValueMapper; -import org.apache.kafka.stream.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.processor.ProcessorMetadata; class KStreamFlatMap extends KafkaProcessor { diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java similarity index 86% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java index 800bb59338b8e..7c4fb538433c6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.ValueMapper; -import org.apache.kafka.stream.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.kstream.ValueMapper; +import org.apache.kafka.streaming.processor.ProcessorMetadata; class KStreamFlatMapValues extends KafkaProcessor { diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java similarity index 89% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index 22ec8fa2536b3..eaf5967d90f9a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -15,20 +15,20 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.ProcessorMetadata; -import org.apache.kafka.stream.processor.PTopologyBuilder; -import org.apache.kafka.stream.processor.KafkaSource; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streaming.processor.KafkaSource; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.stream.KStreamWindowed; -import org.apache.kafka.stream.KeyValueMapper; -import org.apache.kafka.stream.Predicate; -import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.ValueMapper; -import org.apache.kafka.stream.Window; +import org.apache.kafka.streaming.kstream.KStreamWindowed; +import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.kstream.Predicate; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.ValueMapper; +import org.apache.kafka.streaming.kstream.Window; import java.util.ArrayList; import java.util.Arrays; @@ -55,10 +55,10 @@ public class KStreamImpl implements KStream { public static final AtomicInteger INDEX = new AtomicInteger(1); - protected PTopologyBuilder topology; + protected TopologyBuilder topology; protected String name; - public KStreamImpl(PTopologyBuilder topology, String name) { + public KStreamImpl(TopologyBuilder topology, String name) { this.topology = topology; this.name = name; } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java similarity index 93% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamJoin.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index 370eda766856b..236996710b3a6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.ValueJoiner; -import org.apache.kafka.stream.Window; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.kstream.ValueJoiner; +import org.apache.kafka.streaming.kstream.Window; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java similarity index 83% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java index 32bdb02f425a6..0a4526e190ea1 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.KeyValue; -import org.apache.kafka.stream.KeyValueMapper; -import org.apache.kafka.stream.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.processor.ProcessorMetadata; class KStreamMap extends KafkaProcessor { diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java similarity index 85% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java index 32649f8f8eba8..778c4bb552721 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.ValueMapper; -import org.apache.kafka.stream.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.kstream.ValueMapper; +import org.apache.kafka.streaming.processor.ProcessorMetadata; class KStreamMapValues extends KafkaProcessor { diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java similarity index 89% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java index 2229cd862646b..292d5f4d544f4 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSend.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.ProcessorMetadata; -import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.ProcessorContext; class KStreamSend extends KafkaProcessor { diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java similarity index 80% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSource.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java index ae358337991bd..3ad77d0219117 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.PTopologyBuilder; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.TopologyBuilder; public class KStreamSource extends KStreamImpl { private KafkaProcessor source; - public KStreamSource(PTopologyBuilder topology, KafkaProcessor source) { + public KStreamSource(TopologyBuilder topology, KafkaProcessor source) { super(topology, source.name()); this.source = source; } diff --git a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java similarity index 84% rename from stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamWindow.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java index 28f849eb709ec..2288a07485cef 100644 --- a/stream/src/main/java/org/apache/kafka/stream/kstream/internals/KStreamWindow.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package org.apache.kafka.stream.kstream.internals; +package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.PTopologyBuilder; -import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.KStreamWindowed; -import org.apache.kafka.stream.ValueJoiner; -import org.apache.kafka.stream.Window; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamWindowed; +import org.apache.kafka.streaming.kstream.ValueJoiner; +import org.apache.kafka.streaming.kstream.Window; public class KStreamWindow extends KafkaProcessor { @@ -31,7 +31,7 @@ public static final class KStreamWindowedImpl extends KStreamImpl im public KStreamWindow windowed; - public KStreamWindowedImpl(PTopologyBuilder topology, KStreamWindow windowed) { + public KStreamWindowedImpl(TopologyBuilder topology, KStreamWindow windowed) { super(topology, windowed); this.windowed = windowed; } diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/Chooser.java b/stream/src/main/java/org/apache/kafka/streaming/processor/Chooser.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/processor/Chooser.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/Chooser.java index 434db8d23d1d7..0647430f1c093 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/Chooser.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/Chooser.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; public interface Chooser { diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/KafkaProcessor.java b/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/stream/processor/KafkaProcessor.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java index 80249a3860d5e..e76177200e49a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/KafkaProcessor.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; import java.util.ArrayList; import java.util.List; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/KafkaSource.java b/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaSource.java similarity index 93% rename from stream/src/main/java/org/apache/kafka/stream/processor/KafkaSource.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/KafkaSource.java index cea72920cb767..902f14142e9fc 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/KafkaSource.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaSource.java @@ -15,14 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; import org.apache.kafka.common.serialization.Deserializer; public class KafkaSource extends KafkaProcessor { - private static final String SOURCE_NAME = "KAFKA-SOURCE"; - public Deserializer keyDeserializer; public Deserializer valDeserializer; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/Processor.java b/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/processor/Processor.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java index 7ddce341eb72a..9d6f47bf331a8 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/Processor.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java @@ -65,7 +65,7 @@ public interface ProcessorContext { * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; public interface Processor { diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorConfig.java similarity index 99% rename from stream/src/main/java/org/apache/kafka/stream/processor/ProcessorConfig.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorConfig.java index f7f87e1e852f1..80802d20ed238 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorConfig.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorConfig.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorContext.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/stream/processor/ProcessorContext.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java index b26458b5e62e3..bbf58d7188b53 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorContext.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorMetadata.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorMetadata.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/processor/ProcessorMetadata.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorMetadata.java index cef30bbfb4c94..ba773b0f279ca 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/ProcessorMetadata.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorMetadata.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; public class ProcessorMetadata { diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/Punctuator.java b/stream/src/main/java/org/apache/kafka/streaming/processor/Punctuator.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/processor/Punctuator.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/Punctuator.java index 2fece68c4ef2a..4619263db3b4c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/Punctuator.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/Punctuator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; public interface Punctuator { diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/RecordCollector.java b/stream/src/main/java/org/apache/kafka/streaming/processor/RecordCollector.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/processor/RecordCollector.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/RecordCollector.java index 2db7f2c7f6314..e65b2b93f84ac 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/RecordCollector.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/RecordCollector.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/RecordQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/RecordQueue.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/processor/RecordQueue.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/RecordQueue.java index 5e3a345e8600b..0aca77ed0d04b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/RecordQueue.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; import org.apache.kafka.common.TopicPartition; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/RestoreFunc.java b/stream/src/main/java/org/apache/kafka/streaming/processor/RestoreFunc.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/processor/RestoreFunc.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/RestoreFunc.java index ab6e7b25ae495..877c6a49188e6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/RestoreFunc.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/RestoreFunc.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; // TODO: this should be removed once we move to Java 8 public interface RestoreFunc { diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/Stamped.java b/stream/src/main/java/org/apache/kafka/streaming/processor/Stamped.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/processor/Stamped.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/Stamped.java index 79c8eeb1d9859..a174b953092c2 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/Stamped.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/Stamped.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; public class Stamped implements Comparable { diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/StampedRecord.java b/stream/src/main/java/org/apache/kafka/streaming/processor/StampedRecord.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/processor/StampedRecord.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/StampedRecord.java index 2a93f1ef5ecef..96b5e400fecee 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/StampedRecord.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/StampedRecord.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/StateStore.java b/stream/src/main/java/org/apache/kafka/streaming/processor/StateStore.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/processor/StateStore.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/StateStore.java index 71cf5a1cbd648..752037c88258a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/StateStore.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/StateStore.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; /** * A storage engine for managing state maintained by a stream processor. diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/TimestampExtractor.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TimestampExtractor.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/processor/TimestampExtractor.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/TimestampExtractor.java index f135fd8dfaacc..5ab69677d8cb1 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/TimestampExtractor.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TimestampExtractor.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; /** * An interface that allows the KStream framework to extract a timestamp from a key-value pair diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TimestampTracker.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/processor/TimestampTracker.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/TimestampTracker.java index 13acca140a842..87e9710e5fd4f 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/TimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TimestampTracker.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; /** * TimestampTracker is a helper class for a sliding window implementation. diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java similarity index 65% rename from stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 8e3ea80f4915d..2cab55ee9c289 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/PTopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -15,48 +15,61 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor; +package org.apache.kafka.streaming.processor; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.stream.processor.internals.PTopology; +import org.apache.kafka.streaming.processor.internals.ProcessorTopology; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -public class PTopologyBuilder { +public class TopologyBuilder { private Map processorClasses = new HashMap<>(); - private Map sources = new HashMap<>(); + private Map sourceClasses = new HashMap<>(); + private Map topicsToSourceNames = new HashMap<>(); private Map> parents = new HashMap<>(); private Map> children = new HashMap<>(); private class ProcessorClazz { public Class clazz; - public ProcessorMetadata config; + public ProcessorMetadata metadata; - public ProcessorClazz(Class clazz, ProcessorMetadata config) { + public ProcessorClazz(Class clazz, ProcessorMetadata metadata) { this.clazz = clazz; - this.config = config; + this.metadata = metadata; } } - public PTopologyBuilder() {} + private class SourceClazz { + public Deserializer keyDeserializer; + public Deserializer valDeserializer; + + private SourceClazz(Deserializer keyDeserializer, Deserializer valDeserializer) { + this.keyDeserializer = keyDeserializer; + this.valDeserializer = valDeserializer; + } + } + + public TopologyBuilder() {} @SuppressWarnings("unchecked") public final KafkaSource addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { KafkaSource source = new KafkaSource(name, keyDeserializer, valDeserializer); for (String topic : topics) { - if (sources.containsKey(topic)) + if (topicsToSourceNames.containsKey(topic)) throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); - sources.put(name, source); + topicsToSourceNames.put(topic, name); } + sourceClasses.put(name, new SourceClazz(keyDeserializer, valDeserializer)); + return source; } @@ -76,7 +89,7 @@ public final void addProcessor(String name, Class proc parents.put(name, new ArrayList<>()); parents.get(name).add(parent); - // add to parent list + // add to children list if (!children.containsKey(parent)) children.put(parent, new ArrayList<>()); children.get(parent).add(name); @@ -88,31 +101,38 @@ public final void addProcessor(String name, Class proc * Build the topology by creating the processors */ @SuppressWarnings("unchecked") - public PTopology build() { + public ProcessorTopology build() { Map processorMap = new HashMap<>(); + Map topicSourceMap = new HashMap<>(); + + // create sources + try { + for (String name : sourceClasses.keySet()) { + Deserializer keyDeserializer = sourceClasses.get(name).keyDeserializer; + Deserializer valDeserializer = sourceClasses.get(name).valDeserializer; + KafkaSource source = new KafkaSource(name, keyDeserializer, valDeserializer); + processorMap.put(name, source); + } + } catch (Exception e) { + throw new KafkaException("KafkaProcessor(String) constructor failed: this should not happen."); + } // create processors try { for (String name : processorClasses.keySet()) { - ProcessorMetadata config = processorClasses.get(name).config; + ProcessorMetadata metadata = processorClasses.get(name).metadata; Class processorClass = processorClasses.get(name).clazz; - KafkaProcessor processor = processorClass.getConstructor(String.class, ProcessorMetadata.class).newInstance(name, config); + KafkaProcessor processor = processorClass.getConstructor(String.class, ProcessorMetadata.class).newInstance(name, metadata); processorMap.put(name, processor); } } catch (Exception e) { throw new KafkaException("KafkaProcessor(String) constructor failed: this should not happen."); } - // construct processor parent-child relationships - for (String topic : sources.keySet()) { - KafkaSource source = sources.get(topic); - processorMap.put(source.name(), source); - - // chain children to this processor - for (String child : children.get(source.name())) { - KafkaProcessor childProcessor = processorMap.get(child); - source.chain(childProcessor); - } + // construct topics to sources map + for (String topic : topicsToSourceNames.keySet()) { + KafkaSource source = (KafkaSource) processorMap.get(topicsToSourceNames.get(topic)); + topicSourceMap.put(topic, source); } for (KafkaProcessor processor : processorMap.values()) { @@ -121,9 +141,8 @@ public PTopology build() { KafkaProcessor childProcessor = processorMap.get(child); processor.chain(childProcessor); } - } - return new PTopology(processorMap, sources); + return new ProcessorTopology(processorMap, topicSourceMap); } } diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/Ingestor.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Ingestor.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/Ingestor.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/Ingestor.java index 1d1e233f54f1b..b66808abda579 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/Ingestor.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Ingestor.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.common.TopicPartition; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/IngestorImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/IngestorImpl.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/IngestorImpl.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/IngestorImpl.java index 6ba190898ca09..886cb5c799bd7 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/IngestorImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/IngestorImpl.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.CommitType; import org.apache.kafka.clients.consumer.Consumer; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/KStreamThread.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java index a0018a1f7efee..cf1991e0ad181 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.stream.processor.PTopologyBuilder; -import org.apache.kafka.stream.processor.ProcessorConfig; +import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streaming.processor.ProcessorConfig; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.KafkaException; @@ -53,7 +53,7 @@ public class KStreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); - private final PTopologyBuilder builder; + private final TopologyBuilder builder; private final IngestorImpl ingestor; private final RecordCollectorImpl collector; private final ArrayList streamGroups = new ArrayList<>(); @@ -90,7 +90,7 @@ public void onPartitionsRevoked(Consumer consumer, Collection producer = new KafkaProducer<>(config.getProducerProperties(), @@ -251,7 +251,7 @@ private void addPartitions(Collection assignment) { if (context == null) { try { // build the topology and initialize with the context - PTopology topology = builder.build(); + ProcessorTopology topology = builder.build(); context = new ProcessorContextImpl(id, ingestor, topology, collector, config, metrics); diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java similarity index 92% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/MinTimestampTracker.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java index d529e59af8024..1eb0ddb76a88d 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/MinTimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.stream.processor.Stamped; -import org.apache.kafka.stream.processor.TimestampTracker; +import org.apache.kafka.streaming.processor.Stamped; +import org.apache.kafka.streaming.processor.TimestampTracker; import java.util.LinkedList; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/OffsetCheckpoint.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/OffsetCheckpoint.java similarity index 99% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/OffsetCheckpoint.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/OffsetCheckpoint.java index dc143ec244327..db0b90858bea4 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/OffsetCheckpoint.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/OffsetCheckpoint.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.common.TopicPartition; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java similarity index 93% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorContextImpl.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index 2c71b33664c8b..5fafa36381461 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.stream.processor.KafkaSource; -import org.apache.kafka.stream.processor.ProcessorConfig; -import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.processor.RecordCollector; -import org.apache.kafka.stream.processor.StateStore; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.RestoreFunc; -import org.apache.kafka.stream.processor.TimestampExtractor; +import org.apache.kafka.streaming.processor.KafkaSource; +import org.apache.kafka.streaming.processor.ProcessorConfig; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.RecordCollector; +import org.apache.kafka.streaming.processor.StateStore; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.RestoreFunc; +import org.apache.kafka.streaming.processor.TimestampExtractor; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; @@ -56,7 +56,7 @@ public class ProcessorContextImpl implements ProcessorContext { public final StreamGroup streamGroup; private final Metrics metrics; - private final PTopology topology; + private final ProcessorTopology topology; private final RecordCollectorImpl collector; private final ProcessorStateManager stateMgr; @@ -70,7 +70,7 @@ public class ProcessorContextImpl implements ProcessorContext { @SuppressWarnings("unchecked") public ProcessorContextImpl(int id, Ingestor ingestor, - PTopology topology, + ProcessorTopology topology, RecordCollectorImpl collector, ProcessorConfig config, Metrics metrics) throws IOException { diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorStateManager.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java index aeff8e6264595..f286da33a8203 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/ProcessorStateManager.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.stream.processor.RestoreFunc; -import org.apache.kafka.stream.processor.StateStore; +import org.apache.kafka.streaming.processor.RestoreFunc; +import org.apache.kafka.streaming.processor.StateStore; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/PTopology.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java similarity index 91% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/PTopology.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java index 8d515d867e34d..a80f2e55f5321 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/PTopology.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.KafkaSource; -import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.KafkaSource; +import org.apache.kafka.streaming.processor.ProcessorContext; import java.util.ArrayDeque; import java.util.Deque; @@ -30,12 +30,12 @@ import java.util.Map; import java.util.Set; -public class PTopology { +public class ProcessorTopology { private Map processors = new HashMap<>(); private Map topicSources = new HashMap<>(); - public PTopology(Map processors, Map topicSources) { + public ProcessorTopology(Map processors, Map topicSources) { this.processors = processors; this.topicSources = topicSources; } diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationQueue.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationQueue.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationQueue.java index b53095b4c37ec..2e7cd77420b83 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationQueue.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; import java.util.PriorityQueue; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java similarity index 88% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationSchedule.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java index 7b47820c038b2..92ea5581bf8d1 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/PunctuationSchedule.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.stream.processor.Punctuator; -import org.apache.kafka.stream.processor.Stamped; +import org.apache.kafka.streaming.processor.Punctuator; +import org.apache.kafka.streaming.processor.Stamped; public class PunctuationSchedule extends Stamped { diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordCollectorImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollectorImpl.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordCollectorImpl.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollectorImpl.java index b8dc7f4c86ced..878f5ce4026bb 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordCollectorImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollectorImpl.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.stream.processor.RecordCollector; +import org.apache.kafka.streaming.processor.RecordCollector; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordQueueImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java similarity index 90% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordQueueImpl.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java index c0c99014bf3f8..ad4b460c6c7ee 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/RecordQueueImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.KafkaSource; -import org.apache.kafka.stream.processor.RecordQueue; -import org.apache.kafka.stream.processor.StampedRecord; -import org.apache.kafka.stream.processor.TimestampTracker; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.KafkaSource; +import org.apache.kafka.streaming.processor.RecordQueue; +import org.apache.kafka.streaming.processor.StampedRecord; +import org.apache.kafka.streaming.processor.TimestampTracker; import org.apache.kafka.common.TopicPartition; import java.util.ArrayDeque; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java similarity index 88% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/RoundRobinChooser.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java index 33f0fe84f938d..97903fd925002 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/RoundRobinChooser.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.stream.processor.Chooser; -import org.apache.kafka.stream.processor.RecordQueue; +import org.apache.kafka.streaming.processor.Chooser; +import org.apache.kafka.streaming.processor.RecordQueue; import java.util.ArrayDeque; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/StreamGroup.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java index 22f3c53d7253a..d8fdd6b99c80c 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.stream.processor.Chooser; -import org.apache.kafka.stream.processor.KafkaSource; -import org.apache.kafka.stream.processor.Punctuator; -import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.processor.RecordQueue; -import org.apache.kafka.stream.processor.StampedRecord; -import org.apache.kafka.stream.processor.TimestampExtractor; +import org.apache.kafka.streaming.processor.Chooser; +import org.apache.kafka.streaming.processor.KafkaSource; +import org.apache.kafka.streaming.processor.Punctuator; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.RecordQueue; +import org.apache.kafka.streaming.processor.StampedRecord; +import org.apache.kafka.streaming.processor.TimestampExtractor; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; diff --git a/stream/src/main/java/org/apache/kafka/stream/processor/internals/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java similarity index 91% rename from stream/src/main/java/org/apache/kafka/stream/processor/internals/TimeBasedChooser.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java index 7aa4f0afede3d..6889a3e59a6f9 100644 --- a/stream/src/main/java/org/apache/kafka/stream/processor/internals/TimeBasedChooser.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.stream.processor.internals; +package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.stream.processor.Chooser; -import org.apache.kafka.stream.processor.RecordQueue; +import org.apache.kafka.streaming.processor.Chooser; +import org.apache.kafka.streaming.processor.RecordQueue; import java.util.Comparator; import java.util.PriorityQueue; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/Entry.java b/stream/src/main/java/org/apache/kafka/streaming/state/Entry.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/state/Entry.java rename to stream/src/main/java/org/apache/kafka/streaming/state/Entry.java index c02ffac4a18d8..244e07f5b095b 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/Entry.java +++ b/stream/src/main/java/org/apache/kafka/streaming/state/Entry.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream.state; +package org.apache.kafka.streaming.state; public class Entry { diff --git a/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java b/stream/src/main/java/org/apache/kafka/streaming/state/InMemoryKeyValueStore.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java rename to stream/src/main/java/org/apache/kafka/streaming/state/InMemoryKeyValueStore.java index 25e76d1a12eb5..9b194e8883b5d 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/InMemoryKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/streaming/state/InMemoryKeyValueStore.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.stream.state; +package org.apache.kafka.streaming.state; -import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/KeyValueIterator.java b/stream/src/main/java/org/apache/kafka/streaming/state/KeyValueIterator.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/stream/state/KeyValueIterator.java rename to stream/src/main/java/org/apache/kafka/streaming/state/KeyValueIterator.java index 50cc76dd15e2b..6f5c43a2505a6 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/KeyValueIterator.java +++ b/stream/src/main/java/org/apache/kafka/streaming/state/KeyValueIterator.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.kafka.stream.state; +package org.apache.kafka.streaming.state; import java.io.Closeable; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java b/stream/src/main/java/org/apache/kafka/streaming/state/KeyValueStore.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java rename to stream/src/main/java/org/apache/kafka/streaming/state/KeyValueStore.java index 18cf1b26afab3..73518170f188a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/KeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/streaming/state/KeyValueStore.java @@ -17,9 +17,9 @@ * under the License. */ -package org.apache.kafka.stream.state; +package org.apache.kafka.streaming.state; -import org.apache.kafka.stream.processor.StateStore; +import org.apache.kafka.streaming.processor.StateStore; import java.util.List; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/stream/state/MeteredKeyValueStore.java rename to stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java index a2fd7583d0e58..d7664ed5e9b50 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.stream.state; +package org.apache.kafka.streaming.state; -import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.processor.RecordCollector; -import org.apache.kafka.stream.processor.RestoreFunc; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.RecordCollector; +import org.apache.kafka.streaming.processor.RestoreFunc; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.MeasurableStat; diff --git a/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java b/stream/src/main/java/org/apache/kafka/streaming/state/RocksDBKeyValueStore.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java rename to stream/src/main/java/org/apache/kafka/streaming/state/RocksDBKeyValueStore.java index 505582591fe8f..2a5e3a0811eef 100644 --- a/stream/src/main/java/org/apache/kafka/stream/state/RocksDBKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/streaming/state/RocksDBKeyValueStore.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.stream.state; +package org.apache.kafka.streaming.state; -import org.apache.kafka.stream.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.SystemTime; diff --git a/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java b/stream/src/test/java/org/apache/kafka/streaming/FilteredIteratorTest.java similarity index 96% rename from stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java rename to stream/src/test/java/org/apache/kafka/streaming/FilteredIteratorTest.java index 21fd738c6a50f..f2c1f9f5023eb 100644 --- a/stream/src/test/java/org/apache/kafka/stream/FilteredIteratorTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/FilteredIteratorTest.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; import static org.junit.Assert.assertEquals; -import org.apache.kafka.stream.kstream.internals.FilteredIterator; +import org.apache.kafka.streaming.kstream.internals.FilteredIterator; import org.junit.Test; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java similarity index 94% rename from stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java rename to stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java index 05c7924dc335d..0becd1722e572 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java @@ -15,12 +15,15 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopologyBuilder; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.Predicate; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -97,7 +100,7 @@ public boolean apply(Integer key, String value) { private String topic1 = "topic"; >>>>>>> adding files - private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); + private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamFilterTest.java similarity index 91% rename from stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java rename to stream/src/test/java/org/apache/kafka/streaming/KStreamFilterTest.java index d705e1d5c58a1..b55946d14c75f 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/KStreamFilterTest.java @@ -15,12 +15,15 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopologyBuilder; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.Predicate; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -31,7 +34,7 @@ public class KStreamFilterTest { private String topicName = "topic"; - private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); + private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapTest.java similarity index 89% rename from stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java rename to stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapTest.java index ed4a6093585ce..56471048cf3c8 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapTest.java @@ -15,12 +15,16 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopologyBuilder; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -32,7 +36,7 @@ public class KStreamFlatMapTest { private String topicName = "topic"; - private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); + private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapValuesTest.java similarity index 89% rename from stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java rename to stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapValuesTest.java index 554471fc03810..e001f7bd2a52b 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapValuesTest.java @@ -15,12 +15,15 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopologyBuilder; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.ValueMapper; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -32,7 +35,7 @@ public class KStreamFlatMapValuesTest { private String topicName = "topic"; - private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); + private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java similarity index 97% rename from stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java rename to stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java index c0a8c85a35116..e9d3beda17d75 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java @@ -15,13 +15,20 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; 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.stream.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopologyBuilder; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.KStreamWindowed; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.kstream.ValueJoiner; +import org.apache.kafka.streaming.kstream.ValueMapper; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.UnlimitedWindow; @@ -34,7 +41,7 @@ public class KStreamJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); + private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamMapTest.java similarity index 87% rename from stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java rename to stream/src/test/java/org/apache/kafka/streaming/KStreamMapTest.java index 061b1a459f7e0..79306d9ebdd1d 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/KStreamMapTest.java @@ -15,12 +15,16 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopologyBuilder; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -30,7 +34,7 @@ public class KStreamMapTest { private String topicName = "topic"; - private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); + private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java similarity index 88% rename from stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java rename to stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java index 790484008b843..be11bb77611c8 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; <<<<<<< HEAD <<<<<<< HEAD @@ -54,6 +54,7 @@ ======= import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; +<<<<<<< HEAD:stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java <<<<<<< HEAD >>>>>>> wip import org.apache.kafka.stream.internals.KStreamSource; @@ -63,6 +64,13 @@ import org.apache.kafka.stream.kstream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamTopologyBuilder; >>>>>>> Refactor Processor and KStream APIs +======= +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.ValueMapper; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; +>>>>>>> Some package and class renaming, fix KafkaSource constrcution at builder:stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java import org.apache.kafka.test.MockProcessor; <<<<<<< HEAD import org.apache.kafka.test.MockKStreamContext; @@ -77,7 +85,7 @@ public class KStreamMapValuesTest { private String topicName = "topic"; - private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); + private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamSourceTest.java similarity index 89% rename from stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java rename to stream/src/test/java/org/apache/kafka/streaming/KStreamSourceTest.java index 7a5c86d4ec6ea..da2512b56580f 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamSourceTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/KStreamSourceTest.java @@ -15,11 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopologyBuilder; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; import org.apache.kafka.test.MockProcessor; import org.junit.Test; @@ -29,7 +31,7 @@ public class KStreamSourceTest { private String topicName = "topic"; - private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); + private KStreamBuilder topology = new MockKStreamBuilder(); private StringDeserializer keyDeserializer = new StringDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java similarity index 90% rename from stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java rename to stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java index 39f8430b63bce..de5cd622a0d4d 100644 --- a/stream/src/test/java/org/apache/kafka/stream/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java @@ -15,12 +15,15 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.stream.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopologyBuilder; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.UnlimitedWindow; import org.junit.Test; @@ -33,7 +36,7 @@ public class KStreamWindowedTest { private String topicName = "topic"; - private KStreamTopologyBuilder topology = new MockKStreamTopologyBuilder(); + private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); diff --git a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java similarity index 95% rename from stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java rename to stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java index 984f170695715..bc663d418a2b3 100644 --- a/stream/src/test/java/org/apache/kafka/stream/MinTimestampTrackerTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; import static org.junit.Assert.assertEquals; -import org.apache.kafka.stream.processor.internals.MinTimestampTracker; -import org.apache.kafka.stream.processor.Stamped; +import org.apache.kafka.streaming.processor.internals.MinTimestampTracker; +import org.apache.kafka.streaming.processor.Stamped; import org.junit.Test; public class MinTimestampTrackerTest { diff --git a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java similarity index 97% rename from stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java rename to stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java index f5bad4ba024ad..bcf14215327c3 100644 --- a/stream/src/test/java/org/apache/kafka/stream/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java @@ -15,17 +15,17 @@ * limitations under the License. */ -package org.apache.kafka.stream; +package org.apache.kafka.streaming; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.stream.processor.TimestampExtractor; +import org.apache.kafka.streaming.processor.TimestampExtractor; 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.stream.processor.internals.StreamGroup; -import org.apache.kafka.stream.processor.internals.TimeBasedChooser; +import org.apache.kafka.streaming.processor.internals.StreamGroup; +import org.apache.kafka.streaming.processor.internals.TimeBasedChooser; import org.apache.kafka.test.MockIngestor; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.MockSource; diff --git a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java index 62c6af02dcd93..1624feb47ba2a 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java @@ -19,8 +19,8 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.stream.processor.internals.Ingestor; -import org.apache.kafka.stream.processor.internals.StreamGroup; +import org.apache.kafka.streaming.processor.internals.Ingestor; +import org.apache.kafka.streaming.processor.internals.StreamGroup; import java.util.HashMap; import java.util.HashSet; diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopologyBuilder.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamBuilder.java similarity index 78% rename from stream/src/test/java/org/apache/kafka/test/MockKStreamTopologyBuilder.java rename to stream/src/test/java/org/apache/kafka/test/MockKStreamBuilder.java index 43eda5f1eb942..aa3de2b7a050b 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamTopologyBuilder.java +++ b/stream/src/test/java/org/apache/kafka/test/MockKStreamBuilder.java @@ -17,14 +17,10 @@ package org.apache.kafka.test; -import org.apache.kafka.stream.KStreamTopologyBuilder; +import org.apache.kafka.streaming.kstream.KStreamBuilder; -public class MockKStreamTopologyBuilder extends KStreamTopologyBuilder { - public MockKStreamTopologyBuilder() { - super(null); - } - - @Override - public void build() { +public class MockKStreamBuilder extends KStreamBuilder { + public MockKStreamBuilder() { + super(); } } diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java index 1b22440625305..4b1fba293f9e6 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -17,7 +17,7 @@ package org.apache.kafka.test; -import org.apache.kafka.stream.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.KafkaProcessor; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java index cd6f7de1fc8f3..5e9e11b950c9f 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -17,6 +17,7 @@ package org.apache.kafka.test; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD:stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java <<<<<<< HEAD @@ -78,6 +79,13 @@ import org.apache.kafka.stream.processor.RestoreFunc; import org.apache.kafka.stream.processor.StateStore; >>>>>>> Refactor Processor and KStream APIs +======= +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.RecordCollector; +import org.apache.kafka.streaming.processor.RestoreFunc; +import org.apache.kafka.streaming.processor.StateStore; +>>>>>>> Some package and class renaming, fix KafkaSource constrcution at builder import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; diff --git a/stream/src/test/java/org/apache/kafka/test/MockSource.java b/stream/src/test/java/org/apache/kafka/test/MockSource.java index cf1ace900d17c..38c78c22001ef 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockSource.java +++ b/stream/src/test/java/org/apache/kafka/test/MockSource.java @@ -17,7 +17,7 @@ package org.apache.kafka.test; -import org.apache.kafka.stream.processor.KafkaSource; +import org.apache.kafka.streaming.processor.KafkaSource; import org.apache.kafka.common.serialization.Deserializer; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java index 197932a08ddca..adfb0d1b8911f 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java @@ -17,11 +17,11 @@ package org.apache.kafka.test; -import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.KeyValue; -import org.apache.kafka.stream.Window; -import org.apache.kafka.stream.kstream.internals.FilteredIterator; -import org.apache.kafka.stream.processor.Stamped; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.kstream.internals.FilteredIterator; +import org.apache.kafka.streaming.processor.Stamped; import java.util.Iterator; import java.util.LinkedList; From ac028673e8b5f360efe8788861ae171a8ae41167 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 17 Aug 2015 16:53:49 -0700 Subject: [PATCH 164/275] move some classes to internals, hide Chooser / RecordQueue / StampedRecords / TimestampTracker from user facing APIs --- ...StreamProcess.java => KafkaStreaming.java} | 19 ++++++----- ...cessorConfig.java => StreamingConfig.java} | 6 ++-- .../kafka/streaming/examples/KStreamJob.java | 8 ++--- .../streaming/examples/SimpleProcessJob.java | 8 ++--- .../examples/StatefulProcessJob.java | 8 ++--- .../internals}/SlidingWindow.java | 5 ++- .../processor/{ => internals}/Chooser.java | 2 +- .../processor/internals/KStreamThread.java | 20 ++++++------ .../internals/MinTimestampTracker.java | 3 -- .../internals/ProcessorContextImpl.java | 32 +++++++++---------- .../internals/PunctuationSchedule.java | 1 - .../{ => internals}/RecordQueue.java | 4 ++- .../processor/internals/RecordQueueImpl.java | 3 -- .../internals/RoundRobinChooser.java | 3 -- .../processor/{ => internals}/Stamped.java | 2 +- .../{ => internals}/StampedRecord.java | 2 +- .../processor/internals/StreamGroup.java | 3 -- .../processor/internals/TimeBasedChooser.java | 3 -- .../{ => internals}/TimestampTracker.java | 2 +- .../streaming/MinTimestampTrackerTest.java | 2 +- .../apache/kafka/test/UnlimitedWindow.java | 2 +- 21 files changed, 61 insertions(+), 77 deletions(-) rename stream/src/main/java/org/apache/kafka/streaming/{KStreamProcess.java => KafkaStreaming.java} (88%) rename stream/src/main/java/org/apache/kafka/streaming/{processor/ProcessorConfig.java => StreamingConfig.java} (98%) rename stream/src/main/java/org/apache/kafka/streaming/{ => kstream/internals}/SlidingWindow.java (96%) rename stream/src/main/java/org/apache/kafka/streaming/processor/{ => internals}/Chooser.java (94%) rename stream/src/main/java/org/apache/kafka/streaming/processor/{ => internals}/RecordQueue.java (91%) rename stream/src/main/java/org/apache/kafka/streaming/processor/{ => internals}/Stamped.java (95%) rename stream/src/main/java/org/apache/kafka/streaming/processor/{ => internals}/StampedRecord.java (96%) rename stream/src/main/java/org/apache/kafka/streaming/processor/{ => internals}/TimestampTracker.java (96%) diff --git a/stream/src/main/java/org/apache/kafka/streaming/KStreamProcess.java b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java similarity index 88% rename from stream/src/main/java/org/apache/kafka/streaming/KStreamProcess.java rename to stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java index dd53ac51aed50..1569e90bed692 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/KStreamProcess.java +++ b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java @@ -19,7 +19,6 @@ import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.processor.internals.KStreamThread; -import org.apache.kafka.streaming.processor.ProcessorConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,7 +29,7 @@ * sends output to zero or more output topics. *

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

* This streaming instance will co-ordinate with any other instances (whether in this same process, on other processes @@ -38,7 +37,7 @@ * consumed. If instances are added or die, the corresponding {@link KStreamThread} instances will be shutdown or * started in the appropriate processes to balance processing load. *

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

* A simple example might look like this: @@ -48,14 +47,14 @@ * properties config = new properties(props); * config.processor(ExampleStreamProcessor.class); * config.serialization(new StringSerializer(), new StringDeserializer()); - * KStreamProcess container = new KStreamProcess(new MyKStreamTopology(), config); + * KafkaStreaming container = new KafkaStreaming(new MyKStreamTopology(), config); * container.run(); * * */ -public class KStreamProcess implements Runnable { +public class KafkaStreaming implements Runnable { - private static final Logger log = LoggerFactory.getLogger(KStreamProcess.class); + private static final Logger log = LoggerFactory.getLogger(KafkaStreaming.class); // Container States private static final int CREATED = 0; @@ -69,16 +68,16 @@ public class KStreamProcess implements Runnable { private final File stateDir; - public KStreamProcess(TopologyBuilder builder, ProcessorConfig config) throws Exception { - if (config.getClass(ProcessorConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG) == null) + public KafkaStreaming(TopologyBuilder builder, StreamingConfig config) throws Exception { + if (config.getClass(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG) == null) throw new NullPointerException("timestamp extractor is missing"); - this.threads = new KStreamThread[config.getInt(ProcessorConfig.NUM_STREAM_THREADS_CONFIG)]; + this.threads = new KStreamThread[config.getInt(StreamingConfig.NUM_STREAM_THREADS_CONFIG)]; for (int i = 0; i < this.threads.length; i++) { this.threads[i] = new KStreamThread(builder, config); } - this.stateDir = new File(config.getString(ProcessorConfig.STATE_DIR_CONFIG)); + this.stateDir = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG)); } /** diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorConfig.java b/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorConfig.java rename to stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java index 80802d20ed238..5b59197388dc3 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorConfig.java +++ b/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streaming; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; @@ -27,7 +27,7 @@ import java.util.Map; import java.util.Properties; -public class ProcessorConfig extends AbstractConfig { +public class StreamingConfig extends AbstractConfig { private static final ConfigDef CONFIG; @@ -148,7 +148,7 @@ public class ProcessorConfig extends AbstractConfig { TIMESTAMP_EXTRACTOR_CLASS_DOC); } - public ProcessorConfig(Map props) { + public StreamingConfig(Map props) { super(CONFIG, props); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java index d552fec139635..9246998c5964d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java @@ -21,8 +21,8 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.processor.ProcessorConfig; -import org.apache.kafka.streaming.KStreamProcess; +import org.apache.kafka.streaming.StreamingConfig; +import org.apache.kafka.streaming.KafkaStreaming; import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.KeyValueMapper; @@ -33,7 +33,7 @@ public class KStreamJob { public static void main(String[] args) throws Exception { - ProcessorConfig config = new ProcessorConfig(new Properties()); + StreamingConfig config = new StreamingConfig(new Properties()); KStreamBuilder builder = new KStreamBuilder(); StringSerializer stringSerializer = new StringSerializer(); @@ -72,7 +72,7 @@ public boolean apply(String key, Integer value) { streams[0].sendTo("topic2", stringSerializer, intSerializer); streams[1].sendTo("topic3", stringSerializer, intSerializer); - KStreamProcess kstream = new KStreamProcess(builder, config); + KafkaStreaming kstream = new KafkaStreaming(builder, config); kstream.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java index d9ac62e02e438..e65ab50fb8807 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java @@ -17,13 +17,13 @@ package org.apache.kafka.streaming.examples; +import org.apache.kafka.streaming.KafkaStreaming; import org.apache.kafka.streaming.processor.KafkaProcessor; import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.processor.ProcessorConfig; +import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.KStreamProcess; import java.util.Properties; @@ -57,13 +57,13 @@ public void close() { } public static void main(String[] args) throws Exception { - ProcessorConfig config = new ProcessorConfig(new Properties()); + StreamingConfig config = new StreamingConfig(new Properties()); TopologyBuilder builder = new TopologyBuilder(); builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); builder.addProcessor("PROCESS", MyProcessor.class, null, "SOURCE"); - KStreamProcess streaming = new KStreamProcess(builder, config); + KafkaStreaming streaming = new KafkaStreaming(builder, config); streaming.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java index 80286be3eff5b..d1fc9a2270692 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java @@ -19,10 +19,10 @@ import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.KStreamProcess; +import org.apache.kafka.streaming.KafkaStreaming; import org.apache.kafka.streaming.processor.KafkaProcessor; import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.processor.ProcessorConfig; +import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.state.Entry; import org.apache.kafka.streaming.state.InMemoryKeyValueStore; @@ -78,13 +78,13 @@ public void close() { } public static void main(String[] args) throws Exception { - ProcessorConfig config = new ProcessorConfig(new Properties()); + StreamingConfig config = new StreamingConfig(new Properties()); TopologyBuilder builder = new TopologyBuilder(); builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); builder.addProcessor("PROCESS", MyProcessor.class, null, "SOURCE"); - KStreamProcess streaming = new KStreamProcess(builder, config); + KafkaStreaming streaming = new KafkaStreaming(builder, config); streaming.run(); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/SlidingWindow.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/streaming/SlidingWindow.java rename to stream/src/main/java/org/apache/kafka/streaming/kstream/internals/SlidingWindow.java index 2098262cf40e3..2ed323aaf419b 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/SlidingWindow.java @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streaming; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.streaming.kstream.Window; import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.kstream.internals.FilteredIterator; -import org.apache.kafka.streaming.processor.Stamped; +import org.apache.kafka.streaming.processor.internals.Stamped; import java.util.HashMap; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/Chooser.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Chooser.java similarity index 94% rename from stream/src/main/java/org/apache/kafka/streaming/processor/Chooser.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/Chooser.java index 0647430f1c093..121ef11da30aa 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/Chooser.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Chooser.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streaming.processor.internals; public interface Chooser { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java index cf1991e0ad181..c666fc43cd066 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java @@ -21,7 +21,7 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.processor.ProcessorConfig; +import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.KafkaException; @@ -61,7 +61,7 @@ public class KStreamThread extends Thread { private final Metrics metrics; private final Time time; - private final ProcessorConfig config; + private final StreamingConfig config; private final File stateDir; private final long pollTimeMs; private final long commitTimeMs; @@ -90,7 +90,7 @@ public void onPartitionsRevoked(Consumer consumer, Collection) config.getConfiguredInstance(ProcessorConfig.KEY_DESERIALIZER_CLASS_CONFIG, Serializer.class), - (Serializer) config.getConfiguredInstance(ProcessorConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Serializer.class)); + (Serializer) config.getConfiguredInstance(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, Serializer.class), + (Serializer) config.getConfiguredInstance(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Serializer.class)); Consumer consumer = new KafkaConsumer<>(config.getConsumerProperties(), rebalanceCallback, @@ -116,11 +116,11 @@ public KStreamThread(TopologyBuilder builder, ProcessorConfig config) throws Exc new ByteArrayDeserializer()); this.ingestor = new IngestorImpl(consumer, topology.topics()); - this.stateDir = new File(this.config.getString(ProcessorConfig.STATE_DIR_CONFIG)); - this.pollTimeMs = config.getLong(ProcessorConfig.POLL_MS_CONFIG); - this.commitTimeMs = config.getLong(ProcessorConfig.COMMIT_INTERVAL_MS_CONFIG); - this.stateCleanupDelayMs = config.getLong(ProcessorConfig.STATE_CLEANUP_DELAY_MS_CONFIG); - this.totalRecordsToProcess = config.getLong(ProcessorConfig.TOTAL_RECORDS_TO_PROCESS); + this.stateDir = new File(this.config.getString(StreamingConfig.STATE_DIR_CONFIG)); + this.pollTimeMs = config.getLong(StreamingConfig.POLL_MS_CONFIG); + this.commitTimeMs = config.getLong(StreamingConfig.COMMIT_INTERVAL_MS_CONFIG); + this.stateCleanupDelayMs = config.getLong(StreamingConfig.STATE_CLEANUP_DELAY_MS_CONFIG); + this.totalRecordsToProcess = config.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); this.running = true; this.lastCommit = 0; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java index 1eb0ddb76a88d..3e94622f6612b 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java @@ -17,9 +17,6 @@ package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.streaming.processor.Stamped; -import org.apache.kafka.streaming.processor.TimestampTracker; - import java.util.LinkedList; /** diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index 5fafa36381461..6a99feb38c712 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -19,14 +19,6 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.streaming.processor.KafkaSource; -import org.apache.kafka.streaming.processor.ProcessorConfig; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RecordCollector; -import org.apache.kafka.streaming.processor.StateStore; -import org.apache.kafka.streaming.processor.KafkaProcessor; -import org.apache.kafka.streaming.processor.RestoreFunc; -import org.apache.kafka.streaming.processor.TimestampExtractor; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; @@ -34,6 +26,14 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streaming.processor.KafkaSource; +import org.apache.kafka.streaming.StreamingConfig; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.RecordCollector; +import org.apache.kafka.streaming.processor.StateStore; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.RestoreFunc; +import org.apache.kafka.streaming.processor.TimestampExtractor; import org.slf4j.Logger; @@ -72,7 +72,7 @@ public ProcessorContextImpl(int id, Ingestor ingestor, ProcessorTopology topology, RecordCollectorImpl collector, - ProcessorConfig config, + StreamingConfig config, Metrics metrics) throws IOException { this.id = id; this.metrics = metrics; @@ -85,15 +85,15 @@ public ProcessorContextImpl(int id, throw new IllegalArgumentException("topic not subscribed: " + topic); } - this.keySerializer = config.getConfiguredInstance(ProcessorConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class); - this.valSerializer = config.getConfiguredInstance(ProcessorConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class); - this.keyDeserializer = config.getConfiguredInstance(ProcessorConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class); - this.valDeserializer = config.getConfiguredInstance(ProcessorConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + 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); - TimestampExtractor extractor = config.getConfiguredInstance(ProcessorConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); - int bufferedRecordsPerPartition = config.getInt(ProcessorConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + TimestampExtractor extractor = config.getConfiguredInstance(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); + int bufferedRecordsPerPartition = config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); - File stateFile = new File(config.getString(ProcessorConfig.STATE_DIR_CONFIG), Integer.toString(id)); + File stateFile = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG), Integer.toString(id)); Consumer restoreConsumer = new KafkaConsumer<>(config.getConsumerProperties(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java index 92ea5581bf8d1..6f67e3e09dfe0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java @@ -18,7 +18,6 @@ package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.streaming.processor.Punctuator; -import org.apache.kafka.streaming.processor.Stamped; public class PunctuationSchedule extends Stamped { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/RecordQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java similarity index 91% rename from stream/src/main/java/org/apache/kafka/streaming/processor/RecordQueue.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java index 0aca77ed0d04b..0f342837b313c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java @@ -15,9 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.internals.StampedRecord; /** * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java index ad4b460c6c7ee..c8700c6b42fd0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java @@ -20,9 +20,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.streaming.processor.KafkaProcessor; import org.apache.kafka.streaming.processor.KafkaSource; -import org.apache.kafka.streaming.processor.RecordQueue; -import org.apache.kafka.streaming.processor.StampedRecord; -import org.apache.kafka.streaming.processor.TimestampTracker; import org.apache.kafka.common.TopicPartition; import java.util.ArrayDeque; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java index 97903fd925002..744ca8cf713bb 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java @@ -17,9 +17,6 @@ package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.streaming.processor.Chooser; -import org.apache.kafka.streaming.processor.RecordQueue; - import java.util.ArrayDeque; public class RoundRobinChooser implements Chooser { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/Stamped.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/processor/Stamped.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java index a174b953092c2..7a078b681dd4a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/Stamped.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streaming.processor.internals; public class Stamped implements Comparable { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/StampedRecord.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StampedRecord.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/streaming/processor/StampedRecord.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/StampedRecord.java index 96b5e400fecee..e294c365f6346 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/StampedRecord.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StampedRecord.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java index d8fdd6b99c80c..9bef5bdf16540 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java @@ -18,12 +18,9 @@ package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streaming.processor.Chooser; import org.apache.kafka.streaming.processor.KafkaSource; import org.apache.kafka.streaming.processor.Punctuator; import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RecordQueue; -import org.apache.kafka.streaming.processor.StampedRecord; import org.apache.kafka.streaming.processor.TimestampExtractor; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java index 6889a3e59a6f9..ee8f41c1e2d0f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java @@ -18,9 +18,6 @@ package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.streaming.processor.Chooser; -import org.apache.kafka.streaming.processor.RecordQueue; - import java.util.Comparator; import java.util.PriorityQueue; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/streaming/processor/TimestampTracker.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java index 87e9710e5fd4f..80fc20898ad09 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TimestampTracker.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streaming.processor.internals; /** * TimestampTracker is a helper class for a sliding window implementation. diff --git a/stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java index bc663d418a2b3..8346b78447fdd 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java @@ -20,7 +20,7 @@ import static org.junit.Assert.assertEquals; import org.apache.kafka.streaming.processor.internals.MinTimestampTracker; -import org.apache.kafka.streaming.processor.Stamped; +import org.apache.kafka.streaming.processor.internals.Stamped; import org.junit.Test; public class MinTimestampTrackerTest { diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java index adfb0d1b8911f..bea636167e11a 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java @@ -21,7 +21,7 @@ import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.Window; import org.apache.kafka.streaming.kstream.internals.FilteredIterator; -import org.apache.kafka.streaming.processor.Stamped; +import org.apache.kafka.streaming.processor.internals.Stamped; import java.util.Iterator; import java.util.LinkedList; From 3cee6abba24f2a3ff26cf45ae7c8c33b4fde30b1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 18 Aug 2015 14:34:48 -0700 Subject: [PATCH 165/275] Use reflection to clone KafkaProcessor --- .../kafka/streaming/kstream/KStream.java | 7 ++- .../kstream/internals/KStreamBranch.java | 2 +- .../kstream/internals/KStreamFilter.java | 2 +- .../kstream/internals/KStreamFlatMap.java | 2 +- .../internals/KStreamFlatMapValues.java | 2 +- .../kstream/internals/KStreamImpl.java | 24 +++++---- .../kstream/internals/KStreamMap.java | 2 +- .../kstream/internals/KStreamMapValues.java | 2 +- .../kstream/internals/KStreamProcessor.java | 50 +++++++++++++++++++ .../kstream/internals/KStreamSend.java | 2 +- .../streaming/processor/KafkaProcessor.java | 11 ++-- 11 files changed, 80 insertions(+), 26 deletions(-) create mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java index 9256745325de3..04ac6ef4a78df 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java @@ -17,10 +17,9 @@ package org.apache.kafka.streaming.kstream; -import org.apache.kafka.streaming.processor.KafkaProcessor; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.KafkaProcessor; /** * KStream is an abstraction of a stream of key-value pairs. @@ -134,7 +133,7 @@ public interface KStream { /** * Processes all elements in this stream by applying a processor. * - * @param processorClass the class of Processor + * @param processor the class of Processor */ - KStream process(Class> processorClass, ProcessorMetadata config); + KStream process(KafkaProcessor processor); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java index 1b8aba3b6729e..229e07bb6d509 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java @@ -30,7 +30,7 @@ class KStreamBranch extends KafkaProcessor { public KStreamBranch(String name, ProcessorMetadata config) { super(name, config); - if (this.config() == null) + if (this.metadata() == null) throw new IllegalStateException("ProcessorMetadata should be specified."); this.predicates = (Predicate[]) config.value(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java index 58daa2c1c62ab..778abf081083e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java @@ -44,7 +44,7 @@ public PredicateOut(Predicate predicate, boolean filterOut) { public KStreamFilter(String name, ProcessorMetadata config) { super(name, config); - if (this.config() == null) + if (this.metadata() == null) throw new IllegalStateException("ProcessorMetadata should be specified."); this.predicateOut = (PredicateOut) config.value(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java index 7348bd69d6a27..0d420a5c52f3a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java @@ -30,7 +30,7 @@ class KStreamFlatMap extends KafkaProcessor { KStreamFlatMap(String name, ProcessorMetadata config) { super(name, config); - if (this.config() == null) + if (this.metadata() == null) throw new IllegalStateException("ProcessorMetadata should be specified."); this.mapper = (KeyValueMapper>) config.value(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java index 7c4fb538433c6..99a1ba2f69866 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java @@ -29,7 +29,7 @@ class KStreamFlatMapValues extends KafkaProcessor { KStreamFlatMapValues(String name, ProcessorMetadata config) { super(name, config); - if (this.config() == null) + if (this.metadata() == null) throw new IllegalStateException("ProcessorMetadata should be specified."); this.mapper = (ValueMapper>) config.value(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index eaf5967d90f9a..618aa8c3916a2 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -17,12 +17,12 @@ package org.apache.kafka.streaming.kstream.internals; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.processor.KafkaProcessor; import org.apache.kafka.streaming.processor.ProcessorMetadata; import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.processor.KafkaSource; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.kstream.KStreamWindowed; import org.apache.kafka.streaming.kstream.KeyValueMapper; import org.apache.kafka.streaming.kstream.Predicate; @@ -53,6 +53,8 @@ public class KStreamImpl implements KStream { public static final String SOURCE_NAME = "KAFKA-SOURCE-"; + public static final String SEND_NAME = "KAFKA-SEND-"; + public static final AtomicInteger INDEX = new AtomicInteger(1); protected TopologyBuilder topology; @@ -151,11 +153,13 @@ public KStream through(String topic, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { - process(KStreamSend.class, new ProcessorMetadata("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer))); + String sendName = SEND_NAME + INDEX.getAndIncrement(); + + process(new KStreamSend(sendName, new ProcessorMetadata("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer)))); - String name = SOURCE_NAME + INDEX.getAndIncrement(); + String sourceName = SOURCE_NAME + INDEX.getAndIncrement(); - KafkaSource source = topology.addSource(name, keyDeserializer, valDeserializer, topic); + KafkaSource source = topology.addSource(sourceName, keyDeserializer, valDeserializer, topic); return new KStreamSource<>(topology, source); } @@ -163,16 +167,18 @@ public KStream through(String topic, @Override @SuppressWarnings("unchecked") public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { - process(KStreamSend.class, new ProcessorMetadata("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer))); + String name = SEND_NAME + INDEX.getAndIncrement(); + + process(new KStreamSend(name, new ProcessorMetadata("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer)))); } @SuppressWarnings("unchecked") @Override - public KStream process(final Class> clazz, ProcessorMetadata config) { + public KStream process(final KafkaProcessor processor) { String name = PROCESSOR_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, clazz, config, this.name); + topology.addProcessor(name, KStreamProcessor.class, new ProcessorMetadata("Processor", processor), this.name); - return new KStreamImpl(topology, name); + return new KStreamImpl<>(topology, name); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java index 0a4526e190ea1..4ab959cfe7b28 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java @@ -30,7 +30,7 @@ class KStreamMap extends KafkaProcessor { public KStreamMap(String name, ProcessorMetadata config) { super(name, config); - if (this.config() == null) + if (this.metadata() == null) throw new IllegalStateException("ProcessorMetadata should be specified."); this.mapper = (KeyValueMapper) config.value(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java index 778c4bb552721..77d88ebf11004 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java @@ -29,7 +29,7 @@ class KStreamMapValues extends KafkaProcessor { public KStreamMapValues(String name, ProcessorMetadata config) { super(name, config); - if (this.config() == null) + if (this.metadata() == null) throw new IllegalStateException("ProcessorMetadata should be specified."); this.mapper = (ValueMapper) config.value(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java new file mode 100644 index 0000000000000..2679b70743ce6 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java @@ -0,0 +1,50 @@ +/** + * 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.streaming.kstream.internals; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; +import org.apache.kafka.streaming.processor.ProcessorMetadata; + +import java.lang.reflect.InvocationTargetException; + +public class KStreamProcessor extends KafkaProcessor { + + private final Processor processor; + + @SuppressWarnings("unchecked") + public KStreamProcessor(String name, ProcessorMetadata metadata) { + super(name, metadata); + + if (this.metadata() != null) + throw new IllegalStateException("ProcessorMetadata should be null."); + + try { + this.processor = ((KafkaProcessor) metadata.value()).getClass().getDeclaredConstructor(String.class).newInstance(name); + + } catch (Exception e) { + throw new KafkaException(e); + } + } + + @Override + public void process(K1 key, V1 value) { + processor.process(key, value); + } +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java index 292d5f4d544f4..0da4f899bf2e0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java @@ -44,7 +44,7 @@ public TopicSer(String topic, Serializer keySerializer, Serializer implements Processor> parents; private final String name; - private final ProcessorMetadata config; + private final ProcessorMetadata metadata; public boolean initialized; @@ -34,9 +34,9 @@ public KafkaProcessor(String name) { this(name, null); } - public KafkaProcessor(String name, ProcessorMetadata config) { + public KafkaProcessor(String name, ProcessorMetadata metadata) { this.name = name; - this.config = config; + this.metadata = metadata; this.children = new ArrayList<>(); this.parents = new ArrayList<>(); @@ -48,11 +48,10 @@ public String name() { return name; } - public ProcessorMetadata config() { - return config; + public ProcessorMetadata metadata() { + return metadata; } - public List> children() { return children; } From 5c41e829bfe393fe85ab6c2cfe1a8d6439a14eb8 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 18 Aug 2015 16:59:52 -0700 Subject: [PATCH 166/275] Add SinkNode, unify context.send() with processor.forward() --- .../kafka/streaming/kstream/KStream.java | 3 +- .../streaming/kstream/KStreamBuilder.java | 10 +- .../kstream/internals/KStreamFilter.java | 10 +- .../kstream/internals/KStreamImpl.java | 9 +- .../kstream/internals/KStreamMap.java | 11 +- .../kstream/internals/KStreamProcessor.java | 20 +-- .../kstream/internals/KStreamSource.java | 8 +- .../streaming/processor/KafkaProcessor.java | 49 +------- .../kafka/streaming/processor/Processor.java | 6 +- .../streaming/processor/ProcessorContext.java | 2 + .../streaming/processor/TopologyBuilder.java | 88 +++++++++---- .../processor/internals/KStreamThread.java | 2 +- .../internals/ProcessorContextImpl.java | 14 ++- .../processor/internals/ProcessorNode.java | 81 ++++++++++++ .../internals/ProcessorTopology.java | 118 +++++++----------- .../processor/internals/RecordQueue.java | 2 +- .../processor/internals/RecordQueueImpl.java | 8 +- .../processor/internals/SinkNode.java | 66 ++++++++++ .../SourceNode.java} | 18 ++- .../processor/internals/StreamGroup.java | 21 +++- .../kafka/streaming/StreamGroupTest.java | 8 +- .../{MockSource.java => MockSourceNode.java} | 6 +- 22 files changed, 341 insertions(+), 219 deletions(-) create mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java create mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java rename stream/src/main/java/org/apache/kafka/streaming/processor/{KafkaSource.java => internals/SourceNode.java} (66%) rename stream/src/test/java/org/apache/kafka/test/{MockSource.java => MockSourceNode.java} (85%) diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java index 04ac6ef4a78df..e5d300c08f74a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; /** * KStream is an abstraction of a stream of key-value pairs. @@ -135,5 +136,5 @@ public interface KStream { * * @param processor the class of Processor */ - KStream process(KafkaProcessor processor); + KStream process(Processor processor); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java index 30b85c6f9a18d..83777aea9185f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java @@ -18,11 +18,8 @@ package org.apache.kafka.streaming.kstream; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.internals.KStreamImpl; -import org.apache.kafka.streaming.processor.KafkaSource; import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; /** * KStreamBuilder is the class to create KStream instances. @@ -45,7 +42,10 @@ public KStreamBuilder() { */ @SuppressWarnings("unchecked") public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KafkaSource source = addSource(KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.getAndIncrement(), keyDeserializer, valDeserializer, topics); - return new KStreamSource<>(this, source); + String name = KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.getAndIncrement(); + + addSource(name, keyDeserializer, valDeserializer, topics); + + return new KStreamImpl<>(this, name); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java index 778abf081083e..d4d73c60e0bb9 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java @@ -21,7 +21,7 @@ import org.apache.kafka.streaming.kstream.Predicate; import org.apache.kafka.streaming.processor.ProcessorMetadata; -class KStreamFilter extends KafkaProcessor { +class KStreamFilter extends KafkaProcessor { private final PredicateOut predicateOut; @@ -41,20 +41,20 @@ public PredicateOut(Predicate predicate, boolean filterOut) { } @SuppressWarnings("unchecked") - public KStreamFilter(String name, ProcessorMetadata config) { - super(name, config); + public KStreamFilter(ProcessorMetadata metadata) { + super(metadata); if (this.metadata() == null) throw new IllegalStateException("ProcessorMetadata should be specified."); - this.predicateOut = (PredicateOut) config.value(); + this.predicateOut = (PredicateOut) metadata.value(); } @Override public void process(K key, V value) { if ((!predicateOut.filterOut && predicateOut.predicate.apply(key, value)) || (predicateOut.filterOut && !predicateOut.predicate.apply(key, value))) { - forward(key, value); + context.forward(key, value); } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index 618aa8c3916a2..255973f2950e7 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorMetadata; import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.processor.KafkaSource; import org.apache.kafka.streaming.kstream.KStreamWindowed; import org.apache.kafka.streaming.kstream.KeyValueMapper; import org.apache.kafka.streaming.kstream.Predicate; @@ -159,9 +158,9 @@ public KStream through(String topic, String sourceName = SOURCE_NAME + INDEX.getAndIncrement(); - KafkaSource source = topology.addSource(sourceName, keyDeserializer, valDeserializer, topic); + topology.addSource(sourceName, keyDeserializer, valDeserializer, topic); - return new KStreamSource<>(topology, source); + return new KStreamImpl<>(topology, sourceName); } @Override @@ -174,7 +173,7 @@ public void sendTo(String topic, Serializer keySerializer, Serializer valS @SuppressWarnings("unchecked") @Override - public KStream process(final KafkaProcessor processor) { + public KStream process(final Processor processor) { String name = PROCESSOR_NAME + INDEX.getAndIncrement(); topology.addProcessor(name, KStreamProcessor.class, new ProcessorMetadata("Processor", processor), this.name); diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java index 4ab959cfe7b28..f6ca1fe35f77a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java @@ -20,25 +20,26 @@ import org.apache.kafka.streaming.processor.KafkaProcessor; import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.processor.ProcessorMetadata; -class KStreamMap extends KafkaProcessor { +class KStreamMap extends KafkaProcessor { private final KeyValueMapper mapper; @SuppressWarnings("unchecked") - public KStreamMap(String name, ProcessorMetadata config) { - super(name, config); + public KStreamMap(ProcessorMetadata metadata) { + super(metadata); if (this.metadata() == null) throw new IllegalStateException("ProcessorMetadata should be specified."); - this.mapper = (KeyValueMapper) config.value(); + this.mapper = (KeyValueMapper) metadata.value(); } @Override public void process(K1 key, V1 value) { KeyValue newPair = mapper.apply(key, value); - forward(newPair.key, newPair.value); + context.forward(newPair.key, newPair.value); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java index 2679b70743ce6..2e1826ec3e795 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java @@ -17,34 +17,26 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.streaming.processor.KafkaProcessor; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorMetadata; -import java.lang.reflect.InvocationTargetException; +public class KStreamProcessor extends KafkaProcessor { -public class KStreamProcessor extends KafkaProcessor { - - private final Processor processor; + private final Processor processor; @SuppressWarnings("unchecked") - public KStreamProcessor(String name, ProcessorMetadata metadata) { - super(name, metadata); + public KStreamProcessor(ProcessorMetadata metadata) { + super(metadata); if (this.metadata() != null) throw new IllegalStateException("ProcessorMetadata should be null."); - try { - this.processor = ((KafkaProcessor) metadata.value()).getClass().getDeclaredConstructor(String.class).newInstance(name); - - } catch (Exception e) { - throw new KafkaException(e); - } + this.processor = (Processor) metadata.value(); } @Override - public void process(K1 key, V1 value) { + public void process(K key, V value) { processor.process(key, value); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java index 3ad77d0219117..e292ab4ac74a8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java @@ -17,19 +17,19 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.internals.SourceNode; import org.apache.kafka.streaming.processor.TopologyBuilder; public class KStreamSource extends KStreamImpl { - private KafkaProcessor source; + private SourceNode source; - public KStreamSource(TopologyBuilder topology, KafkaProcessor source) { + public KStreamSource(TopologyBuilder topology, SourceNode source) { super(topology, source.name()); this.source = source; } - public KafkaProcessor source() { + public SourceNode source() { return source; } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java b/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java index b2c92afed7be0..cc0cf943a9848 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java @@ -17,61 +17,20 @@ package org.apache.kafka.streaming.processor; -import java.util.ArrayList; -import java.util.List; +public abstract class KafkaProcessor implements Processor, Punctuator { -public abstract class KafkaProcessor implements Processor, Punctuator { - - private final List> children; - private final List> parents; - - private final String name; private final ProcessorMetadata metadata; - public boolean initialized; + protected ProcessorContext context; - public KafkaProcessor(String name) { - this(name, null); - } - - public KafkaProcessor(String name, ProcessorMetadata metadata) { - this.name = name; + public KafkaProcessor(ProcessorMetadata metadata) { this.metadata = metadata; - - this.children = new ArrayList<>(); - this.parents = new ArrayList<>(); - - this.initialized = false; - } - - public String name() { - return name; } public ProcessorMetadata metadata() { return metadata; } - public List> children() { - return children; - } - - public List> parents() { - return parents; - } - - public final void chain(KafkaProcessor child) { - child.parents.add(this); - children.add(child); - } - - @Override - public final void forward(K2 key, V2 value) { - for (KafkaProcessor child : children) { - child.process(key, value); - } - } - /* Following functions can be overridden by users */ @Override @@ -81,7 +40,7 @@ public void punctuate(long streamTime) { @Override public void init(ProcessorContext context) { - // do nothing + this.context = context; } @Override diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java b/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java index 9d6f47bf331a8..c74fbc8ab370e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java @@ -67,13 +67,11 @@ public interface ProcessorContext { package org.apache.kafka.streaming.processor; -public interface Processor { +public interface Processor { void init(ProcessorContext context); - void process(K1 key, V1 value); - - void forward(K2 key, V2 value); + void process(K key, V value); void close(); >>>>>>> wip diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java index bbf58d7188b53..787d91c8f2e5f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java @@ -96,6 +96,8 @@ public interface ProcessorContext { */ void flush(); + void forward(K key, V value); + void send(String topic, Object key, Object value); void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 2cab55ee9c289..d7d55620f2ff4 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -19,7 +19,11 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streaming.processor.internals.ProcessorNode; import org.apache.kafka.streaming.processor.internals.ProcessorTopology; +import org.apache.kafka.streaming.processor.internals.SinkNode; +import org.apache.kafka.streaming.processor.internals.SourceNode; import java.util.ArrayList; import java.util.HashMap; @@ -30,7 +34,9 @@ public class TopologyBuilder { private Map processorClasses = new HashMap<>(); private Map sourceClasses = new HashMap<>(); + private Map sinkClasses = new HashMap<>(); private Map topicsToSourceNames = new HashMap<>(); + private Map topicsToSinkNames = new HashMap<>(); private Map> parents = new HashMap<>(); private Map> children = new HashMap<>(); @@ -55,12 +61,20 @@ private SourceClazz(Deserializer keyDeserializer, Deserializer valDeserializer) } } + private class SinkClazz { + public Serializer keySerializer; + public Serializer valSerializer; + + private SinkClazz(Serializer keySerializer, Serializer valSerializer) { + this.keySerializer = keySerializer; + this.valSerializer = valSerializer; + } + } + public TopologyBuilder() {} @SuppressWarnings("unchecked") - public final KafkaSource addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - KafkaSource source = new KafkaSource(name, keyDeserializer, valDeserializer); - + public final void addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { for (String topic : topics) { if (topicsToSourceNames.containsKey(topic)) throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); @@ -69,8 +83,17 @@ public final KafkaSource addSource(String name, Deserializer keyDeserializer, De } sourceClasses.put(name, new SourceClazz(keyDeserializer, valDeserializer)); + } - return source; + public final void addSink(String name, Serializer keySerializer, Serializer valSerializer, String... topics) { + for (String topic : topics) { + if (topicsToSinkNames.containsKey(topic)) + throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); + + topicsToSinkNames.put(topic, name); + } + + sinkClasses.put(name, new SinkClazz(keySerializer, valSerializer)); } public final void addProcessor(String name, Class processorClass, ProcessorMetadata config, String... parentNames) { @@ -102,28 +125,34 @@ public final void addProcessor(String name, Class proc */ @SuppressWarnings("unchecked") public ProcessorTopology build() { - Map processorMap = new HashMap<>(); - Map topicSourceMap = new HashMap<>(); + Map processorMap = new HashMap<>(); + Map topicSourceMap = new HashMap<>(); + Map topicSinkMap = new HashMap<>(); // create sources - try { - for (String name : sourceClasses.keySet()) { - Deserializer keyDeserializer = sourceClasses.get(name).keyDeserializer; - Deserializer valDeserializer = sourceClasses.get(name).valDeserializer; - KafkaSource source = new KafkaSource(name, keyDeserializer, valDeserializer); - processorMap.put(name, source); - } - } catch (Exception e) { - throw new KafkaException("KafkaProcessor(String) constructor failed: this should not happen."); + for (String name : sourceClasses.keySet()) { + Deserializer keyDeserializer = sourceClasses.get(name).keyDeserializer; + Deserializer valDeserializer = sourceClasses.get(name).valDeserializer; + SourceNode node = new SourceNode(name, keyDeserializer, valDeserializer); + processorMap.put(name, node); } - // create processors + // create sinks + for (String name : sinkClasses.keySet()) { + Serializer keySerializer = sinkClasses.get(name).keySerializer; + Serializer valSerializer = sinkClasses.get(name).valSerializer; + SinkNode node = new SinkNode(name, keySerializer, valSerializer); + processorMap.put(name, node); + } + + // create normal processors try { for (String name : processorClasses.keySet()) { ProcessorMetadata metadata = processorClasses.get(name).metadata; Class processorClass = processorClasses.get(name).clazz; - KafkaProcessor processor = processorClass.getConstructor(String.class, ProcessorMetadata.class).newInstance(name, metadata); - processorMap.put(name, processor); + KafkaProcessor processor = processorClass.getConstructor(ProcessorMetadata.class).newInstance(metadata); + ProcessorNode node = new ProcessorNode(name, processor); + processorMap.put(name, node); } } catch (Exception e) { throw new KafkaException("KafkaProcessor(String) constructor failed: this should not happen."); @@ -131,18 +160,25 @@ public ProcessorTopology build() { // construct topics to sources map for (String topic : topicsToSourceNames.keySet()) { - KafkaSource source = (KafkaSource) processorMap.get(topicsToSourceNames.get(topic)); - topicSourceMap.put(topic, source); + SourceNode node = (SourceNode) processorMap.get(topicsToSourceNames.get(topic)); + topicSourceMap.put(topic, node); + } + + // construct topics to sinks map + for (String topic : topicsToSinkNames.keySet()) { + SinkNode node = (SinkNode) processorMap.get(topicsToSourceNames.get(topic)); + node.addTopic(topic); + topicSinkMap.put(topic, node); } - for (KafkaProcessor processor : processorMap.values()) { - // chain children to this processor - for (String child : children.get(processor.name())) { - KafkaProcessor childProcessor = processorMap.get(child); - processor.chain(childProcessor); + // chain children to parents to build the DAG + for (ProcessorNode node : processorMap.values()) { + for (String child : children.get(node.name())) { + ProcessorNode childNode = processorMap.get(child); + node.chain(childNode); } } - return new ProcessorTopology(processorMap, topicSourceMap); + return new ProcessorTopology(processorMap, topicSourceMap, topicSinkMap); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java index c666fc43cd066..005307b1d69c0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java @@ -114,7 +114,7 @@ public KStreamThread(TopologyBuilder builder, StreamingConfig config) throws Exc rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); - this.ingestor = new IngestorImpl(consumer, topology.topics()); + this.ingestor = new IngestorImpl(consumer, topology.sourceTopics()); this.stateDir = new File(this.config.getString(StreamingConfig.STATE_DIR_CONFIG)); this.pollTimeMs = config.getLong(StreamingConfig.POLL_MS_CONFIG); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index 6a99feb38c712..d4de6f8302a4e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.processor.KafkaSource; import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.processor.RecordCollector; @@ -80,7 +79,7 @@ public ProcessorContextImpl(int id, this.topology = topology; this.collector = collector; - for (String topic : this.topology.topics()) { + for (String topic : this.topology.sourceTopics()) { if (!ingestor.topics().contains(topic)) throw new IllegalArgumentException("topic not subscribed: " + topic); } @@ -106,7 +105,7 @@ public ProcessorContextImpl(int id, public void addPartition(TopicPartition partition) { // update the partition -> source stream map - KafkaSource source = topology.source(partition.topic()); + SourceNode source = topology.source(partition.topic()); this.streamGroup.addPartition(partition, source); this.ingestor.addPartitionStreamToGroup(this.streamGroup, partition); @@ -231,6 +230,15 @@ public long timestamp() { return streamGroup.record().timestamp; } + @Override + @SuppressWarnings("unchecked") + public void forward(K key, V value) { + for (ProcessorNode childNode : (List>) streamGroup.node().children()) { + streamGroup.setNode(childNode); + childNode.process(key, value); + } + } + @Override public void send(String topic, Object key, Object value) { collector.send(new ProducerRecord<>(topic, key, value)); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java new file mode 100644 index 0000000000000..efdf11add7637 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java @@ -0,0 +1,81 @@ +/** + * 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.streaming.processor.internals; + +import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.ProcessorContext; + +import java.util.ArrayList; +import java.util.List; + +public class ProcessorNode { + + private final List> children; + private final List> parents; + + private final String name; + private final KafkaProcessor processor; + + public boolean initialized; + + public ProcessorNode(String name) { + this(name, null); + } + + public ProcessorNode(String name, KafkaProcessor processor) { + this.name = name; + this.processor = processor; + this.parents = new ArrayList<>(); + this.children = new ArrayList<>(); + + this.initialized = false; + } + + public String name() { + return name; + } + + public KafkaProcessor processor() { + return processor; + } + + public List> parents() { + return parents; + } + + public List> children() { + return children; + } + + public final void chain(ProcessorNode child) { + child.parents.add(this); + children.add(child); + } + + public void init(ProcessorContext context) { + processor.init(context); + } + + public void process(K1 key, V1 value) { + processor.process(key, value); + } + + public void close() { + processor.close(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java index a80f2e55f5321..364d3d640a5b0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java @@ -17,110 +17,76 @@ package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.streaming.processor.KafkaProcessor; -import org.apache.kafka.streaming.processor.KafkaSource; import org.apache.kafka.streaming.processor.ProcessorContext; -import java.util.ArrayDeque; -import java.util.Deque; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; public class ProcessorTopology { - private Map processors = new HashMap<>(); - private Map topicSources = new HashMap<>(); + private Map processors = new HashMap<>(); + private Map sourceTopics = new HashMap<>(); + private Map sinkTopics = new HashMap<>(); - public ProcessorTopology(Map processors, Map topicSources) { + public ProcessorTopology(Map processors, + Map sourceTopics, + Map sinkTopics) { this.processors = processors; - this.topicSources = topicSources; + this.sourceTopics = sourceTopics; + this.sinkTopics = sinkTopics; } - public Set sources() { - Set sources = new HashSet<>(); - for (KafkaSource source : this.topicSources.values()) { - sources.add(source); - } - - return sources; + public Set sourceTopics() { + return sourceTopics.keySet(); } - public Set topics() { - return topicSources.keySet(); + public Set sinkTopics() { + return sinkTopics.keySet(); } - public KafkaSource source(String topic) { - return topicSources.get(topic); - } - - public Deserializer keyDeser(String topic) { - KafkaSource source = topicSources.get(topic); - - if (source == null) - throw new IllegalStateException("The topic " + topic + " is unknown."); - - return source.keyDeserializer; + public SourceNode source(String topic) { + return sourceTopics.get(topic); } - public Deserializer valueDeser(String topic) { - KafkaSource source = topicSources.get(topic); - - if (source == null) - throw new IllegalStateException("The topic " + topic + " is unknown."); - - return source.valDeserializer; + public SinkNode sink(String topic) { + return sinkTopics.get(topic); } /** - * initialize the processors following the DAG ordering - * such that parents are always created and initialized before children + * Initialize the processors following the DAG reverse ordering + * such that parents are always initialized before children */ @SuppressWarnings("unchecked") - public final void init(ProcessorContext context) { - Deque deque = new ArrayDeque<>(); - + public void init(ProcessorContext context) { // initialize sources - for (String topic : topicSources.keySet()) { - KafkaSource source = topicSources.get(topic); - - source.init(context); - source.initialized = true; + for (String topic : sourceTopics.keySet()) { + SourceNode source = sourceTopics.get(topic); - // put source children to be traversed first - for (KafkaProcessor childProcessor : (List) source.children()) { - deque.addLast(childProcessor); - } + init(source, context); } + } - // traverse starting at the sources' children, and initialize the processor - // if 1) it has not been initialized, 2) all its parents are initialized - while (!deque.isEmpty()) { - KafkaProcessor processor = deque.pollFirst(); - - if (processor.initialized) - continue; - - boolean parentsInitialized = true; - - for (KafkaProcessor parentProcessor : (List) processor.parents()) { - if (!parentProcessor.initialized) { - parentsInitialized = false; - break; - } + /** + * Initialize the current processor node by first initializing + * its parent nodes first, then the processor itself + */ + @SuppressWarnings("unchecked") + private void init(ProcessorNode node, ProcessorContext context) { + for (ProcessorNode parentNode : (List) node.parents()) { + if (!parentNode.initialized) { + init(parentNode, context); } + } - if (parentsInitialized) { - processor.init(context); - processor.initialized = true; + node.init(context); + node.initialized = true; - // put source children to be traversed first - for (KafkaProcessor childProcessor : (List) processor.children()) { - deque.addLast(childProcessor); - } + // try to initialize its children + for (ProcessorNode childNode : (List) node.children()) { + if (!childNode.initialized) { + init(childNode, context); } } } @@ -128,11 +94,11 @@ public final void init(ProcessorContext context) { public final void close() { // close the processors // TODO: do we need to follow the DAG ordering - for (KafkaProcessor processor : processors.values()) { - processor.close(); + for (ProcessorNode processorNode : processors.values()) { + processorNode.close(); } processors.clear(); - topicSources.clear(); + sourceTopics.clear(); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java index 0f342837b313c..9d82531a37d95 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java @@ -38,7 +38,7 @@ public interface RecordQueue { * * @return KafkaProcessor */ - KafkaProcessor source(); + ProcessorNode source(); /** * Adds a StampedRecord to the queue diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java index c8700c6b42fd0..2787af9ca3b5c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java @@ -18,14 +18,12 @@ package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streaming.processor.KafkaProcessor; -import org.apache.kafka.streaming.processor.KafkaSource; import org.apache.kafka.common.TopicPartition; import java.util.ArrayDeque; public class RecordQueueImpl implements RecordQueue { - private final KafkaSource source; + private final SourceNode source; private final TopicPartition partition; private final ArrayDeque queue = new ArrayDeque<>(); @@ -39,14 +37,14 @@ public class RecordQueueImpl implements RecordQueue { * @param source the instance of KStreamImpl that receives records * @param timestampTracker TimestampTracker */ - public RecordQueueImpl(TopicPartition partition, KafkaSource source, TimestampTracker> timestampTracker) { + public RecordQueueImpl(TopicPartition partition, SourceNode source, TimestampTracker> timestampTracker) { this.partition = partition; this.source = source; this.timestampTracker = timestampTracker; } @Override - public KafkaProcessor source() { + public ProcessorNode source() { return source; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java new file mode 100644 index 0000000000000..36ebd3d53fe37 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java @@ -0,0 +1,66 @@ +/** + * 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.streaming.processor.internals; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.RecordCollector; + +import java.util.ArrayList; +import java.util.List; + +public class SinkNode extends ProcessorNode { + + private final Serializer keySerializer; + private final Serializer valSerializer; + private final List topics; + + private ProcessorContext context; + + public SinkNode(String name, Serializer keySerializer, Serializer valSerializer) { + super(name); + + this.topics = new ArrayList<>(); + this.keySerializer = keySerializer; + this.valSerializer = valSerializer; + } + + public void addTopic(String topic) { + this.topics.add(topic); + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + @Override + public void process(K key, V value) { + // send to all the registered topics + RecordCollector collector = context.recordCollector(); + for (String topic : topics) { + collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); + } + } + + @Override + public void close() { + // do nothing + } +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaSource.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java similarity index 66% rename from stream/src/main/java/org/apache/kafka/streaming/processor/KafkaSource.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java index 902f14142e9fc..5634d76019d41 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaSource.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java @@ -15,16 +15,18 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.internals.ProcessorNode; -public class KafkaSource extends KafkaProcessor { +public class SourceNode extends ProcessorNode { - public Deserializer keyDeserializer; - public Deserializer valDeserializer; + public Deserializer keyDeserializer; + public Deserializer valDeserializer; - public KafkaSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer) { + public SourceNode(String name, Deserializer keyDeserializer, Deserializer valDeserializer) { super(name); this.keyDeserializer = keyDeserializer; @@ -38,11 +40,15 @@ public void init(ProcessorContext context) { @Override public void process(K key, V value) { - forward(key, value); + // just forward to all children + for (ProcessorNode childNode : this.children()) { + childNode.process(key, value); + } } @Override public void close() { // do nothing } + } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java index 9bef5bdf16540..bd2b928cbe6d2 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java @@ -18,7 +18,6 @@ package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streaming.processor.KafkaSource; import org.apache.kafka.streaming.processor.Punctuator; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.processor.TimestampExtractor; @@ -53,6 +52,7 @@ public class StreamGroup { private long streamTime = -1; private boolean commitRequested = false; private StampedRecord currRecord = null; + private ProcessorNode currNode = null; private volatile int buffered = 0; /** @@ -81,6 +81,14 @@ public StampedRecord record() { return currRecord; } + public ProcessorNode node() { + return currNode; + } + + public void setNode(ProcessorNode node) { + currNode = node; + } + public Set partitions() { return stash.keySet(); } @@ -92,7 +100,7 @@ public Set partitions() { * @param source the instance of KStreamImpl */ @SuppressWarnings("unchecked") - public void addPartition(TopicPartition partition, KafkaSource source) { + public void addPartition(TopicPartition partition, SourceNode source) { synchronized (this) { RecordQueue recordQueue = stash.get(partition); @@ -131,8 +139,8 @@ private void ingestNewRecords() { ConsumerRecord record = iterator.next(); // deserialize the raw record, extract the timestamp and put into the queue - Deserializer keyDeserializer = ((KafkaSource) recordQueue.source()).keyDeserializer; - Deserializer valDeserializer = ((KafkaSource) recordQueue.source()).valDeserializer; + Deserializer keyDeserializer = ((SourceNode) recordQueue.source()).keyDeserializer; + Deserializer valDeserializer = ((SourceNode) recordQueue.source()).valDeserializer; Object key = keyDeserializer.deserialize(record.topic(), record.key()); Object value = valDeserializer.deserialize(record.topic(), record.value()); @@ -187,10 +195,11 @@ public boolean process() { long trackedTimestamp = recordQueue.trackedTimestamp(); currRecord = recordQueue.next(); + currNode = recordQueue.source(); if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - recordQueue.source().process(currRecord.key(), currRecord.value()); + currNode.process(currRecord.key(), currRecord.value()); consumedOffsets.put(recordQueue.partition(), currRecord.offset()); // TODO: local state flush and downstream producer flush @@ -245,7 +254,7 @@ public void close() { stash.clear(); } - protected RecordQueue createRecordQueue(TopicPartition partition, KafkaSource source) { + protected RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { return new RecordQueueImpl(partition, source, new MinTimestampTracker>()); } diff --git a/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java index bcf14215327c3..94ed0a6646f91 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java @@ -28,7 +28,7 @@ import org.apache.kafka.streaming.processor.internals.TimeBasedChooser; import org.apache.kafka.test.MockIngestor; import org.apache.kafka.test.MockProcessorContext; -import org.apache.kafka.test.MockSource; +import org.apache.kafka.test.MockSourceNode; import org.junit.Test; import java.util.Arrays; @@ -106,9 +106,9 @@ public long extract(String topic, Object key, Object value) { TopicPartition partition1 = new TopicPartition("topic1", 1); TopicPartition partition2 = new TopicPartition("topic2", 1); - MockSource source1 = new MockSource(deserializer, deserializer); - MockSource source2 = new MockSource(deserializer, deserializer); - MockSource source3 = new MockSource(deserializer, deserializer); + MockSourceNode source1 = new MockSourceNode(deserializer, deserializer); + MockSourceNode source2 = new MockSourceNode(deserializer, deserializer); + MockSourceNode source3 = new MockSourceNode(deserializer, deserializer); streamGroup.addPartition(partition1, source1); mockIngestor.addPartitionStreamToGroup(streamGroup, partition1); diff --git a/stream/src/test/java/org/apache/kafka/test/MockSource.java b/stream/src/test/java/org/apache/kafka/test/MockSourceNode.java similarity index 85% rename from stream/src/test/java/org/apache/kafka/test/MockSource.java rename to stream/src/test/java/org/apache/kafka/test/MockSourceNode.java index 38c78c22001ef..cd29a317c4ff9 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockSource.java +++ b/stream/src/test/java/org/apache/kafka/test/MockSourceNode.java @@ -17,12 +17,12 @@ package org.apache.kafka.test; -import org.apache.kafka.streaming.processor.KafkaSource; +import org.apache.kafka.streaming.processor.internals.SourceNode; import org.apache.kafka.common.serialization.Deserializer; import java.util.ArrayList; -public class MockSource extends KafkaSource { +public class MockSourceNode extends SourceNode { public Deserializer keyDeserializer; public Deserializer valDeserializer; @@ -31,7 +31,7 @@ public class MockSource extends KafkaSource { public ArrayList keys = new ArrayList<>(); public ArrayList values = new ArrayList<>(); - public MockSource(Deserializer keyDeserializer, Deserializer valDeserializer) { + public MockSourceNode(Deserializer keyDeserializer, Deserializer valDeserializer) { super(keyDeserializer, valDeserializer); this.keyDeserializer = keyDeserializer; From 070ab00d8281da0ff023af84e9bdbc34b56367f7 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 19 Aug 2015 11:52:37 -0700 Subject: [PATCH 167/275] Fix flatMap by adding a KeyValueFlatMap --- .../kafka/streaming/kstream/KStream.java | 2 +- .../streaming/kstream/KeyValueFlatMap.java | 23 +++++++++++++++++++ .../kstream/internals/KStreamFlatMap.java | 20 ++++++++-------- .../kstream/internals/KStreamImpl.java | 5 ++-- .../processor/internals/SourceNode.java | 3 +-- 5 files changed, 38 insertions(+), 15 deletions(-) create mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueFlatMap.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java index e5d300c08f74a..271a89bc868c5 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java @@ -70,7 +70,7 @@ public interface KStream { * @param the value type of the new stream * @return KStream */ - KStream flatMap(KeyValueMapper> mapper); + KStream flatMap(KeyValueFlatMap mapper); /** * Creates a new stream by applying a mapper to all values of this stream and using the values in the resulting Iterable diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueFlatMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueFlatMap.java new file mode 100644 index 0000000000000..16c96dc21310d --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueFlatMap.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.kstream; + +public interface KeyValueFlatMap { + + Iterable> apply(K1 key, V1 value); +} \ No newline at end of file diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java index 0d420a5c52f3a..408c7cff3d8b2 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java @@ -18,29 +18,29 @@ package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.streaming.processor.KafkaProcessor; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.KeyValueFlatMap; -class KStreamFlatMap extends KafkaProcessor { +class KStreamFlatMap extends KafkaProcessor { - private final KeyValueMapper> mapper; + private final KeyValueFlatMap mapper; @SuppressWarnings("unchecked") - KStreamFlatMap(String name, ProcessorMetadata config) { - super(name, config); + KStreamFlatMap(ProcessorMetadata metadata) { + super(metadata); if (this.metadata() == null) throw new IllegalStateException("ProcessorMetadata should be specified."); - this.mapper = (KeyValueMapper>) config.value(); + this.mapper = (KeyValueFlatMap) metadata.value(); } @Override public void process(K1 key, V1 value) { - KeyValue> newPair = mapper.apply(key, value); - for (V2 v : newPair.value) { - forward(newPair.key, v); + Iterable> pairs = mapper.apply(key, value); + for (KeyValue pair : pairs) { + context.forward(pair.key, pair.value); } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index 255973f2950e7..619a0af77fe30 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streaming.kstream.KeyValueFlatMap; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorMetadata; import org.apache.kafka.streaming.processor.TopologyBuilder; @@ -102,10 +103,10 @@ public KStream mapValues(ValueMapper mapper) { @SuppressWarnings("unchecked") @Override - public KStream flatMap(KeyValueMapper> mapper) { + public KStream flatMap(KeyValueFlatMap mapper) { String name = FLATMAP_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFlatMap.class, new ProcessorMetadata("Mapper", mapper), this.name); + topology.addProcessor(name, KStreamFlatMap.class, new ProcessorMetadata("FlatMapper", mapper), this.name); return new KStreamImpl<>(topology, name); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java index 5634d76019d41..e86fe9c4d661d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.internals.ProcessorNode; public class SourceNode extends ProcessorNode { @@ -51,4 +50,4 @@ public void close() { // do nothing } -} +} \ No newline at end of file From 24f36c082ec2bc5a6d55810968ec2e183f1445d2 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 20 Aug 2015 16:05:04 -0700 Subject: [PATCH 168/275] WIP towards the stateless plan API --- .../kafka/streaming/KafkaStreaming.java | 16 +- .../processor/internals/Chooser.java | 28 -- .../processor/internals/PartitionGroup.java | 139 ++++++++++ .../internals/ProcessorContextImpl.java | 2 +- .../internals/ProcessorStateManager.java | 1 + .../processor/internals/RecordQueue.java | 70 +++-- .../processor/internals/RecordQueueImpl.java | 129 --------- .../internals/RoundRobinChooser.java | 45 ---- .../processor/internals/StreamGroup.java | 51 ++-- .../processor/internals/StreamTask.java | 230 ++++++++++++++++ .../{KStreamThread.java => StreamThread.java} | 100 +++---- .../processor/internals/TimeBasedChooser.java | 60 ----- .../internals => state}/OffsetCheckpoint.java | 2 +- .../kafka/streaming/StreamGroupTest.java | 2 - .../internals/FilteredIteratorTest.java | 95 +++++++ .../internals/KStreamBranchTest.java | 90 +++++++ .../internals/KStreamFilterTest.java | 84 ++++++ .../internals/KStreamFlatMapTest.java | 80 ++++++ .../internals/KStreamFlatMapValuesTest.java | 77 ++++++ .../streaming/internals/KStreamJoinTest.java | 249 ++++++++++++++++++ .../streaming/internals/KStreamMapTest.java | 73 +++++ .../internals/KStreamMapValuesTest.java | 71 +++++ .../internals/KStreamSourceTest.java | 59 +++++ .../internals/KStreamWindowedTest.java | 91 +++++++ .../internals/MinTimestampTrackerTest.java | 95 +++++++ .../streaming/internals/StreamGroupTest.java | 163 ++++++++++++ 26 files changed, 1744 insertions(+), 358 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/Chooser.java create mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java create mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java rename stream/src/main/java/org/apache/kafka/streaming/processor/internals/{KStreamThread.java => StreamThread.java} (81%) delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java rename stream/src/main/java/org/apache/kafka/streaming/{processor/internals => state}/OffsetCheckpoint.java (99%) create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/FilteredIteratorTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamBranchTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFilterTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapValuesTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamJoinTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapValuesTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamSourceTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/MinTimestampTrackerTest.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java index 1569e90bed692..35d1b751a02e3 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java +++ b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java @@ -18,7 +18,7 @@ package org.apache.kafka.streaming; import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.processor.internals.KStreamThread; +import org.apache.kafka.streaming.processor.internals.StreamThread; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,7 +34,7 @@ *

* 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 KStreamThread} instances will be shutdown or + * consumed. If instances are added or die, the corresponding {@link StreamThread} instances will be shutdown or * started in the appropriate processes to balance processing load. *

* Internally the {@link KafkaStreaming} instance contains a normal {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer} @@ -64,7 +64,7 @@ public class KafkaStreaming implements Runnable { private int state = CREATED; private final Object lock = new Object(); - private final KStreamThread[] threads; + private final StreamThread[] threads; private final File stateDir; @@ -72,9 +72,9 @@ public KafkaStreaming(TopologyBuilder builder, StreamingConfig config) throws Ex if (config.getClass(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG) == null) throw new NullPointerException("timestamp extractor is missing"); - this.threads = new KStreamThread[config.getInt(StreamingConfig.NUM_STREAM_THREADS_CONFIG)]; + this.threads = new StreamThread[config.getInt(StreamingConfig.NUM_STREAM_THREADS_CONFIG)]; for (int i = 0; i < this.threads.length; i++) { - this.threads[i] = new KStreamThread(builder, config); + this.threads[i] = new StreamThread(builder, config); } this.stateDir = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG)); @@ -90,7 +90,7 @@ public void run() { if (!stateDir.exists() && !stateDir.mkdirs()) throw new IllegalArgumentException("Failed to create state directory: " + stateDir.getAbsolutePath()); - for (KStreamThread thread : threads) thread.start(); + for (StreamThread thread : threads) thread.start(); log.info("Start-up complete"); } else { throw new IllegalStateException("This container was already started"); @@ -108,10 +108,10 @@ public void run() { if (state == STOPPING) { log.info("Shutting down the container"); - for (KStreamThread thread : threads) + for (StreamThread thread : threads) thread.close(); - for (KStreamThread thread : threads) { + for (StreamThread thread : threads) { try { thread.join(); } catch (InterruptedException ex) { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Chooser.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Chooser.java deleted file mode 100644 index 121ef11da30aa..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Chooser.java +++ /dev/null @@ -1,28 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor.internals; - -public interface Chooser { - - void add(RecordQueue queue); - - RecordQueue next(); - - void close(); - -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java new file mode 100644 index 0000000000000..c5a530b6666ea --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java @@ -0,0 +1,139 @@ +/** + * 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.streaming.processor.internals; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; + +import java.util.Comparator; +import java.util.Map; +import java.util.PriorityQueue; + +/** + * A PartitionGroup is composed from a set of partitions. + */ +public class PartitionGroup { + + private final Map partitionQueues; + + private final PriorityQueue queuesByTime; + + private volatile int totalBuffered; + + public PartitionGroup(Map partitionQueues) { + + this.partitionQueues = partitionQueues; + this.queuesByTime = new PriorityQueue<>(new Comparator() { + 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; + } + }); + + totalBuffered = 0; + } + + /** + * Get the next record from the partition with the lowest timestamp to be processed + */ + public StampedRecord nextRecord() { + + // Get the partition with the lowest timestamp. + RecordQueue recordQueue = queuesByTime.poll(); + + // Get the first record from this partition's queue. + StampedRecord record = recordQueue.next(); + + totalBuffered--; + + // Update the partition's timestamp and re-order it with other partitions. + if (recordQueue.size() > 0) { + queuesByTime.offer(recordQueue); + } + + return record; + } + + /** + * Put a timestamped record associated into its corresponding partition's queues. + */ + public void putRecord(StampedRecord record, TopicPartition partition) { + if (record.partition() != partition.partition() || !record.topic().equals(partition.topic())) + throw new KafkaException("The specified partition is different from the record's associated partition."); + + RecordQueue recordQueue = partitionQueues.get(partition); + + if (recordQueue == null) + throw new KafkaException("Record's partition does not belong to this partition-group."); + + boolean wasEmpty = recordQueue.isEmpty(); + + recordQueue.add(record); + + totalBuffered++; + + // add this record queue to be considered for processing in the future if it was empty before + if (wasEmpty) + queuesByTime.offer(recordQueue); + } + + public Deserializer keyDeserializer(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.source().keyDeserializer; + } + + public Deserializer valDeserializer(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.source().valDeserializer; + } + + public long timestamp() { + + // return the timestamp of this partition-group as the smallest partition timestamp + if (queuesByTime.isEmpty()) + return -1L; + 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; + } +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index d4de6f8302a4e..dd6d3c48602d3 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -96,7 +96,7 @@ public ProcessorContextImpl(int id, Consumer restoreConsumer = new KafkaConsumer<>(config.getConsumerProperties(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); - this.streamGroup = new StreamGroup(this, this.ingestor, new TimeBasedChooser(), extractor, bufferedRecordsPerPartition); + this.streamGroup = new StreamGroup(this, this.ingestor, extractor, bufferedRecordsPerPartition); stateMgr.init(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java index f286da33a8203..58d18ede0f0f5 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java @@ -23,6 +23,7 @@ import org.apache.kafka.streaming.processor.StateStore; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streaming.state.OffsetCheckpoint; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java index 9d82531a37d95..aaa656d008f73 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java @@ -17,69 +17,107 @@ package org.apache.kafka.streaming.processor.internals; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streaming.processor.KafkaProcessor; -import org.apache.kafka.streaming.processor.internals.StampedRecord; + +import java.util.ArrayDeque; /** * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). */ -public interface RecordQueue { +public class RecordQueue { + + private final SourceNode source; + private final TopicPartition partition; + private final ArrayDeque queue = new ArrayDeque<>(); + private final TimestampTracker> timestampTracker = new MinTimestampTracker<>(); + + private long offset; /** - * Returns the partition with which this queue is associated + * Creates a new instance of RecordQueue * - * @return TopicPartition + * @param partition partition + * @param source source node */ - TopicPartition partition(); + public RecordQueue(TopicPartition partition, SourceNode source) { + this.partition = partition; + this.source = source; + } + + public SourceNode source() { + return source; + } /** - * Returns the corresponding source processor with this queue + * Returns the partition with which this queue is associated * - * @return KafkaProcessor + * @return TopicPartition */ - ProcessorNode source(); + public TopicPartition partition() { + return partition; + } /** * Adds a StampedRecord to the queue * * @param record StampedRecord */ - void add(StampedRecord record); + public void add(StampedRecord record) { + queue.addLast(record); + + offset = record.offset(); + timestampTracker.addStampedElement(record); + } /** * Returns the next record fro the queue * * @return StampedRecord */ - StampedRecord next(); + public StampedRecord next() { + StampedRecord elem = queue.pollFirst(); + + if (elem == null) return null; + + timestampTracker.removeStampedElement(elem); + + return elem; + } /** * Returns the highest offset in the queue * * @return offset */ - long offset(); + public long offset() { + return offset; + } /** * Returns the number of records in the queue * * @return the number of records */ - int size(); + public int size() { + return queue.size(); + } /** * Tests if the queue is empty * * @return true if the queue is empty, otherwise false */ - boolean isEmpty(); + public boolean isEmpty() { + return queue.isEmpty(); + } /** * Returns a timestamp tracked by the TimestampTracker * * @return timestamp */ - long trackedTimestamp(); - + public long timestamp() { + return timestampTracker.get(); + } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java deleted file mode 100644 index 2787af9ca3b5c..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueueImpl.java +++ /dev/null @@ -1,129 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor.internals; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.TopicPartition; - -import java.util.ArrayDeque; - -public class RecordQueueImpl implements RecordQueue { - private final SourceNode source; - private final TopicPartition partition; - private final ArrayDeque queue = new ArrayDeque<>(); - - private TimestampTracker> timestampTracker; - private long offset; - - /** - * Creates a new instance of RecordQueue - * - * @param partition partition - * @param source the instance of KStreamImpl that receives records - * @param timestampTracker TimestampTracker - */ - public RecordQueueImpl(TopicPartition partition, SourceNode source, TimestampTracker> timestampTracker) { - this.partition = partition; - this.source = source; - this.timestampTracker = timestampTracker; - } - - @Override - public ProcessorNode source() { - return source; - } - - /** - * Returns the partition with which this queue is associated - * - * @return TopicPartition - */ - @Override - public TopicPartition partition() { - return partition; - } - - /** - * Adds a StampedRecord to the queue - * - * @param record StampedRecord - */ - @Override - public void add(StampedRecord record) { - queue.addLast(record); - - offset = record.offset(); - timestampTracker.addStampedElement(record); - } - - /** - * Returns the next record fro the queue - * - * @return StampedRecord - */ - @Override - public StampedRecord next() { - StampedRecord elem = queue.pollFirst(); - - if (elem == null) return null; - - timestampTracker.removeStampedElement(elem); - - return elem; - } - - /** - * Returns the highest offset in the queue - * - * @return offset - */ - @Override - public long offset() { - return offset; - } - - /** - * Returns the number of records in the queue - * - * @return the number of records - */ - @Override - public int size() { - return queue.size(); - } - - /** - * Tests if the queue is empty - * - * @return true if the queue is empty, otherwise false - */ - @Override - public boolean isEmpty() { - return queue.isEmpty(); - } - - /** - * Returns a timestamp tracked by the TimestampTracker - * - * @return timestamp - */ - @Override - public long trackedTimestamp() { - return timestampTracker.get(); - } -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java deleted file mode 100644 index 744ca8cf713bb..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RoundRobinChooser.java +++ /dev/null @@ -1,45 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor.internals; - -import java.util.ArrayDeque; - -public class RoundRobinChooser implements Chooser { - - private final ArrayDeque deque; - - public RoundRobinChooser() { - deque = new ArrayDeque<>(); - } - - @Override - public void add(RecordQueue queue) { - deque.offer(queue); - } - - @Override - public RecordQueue next() { - return deque.poll(); - } - - @Override - public void close() { - deque.clear(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java index bd2b928cbe6d2..af1954b23d1ac 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java @@ -26,9 +26,11 @@ import java.util.ArrayDeque; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.PriorityQueue; import java.util.Set; /** @@ -36,15 +38,25 @@ */ public class StreamGroup { - private final ProcessorContext context; private final Ingestor ingestor; - private final Chooser chooser; + private final ProcessorContext context; private final TimestampExtractor timestampExtractor; - private final Map stash = new HashMap<>(); + + private final Map queuesPerPartition = new HashMap<>(); + private final PriorityQueue queuesByTime = new PriorityQueue<>(new Comparator() { + 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; + } + }); private final int desiredUnprocessed; - // TODO: merge stash, consumedOffset, and newRecordBuffer into sth. like partition metadata + // TODO: merge queuesPerPartition, consumedOffset, and newRecordBuffer into sth. like partition metadata private final Map consumedOffsets; private final PunctuationQueue punctuationQueue = new PunctuationQueue(); private final ArrayDeque newRecordBuffer = new ArrayDeque<>(); @@ -60,18 +72,15 @@ public class StreamGroup { * * @param context the task context * @param ingestor the instance of {@link Ingestor} - * @param chooser the instance of {@link Chooser} * @param timestampExtractor the instance of {@link TimestampExtractor} * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic */ public StreamGroup(ProcessorContext context, Ingestor ingestor, - Chooser chooser, TimestampExtractor timestampExtractor, int desiredUnprocessedPerPartition) { this.context = context; this.ingestor = ingestor; - this.chooser = chooser; this.timestampExtractor = timestampExtractor; this.desiredUnprocessed = desiredUnprocessedPerPartition; this.consumedOffsets = new HashMap<>(); @@ -90,7 +99,7 @@ public void setNode(ProcessorNode node) { } public Set partitions() { - return stash.keySet(); + return queuesPerPartition.keySet(); } /** @@ -102,10 +111,10 @@ public Set partitions() { @SuppressWarnings("unchecked") public void addPartition(TopicPartition partition, SourceNode source) { synchronized (this) { - RecordQueue recordQueue = stash.get(partition); + RecordQueue recordQueue = queuesPerPartition.get(partition); if (recordQueue == null) { - stash.put(partition, createRecordQueue(partition, source)); + queuesPerPartition.put(partition, createRecordQueue(partition, source)); } else { throw new IllegalStateException("duplicate partition"); } @@ -131,7 +140,7 @@ private void ingestNewRecords() { TopicPartition partition = newRecords.partition; Iterator> iterator = newRecords.iterator; - RecordQueue recordQueue = stash.get(partition); + RecordQueue recordQueue = queuesPerPartition.get(partition); if (recordQueue != null) { boolean wasEmpty = recordQueue.isEmpty(); @@ -152,7 +161,10 @@ private void ingestNewRecords() { } int queueSize = recordQueue.size(); - if (wasEmpty && queueSize > 0) chooser.add(recordQueue); + + // add this record queue to be considered for processing + if (wasEmpty && queueSize > 0) + queuesByTime.offer(recordQueue); // if we have buffered enough for this partition, pause if (queueSize >= this.desiredUnprocessed) { @@ -182,7 +194,9 @@ public boolean process() { boolean readyForNextExecution = false; ingestNewRecords(); - RecordQueue recordQueue = chooser.next(); + // take the next record queue with the smallest estimated timestamp to process + RecordQueue recordQueue = queuesByTime.poll(); + if (recordQueue == null) { return false; } @@ -193,7 +207,7 @@ public boolean process() { ingestor.unpause(recordQueue.partition(), recordQueue.offset()); } - long trackedTimestamp = recordQueue.trackedTimestamp(); + long trackedTimestamp = recordQueue.timestamp(); currRecord = recordQueue.next(); currNode = recordQueue.source(); @@ -219,9 +233,10 @@ public boolean process() { new TopicPartition(currRecord.topic(), currRecord.partition()), currRecord.offset())); + // update this record queue's estimated timestamp if (recordQueue.size() > 0) { readyForNextExecution = true; - chooser.add(recordQueue); + queuesByTime.offer(recordQueue); } buffered--; @@ -250,12 +265,12 @@ public void commitOffset() { } public void close() { - chooser.close(); - stash.clear(); + queuesByTime.clear(); + queuesPerPartition.clear(); } protected RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { - return new RecordQueueImpl(partition, source, new MinTimestampTracker>()); + return new RecordQueue(partition, source); } private static class NewRecords { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java new file mode 100644 index 0000000000000..f5496ec62428e --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -0,0 +1,230 @@ +/** + * 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.streaming.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.Punctuator; +import org.apache.kafka.streaming.processor.TimestampExtractor; + +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +/** + * A StreamTask is associated with a {@link PartitionGroup} which is assigned to a StreamThread for processing. + */ +public class StreamTask { + + private final int id; + private final int desiredUnprocessed; + + private final Consumer consumer; + private final PartitionGroup partitionGroup; + private final TimestampExtractor timestampExtractor; + + private final Map consumedOffsets; + private final PunctuationQueue punctuationQueue = new PunctuationQueue(); + private final ArrayDeque newRecordBuffer = new ArrayDeque<>(); + + private long streamTime = -1; + private boolean commitRequested = false; + private StampedRecord currRecord = null; + private ProcessorNode currNode = null; + + /** + * Creates StreamGroup + * + * @param consumer the instance of {@link Consumer} + * @param partitions the instance of {@link TimestampExtractor} + * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic + */ + public StreamTask(int id, + Consumer consumer, + ProcessorTopology topology, + Collection partitions, + int desiredUnprocessedPerPartition) { + this.id = id; + this.consumer = consumer; + this.desiredUnprocessed = desiredUnprocessedPerPartition; + this.consumedOffsets = new HashMap<>(); + + // create partition queues and pipe them to corresponding source nodes in the topology + Map partitionQueues = new HashMap<>(); + for (TopicPartition partition : partitions) { + RecordQueue queue = createRecordQueue(partition, topology.source(partition.topic())); + partitionQueues.put(partition, queue); + } + this.partitionGroup = new PartitionGroup(partitionQueues); + } + + public int id() { + return id; + } + + public StampedRecord record() { + return currRecord; + } + + public ProcessorNode node() { + return currNode; + } + + public void setNode(ProcessorNode node) { + currNode = node; + } + + public Set partitions() { + return queuesPerPartition.keySet(); + } + + /** + * Adds records to queues + * + * @param partition the partition + * @param iterator the iterator of records + */ + @SuppressWarnings("unchecked") + public void addRecords(TopicPartition partition, Iterator> iterator) { + + // get deserializers for this partition + Deserializer keyDeserializer = partitionGroup.keyDeserializer(partition); + Deserializer valDeserializer = partitionGroup.valDeserializer(partition); + + while (iterator.hasNext()) { + + ConsumerRecord rawRecord = iterator.next(); + + // deserialize the raw record, extract the timestamp and put into the queue + Object key = keyDeserializer.deserialize(rawRecord.topic(), rawRecord.key()); + Object value = valDeserializer.deserialize(rawRecord.topic(), rawRecord.value()); + long timestamp = timestampExtractor.extract(rawRecord.topic(), key, value); + + StampedRecord stampedRecord = new StampedRecord(new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), key, value), timestamp); + + partitionGroup.putRecord(stampedRecord, partition); + } + } + + /** + * Schedules a punctuation for the processor + * + * @param punctuator the punctuator requesting scheduler + * @param interval the interval in milliseconds + */ + public void schedule(Punctuator punctuator, long interval) { + punctuationQueue.schedule(new PunctuationSchedule(punctuator, interval)); + } + + /** + * Processes one record + */ + @SuppressWarnings("unchecked") + public boolean process() { + synchronized (this) { + boolean readyForNextExecution = false; + + // take the next record queue with the smallest estimated timestamp to process + long timestamp = partitionGroup.timestamp(); + StampedRecord record = partitionGroup.nextRecord(); + + currRecord = recordQueue.next(); + currNode = recordQueue.source(); + + if (streamTime < timestamp) streamTime = timestamp; + + currNode.process(currRecord.key(), currRecord.value()); + consumedOffsets.put(recordQueue.partition(), currRecord.offset()); + + // TODO: local state flush and downstream producer flush + // need to be done altogether with offset commit atomically + if (commitRequested) { + // flush local state + context.flush(); + + // flush produced records in the downstream + context.recordCollector().flush(); + + // commit consumed offsets + ingestor.commit(consumedOffsets()); + } + + if (commitRequested) ingestor.commit(Collections.singletonMap( + new TopicPartition(currRecord.topic(), currRecord.partition()), + currRecord.offset())); + + // update this record queue's estimated timestamp + if (recordQueue.size() > 0) { + readyForNextExecution = true; + } + + if (recordQueue.size() == this.desiredUnprocessed) { + ingestor.unpause(recordQueue.partition(), recordQueue.offset()); + } + + buffered--; + currRecord = null; + + punctuationQueue.mayPunctuate(streamTime); + + return readyForNextExecution; + } + } + + /** + * Returns consumed offsets + * + * @return the map of partition to consumed offset + */ + public Map consumedOffsets() { + return this.consumedOffsets; + } + + /** + * Request committing the current record's offset + */ + public void commitOffset() { + this.commitRequested = true; + } + + public void close() { + queuesByTime.clear(); + queuesPerPartition.clear(); + } + + protected RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { + return new RecordQueue(partition, source); + } + + private static class NewRecords { + final TopicPartition partition; + final Iterator> iterator; + + NewRecords(TopicPartition partition, Iterator> iterator) { + this.partition = partition; + this.iterator = iterator; + } + } +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java similarity index 81% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index 005307b1d69c0..05123a54acd0a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/KStreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -19,8 +19,8 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; +import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; @@ -43,21 +43,19 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Map; -public class KStreamThread extends Thread { +public class StreamThread extends Thread { - private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); + private static final Logger log = LoggerFactory.getLogger(StreamThread.class); - private final TopologyBuilder builder; - private final IngestorImpl ingestor; + private final Consumer consumer; + private final ProcessorTopology topology; private final RecordCollectorImpl collector; - private final ArrayList streamGroups = new ArrayList<>(); - private final Map kstreamContexts = new HashMap<>(); + private final Map tasks = new HashMap<>(); private final Metrics metrics; private final Time time; @@ -77,7 +75,6 @@ public class KStreamThread extends Thread { protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { @Override public void onPartitionsAssigned(Consumer consumer, Collection assignment) { - ingestor.init(); addPartitions(assignment); } @@ -85,23 +82,18 @@ public void onPartitionsAssigned(Consumer consumer, Collection consumer, Collection assignment) { commitAll(time.milliseconds()); removePartitions(); - ingestor.clear(); } }; @SuppressWarnings("unchecked") - public KStreamThread(TopologyBuilder builder, StreamingConfig config) throws Exception { + public StreamThread(ProcessorTopology topology, StreamingConfig config) throws Exception { super(); - this.metrics = new Metrics(); this.config = config; - this.builder = builder; + this.topology = topology; this.streamingMetrics = new KafkaStreamingMetrics(); - // build the topology without initialization to get the topics for consumer - ProcessorTopology topology = builder.build(); - // create the producer and consumer clients Producer producer = new KafkaProducer<>(config.getProducerProperties(), new ByteArraySerializer(), @@ -110,11 +102,10 @@ public KStreamThread(TopologyBuilder builder, StreamingConfig config) throws Exc (Serializer) config.getConfiguredInstance(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, Serializer.class), (Serializer) config.getConfiguredInstance(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Serializer.class)); - Consumer consumer = new KafkaConsumer<>(config.getConsumerProperties(), + consumer = new KafkaConsumer<>(config.getConsumerProperties(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); - this.ingestor = new IngestorImpl(consumer, topology.sourceTopics()); this.stateDir = new File(this.config.getString(StreamingConfig.STATE_DIR_CONFIG)); this.pollTimeMs = config.getLong(StreamingConfig.POLL_MS_CONFIG); @@ -127,6 +118,8 @@ public KStreamThread(TopologyBuilder builder, StreamingConfig config) throws Exc this.nextStateCleaning = Long.MAX_VALUE; this.recordsProcessed = 0; this.time = new SystemTime(); + + this.metrics = new Metrics(); } /** @@ -136,7 +129,6 @@ public KStreamThread(TopologyBuilder builder, StreamingConfig config) throws Exc public synchronized void run() { log.info("Starting a kstream thread"); try { - ingestor.open(); runLoop(); } catch (RuntimeException e) { log.error("Uncaught error during processing: ", e); @@ -151,7 +143,7 @@ private void shutdown() { commitAll(time.milliseconds()); collector.close(); - ingestor.close(); + consumer.close(); removePartitions(); log.info("kstream thread shutdown complete"); } @@ -168,10 +160,20 @@ private void runLoop() { boolean readyForNextExecution = false; while (stillRunning()) { - ingestor.poll(readyForNextExecution ? 0 : this.pollTimeMs); + // try to fetch some records and put them to tasks' queues + // TODO: we may not need to poll every iteration + ConsumerRecords records = consumer.poll(readyForNextExecution ? 0 : this.pollTimeMs); - for (StreamGroup group : this.streamGroups) { - readyForNextExecution = group.process(); + for (StreamTask task : tasks.values()) { + for (TopicPartition partition : task.partitions()) { + task.addRecords(partition, records.records(partition).iterator()); + } + } + + // try to process one record from each task + // TODO: we may want to process more than one record in each iteration + for (StreamTask task : tasks.values()) { + readyForNextExecution = task.process(); } maybeCommit(); @@ -203,8 +205,10 @@ private void maybeCommit() { } private void commitAll(long now) { + + /* Map commit = new HashMap<>(); - for (ProcessorContextImpl context : kstreamContexts.values()) { + for (ProcessorContextImpl context : tasks.values()) { context.flush(); commit.putAll(context.consumedOffsets()); } @@ -217,6 +221,7 @@ private void commitAll(long now) { ingestor.commit(commit); // TODO: can this be async? streamingMetrics.commitTime.record(now - lastCommit); } + */ } /* delete any state dirs that aren't for active contexts */ @@ -228,7 +233,7 @@ private void maybeCleanState() { for (File dir : stateDirs) { try { Integer id = Integer.parseInt(dir.getName()); - if (!kstreamContexts.keySet().contains(id)) { + if (!tasks.keySet().contains(id)) { log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), stateCleanupDelayMs); Utils.rm(dir); } @@ -245,45 +250,40 @@ private void maybeCleanState() { private void addPartitions(Collection assignment) { HashSet partitions = new HashSet<>(assignment); + // TODO: change this hard-coded co-partitioning behavior for (TopicPartition partition : partitions) { - final Integer id = partition.partition(); // TODO: switch this to the group id - ProcessorContextImpl context = kstreamContexts.get(id); - if (context == null) { - try { - // build the topology and initialize with the context - ProcessorTopology topology = builder.build(); - - context = new ProcessorContextImpl(id, ingestor, topology, collector, config, metrics); - - topology.init(context); - - context.initialized(); - - kstreamContexts.put(id, context); - } catch (Exception e) { - throw new KafkaException(e); - } - - streamGroups.add(context.streamGroup); + final Integer id = partition.partition(); + StreamTask task = tasks.get(id); + if (task == null) { + // get the partitions for the task + HashSet partitionsForTask = new HashSet<>(); + for (TopicPartition part : partitions) + if (part.partition() == id) + partitionsForTask.add(part); + + // creat the task + task = new StreamTask(id, consumer, topology, partitionsForTask, config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG)); + + tasks.put(id, task); } - - context.addPartition(partition); } nextStateCleaning = time.milliseconds() + stateCleanupDelayMs; } private void removePartitions() { - for (ProcessorContextImpl context : kstreamContexts.values()) { - log.info("Removing task context {}", context.id()); + + // TODO: change this clearing tasks behavior + for (StreamTask task : tasks.values()) { + log.info("Removing task {}", task.id()); try { - context.close(); + task.close(); } catch (Exception e) { throw new KafkaException(e); } streamingMetrics.processorDestruction.record(); } - streamGroups.clear(); + tasks.clear(); } private class KafkaStreamingMetrics { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java deleted file mode 100644 index ee8f41c1e2d0f..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimeBasedChooser.java +++ /dev/null @@ -1,60 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor.internals; - - -import java.util.Comparator; -import java.util.PriorityQueue; - -public class TimeBasedChooser implements Chooser { - - private final PriorityQueue pq; - - public TimeBasedChooser() { - this(new Comparator() { - public int compare(RecordQueue queue1, RecordQueue queue2) { - long time1 = queue1.trackedTimestamp(); - long time2 = queue2.trackedTimestamp(); - - if (time1 < time2) return -1; - if (time1 > time2) return 1; - return 0; - } - }); - } - - private TimeBasedChooser(Comparator comparator) { - pq = new PriorityQueue<>(3, comparator); - } - - @Override - public void add(RecordQueue queue) { - pq.offer(queue); - } - - @Override - public RecordQueue next() { - return pq.poll(); - } - - @Override - public void close() { - pq.clear(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/OffsetCheckpoint.java b/stream/src/main/java/org/apache/kafka/streaming/state/OffsetCheckpoint.java similarity index 99% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/OffsetCheckpoint.java rename to stream/src/main/java/org/apache/kafka/streaming/state/OffsetCheckpoint.java index db0b90858bea4..65c1ecc97fb8e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/OffsetCheckpoint.java +++ b/stream/src/main/java/org/apache/kafka/streaming/state/OffsetCheckpoint.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streaming.state; import org.apache.kafka.common.TopicPartition; diff --git a/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java index 94ed0a6646f91..a53cf20d72286 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.processor.internals.StreamGroup; -import org.apache.kafka.streaming.processor.internals.TimeBasedChooser; import org.apache.kafka.test.MockIngestor; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.MockSourceNode; @@ -92,7 +91,6 @@ public void testAddPartition() { StreamGroup streamGroup = new StreamGroup( new MockProcessorContext(serializer, deserializer), mockIngestor, - new TimeBasedChooser(), new TimestampExtractor() { public long extract(String topic, Object key, Object value) { if (topic.equals("topic1")) diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/FilteredIteratorTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/FilteredIteratorTest.java new file mode 100644 index 0000000000000..2bd658c826a0c --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/FilteredIteratorTest.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.internals; + +import static org.junit.Assert.assertEquals; + +import org.apache.kafka.streaming.kstream.internals.FilteredIterator; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class FilteredIteratorTest { + + @Test + public void testFiltering() { + List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 3 == 0) return i.toString(); + return null; + } + }; + + List expected = Arrays.asList("3", "9", "6", "3"); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + + @Test + public void testEmptySource() { + List list = new ArrayList(); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 3 == 0) return i.toString(); + return null; + } + }; + + List expected = new ArrayList(); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + + @Test + public void testNoMatch() { + List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 7 == 0) return i.toString(); + return null; + } + }; + + List expected = new ArrayList(); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamBranchTest.java new file mode 100644 index 0000000000000..518d00f43d7e7 --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamBranchTest.java @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.Predicate; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; +import org.apache.kafka.test.MockProcessor; +import org.junit.Test; + +import java.lang.reflect.Array; + +import static org.junit.Assert.assertEquals; + +public class KStreamBranchTest { + + private String topic1 = "topic"; + + private KStreamBuilder topology = new MockKStreamBuilder(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @SuppressWarnings("unchecked") + @Test + public void testKStreamBranch() { + + Predicate isEven = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 2) == 0; + } + }; + Predicate isMultipleOfThree = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 3) == 0; + } + }; + Predicate isOdd = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 2) != 0; + } + }; + + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + + KStream stream; + KStream[] branches; + MockProcessor[] processors; + + stream = topology.from(keyDeserializer, valDeserializer, topic1); + branches = stream.branch(isEven, isMultipleOfThree, isOdd); + + assertEquals(3, branches.length); + + processors = (MockProcessor[]) Array.newInstance(MockProcessor.class, branches.length); + for (int i = 0; i < branches.length; i++) { + processors[i] = new MockProcessor<>(); + branches[i].process(processors[i]); + } + + for (int i = 0; i < expectedKeys.length; i++) { + ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(3, processors[0].processed.size()); + assertEquals(2, processors[1].processed.size()); + assertEquals(4, processors[2].processed.size()); + } +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFilterTest.java new file mode 100644 index 0000000000000..276302b738bbf --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFilterTest.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.Predicate; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; +import org.apache.kafka.test.MockProcessor; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamFilterTest { + + private String topicName = "topic"; + + private KStreamBuilder topology = new MockKStreamBuilder(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + private Predicate isMultipleOfThree = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 3) == 0; + } + }; + + @Test + public void testFilter() { + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + + KStream stream; + MockProcessor processor; + + processor = new MockProcessor<>(); + stream = topology.from(keyDeserializer, valDeserializer, topicName); + stream.filter(isMultipleOfThree).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(2, processor.processed.size()); + } + + @Test + public void testFilterOut() { + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + + KStream stream; + MockProcessor processor; + + processor = new MockProcessor<>(); + stream = topology.from(keyDeserializer, valDeserializer, topicName); + stream.filterOut(isMultipleOfThree).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(5, processor.processed.size()); + } + +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapTest.java new file mode 100644 index 0000000000000..82b36308a0c8b --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapTest.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; +import org.apache.kafka.test.MockProcessor; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +import java.util.ArrayList; + +public class KStreamFlatMapTest { + + private String topicName = "topic"; + + private KStreamBuilder topology = new MockKStreamBuilder(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testFlatMap() { + + KeyValueMapper> mapper = + new KeyValueMapper>() { + @Override + public KeyValue> apply(Integer key, String value) { + ArrayList result = new ArrayList(); + for (int i = 0; i < key; i++) { + result.add(value); + } + return KeyValue.pair(Integer.toString(key * 10), (Iterable) result); + } + }; + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream; + MockProcessor processor; + + processor = new MockProcessor<>(); + stream = topology.from(keyDeserializer, valDeserializer, topicName); + stream.flatMap(mapper).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(6, processor.processed.size()); + + String[] expected = new String[]{"10:V1", "20:V2", "20:V2", "30:V3", "30:V3", "30:V3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapValuesTest.java new file mode 100644 index 0000000000000..9c5863bf3ed19 --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapValuesTest.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.ValueMapper; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; +import org.apache.kafka.test.MockProcessor; +import org.junit.Test; + +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +public class KStreamFlatMapValuesTest { + + private String topicName = "topic"; + + private KStreamBuilder topology = new MockKStreamBuilder(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testFlatMapValues() { + + ValueMapper> mapper = + new ValueMapper>() { + @Override + public Iterable apply(String value) { + ArrayList result = new ArrayList(); + result.add(value.toLowerCase()); + result.add(value); + return result; + } + }; + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream; + MockProcessor processor; + + processor = new MockProcessor<>(); + stream = topology.from(keyDeserializer, valDeserializer, topicName); + stream.flatMapValues(mapper).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(8, processor.processed.size()); + + String[] expected = new String[]{"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)); + } + } +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamJoinTest.java new file mode 100644 index 0000000000000..bdc2b23a82e5b --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamJoinTest.java @@ -0,0 +1,249 @@ +/** + * 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.streaming.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.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.KStreamWindowed; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.kstream.ValueJoiner; +import org.apache.kafka.streaming.kstream.ValueMapper; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; +import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.UnlimitedWindow; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamJoinTest { + + private String topic1 = "topic1"; + private String topic2 = "topic2"; + + private KStreamBuilder topology = new MockKStreamBuilder(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + private ValueJoiner joiner = new ValueJoiner() { + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } + }; + + private ValueMapper valueMapper = new ValueMapper() { + @Override + public String apply(String value) { + return "#" + value; + } + }; + + private ValueMapper> valueMapper2 = new ValueMapper>() { + @Override + public Iterable apply(String value) { + return (Iterable) Utils.mkSet(value); + } + }; + + private KeyValueMapper keyValueMapper = + new KeyValueMapper() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(key, value); + } + }; + + KeyValueMapper> keyValueMapper2 = + new KeyValueMapper>() { + @Override + public KeyValue> apply(Integer key, String value) { + return KeyValue.pair(key, (Iterable) Utils.mkSet(value)); + } + }; + + + @Test + public void testJoin() { + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream1; + KStream stream2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + MockProcessor processor; + String[] expected; + + processor = new MockProcessor<>(); + stream1 = topology.from(keyDeserializer, valDeserializer, topic1); + stream2 = topology.from(keyDeserializer, valDeserializer, topic2); + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); + + windowed1.join(windowed2, joiner).process(processor); + + MockProcessorContext context = new MockProcessorContext(null, null); + topology.init(context); + context.setTime(0L); + + // push two items to the main stream. the other stream's window is empty + + for (int i = 0; i < 2; i++) { + ((KStreamSource) stream1).source().process(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++) { + ((KStreamSource) stream2).source().process(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++) { + ((KStreamSource) stream1).source().process(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++) { + ((KStreamSource) stream2).source().process(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)); + } + } + + @Test + public void testJoinPrior() { + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream1; + KStream stream2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + MockProcessor processor; + String[] expected; + + processor = new MockProcessor<>(); + stream1 = topology.from(keyDeserializer, valDeserializer, topic1); + stream2 = topology.from(keyDeserializer, valDeserializer, topic2); + windowed1 = stream1.with(new UnlimitedWindow()); + windowed2 = stream2.with(new UnlimitedWindow()); + + windowed1.joinPrior(windowed2, joiner).process(processor); + + MockProcessorContext context = new MockProcessorContext(null, null); + topology.init(context); + + // push two items to the main stream. the other stream's window is empty + + for (int i = 0; i < 2; i++) { + context.setTime(i); + + ((KStreamSource) stream1).source().process(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 + // no corresponding item in the main window has a newer timestamp + + for (int i = 0; i < 2; i++) { + context.setTime(i + 1); + + ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]); + } + + assertEquals(0, processor.processed.size()); + + processor.processed.clear(); + + // push all items with newer timestamps to the main stream. this should produce two items. + + for (int i = 0; i < expectedKeys.length; i++) { + context.setTime(i + 2); + + ((KStreamSource) stream1).source().process(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 with older timestamps to the other stream. this should produce six items + for (int i = 0; i < expectedKeys.length; i++) { + context.setTime(i); + + ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]); + } + + assertEquals(6, processor.processed.size()); + + expected = new String[]{"0:X0+Y0", "0:X0+Y0", "1:X1+Y1", "1:X1+Y1", "2:X2+Y2", "3:X3+Y3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + + // TODO: test for joinability +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapTest.java new file mode 100644 index 0000000000000..7104e5efefec4 --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapTest.java @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.KeyValueMapper; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; +import org.apache.kafka.test.MockProcessor; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamMapTest { + + private String topicName = "topic"; + + private KStreamBuilder topology = new MockKStreamBuilder(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testMap() { + + KeyValueMapper mapper = + new KeyValueMapper() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(value, key); + } + }; + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream; + MockProcessor processor; + + processor = new MockProcessor<>(); + stream = topology.from(keyDeserializer, valDeserializer, topicName); + stream.map(mapper).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(4, processor.processed.size()); + + String[] expected = new String[]{"V0:0", "V1:1", "V2:2", "V3:3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapValuesTest.java new file mode 100644 index 0000000000000..9cca557bc85d9 --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapValuesTest.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.ValueMapper; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; +import org.apache.kafka.test.MockProcessor; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamMapValuesTest { + + private String topicName = "topic"; + + private KStreamBuilder topology = new MockKStreamBuilder(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testFlatMapValues() { + + ValueMapper mapper = + new ValueMapper() { + @Override + public Integer apply(String value) { + return value.length(); + } + }; + + final int[] expectedKeys = new int[]{1, 10, 100, 1000}; + + KStream stream; + MockProcessor processor = new MockProcessor<>(); + stream = topology.from(keyDeserializer, valDeserializer, topicName); + stream.mapValues(mapper).process(processor); + + for (int i = 0; i < expectedKeys.length; i++) { + ((KStreamSource) stream).source().process(expectedKeys[i], Integer.toString(expectedKeys[i])); + } + + assertEquals(4, processor.processed.size()); + + String[] expected = new String[]{"1:1", "10:2", "100:3", "1000:4"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamSourceTest.java new file mode 100644 index 0000000000000..2f792f415605a --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamSourceTest.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.internals; + +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; +import org.apache.kafka.test.MockProcessor; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamSourceTest { + + private String topicName = "topic"; + + private KStreamBuilder topology = new MockKStreamBuilder(); + private StringDeserializer keyDeserializer = new StringDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testKStreamSource() { + + MockProcessor processor = new MockProcessor<>(); + + KStream stream = topology.from(keyDeserializer, valDeserializer, topicName); + stream.process(processor); + + final String[] expectedKeys = new String[]{"k1", "k2", "k3"}; + final String[] expectedValues = new String[]{"v1", "v2", "v3"}; + + for (int i = 0; i < expectedKeys.length; i++) { + ((KStreamSource) stream).source().process(expectedKeys[i], expectedValues[i]); + } + + assertEquals(3, processor.processed.size()); + + for (int i = 0; i < expectedKeys.length; i++) { + assertEquals(expectedKeys[i] + ":" + expectedValues[i], processor.processed.get(i)); + } + } +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java new file mode 100644 index 0000000000000..88e92a551a981 --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java @@ -0,0 +1,91 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.kstream.internals.KStreamSource; +import org.apache.kafka.test.MockKStreamBuilder; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.UnlimitedWindow; +import org.junit.Test; + +import java.util.Iterator; + +import static org.junit.Assert.assertEquals; + +public class KStreamWindowedTest { + + private String topicName = "topic"; + + private KStreamBuilder topology = new MockKStreamBuilder(); + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testWindowedStream() { + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream; + Window window; + + window = new UnlimitedWindow<>(); + stream = topology.from(keyDeserializer, valDeserializer, topicName); + stream.with(window); + + MockProcessorContext context = new MockProcessorContext(null, null); + topology.init(context); + context.setTime(0L); + + // two items in the window + + for (int i = 0; i < 2; i++) { + ((KStreamSource) stream).source().process(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++) { + ((KStreamSource) stream).source().process(expectedKeys[i], "Y" + expectedKeys[i]); + } + + assertEquals(2, countItem(window.find(0, 0L))); + assertEquals(2, countItem(window.find(1, 0L))); + assertEquals(1, countItem(window.find(2, 0L))); + assertEquals(1, countItem(window.find(3, 0L))); + } + + + private int countItem(Iterator iter) { + int i = 0; + while (iter.hasNext()) { + i++; + iter.next(); + } + return i; + } +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/MinTimestampTrackerTest.java new file mode 100644 index 0000000000000..77836967a50b3 --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/MinTimestampTrackerTest.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.internals; + +import static org.junit.Assert.assertEquals; + +import org.apache.kafka.streaming.processor.internals.MinTimestampTracker; +import org.apache.kafka.streaming.processor.internals.Stamped; +import org.junit.Test; + +public class MinTimestampTrackerTest { + + private Stamped elem(long timestamp) { + return new Stamped<>("", timestamp); + } + + @SuppressWarnings("unchecked") + @Test + public void testTracking() { + MinTimestampTracker 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.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); + + // add 101 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); + + // remove 100 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(101L, tracker.get()); + + // add 102 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(101L, tracker.get()); + + // add 98 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // add 99 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // add 100 + tracker.addStampedElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // remove 101 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); + + // remove 102 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); + + // remove 98 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(99L, tracker.get()); + + // remove 99 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(100L, tracker.get()); + + // remove 100 + tracker.removeStampedElement((Stamped) elems[removalIndex++]); + assertEquals(-1L, tracker.get()); + + assertEquals(insertionIndex, removalIndex); + } + +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java new file mode 100644 index 0000000000000..daf28eb3aee80 --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java @@ -0,0 +1,163 @@ +/** + * 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.streaming.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streaming.processor.TimestampExtractor; +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.streaming.processor.internals.StreamGroup; +import org.apache.kafka.test.MockIngestor; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.MockSourceNode; +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class StreamGroupTest { + + private static Serializer serializer = new IntegerSerializer(); + private static Deserializer deserializer = new IntegerDeserializer(); + + @SuppressWarnings("unchecked") + @Test + public void testAddPartition() { + + MockIngestor mockIngestor = new MockIngestor(); + + StreamGroup streamGroup = new StreamGroup( + new MockProcessorContext(serializer, deserializer), + mockIngestor, + new TimestampExtractor() { + public long extract(String topic, Object key, Object value) { + if (topic.equals("topic1")) + return ((Integer) key).longValue(); + else + return ((Integer) key).longValue() / 10L + 5L; + } + }, + 3 + ); + + TopicPartition partition1 = new TopicPartition("topic1", 1); + TopicPartition partition2 = new TopicPartition("topic2", 1); + MockSourceNode source1 = new MockSourceNode(deserializer, deserializer); + MockSourceNode source2 = new MockSourceNode(deserializer, deserializer); + MockSourceNode source3 = new MockSourceNode(deserializer, deserializer); + + streamGroup.addPartition(partition1, source1); + mockIngestor.addPartitionStreamToGroup(streamGroup, partition1); + + streamGroup.addPartition(partition2, source2); + mockIngestor.addPartitionStreamToGroup(streamGroup, partition2); + + Exception exception = null; + try { + streamGroup.addPartition(partition1, source3); + } catch (Exception ex) { + exception = ex; + } + assertTrue(exception != null); + + byte[] recordValue = serializer.serialize(null, new Integer(10)); + + mockIngestor.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 1, serializer.serialize(partition1.topic(), new Integer(10)), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 2, serializer.serialize(partition1.topic(), new Integer(20)), recordValue) + )); + + mockIngestor.addRecords(partition2, records( + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 1, serializer.serialize(partition1.topic(), new Integer(300)), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 2, serializer.serialize(partition1.topic(), new Integer(400)), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 3, serializer.serialize(partition1.topic(), new Integer(500)), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 4, serializer.serialize(partition1.topic(), new Integer(600)), recordValue) + )); + + streamGroup.process(); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 0); + + assertEquals(mockIngestor.paused.size(), 1); + assertTrue(mockIngestor.paused.contains(partition2)); + + mockIngestor.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 3, serializer.serialize(partition1.topic(), new Integer(30)), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 4, serializer.serialize(partition1.topic(), new Integer(40)), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 5, serializer.serialize(partition1.topic(), new Integer(50)), recordValue) + )); + + streamGroup.process(); + assertEquals(source1.numReceived, 2); + assertEquals(source2.numReceived, 0); + + assertEquals(mockIngestor.paused.size(), 2); + assertTrue(mockIngestor.paused.contains(partition1)); + assertTrue(mockIngestor.paused.contains(partition2)); + + streamGroup.process(); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 0); + + streamGroup.process(); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 1); + + assertEquals(mockIngestor.paused.size(), 1); + assertTrue(mockIngestor.paused.contains(partition2)); + + streamGroup.process(); + assertEquals(source1.numReceived, 4); + assertEquals(source2.numReceived, 1); + + assertEquals(mockIngestor.paused.size(), 1); + + streamGroup.process(); + assertEquals(source1.numReceived, 4); + assertEquals(source2.numReceived, 2); + + assertEquals(mockIngestor.paused.size(), 0); + + streamGroup.process(); + assertEquals(source1.numReceived, 5); + assertEquals(source2.numReceived, 2); + + streamGroup.process(); + assertEquals(source1.numReceived, 5); + assertEquals(source2.numReceived, 3); + + streamGroup.process(); + assertEquals(source1.numReceived, 5); + assertEquals(source2.numReceived, 4); + + assertEquals(mockIngestor.paused.size(), 0); + + streamGroup.process(); + assertEquals(source1.numReceived, 5); + assertEquals(source2.numReceived, 4); + } + + private Iterable> records(ConsumerRecord... recs) { + return Arrays.asList(recs); + } +} From 63eee5dca65443ffdbc5e46841b5d4be428c69e4 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 25 Aug 2015 16:59:09 -0700 Subject: [PATCH 169/275] wip: refactor StreamTask and ProcessorContext, fix RecordQueue timestamp tracking --- .../streaming/examples/SimpleProcessJob.java | 4 +- .../examples/StatefulProcessJob.java | 4 +- .../kafka/streaming/kstream/KStream.java | 1 - .../kstream/internals/KStreamBranch.java | 4 +- .../kstream/internals/KStreamFilter.java | 4 +- .../kstream/internals/KStreamFlatMap.java | 4 +- .../internals/KStreamFlatMapValues.java | 4 +- .../kstream/internals/KStreamImpl.java | 1 - .../kstream/internals/KStreamJoin.java | 10 +- .../kstream/internals/KStreamMap.java | 5 +- .../kstream/internals/KStreamMapValues.java | 4 +- .../kstream/internals/KStreamProcessor.java | 3 +- .../kstream/internals/KStreamSend.java | 4 +- .../kstream/internals/KStreamWindow.java | 4 +- .../streaming/processor/KafkaProcessor.java | 50 ------- .../kafka/streaming/processor/Processor.java | 2 + .../streaming/processor/ProcessorContext.java | 15 +- .../streaming/processor/TopologyBuilder.java | 23 ++- .../internals/MinTimestampTracker.java | 58 -------- .../processor/internals/PartitionGroup.java | 82 ++++++++--- .../internals/ProcessorContextImpl.java | 98 +++---------- .../processor/internals/ProcessorNode.java | 8 +- .../internals/ProcessorStateManager.java | 16 +-- .../internals/ProcessorTopology.java | 12 +- .../processor/internals/RecordQueue.java | 51 +++---- .../processor/internals/Stamped.java | 1 + .../processor/internals/StreamTask.java | 136 +++++++----------- .../processor/internals/StreamThread.java | 16 ++- .../processor/internals/TimestampTracker.java | 56 -------- .../streaming/state/OffsetCheckpoint.java | 2 +- .../streaming/MinTimestampTrackerTest.java | 95 ------------ .../internals/MinTimestampTrackerTest.java | 95 ------------ .../org/apache/kafka/test/MockProcessor.java | 4 +- .../kafka/test/MockProcessorContext.java | 6 +- 34 files changed, 237 insertions(+), 645 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/MinTimestampTrackerTest.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java index e65ab50fb8807..22d6425e7aadf 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java @@ -18,7 +18,7 @@ package org.apache.kafka.streaming.examples; import org.apache.kafka.streaming.KafkaStreaming; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.ProcessorContext; @@ -29,7 +29,7 @@ public class SimpleProcessJob { - private static class MyProcessor extends KafkaProcessor { + private static class MyProcessor extends Processor { private ProcessorContext context; public MyProcessor(String name) { diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java index d1fc9a2270692..6a01551b4972a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.streaming.KafkaStreaming; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.ProcessorContext; @@ -33,7 +33,7 @@ public class StatefulProcessJob { - private static class MyProcessor extends KafkaProcessor { + private static class MyProcessor extends Processor { private ProcessorContext context; private KeyValueStore kvStore; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java index 271a89bc868c5..fc23e2d6d530d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.processor.KafkaProcessor; import org.apache.kafka.streaming.processor.Processor; /** diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java index 229e07bb6d509..25189f677c994 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java @@ -18,11 +18,11 @@ package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorMetadata; import org.apache.kafka.streaming.kstream.Predicate; -class KStreamBranch extends KafkaProcessor { +class KStreamBranch extends Processor { private final Predicate[] predicates; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java index d4d73c60e0bb9..b467a9d3d0fc3 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java @@ -17,11 +17,11 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.Predicate; import org.apache.kafka.streaming.processor.ProcessorMetadata; -class KStreamFilter extends KafkaProcessor { +class KStreamFilter extends Processor { private final PredicateOut predicateOut; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java index 408c7cff3d8b2..89543b704e986 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java @@ -17,12 +17,12 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorMetadata; import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.KeyValueFlatMap; -class KStreamFlatMap extends KafkaProcessor { +class KStreamFlatMap extends Processor { private final KeyValueFlatMap mapper; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java index 99a1ba2f69866..0df01e62358ed 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java @@ -17,11 +17,11 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.ValueMapper; import org.apache.kafka.streaming.processor.ProcessorMetadata; -class KStreamFlatMapValues extends KafkaProcessor { +class KStreamFlatMapValues extends Processor { private final ValueMapper> mapper; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index 619a0af77fe30..14b09c52bceb3 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.kstream.KeyValueFlatMap; -import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorMetadata; import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.kstream.KStreamWindowed; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index 236996710b3a6..7751a7f3603f3 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -17,14 +17,14 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.ValueJoiner; import org.apache.kafka.streaming.kstream.Window; import java.util.Iterator; -class KStreamJoin extends KafkaProcessor { +class KStreamJoin extends Processor { private static final String JOIN_NAME = "KAFKA-JOIN"; private static final String JOIN_OTHER_NAME = "KAFKA-JOIN-OTHER"; @@ -38,7 +38,7 @@ private static abstract class Finder { private final Finder finder1; private final Finder finder2; private final ValueJoiner joiner; - final KafkaProcessor processorForOtherStream; + final Processor processorForOtherStream; private ProcessorContext context; @@ -100,8 +100,8 @@ public void process(K key, V1 value) { } } - private KafkaProcessor processorForOther() { - return new KafkaProcessor(JOIN_OTHER_NAME) { + private Processor processorForOther() { + return new Processor(JOIN_OTHER_NAME) { @SuppressWarnings("unchecked") @Override diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java index f6ca1fe35f77a..903fb785a7561 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java @@ -17,13 +17,12 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.processor.ProcessorMetadata; -class KStreamMap extends KafkaProcessor { +class KStreamMap extends Processor { private final KeyValueMapper mapper; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java index 77d88ebf11004..a8aac1285039e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java @@ -17,11 +17,11 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.ValueMapper; import org.apache.kafka.streaming.processor.ProcessorMetadata; -class KStreamMapValues extends KafkaProcessor { +class KStreamMapValues extends Processor { private final ValueMapper mapper; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java index 2e1826ec3e795..486b67e3c7625 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java @@ -17,11 +17,10 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.streaming.processor.KafkaProcessor; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorMetadata; -public class KStreamProcessor extends KafkaProcessor { +public class KStreamProcessor extends Processor { private final Processor processor; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java index 0da4f899bf2e0..12b5a8ffa2837 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java @@ -18,11 +18,11 @@ package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorMetadata; import org.apache.kafka.streaming.processor.ProcessorContext; -class KStreamSend extends KafkaProcessor { +class KStreamSend extends Processor { private ProcessorContext context; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java index 2288a07485cef..2a6628ab5859a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java @@ -17,7 +17,7 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.KStream; @@ -25,7 +25,7 @@ import org.apache.kafka.streaming.kstream.ValueJoiner; import org.apache.kafka.streaming.kstream.Window; -public class KStreamWindow extends KafkaProcessor { +public class KStreamWindow extends Processor { public static final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java b/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java deleted file mode 100644 index cc0cf943a9848..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/KafkaProcessor.java +++ /dev/null @@ -1,50 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor; - -public abstract class KafkaProcessor implements Processor, Punctuator { - - private final ProcessorMetadata metadata; - - protected ProcessorContext context; - - public KafkaProcessor(ProcessorMetadata metadata) { - this.metadata = metadata; - } - - public ProcessorMetadata metadata() { - return metadata; - } - - /* Following functions can be overridden by users */ - - @Override - public void punctuate(long streamTime) { - // do nothing - } - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void close() { - // do nothing - } -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java b/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java index c74fbc8ab370e..57a618b2649f0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java @@ -73,6 +73,8 @@ public interface Processor { void process(K key, V value); + void punctuate(long streamTime); + void close(); >>>>>>> wip } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java index 787d91c8f2e5f..4bb46604377de 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java @@ -63,13 +63,6 @@ public interface ProcessorContext { */ Deserializer valueDeserializer(); - /** - * Returns a RecordCollector - * - * @return RecordCollector - */ - RecordCollector recordCollector(); - /** * Returns the state directory for the partition. * @@ -96,13 +89,9 @@ public interface ProcessorContext { */ void flush(); - void forward(K key, V value); - - void send(String topic, Object key, Object value); + void schedule(Processor processor, long interval); - void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); - - void schedule(KafkaProcessor processor, long interval); + void forward(K key, V value); void commit(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index d7d55620f2ff4..591b212963b28 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -42,12 +42,10 @@ public class TopologyBuilder { private Map> children = new HashMap<>(); private class ProcessorClazz { - public Class clazz; - public ProcessorMetadata metadata; + public ProcessorFactory factory; - public ProcessorClazz(Class clazz, ProcessorMetadata metadata) { - this.clazz = clazz; - this.metadata = metadata; + public ProcessorClazz(ProcessorFactory factory) { + this.factory = factory; } } @@ -96,11 +94,11 @@ public final void addSink(String name, Serializer keySerializer, Serializer valS sinkClasses.put(name, new SinkClazz(keySerializer, valSerializer)); } - public final void addProcessor(String name, Class processorClass, ProcessorMetadata config, String... parentNames) { + public final void addProcessor(String name, ProcessorFactory factory, String... parentNames) { if (processorClasses.containsKey(name)) throw new IllegalArgumentException("Processor " + name + " is already added."); - processorClasses.put(name, new ProcessorClazz(processorClass, config)); + processorClasses.put(name, new ProcessorClazz(factory)); if (parentNames != null) { for (String parent : parentNames) { @@ -145,17 +143,16 @@ public ProcessorTopology build() { processorMap.put(name, node); } - // create normal processors + // create processors try { for (String name : processorClasses.keySet()) { - ProcessorMetadata metadata = processorClasses.get(name).metadata; - Class processorClass = processorClasses.get(name).clazz; - KafkaProcessor processor = processorClass.getConstructor(ProcessorMetadata.class).newInstance(metadata); + ProcessorFactory processorFactory = processorClasses.get(name).factory; + Processor processor = processorFactory.build(); ProcessorNode node = new ProcessorNode(name, processor); processorMap.put(name, node); } } catch (Exception e) { - throw new KafkaException("KafkaProcessor(String) constructor failed: this should not happen."); + throw new KafkaException("Processor(String) constructor failed: this should not happen."); } // construct topics to sources map @@ -167,8 +164,8 @@ public ProcessorTopology build() { // construct topics to sinks map for (String topic : topicsToSinkNames.keySet()) { SinkNode node = (SinkNode) processorMap.get(topicsToSourceNames.get(topic)); - node.addTopic(topic); topicSinkMap.put(topic, node); + node.addTopic(topic); } // chain children to parents to build the DAG diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java deleted file mode 100644 index 3e94622f6612b..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java +++ /dev/null @@ -1,58 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor.internals; - -import java.util.LinkedList; - -/** - * MinTimestampTracker is a helper class for a sliding window implementation. - * It is assumed that elements are added or removed in a FIFO manner. - * It maintains the minimum timestamp of stamped elements that were added but not yet removed. - */ -public class MinTimestampTracker implements TimestampTracker { - - private final LinkedList> descendingSubsequence = new LinkedList>(); - - public void addStampedElement(Stamped elem) { - if (elem == null) throw new NullPointerException(); - - Stamped minElem = descendingSubsequence.peekLast(); - while (minElem != null && minElem.timestamp >= elem.timestamp) { - descendingSubsequence.removeLast(); - minElem = descendingSubsequence.peekLast(); - } - descendingSubsequence.offerLast(elem); - } - - public void removeStampedElement(Stamped elem) { - if (elem != null && descendingSubsequence.peekFirst() == elem) - descendingSubsequence.removeFirst(); - } - - public int size() { - return descendingSubsequence.size(); - } - - public long get() { - Stamped stamped = descendingSubsequence.peekFirst(); - if (stamped == null) return -1L; - - return stamped.timestamp; - } - -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java index c5a530b6666ea..dbe96b4eb0da0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java @@ -22,24 +22,30 @@ import org.apache.kafka.common.serialization.Deserializer; import java.util.Comparator; +import java.util.HashMap; import java.util.Map; import java.util.PriorityQueue; +import java.util.Set; /** - * A PartitionGroup is composed from a set of partitions. + * A PartitionGroup is composed from a set of partitions; it also keeps track + * of the consumed offsets for each of its partitions. */ public class PartitionGroup { private final Map partitionQueues; + private final Map consumedOffsets; + private final PriorityQueue queuesByTime; - private volatile int totalBuffered; + // since task is thread-safe, we do not need to synchronize on local variables + private int totalBuffered; public PartitionGroup(Map partitionQueues) { - - this.partitionQueues = partitionQueues; this.queuesByTime = new PriorityQueue<>(new Comparator() { + + @Override public int compare(RecordQueue queue1, RecordQueue queue2) { long time1 = queue1.timestamp(); long time2 = queue2.timestamp(); @@ -50,28 +56,41 @@ public int compare(RecordQueue queue1, RecordQueue queue2) { } }); - totalBuffered = 0; + this.partitionQueues = partitionQueues; + + this.consumedOffsets = new HashMap<>(); + + this.totalBuffered = 0; } /** - * Get the next record from the partition with the lowest timestamp to be processed + * Process one record from this partition group's queues + * as the first record from the lowest stamped partition, + * return its partition when completed */ - public StampedRecord nextRecord() { - - // Get the partition with the lowest timestamp. + @SuppressWarnings("unchecked") + public TopicPartition processRecord() { + // get the partition with the lowest timestamp. RecordQueue recordQueue = queuesByTime.poll(); - // Get the first record from this partition's queue. - StampedRecord record = recordQueue.next(); + // get the first record from this partition's queue. + StampedRecord record = recordQueue.get(); totalBuffered--; - // Update the partition's timestamp and re-order it with other partitions. + // process the record by passing it to the source node of the topology + recordQueue.source().process(record.key(), record.value()); + + // update the partition's timestamp and re-order it against other partitions. if (recordQueue.size() > 0) { queuesByTime.offer(recordQueue); } - return record; + // update the consumed offset map. + consumedOffsets.put(recordQueue.partition(), record.offset()); + + // return the processed record's partition + return recordQueue.partition(); } /** @@ -93,8 +112,9 @@ public void putRecord(StampedRecord record, TopicPartition partition) { totalBuffered++; // add this record queue to be considered for processing in the future if it was empty before - if (wasEmpty) + if (wasEmpty) { queuesByTime.offer(recordQueue); + } } public Deserializer keyDeserializer(TopicPartition partition) { @@ -115,13 +135,33 @@ public Deserializer valDeserializer(TopicPartition partition) { return recordQueue.source().valDeserializer; } - public long timestamp() { + public Set partitions() { + return partitionQueues.keySet(); + } - // return the timestamp of this partition-group as the smallest partition timestamp - if (queuesByTime.isEmpty()) + /** + * Return the timestamp of this partition group as the smallest + * partition timestamp among all its partitions + */ + public long timestamp() { + if (queuesByTime.isEmpty()) { return -1L; - else + } else { return queuesByTime.peek().timestamp(); + } + } + + public long offset(TopicPartition partition) { + Long offset = consumedOffsets.get(partition); + + if (offset == null) + throw new KafkaException("No record has ever been consumed for this partition."); + + return offset; + } + + public Map offsets() { + return consumedOffsets; } public int numbuffered(TopicPartition partition) { @@ -136,4 +176,10 @@ public int numbuffered(TopicPartition partition) { public int numbuffered() { return totalBuffered; } + + public void close() { + queuesByTime.clear(); + consumedOffsets.clear(); + partitionQueues.clear(); + } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index dd6d3c48602d3..6b9137abb83c4 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -30,7 +30,7 @@ import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.processor.RecordCollector; import org.apache.kafka.streaming.processor.StateStore; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.RestoreFunc; import org.apache.kafka.streaming.processor.TimestampExtractor; @@ -50,13 +50,9 @@ public class ProcessorContextImpl implements ProcessorContext { private static final Logger log = LoggerFactory.getLogger(ProcessorContextImpl.class); - public final int id; - public final Ingestor ingestor; - public final StreamGroup streamGroup; - + private final int id; + private final StreamTask task; private final Metrics metrics; - private final ProcessorTopology topology; - private final RecordCollectorImpl collector; private final ProcessorStateManager stateMgr; private final Serializer keySerializer; @@ -65,50 +61,34 @@ public class ProcessorContextImpl implements ProcessorContext { private final Deserializer valDeserializer; private boolean initialized; + private ProcessorNode currNode; + private StampedRecord currRecord; @SuppressWarnings("unchecked") public ProcessorContextImpl(int id, - Ingestor ingestor, - ProcessorTopology topology, - RecordCollectorImpl collector, + StreamTask task, StreamingConfig config, Metrics metrics) throws IOException { this.id = id; + this.task = task; this.metrics = metrics; - this.ingestor = ingestor; - this.topology = topology; - this.collector = collector; - - for (String topic : this.topology.sourceTopics()) { - if (!ingestor.topics().contains(topic)) - throw new IllegalArgumentException("topic not subscribed: " + topic); - } 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); - TimestampExtractor extractor = config.getConfiguredInstance(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); - int bufferedRecordsPerPartition = config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); - File stateFile = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG), Integer.toString(id)); - Consumer restoreConsumer = new KafkaConsumer<>(config.getConsumerProperties(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); - this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); - this.streamGroup = new StreamGroup(this, this.ingestor, extractor, bufferedRecordsPerPartition); - - stateMgr.init(); - - initialized = false; - } + Consumer restoreConsumer = new KafkaConsumer<>( + config.getConsumerProperties(), + null /* no callback for restore consumer */, + new ByteArrayDeserializer(), + new ByteArrayDeserializer()); - public void addPartition(TopicPartition partition) { - // update the partition -> source stream map - SourceNode source = topology.source(partition.topic()); + this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); - this.streamGroup.addPartition(partition, source); - this.ingestor.addPartitionStreamToGroup(this.streamGroup, partition); + this.initialized = false; } @Override @@ -116,10 +96,10 @@ public boolean joinable(ProcessorContext o) { ProcessorContextImpl other = (ProcessorContextImpl) o; - if (this.streamGroup != other.streamGroup) + if (this.task != other.task) return false; - Set partitions = this.streamGroup.partitions(); + Set partitions = this.task.partitions(); Map> partitionsById = new HashMap<>(); int firstId = -1; for (TopicPartition partition : partitions) { @@ -171,11 +151,6 @@ public Deserializer valueDeserializer() { return this.valDeserializer; } - @Override - public RecordCollector recordCollector() { - return collector; - } - @Override public File stateDir() { return stateMgr.baseDir(); @@ -199,13 +174,6 @@ public void flush() { stateMgr.flush(); } - public String topic() { - if (streamGroup.record() == null) - throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); - - return streamGroup.record().topic(); - } - @Override public int partition() { if (streamGroup.record() == null) @@ -239,41 +207,13 @@ public void forward(K key, V value) { } } - @Override - public void send(String topic, Object key, Object value) { - collector.send(new ProducerRecord<>(topic, key, value)); - } - - @Override - public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { - if (keySerializer == null || valSerializer == null) - throw new IllegalStateException("key and value serializers must be specified"); - - collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); - } - @Override public void commit() { - streamGroup.commitOffset(); + task.commitOffset(); } @Override - public void schedule(KafkaProcessor processor, long interval) { - streamGroup.schedule(processor, interval); + public void schedule(Processor processor, long interval) { + task.schedule(processor, interval); } - - public void initialized() { - initialized = true; - } - - public Map consumedOffsets() { - return streamGroup.consumedOffsets(); - } - - public void close() throws Exception { - topology.close(); - stateMgr.close(collector.offsets()); - streamGroup.close(); - } - } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java index efdf11add7637..167863172149e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java @@ -17,7 +17,7 @@ package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; import java.util.ArrayList; @@ -29,7 +29,7 @@ public class ProcessorNode { private final List> parents; private final String name; - private final KafkaProcessor processor; + private final Processor processor; public boolean initialized; @@ -37,7 +37,7 @@ public ProcessorNode(String name) { this(name, null); } - public ProcessorNode(String name, KafkaProcessor processor) { + public ProcessorNode(String name, Processor processor) { this.name = name; this.processor = processor; this.parents = new ArrayList<>(); @@ -50,7 +50,7 @@ public String name() { return name; } - public KafkaProcessor processor() { + public Processor processor() { return processor; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java index 58d18ede0f0f5..5052421e0fe9f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java @@ -44,13 +44,19 @@ public class ProcessorStateManager { private final Map restoredOffsets; private final Map checkpointedOffsets; - public ProcessorStateManager(int id, File baseDir, Consumer restoreConsumer) { + public ProcessorStateManager(int id, File baseDir, Consumer restoreConsumer) throws IOException { this.id = id; this.baseDir = baseDir; this.stores = new HashMap<>(); this.restoreConsumer = restoreConsumer; this.restoredOffsets = new HashMap<>(); this.checkpointedOffsets = new HashMap<>(); + + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + this.checkpointedOffsets.putAll(checkpoint.read()); + + // delete the checkpoint file after finish loading its stored offsets + checkpoint.delete(); } public File baseDir() { @@ -61,14 +67,6 @@ public Consumer restoreConsumer() { return this.restoreConsumer; } - public void init() throws IOException { - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); - this.checkpointedOffsets.putAll(checkpoint.read()); - - // delete the checkpoint file after finish loading its stored offsets - checkpoint.delete(); - } - public void register(StateStore store, RestoreFunc restoreFunc) { if (store.name().equals(CHECKPOINT_FILE_NAME)) throw new IllegalArgumentException("Illegal store name: " + CHECKPOINT_FILE_NAME); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java index 364d3d640a5b0..6ae7c3175096d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java @@ -19,6 +19,7 @@ import org.apache.kafka.streaming.processor.ProcessorContext; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -54,6 +55,14 @@ public SinkNode sink(String topic) { return sinkTopics.get(topic); } + public Collection sources() { + return sourceTopics.values(); + } + + public Collection sinks() { + return sinkTopics.values(); + } + /** * Initialize the processors following the DAG reverse ordering * such that parents are always initialized before children @@ -93,12 +102,13 @@ private void init(ProcessorNode node, ProcessorContext context) { public final void close() { // close the processors - // TODO: do we need to follow the DAG ordering + // TODO: do we need to follow the DAG ordering? for (ProcessorNode processorNode : processors.values()) { processorNode.close(); } processors.clear(); sourceTopics.clear(); + sinkTopics.clear(); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java index aaa656d008f73..b97cfbe544e1d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java @@ -17,22 +17,24 @@ package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import java.util.ArrayDeque; +import java.util.PriorityQueue; /** - * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). + * 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 queue = new ArrayDeque<>(); - private final TimestampTracker> timestampTracker = new MinTimestampTracker<>(); + private final ArrayDeque fifoQueue = new ArrayDeque<>(); + private final PriorityQueue timeQueue = new PriorityQueue<>(); - private long offset; + private long partitionTime = -1L; /** * Creates a new instance of RecordQueue @@ -64,34 +66,33 @@ public TopicPartition partition() { * @param record StampedRecord */ public void add(StampedRecord record) { - queue.addLast(record); + fifoQueue.addLast(record); - offset = record.offset(); - timestampTracker.addStampedElement(record); + // only add it to the timestamp tracker queue if its timestamp + // is no smaller than the current partition timestamp + if (record.timestamp >= partitionTime) { + timeQueue.offer(record); + } } /** - * Returns the next record fro the queue + * Returns the get record fro the queue * * @return StampedRecord */ - public StampedRecord next() { - StampedRecord elem = queue.pollFirst(); + public StampedRecord get() { + StampedRecord elem = fifoQueue.pollFirst(); if (elem == null) return null; - timestampTracker.removeStampedElement(elem); + // try to advance the partition timestamp if necessary + timeQueue.remove(elem); - return elem; - } + if (!timeQueue.isEmpty()) { + partitionTime = timeQueue.peek().timestamp; + } - /** - * Returns the highest offset in the queue - * - * @return offset - */ - public long offset() { - return offset; + return elem; } /** @@ -100,7 +101,7 @@ public long offset() { * @return the number of records */ public int size() { - return queue.size(); + return fifoQueue.size(); } /** @@ -109,15 +110,15 @@ public int size() { * @return true if the queue is empty, otherwise false */ public boolean isEmpty() { - return queue.isEmpty(); + return fifoQueue.isEmpty(); } /** - * Returns a timestamp tracked by the TimestampTracker + * Returns the tracked partition timestamp * * @return timestamp */ public long timestamp() { - return timestampTracker.get(); + return partitionTime; } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java index 7a078b681dd4a..dec23759afc12 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java @@ -27,6 +27,7 @@ public Stamped(V value, long timestamp) { this.timestamp = timestamp; } + @Override public int compareTo(Object other) { long otherTimestamp = ((Stamped) other).timestamp; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java index f5496ec62428e..df99ea28e6259 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -17,87 +17,82 @@ package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.processor.Punctuator; import org.apache.kafka.streaming.processor.TimestampExtractor; -import java.util.ArrayDeque; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.Set; /** - * A StreamTask is associated with a {@link PartitionGroup} which is assigned to a StreamThread for processing. + * A StreamTask is associated with a {@link PartitionGroup}, and is assigned to a StreamThread for processing. */ public class StreamTask { private final int id; - private final int desiredUnprocessed; + private final int maxBufferedSize; private final Consumer consumer; private final PartitionGroup partitionGroup; + private final PunctuationQueue punctuationQueue; + private final ProcessorContext processorContext; private final TimestampExtractor timestampExtractor; - private final Map consumedOffsets; - private final PunctuationQueue punctuationQueue = new PunctuationQueue(); - private final ArrayDeque newRecordBuffer = new ArrayDeque<>(); - - private long streamTime = -1; private boolean commitRequested = false; - private StampedRecord currRecord = null; - private ProcessorNode currNode = null; /** * Creates StreamGroup * - * @param consumer the instance of {@link Consumer} - * @param partitions the instance of {@link TimestampExtractor} - * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic + * @param id the ID of this task + * @param consumer the instance of {@link Consumer} + * @param topology the instance of {@link ProcessorTopology} + * @param partitions the collection of assigned {@link TopicPartition} + * @param config the {@link StreamingConfig} specified by the user */ public StreamTask(int id, Consumer consumer, ProcessorTopology topology, Collection partitions, - int desiredUnprocessedPerPartition) { + StreamingConfig config) { + this.id = id; this.consumer = consumer; - this.desiredUnprocessed = desiredUnprocessedPerPartition; - this.consumedOffsets = new HashMap<>(); + this.punctuationQueue = new PunctuationQueue(); + this.maxBufferedSize = config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + this.timestampExtractor = config.getConfiguredInstance(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); - // create partition queues and pipe them to corresponding source nodes in the topology + // create queues for each assigned partition and associate them + // to corresponding source nodes in the processor topology Map partitionQueues = new HashMap<>(); + for (TopicPartition partition : partitions) { - RecordQueue queue = createRecordQueue(partition, topology.source(partition.topic())); + SourceNode source = topology.source(partition.topic()); + RecordQueue queue = createRecordQueue(partition, source); partitionQueues.put(partition, queue); } - this.partitionGroup = new PartitionGroup(partitionQueues); - } - public int id() { - return id; - } + this.partitionGroup = new PartitionGroup(partitionQueues); - public StampedRecord record() { - return currRecord; - } + // initialize the topology with its own context + this.processorContext = new ProcessorContextImpl(id, this, config, ) - public ProcessorNode node() { - return currNode; + topology.init(); } - public void setNode(ProcessorNode node) { - currNode = node; + public int id() { + return id; } public Set partitions() { - return queuesPerPartition.keySet(); + return this.partitionGroup.partitions(); } /** @@ -146,62 +141,38 @@ public boolean process() { synchronized (this) { boolean readyForNextExecution = false; - // take the next record queue with the smallest estimated timestamp to process - long timestamp = partitionGroup.timestamp(); - StampedRecord record = partitionGroup.nextRecord(); - - currRecord = recordQueue.next(); - currNode = recordQueue.source(); - - if (streamTime < timestamp) streamTime = timestamp; - - currNode.process(currRecord.key(), currRecord.value()); - consumedOffsets.put(recordQueue.partition(), currRecord.offset()); + // process the next record from the partition-group queue + // with the smallest estimated timestamp to process + TopicPartition partition = partitionGroup.processRecord(); - // TODO: local state flush and downstream producer flush - // need to be done altogether with offset commit atomically if (commitRequested) { - // flush local state - context.flush(); - - // flush produced records in the downstream - context.recordCollector().flush(); - - // commit consumed offsets - ingestor.commit(consumedOffsets()); + // TODO: flush the following states atomically + // 1) flush local state + // 2) commit consumed offsets + // 3) flush produced records in the downstream } - if (commitRequested) ingestor.commit(Collections.singletonMap( - new TopicPartition(currRecord.topic(), currRecord.partition()), - currRecord.offset())); - - // update this record queue's estimated timestamp - if (recordQueue.size() > 0) { + // we can continue processing this task as long as its + // partition group still have buffered records + if (partitionGroup.numbuffered() > 0) { readyForNextExecution = true; } - if (recordQueue.size() == this.desiredUnprocessed) { - ingestor.unpause(recordQueue.partition(), recordQueue.offset()); + // if after processing this record, 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); } - buffered--; - currRecord = null; - - punctuationQueue.mayPunctuate(streamTime); + // possibly trigger registered punctuation functions if + // partition group's time has reached the defined stamp + long timestamp = partitionGroup.timestamp(); + punctuationQueue.mayPunctuate(timestamp); return readyForNextExecution; } } - /** - * Returns consumed offsets - * - * @return the map of partition to consumed offset - */ - public Map consumedOffsets() { - return this.consumedOffsets; - } - /** * Request committing the current record's offset */ @@ -210,21 +181,10 @@ public void commitOffset() { } public void close() { - queuesByTime.clear(); - queuesPerPartition.clear(); + this.partitionGroup.close(); } protected RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { return new RecordQueue(partition, source); } - - private static class NewRecords { - final TopicPartition partition; - final Iterator> iterator; - - NewRecords(TopicPartition partition, Iterator> iterator) { - this.partition = partition; - this.iterator = iterator; - } - } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index 05123a54acd0a..60229bc35893e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -39,6 +39,8 @@ 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.streaming.processor.TimestampExtractor; +import org.apache.kafka.streaming.processor.TopologyBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,7 +55,7 @@ public class StreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(StreamThread.class); private final Consumer consumer; - private final ProcessorTopology topology; + private final TopologyBuilder builder; private final RecordCollectorImpl collector; private final Map tasks = new HashMap<>(); private final Metrics metrics; @@ -86,11 +88,11 @@ public void onPartitionsRevoked(Consumer consumer, Collection assignment) { if (part.partition() == id) partitionsForTask.add(part); - // creat the task - task = new StreamTask(id, consumer, topology, partitionsForTask, config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG)); + // create the task + task = new StreamTask(id, consumer, builder.build(), partitionsForTask, config); tasks.put(id, task); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java deleted file mode 100644 index 80fc20898ad09..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java +++ /dev/null @@ -1,56 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor.internals; - -/** - * TimestampTracker is a helper class for a sliding window implementation. - * It is assumed that elements are added or removed in a FIFO manner. - * It maintains the timestamp, like the min timestamp, the max timestamp, etc. of stamped elements - * that were added but not yet removed. - */ -public interface TimestampTracker { - - /** - * Adds a stamped elements to this tracker. - * - * @param elem the added element - */ - void addStampedElement(Stamped elem); - - /** - * Removed a stamped elements to this tracker. - * - * @param elem the removed element - */ - void removeStampedElement(Stamped elem); - - /** - * Returns the timestamp - * - * @return timestamp, or -1L when empty - */ - long get(); - - /** - * Returns the size of internal structure. The meaning of "size" depends on the implementation. - * - * @return size - */ - int size(); - -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/state/OffsetCheckpoint.java b/stream/src/main/java/org/apache/kafka/streaming/state/OffsetCheckpoint.java index 65c1ecc97fb8e..68bc35bc42f6f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/state/OffsetCheckpoint.java +++ b/stream/src/main/java/org/apache/kafka/streaming/state/OffsetCheckpoint.java @@ -43,7 +43,7 @@ * . * <topic_name_n> <partition_n> <offset_n> * - * The first line contains a number designating the format version (currently 0), the next line contains + * 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. */ diff --git a/stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java deleted file mode 100644 index 8346b78447fdd..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/MinTimestampTrackerTest.java +++ /dev/null @@ -1,95 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming; - -import static org.junit.Assert.assertEquals; - -import org.apache.kafka.streaming.processor.internals.MinTimestampTracker; -import org.apache.kafka.streaming.processor.internals.Stamped; -import org.junit.Test; - -public class MinTimestampTrackerTest { - - private Stamped elem(long timestamp) { - return new Stamped<>("", timestamp); - } - - @SuppressWarnings("unchecked") - @Test - public void testTracking() { - MinTimestampTracker 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.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(100L, tracker.get()); - - // add 101 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(100L, tracker.get()); - - // remove 100 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(101L, tracker.get()); - - // add 102 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(101L, tracker.get()); - - // add 98 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); - - // add 99 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); - - // add 100 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); - - // remove 101 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(98L, tracker.get()); - - // remove 102 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(98L, tracker.get()); - - // remove 98 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(99L, tracker.get()); - - // remove 99 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(100L, tracker.get()); - - // remove 100 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(-1L, tracker.get()); - - assertEquals(insertionIndex, removalIndex); - } - -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/MinTimestampTrackerTest.java deleted file mode 100644 index 77836967a50b3..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/MinTimestampTrackerTest.java +++ /dev/null @@ -1,95 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.internals; - -import static org.junit.Assert.assertEquals; - -import org.apache.kafka.streaming.processor.internals.MinTimestampTracker; -import org.apache.kafka.streaming.processor.internals.Stamped; -import org.junit.Test; - -public class MinTimestampTrackerTest { - - private Stamped elem(long timestamp) { - return new Stamped<>("", timestamp); - } - - @SuppressWarnings("unchecked") - @Test - public void testTracking() { - MinTimestampTracker 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.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(100L, tracker.get()); - - // add 101 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(100L, tracker.get()); - - // remove 100 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(101L, tracker.get()); - - // add 102 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(101L, tracker.get()); - - // add 98 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); - - // add 99 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); - - // add 100 - tracker.addStampedElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); - - // remove 101 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(98L, tracker.get()); - - // remove 102 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(98L, tracker.get()); - - // remove 98 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(99L, tracker.get()); - - // remove 99 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(100L, tracker.get()); - - // remove 100 - tracker.removeStampedElement((Stamped) elems[removalIndex++]); - assertEquals(-1L, tracker.get()); - - assertEquals(insertionIndex, removalIndex); - } - -} diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java index 4b1fba293f9e6..81dbf49a5fe87 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessor.java @@ -17,11 +17,11 @@ package org.apache.kafka.test; -import org.apache.kafka.streaming.processor.KafkaProcessor; +import org.apache.kafka.streaming.processor.Processor; import java.util.ArrayList; -public class MockProcessor extends KafkaProcessor { +public class MockProcessor extends Processor { public final ArrayList processed = new ArrayList<>(); public final ArrayList punctuated = new ArrayList<>(); diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 5e9e11b950c9f..7f9c7f6b626eb 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -17,6 +17,7 @@ package org.apache.kafka.test; +<<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD:stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java @@ -81,6 +82,9 @@ >>>>>>> Refactor Processor and KStream APIs ======= import org.apache.kafka.streaming.processor.KafkaProcessor; +======= +import org.apache.kafka.streaming.processor.Processor; +>>>>>>> wip: refactor StreamTask and ProcessorContext, fix RecordQueue timestamp tracking import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.processor.RecordCollector; import org.apache.kafka.streaming.processor.RestoreFunc; @@ -273,7 +277,7 @@ public void send(String topic, Object key, Object value, Serializer keyS } @Override - public void schedule(KafkaProcessor processor, long interval) { + public void schedule(Processor processor, long interval) { throw new UnsupportedOperationException("schedule() not supported"); } From 60e953bd14f0fa5d6bde8399f223fdb71a4accf9 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 25 Aug 2015 17:47:39 -0700 Subject: [PATCH 170/275] record current Node and current Record in StreamTask, add ProcessorFactory --- .../kafka/stream/topology/SlidingWindow.java | 12 +- .../streaming/processor/ProcessorContext.java | 5 - ...{Punctuator.java => ProcessorFactory.java} | 4 +- .../streaming/processor/RecordCollector.java | 30 ---- .../processor/internals/Ingestor.java | 41 ------ .../processor/internals/IngestorImpl.java | 133 ------------------ .../processor/internals/PartitionGroup.java | 62 +++----- .../internals/ProcessorContextImpl.java | 35 +++-- .../processor/internals/PunctuationQueue.java | 2 +- .../internals/PunctuationSchedule.java | 10 +- ...ollectorImpl.java => RecordCollector.java} | 10 +- .../processor/internals/RecordQueue.java | 6 +- .../processor/internals/SinkNode.java | 3 +- .../processor/internals/StreamTask.java | 58 ++++++-- .../processor/internals/StreamThread.java | 7 +- .../streaming/state/MeteredKeyValueStore.java | 5 +- .../kafka/streaming/StreamGroupTest.java | 1 - .../streaming/internals/StreamGroupTest.java | 1 - .../org/apache/kafka/test/MockIngestor.java | 72 ---------- .../kafka/test/MockProcessorContext.java | 24 +--- 20 files changed, 126 insertions(+), 395 deletions(-) rename stream/src/main/java/org/apache/kafka/streaming/processor/{Punctuator.java => ProcessorFactory.java} (92%) delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/RecordCollector.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/Ingestor.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/IngestorImpl.java rename stream/src/main/java/org/apache/kafka/streaming/processor/internals/{RecordCollectorImpl.java => RecordCollector.java} (90%) delete mode 100644 stream/src/test/java/org/apache/kafka/test/MockIngestor.java diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java index 7bbe03ea58ceb..de157a4e04f8a 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java @@ -23,12 +23,22 @@ import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serializer; +<<<<<<< HEAD:stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java import org.apache.kafka.stream.KStreamContext; import org.apache.kafka.stream.RecordCollector; import org.apache.kafka.stream.RestoreFunc; import org.apache.kafka.stream.topology.internals.WindowSupport; import org.apache.kafka.stream.util.FilteredIterator; import org.apache.kafka.clients.processor.internals.Stamped; +======= +import org.apache.kafka.streaming.kstream.internals.FilteredIterator; +import org.apache.kafka.streaming.kstream.internals.WindowSupport; +import org.apache.kafka.streaming.processor.internals.ProcessorContextImpl; +import org.apache.kafka.streaming.processor.internals.RecordCollector; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.RestoreFunc; +import org.apache.kafka.streaming.processor.internals.Stamped; +>>>>>>> record current Node and current Record in StreamTask, add ProcessorFactory:stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java import java.util.HashMap; import java.util.Iterator; @@ -173,7 +183,7 @@ public void flush() { IntegerSerializer intSerializer = new IntegerSerializer(); ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); - RecordCollector collector = context.recordCollector(); + RecordCollector collector = ((ProcessorContextImpl)context).recordCollector(); for (Map.Entry> entry : map.entrySet()) { ValueList values = entry.getValue(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java index 4bb46604377de..15d2e2981eee7 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java @@ -84,11 +84,6 @@ public interface ProcessorContext { */ void register(StateStore store, RestoreFunc restoreFunc); - /** - * Flush the local state of this context - */ - void flush(); - void schedule(Processor processor, long interval); void forward(K key, V value); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/Punctuator.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorFactory.java similarity index 92% rename from stream/src/main/java/org/apache/kafka/streaming/processor/Punctuator.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorFactory.java index 4619263db3b4c..1d1c4816d8cce 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/Punctuator.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorFactory.java @@ -17,7 +17,7 @@ package org.apache.kafka.streaming.processor; -public interface Punctuator { +public interface ProcessorFactory { - void punctuate(long streamTime); + Processor build(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/RecordCollector.java b/stream/src/main/java/org/apache/kafka/streaming/processor/RecordCollector.java deleted file mode 100644 index e65b2b93f84ac..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/RecordCollector.java +++ /dev/null @@ -1,30 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor; - -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.Serializer; - -public interface RecordCollector { - - void send(ProducerRecord record); - - void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer); - - void flush(); -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Ingestor.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Ingestor.java deleted file mode 100644 index b66808abda579..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Ingestor.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor.internals; - -import org.apache.kafka.common.TopicPartition; - -import java.util.Map; -import java.util.Set; - -public interface Ingestor { - - Set topics(); - - void poll(long timeoutMs); - - void pause(TopicPartition partition); - - void unpause(TopicPartition partition, long offset); - - void commit(Map offsets); - - int numPartitions(String topic); - - void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition); - -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/IngestorImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/IngestorImpl.java deleted file mode 100644 index 886cb5c799bd7..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/IngestorImpl.java +++ /dev/null @@ -1,133 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor.internals; - -import org.apache.kafka.clients.consumer.CommitType; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.common.TopicPartition; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -public class IngestorImpl implements Ingestor { - - private static final Logger log = LoggerFactory.getLogger(IngestorImpl.class); - - private final Set topics; - private final Consumer consumer; - private final Set unpaused = new HashSet<>(); - private final Map partitionGroups = new HashMap<>(); - - public IngestorImpl(Consumer consumer, Set topics) { - this.consumer = consumer; - this.topics = Collections.unmodifiableSet(topics); - for (String topic : this.topics) consumer.subscribe(topic); - } - - public void open() { - for (String topic : this.topics) consumer.subscribe(topic); - } - - public void init() { - unpaused.clear(); - unpaused.addAll(consumer.subscriptions()); - } - - @Override - public Set topics() { - return topics; - } - - @Override - public void poll(long timeoutMs) { - synchronized (this) { - ConsumerRecords records = consumer.poll(timeoutMs); - - for (TopicPartition partition : unpaused) { - StreamGroup streamGroup = partitionGroups.get(partition); - - if (streamGroup != null) - streamGroup.addRecords(partition, records.records(partition).iterator()); - else - log.warn("unused topic: " + partition.topic()); - } - } - } - - @Override - public void pause(TopicPartition partition) { - synchronized (this) { - consumer.seek(partition, Long.MAX_VALUE); // hack: stop consuming from this partition by setting a big offset - unpaused.remove(partition); - } - } - - @Override - public void unpause(TopicPartition partition, long lastOffset) { - synchronized (this) { - consumer.seek(partition, lastOffset); - unpaused.add(partition); - } - } - - @Override - public void commit(Map offsets) { - synchronized (this) { - consumer.commit(offsets, CommitType.SYNC); - } - } - - @Override - public int numPartitions(String topic) { - return consumer.partitionsFor(topic).size(); - } - - @SuppressWarnings("unchecked") - @Override - public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition) { - synchronized (this) { - partitionGroups.put(partition, streamGroup); - unpaused.add(partition); - } - } - - public void clear() { - unpaused.clear(); - partitionGroups.clear(); - } - - public boolean commitNeeded(Map offsets) { - for (TopicPartition tp : offsets.keySet()) { - if (consumer.committed(tp) != offsets.get(tp)) { - return true; - } - } - return false; - } - - public void close() { - consumer.close(); - clear(); - } -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java index dbe96b4eb0da0..a473c4eff0891 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java @@ -28,15 +28,12 @@ import java.util.Set; /** - * A PartitionGroup is composed from a set of partitions; it also keeps track - * of the consumed offsets for each of its partitions. + * A PartitionGroup is composed from a set of partitions. */ public class PartitionGroup { private final Map partitionQueues; - private final Map consumedOffsets; - private final PriorityQueue queuesByTime; // since task is thread-safe, we do not need to synchronize on local variables @@ -58,39 +55,40 @@ public int compare(RecordQueue queue1, RecordQueue queue2) { this.partitionQueues = partitionQueues; - this.consumedOffsets = new HashMap<>(); - this.totalBuffered = 0; } /** - * Process one record from this partition group's queues - * as the first record from the lowest stamped partition, - * return its partition when completed + * Get one record from the specified partition queue */ - @SuppressWarnings("unchecked") - public TopicPartition processRecord() { - // get the partition with the lowest timestamp. - RecordQueue recordQueue = queuesByTime.poll(); + public StampedRecord getRecord(RecordQueue queue) { + // get the first record from this queue. + StampedRecord record = queue.poll(); + + // update the partition's timestamp and re-order it against other partitions. + + queuesByTime.remove(queue); - // get the first record from this partition's queue. - StampedRecord record = recordQueue.get(); + if (queue.size() > 0) { + queuesByTime.offer(queue); + } totalBuffered--; - // process the record by passing it to the source node of the topology - recordQueue.source().process(record.key(), record.value()); + return record; + } - // update the partition's timestamp and re-order it against other partitions. - if (recordQueue.size() > 0) { - queuesByTime.offer(recordQueue); - } + /** + * Get the next partition queue that has the lowest timestamp to process + */ + public RecordQueue nextQueue() { + // get the partition with the lowest timestamp + RecordQueue recordQueue = queuesByTime.peek(); - // update the consumed offset map. - consumedOffsets.put(recordQueue.partition(), record.offset()); + if (recordQueue == null) + throw new KafkaException("No records have ever been added to this partition group yet."); - // return the processed record's partition - return recordQueue.partition(); + return recordQueue; } /** @@ -151,19 +149,6 @@ public long timestamp() { } } - public long offset(TopicPartition partition) { - Long offset = consumedOffsets.get(partition); - - if (offset == null) - throw new KafkaException("No record has ever been consumed for this partition."); - - return offset; - } - - public Map offsets() { - return consumedOffsets; - } - public int numbuffered(TopicPartition partition) { RecordQueue recordQueue = partitionQueues.get(partition); @@ -179,7 +164,6 @@ public int numbuffered() { public void close() { queuesByTime.clear(); - consumedOffsets.clear(); partitionQueues.clear(); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index 6b9137abb83c4..1691632d879a0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; @@ -28,11 +27,9 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RecordCollector; import org.apache.kafka.streaming.processor.StateStore; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.RestoreFunc; -import org.apache.kafka.streaming.processor.TimestampExtractor; import org.slf4j.Logger; @@ -53,6 +50,7 @@ public class ProcessorContextImpl implements ProcessorContext { private final int id; private final StreamTask task; private final Metrics metrics; + private final RecordCollector collector; private final ProcessorStateManager stateMgr; private final Serializer keySerializer; @@ -61,17 +59,17 @@ public class ProcessorContextImpl implements ProcessorContext { private final Deserializer valDeserializer; private boolean initialized; - private ProcessorNode currNode; - private StampedRecord currRecord; @SuppressWarnings("unchecked") public ProcessorContextImpl(int id, StreamTask task, StreamingConfig config, + RecordCollector collector, Metrics metrics) throws IOException { this.id = id; this.task = task; this.metrics = metrics; + this.collector = collector; this.keySerializer = config.getConfiguredInstance(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class); this.valSerializer = config.getConfiguredInstance(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class); @@ -91,6 +89,10 @@ public ProcessorContextImpl(int id, this.initialized = false; } + public RecordCollector recordCollector() { + return this.collector; + } + @Override public boolean joinable(ProcessorContext o) { @@ -170,39 +172,42 @@ public void register(StateStore store, RestoreFunc restoreFunc) { } @Override - public void flush() { - stateMgr.flush(); + 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 (streamGroup.record() == null) + if (task.record() == null) throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); - return streamGroup.record().partition(); + return task.record().partition(); } @Override public long offset() { - if (this.streamGroup.record() == null) + 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.streamGroup.record().offset(); + return this.task.record().offset(); } @Override public long timestamp() { - if (streamGroup.record() == null) + if (task.record() == null) throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); - return streamGroup.record().timestamp; + return task.record().timestamp; } @Override @SuppressWarnings("unchecked") public void forward(K key, V value) { - for (ProcessorNode childNode : (List>) streamGroup.node().children()) { - streamGroup.setNode(childNode); + for (ProcessorNode childNode : (List>) task.node().children()) { + task.node(childNode); childNode.process(key, value); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationQueue.java index 2e7cd77420b83..68eb1bdfba7a8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationQueue.java @@ -41,7 +41,7 @@ public void mayPunctuate(long streamTime) { while (top != null && top.timestamp <= streamTime) { PunctuationSchedule sched = top; pq.poll(); - sched.punctuator().punctuate(streamTime); + sched.processor().punctuate(streamTime); pq.add(sched.next()); top = pq.peek(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java index 6f67e3e09dfe0..0d55876d10580 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java @@ -17,18 +17,18 @@ package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.streaming.processor.Punctuator; +import org.apache.kafka.streaming.processor.Processor; -public class PunctuationSchedule extends Stamped { +public class PunctuationSchedule extends Stamped { final long interval; - public PunctuationSchedule(Punctuator punctuator, long interval) { - super(punctuator, System.currentTimeMillis() + interval); + public PunctuationSchedule(Processor processor, long interval) { + super(processor, System.currentTimeMillis() + interval); this.interval = interval; } - public Punctuator punctuator() { + public Processor processor() { return value; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollectorImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollector.java similarity index 90% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollectorImpl.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollector.java index 878f5ce4026bb..1be19aebcd729 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollectorImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollector.java @@ -17,7 +17,6 @@ package org.apache.kafka.streaming.processor.internals; -import org.apache.kafka.streaming.processor.RecordCollector; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -30,9 +29,9 @@ import java.util.HashMap; import java.util.Map; -public class RecordCollectorImpl implements RecordCollector { +public class RecordCollector { - private static final Logger log = LoggerFactory.getLogger(RecordCollectorImpl.class); + private static final Logger log = LoggerFactory.getLogger(RecordCollector.class); private final Producer producer; private final Map offsets; @@ -50,26 +49,23 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { private final Serializer valueSerializer; - public RecordCollectorImpl(Producer producer, Serializer keySerializer, Serializer valueSerializer) { + public RecordCollector(Producer producer, Serializer keySerializer, Serializer valueSerializer) { this.producer = producer; this.offsets = new HashMap<>(); this.keySerializer = keySerializer; this.valueSerializer = valueSerializer; } - @Override public void send(ProducerRecord record) { send(record, this.keySerializer, this.valueSerializer); } - @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); this.producer.send(new ProducerRecord<>(record.topic(), keyBytes, valBytes), callback); } - @Override public void flush() { this.producer.flush(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java index b97cfbe544e1d..6e9210d9076d1 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java @@ -61,7 +61,7 @@ public TopicPartition partition() { } /** - * Adds a StampedRecord to the queue + * Add a {@link StampedRecord} into the queue * * @param record StampedRecord */ @@ -76,11 +76,11 @@ public void add(StampedRecord record) { } /** - * Returns the get record fro the queue + * Get the next {@link StampedRecord} from the queue * * @return StampedRecord */ - public StampedRecord get() { + public StampedRecord poll() { StampedRecord elem = fifoQueue.pollFirst(); if (elem == null) return null; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java index 36ebd3d53fe37..f3cbadf2deda4 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java @@ -20,7 +20,6 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RecordCollector; import java.util.ArrayList; import java.util.List; @@ -53,7 +52,7 @@ public void init(ProcessorContext context) { @Override public void process(K key, V value) { // send to all the registered topics - RecordCollector collector = context.recordCollector(); + RecordCollector collector = ((ProcessorContextImpl)context).recordCollector(); for (String topic : topics) { collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java index df99ea28e6259..04795092355b6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -17,15 +17,18 @@ package org.apache.kafka.streaming.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.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.streaming.StreamingConfig; +import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.Punctuator; import org.apache.kafka.streaming.processor.TimestampExtractor; +import java.io.IOException; import java.util.Collection; import java.util.HashMap; import java.util.Iterator; @@ -46,7 +49,11 @@ public class StreamTask { private final ProcessorContext processorContext; private final TimestampExtractor timestampExtractor; + private final Map consumedOffsets; + private boolean commitRequested = false; + private StampedRecord currRecord = null; + private ProcessorNode currNode = null; /** * Creates StreamGroup @@ -61,6 +68,7 @@ public StreamTask(int id, Consumer consumer, ProcessorTopology topology, Collection partitions, + RecordCollector collector, StreamingConfig config) { this.id = id; @@ -82,9 +90,16 @@ public StreamTask(int id, this.partitionGroup = new PartitionGroup(partitionQueues); // initialize the topology with its own context - this.processorContext = new ProcessorContextImpl(id, this, config, ) + try { + this.processorContext = new ProcessorContextImpl(id, this, config, collector, new Metrics()); + } catch (IOException e) { + throw new KafkaException("Error while creating the state manager in processor context."); + } + + topology.init(this.processorContext); - topology.init(); + // initialize the consumed offset cache + this.consumedOffsets = new HashMap<>(); } public int id() { @@ -126,11 +141,11 @@ public void addRecords(TopicPartition partition, Iterator consumer; private final TopologyBuilder builder; - private final RecordCollectorImpl collector; + private final RecordCollector collector; private final Map tasks = new HashMap<>(); private final Metrics metrics; private final Time time; @@ -100,7 +99,7 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce Producer producer = new KafkaProducer<>(config.getProducerProperties(), new ByteArraySerializer(), new ByteArraySerializer()); - this.collector = new RecordCollectorImpl(producer, + this.collector = new RecordCollector(producer, (Serializer) config.getConfiguredInstance(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, Serializer.class), (Serializer) config.getConfiguredInstance(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Serializer.class)); @@ -264,7 +263,7 @@ private void addPartitions(Collection assignment) { partitionsForTask.add(part); // create the task - task = new StreamTask(id, consumer, builder.build(), partitionsForTask, config); + task = new StreamTask(id, consumer, builder.build(), partitionsForTask, collector, config); tasks.put(id, task); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java index d7664ed5e9b50..93f7c093a7107 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java @@ -18,7 +18,6 @@ package org.apache.kafka.streaming.state; import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RecordCollector; import org.apache.kafka.streaming.processor.RestoreFunc; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.MetricName; @@ -32,6 +31,8 @@ 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.streaming.processor.internals.ProcessorContextImpl; +import org.apache.kafka.streaming.processor.internals.RecordCollector; import java.util.HashSet; import java.util.List; @@ -212,7 +213,7 @@ public void flush() { } private void logChange() { - RecordCollector collector = context.recordCollector(); + RecordCollector collector = ((ProcessorContextImpl)context).recordCollector(); Serializer keySerializer = (Serializer) context.keySerializer(); Serializer valueSerializer = (Serializer) context.valueSerializer(); diff --git a/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java index a53cf20d72286..4cf2d99556262 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.processor.internals.StreamGroup; -import org.apache.kafka.test.MockIngestor; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.MockSourceNode; import org.junit.Test; diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java index daf28eb3aee80..496174ad52872 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.processor.internals.StreamGroup; -import org.apache.kafka.test.MockIngestor; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.MockSourceNode; import org.junit.Test; diff --git a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java b/stream/src/test/java/org/apache/kafka/test/MockIngestor.java deleted file mode 100644 index 1624feb47ba2a..0000000000000 --- a/stream/src/test/java/org/apache/kafka/test/MockIngestor.java +++ /dev/null @@ -1,72 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.test; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streaming.processor.internals.Ingestor; -import org.apache.kafka.streaming.processor.internals.StreamGroup; - -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -public class MockIngestor implements Ingestor { - - private HashMap streamSynchronizers = new HashMap<>(); - - public HashSet paused = new HashSet<>(); - - @Override - public Set topics() { - return null; - } - - @Override - public void poll(long timeoutMs) { - } - - @Override - public void pause(TopicPartition partition) { - paused.add(partition); - } - - @Override - public void unpause(TopicPartition partition, long offset) { - paused.remove(partition); - } - - @Override - public void commit(Map offsets) { /* do nothing */} - - @Override - public int numPartitions(String topic) { - return 1; - } - - @Override - public void addPartitionStreamToGroup(StreamGroup streamGroup, TopicPartition partition) { - streamSynchronizers.put(partition, streamGroup); - } - - public void addRecords(TopicPartition partition, Iterable> records) { - streamSynchronizers.get(partition).addRecords(partition, records.iterator()); - } - -} diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 7f9c7f6b626eb..3edae79e66092 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -86,7 +86,6 @@ import org.apache.kafka.streaming.processor.Processor; >>>>>>> wip: refactor StreamTask and ProcessorContext, fix RecordQueue timestamp tracking import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RecordCollector; import org.apache.kafka.streaming.processor.RestoreFunc; import org.apache.kafka.streaming.processor.StateStore; >>>>>>> Some package and class renaming, fix KafkaSource constrcution at builder @@ -241,11 +240,6 @@ public Deserializer valueDeserializer() { return deserializer; } - @Override - public RecordCollector recordCollector() { - throw new UnsupportedOperationException("recordCollector() not supported."); - } - @Override public File stateDir() { throw new UnsupportedOperationException("stateDir() not supported."); @@ -262,23 +256,13 @@ public void register(StateStore store, RestoreFunc func) { } @Override - public void flush() { - throw new UnsupportedOperationException("flush() not supported."); - } - - @Override - public void send(String topic, Object key, Object value) { - throw new UnsupportedOperationException("send() not supported."); - } - - @Override - public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { - throw new UnsupportedOperationException("send() not supported."); + public void schedule(Processor processor, long interval) { + throw new UnsupportedOperationException("schedule() not supported"); } @Override - public void schedule(Processor processor, long interval) { - throw new UnsupportedOperationException("schedule() not supported"); + public void forward(K key, V value) { + throw new UnsupportedOperationException("forward() not supported"); } @Override From 0f59d1658ef1b06f32d34033a9e08eba2475c33b Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 26 Aug 2015 12:13:06 -0700 Subject: [PATCH 171/275] address Yasu's comments --- .../internals/MinTimestampTracker.java | 59 ++++++++++++ .../processor/internals/RecordQueue.java | 25 +++-- .../processor/internals/StreamTask.java | 2 +- .../processor/internals/TimestampTracker.java | 58 ++++++++++++ .../internals/MinTimestampTrackerTest.java | 93 +++++++++++++++++++ 5 files changed, 222 insertions(+), 15 deletions(-) create mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java create mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java new file mode 100644 index 0000000000000..c9bc580ece19b --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.processor.internals; + +import java.util.LinkedList; + +/** + * MinTimestampTracker implements {@link TimestampTracker} that maintains the min + * timestamp of the maintained stamped elements. + */ +public class MinTimestampTracker implements TimestampTracker { + + private final LinkedList> descendingSubsequence = new LinkedList<>(); + + public void addElement(Stamped elem) { + if (elem == null) throw new NullPointerException(); + + Stamped minElem = descendingSubsequence.peekLast(); + while (minElem != null && minElem.timestamp >= elem.timestamp) { + descendingSubsequence.removeLast(); + minElem = descendingSubsequence.peekLast(); + } + descendingSubsequence.offerLast(elem); + } + + public void removeElement(Stamped elem) { + if (elem != null && descendingSubsequence.peekFirst() == elem) + descendingSubsequence.removeFirst(); + } + + public int size() { + return descendingSubsequence.size(); + } + + public long get() { + Stamped stamped = descendingSubsequence.peekFirst(); + + if (stamped == null) + return TimestampTracker.NOT_KNOWN; + else + return stamped.timestamp; + } + +} \ No newline at end of file diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java index 6e9210d9076d1..93e9420e5bd57 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java @@ -17,10 +17,10 @@ package org.apache.kafka.streaming.processor.internals; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import java.util.ArrayDeque; -import java.util.PriorityQueue; /** * RecordQueue is a FIFO queue of {@link StampedRecord} (ConsumerRecord + timestamp). It also keeps track of the @@ -32,9 +32,9 @@ public class RecordQueue { private final SourceNode source; private final TopicPartition partition; private final ArrayDeque fifoQueue = new ArrayDeque<>(); - private final PriorityQueue timeQueue = new PriorityQueue<>(); + private final TimestampTracker> timeTracker = new MinTimestampTracker<>(); - private long partitionTime = -1L; + private long partitionTime = TimestampTracker.NOT_KNOWN; /** * Creates a new instance of RecordQueue @@ -67,12 +67,7 @@ public TopicPartition partition() { */ public void add(StampedRecord record) { fifoQueue.addLast(record); - - // only add it to the timestamp tracker queue if its timestamp - // is no smaller than the current partition timestamp - if (record.timestamp >= partitionTime) { - timeQueue.offer(record); - } + timeTracker.addElement(record); } /** @@ -85,12 +80,14 @@ public StampedRecord poll() { if (elem == null) return null; - // try to advance the partition timestamp if necessary - timeQueue.remove(elem); + timeTracker.removeElement(elem); + + // only advance the partition timestamp if its currently + // tracked min timestamp has exceeded its value + long timestamp = timeTracker.get(); - if (!timeQueue.isEmpty()) { - partitionTime = timeQueue.peek().timestamp; - } + if (timestamp > partitionTime) + partitionTime = timestamp; return elem; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java index 04795092355b6..751f504c8f778 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -56,7 +56,7 @@ public class StreamTask { private ProcessorNode currNode = null; /** - * Creates StreamGroup + * Create {@link StreamTask} with its assigned partitions * * @param id the ID of this task * @param consumer the instance of {@link Consumer} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java new file mode 100644 index 0000000000000..fdd79dc8dfdbc --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.processor.internals; + +/** + * TimestampTracker is a helper class for a sliding window implementation. + * It is assumed that stamped elements are added or removed in a FIFO manner. + * It maintains the timestamp, such as the min timestamp, the max timestamp, etc. + * of stamped elements that were added but not yet removed. + */ +public interface TimestampTracker { + + static final long NOT_KNOWN = -1L; + + /** + * Adds a stamped elements to this tracker. + * + * @param elem the added element + */ + void addElement(Stamped elem); + + /** + * Removed a stamped elements to this tracker. + * + * @param elem the removed element + */ + void removeElement(Stamped elem); + + /** + * Returns the current tracked timestamp + * + * @return timestamp, or {@link #NOT_KNOWN} when empty + */ + long get(); + + /** + * Returns the size of internal structure. The meaning of "size" depends on the implementation. + * + * @return size + */ + int size(); + +} \ No newline at end of file diff --git a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java new file mode 100644 index 0000000000000..74be3f43122df --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java @@ -0,0 +1,93 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.processor.internals; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +public class MinTimestampTrackerTest { + + private Stamped elem(long timestamp) { + return new Stamped<>("", timestamp); + } + + @SuppressWarnings("unchecked") + @Test + public void testTracking() { + TimestampTracker tracker = new MinTimestampTracker<>(); + + Object[] elems = new Object[]{ + elem(100), elem(101), elem(102), elem(98), elem(99), elem(100) + }; + + int insertionIndex = 0; + int removalIndex = 0; + + // add 100 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); + + // add 101 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); + + // remove 100 + tracker.addElement((Stamped) elems[removalIndex++]); + assertEquals(101L, tracker.get()); + + // add 102 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(101L, tracker.get()); + + // add 98 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // add 99 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // add 100 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // remove 101 + tracker.addElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); + + // remove 102 + tracker.addElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); + + // remove 98 + tracker.addElement((Stamped) elems[removalIndex++]); + assertEquals(99L, tracker.get()); + + // remove 99 + tracker.addElement((Stamped) elems[removalIndex++]); + assertEquals(100L, tracker.get()); + + // remove 100 + tracker.addElement((Stamped) elems[removalIndex++]); + assertEquals(-1L, tracker.get()); + + assertEquals(insertionIndex, removalIndex); + } + +} \ No newline at end of file From a94c32852f1e7fe69b7594a9abe2d35c68faf29f Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 26 Aug 2015 14:40:04 -0700 Subject: [PATCH 172/275] refactor TopologyBuilder --- .../streaming/processor/TopologyBuilder.java | 193 ++++++++++-------- .../processor/internals/ProcessorNode.java | 4 - .../internals/ProcessorTopology.java | 68 ++---- .../processor/internals/SinkNode.java | 14 +- 4 files changed, 128 insertions(+), 151 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 591b212963b28..651299b142e3a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -27,95 +27,128 @@ import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; public class TopologyBuilder { - private Map processorClasses = new HashMap<>(); - private Map sourceClasses = new HashMap<>(); - private Map sinkClasses = new HashMap<>(); - private Map topicsToSourceNames = new HashMap<>(); - private Map topicsToSinkNames = new HashMap<>(); + // list of node factories in a topological order + private ArrayList nodeFactories = new ArrayList<>(); - private Map> parents = new HashMap<>(); - private Map> children = new HashMap<>(); + private Set nodeNames = new HashSet<>(); + private Set sourceTopicNames = new HashSet<>(); - private class ProcessorClazz { - public ProcessorFactory factory; + private interface NodeFactory { + ProcessorNode build(); + } + + private class ProcessorNodeFactory implements NodeFactory { + public final String[] parents; + private final String name; + private final ProcessorFactory factory; - public ProcessorClazz(ProcessorFactory factory) { + public ProcessorNodeFactory(String name, String[] parents, ProcessorFactory factory) { + this.name = name; + this.parents = parents.clone(); this.factory = factory; } + + public ProcessorNode build() { + Processor processor = factory.build(); + return new ProcessorNode(name, processor); + } } - private class SourceClazz { - public Deserializer keyDeserializer; - public Deserializer valDeserializer; + private class SourceNodeFactory implements NodeFactory { + public final String[] topics; + private final String name; + private Deserializer keyDeserializer; + private Deserializer valDeserializer; - private SourceClazz(Deserializer keyDeserializer, 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; } - } - private class SinkClazz { - public Serializer keySerializer; - public Serializer valSerializer; + public ProcessorNode build() { + return new SourceNode(name, keyDeserializer, valDeserializer); + } + } - private SinkClazz(Serializer keySerializer, Serializer valSerializer) { + 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; } + public ProcessorNode build() { + return new SinkNode(name, topic, keySerializer, valSerializer); + } } public TopologyBuilder() {} - @SuppressWarnings("unchecked") public final void addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + if (nodeNames.contains(name)) + throw new IllegalArgumentException("Processor " + name + " is already added."); + for (String topic : topics) { - if (topicsToSourceNames.containsKey(topic)) + if (sourceTopicNames.contains(topic)) throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); - - topicsToSourceNames.put(topic, name); } - sourceClasses.put(name, new SourceClazz(keyDeserializer, valDeserializer)); + nodeNames.add(name); + nodeFactories.add(new SourceNodeFactory(name, topics, keyDeserializer, valDeserializer)); } - public final void addSink(String name, Serializer keySerializer, Serializer valSerializer, String... topics) { - for (String topic : topics) { - if (topicsToSinkNames.containsKey(topic)) - throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); + public final void addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, String... parentNames) { + if (nodeNames.contains(name)) + throw new IllegalArgumentException("Processor " + name + " is already added."); - topicsToSinkNames.put(topic, name); + if (parentNames != null) { + for (String parent : parentNames) { + if (parent.equals(name)) { + throw new IllegalArgumentException("Processor " + name + " cannot be a parent of itself"); + } + if (!nodeNames.contains(parent)) { + throw new IllegalArgumentException("Parent processor " + parent + " is not added yet."); + } + } } - sinkClasses.put(name, new SinkClazz(keySerializer, valSerializer)); + nodeNames.add(name); + nodeFactories.add(new SinkNodeFactory(name, parentNames, topic, keySerializer, valSerializer)); } public final void addProcessor(String name, ProcessorFactory factory, String... parentNames) { - if (processorClasses.containsKey(name)) + if (nodeNames.contains(name)) throw new IllegalArgumentException("Processor " + name + " is already added."); - processorClasses.put(name, new ProcessorClazz(factory)); - if (parentNames != null) { for (String parent : parentNames) { - if (!processorClasses.containsKey(parent)) + if (parent.equals(name)) { + throw new IllegalArgumentException("Processor " + name + " cannot be a parent of itself"); + } + if (!nodeNames.contains(parent)) { throw new IllegalArgumentException("Parent processor " + parent + " is not added yet."); - - // add to parent list - if (!parents.containsKey(name)) - parents.put(name, new ArrayList<>()); - parents.get(name).add(parent); - - // add to children list - if (!children.containsKey(parent)) - children.put(parent, new ArrayList<>()); - children.get(parent).add(name); + } } } + + nodeNames.add(name); + nodeFactories.add(new ProcessorNodeFactory(name, parentNames, factory)); } /** @@ -123,59 +156,41 @@ public final void addProcessor(String name, ProcessorFactory factory, String... */ @SuppressWarnings("unchecked") public ProcessorTopology build() { + List processorNodes = new ArrayList<>(nodeFactories.size()); Map processorMap = new HashMap<>(); Map topicSourceMap = new HashMap<>(); Map topicSinkMap = new HashMap<>(); - // create sources - for (String name : sourceClasses.keySet()) { - Deserializer keyDeserializer = sourceClasses.get(name).keyDeserializer; - Deserializer valDeserializer = sourceClasses.get(name).valDeserializer; - SourceNode node = new SourceNode(name, keyDeserializer, valDeserializer); - processorMap.put(name, node); - } - - // create sinks - for (String name : sinkClasses.keySet()) { - Serializer keySerializer = sinkClasses.get(name).keySerializer; - Serializer valSerializer = sinkClasses.get(name).valSerializer; - SinkNode node = new SinkNode(name, keySerializer, valSerializer); - processorMap.put(name, node); - } - - // create processors try { - for (String name : processorClasses.keySet()) { - ProcessorFactory processorFactory = processorClasses.get(name).factory; - Processor processor = processorFactory.build(); - ProcessorNode node = new ProcessorNode(name, processor); - processorMap.put(name, node); + // 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).chain(node); + } + } else if (factory instanceof SourceNodeFactory) { + for (String topic : ((SourceNodeFactory)factory).topics) { + topicSourceMap.put(topic, (SourceNode) node); + } + } else if (factory instanceof SinkNodeFactory) { + String topic = ((SinkNodeFactory) factory).topic; + topicSinkMap.put(topic, (SinkNode) node); + + for (String parent : ((SinkNodeFactory) factory).parents) { + processorMap.get(parent).chain(node); + } + } else { + throw new IllegalStateException("unknown factory class: " + factory.getClass().getName()); + } } } catch (Exception e) { - throw new KafkaException("Processor(String) constructor failed: this should not happen."); - } - - // construct topics to sources map - for (String topic : topicsToSourceNames.keySet()) { - SourceNode node = (SourceNode) processorMap.get(topicsToSourceNames.get(topic)); - topicSourceMap.put(topic, node); - } - - // construct topics to sinks map - for (String topic : topicsToSinkNames.keySet()) { - SinkNode node = (SinkNode) processorMap.get(topicsToSourceNames.get(topic)); - topicSinkMap.put(topic, node); - node.addTopic(topic); - } - - // chain children to parents to build the DAG - for (ProcessorNode node : processorMap.values()) { - for (String child : children.get(node.name())) { - ProcessorNode childNode = processorMap.get(child); - node.chain(childNode); - } + throw new KafkaException("ProcessorNode construction failed: this should not happen."); } - return new ProcessorTopology(processorMap, topicSourceMap, topicSinkMap); + return new ProcessorTopology(processorNodes, topicSourceMap, topicSinkMap); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java index 167863172149e..480c577754fdb 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java @@ -31,8 +31,6 @@ public class ProcessorNode { private final String name; private final Processor processor; - public boolean initialized; - public ProcessorNode(String name) { this(name, null); } @@ -42,8 +40,6 @@ public ProcessorNode(String name, Processor processor) { this.processor = processor; this.parents = new ArrayList<>(); this.children = new ArrayList<>(); - - this.initialized = false; } public String name() { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java index 6ae7c3175096d..bf4ddd796ece8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java @@ -27,88 +27,60 @@ public class ProcessorTopology { - private Map processors = new HashMap<>(); - private Map sourceTopics = new HashMap<>(); - private Map sinkTopics = new HashMap<>(); + private List processors; + private Map sourceByTopics; + private Map sinkByTopics; - public ProcessorTopology(Map processors, - Map sourceTopics, - Map sinkTopics) { + public ProcessorTopology(List processors, + Map sourceByTopics, + Map sinkByTopics) { this.processors = processors; - this.sourceTopics = sourceTopics; - this.sinkTopics = sinkTopics; + this.sourceByTopics = sourceByTopics; + this.sinkByTopics = sinkByTopics; } public Set sourceTopics() { - return sourceTopics.keySet(); + return sourceByTopics.keySet(); } public Set sinkTopics() { - return sinkTopics.keySet(); + return sinkByTopics.keySet(); } public SourceNode source(String topic) { - return sourceTopics.get(topic); + return sourceByTopics.get(topic); } public SinkNode sink(String topic) { - return sinkTopics.get(topic); + return sinkByTopics.get(topic); } public Collection sources() { - return sourceTopics.values(); + return sourceByTopics.values(); } public Collection sinks() { - return sinkTopics.values(); + return sinkByTopics.values(); } /** * Initialize the processors following the DAG reverse ordering * such that parents are always initialized before children */ - @SuppressWarnings("unchecked") public void init(ProcessorContext context) { - // initialize sources - for (String topic : sourceTopics.keySet()) { - SourceNode source = sourceTopics.get(topic); - - init(source, context); - } - } - - /** - * Initialize the current processor node by first initializing - * its parent nodes first, then the processor itself - */ - @SuppressWarnings("unchecked") - private void init(ProcessorNode node, ProcessorContext context) { - for (ProcessorNode parentNode : (List) node.parents()) { - if (!parentNode.initialized) { - init(parentNode, context); - } - } - - node.init(context); - node.initialized = true; - - // try to initialize its children - for (ProcessorNode childNode : (List) node.children()) { - if (!childNode.initialized) { - init(childNode, context); - } + for (ProcessorNode node : processors) { + node.init(context); } } public final void close() { // close the processors - // TODO: do we need to follow the DAG ordering? - for (ProcessorNode processorNode : processors.values()) { - processorNode.close(); + for (ProcessorNode node : processors) { + node.close(); } processors.clear(); - sourceTopics.clear(); - sinkTopics.clear(); + sourceByTopics.clear(); + sinkByTopics.clear(); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java index f3cbadf2deda4..0daa054657130 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java @@ -26,24 +26,20 @@ public class SinkNode extends ProcessorNode { + private final String topic; private final Serializer keySerializer; private final Serializer valSerializer; - private final List topics; private ProcessorContext context; - public SinkNode(String name, Serializer keySerializer, Serializer valSerializer) { + public SinkNode(String name, String topic, Serializer keySerializer, Serializer valSerializer) { super(name); - this.topics = new ArrayList<>(); + this.topic = topic; this.keySerializer = keySerializer; this.valSerializer = valSerializer; } - public void addTopic(String topic) { - this.topics.add(topic); - } - @Override public void init(ProcessorContext context) { this.context = context; @@ -53,9 +49,7 @@ public void init(ProcessorContext context) { public void process(K key, V value) { // send to all the registered topics RecordCollector collector = ((ProcessorContextImpl)context).recordCollector(); - for (String topic : topics) { - collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); - } + collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); } @Override From 82108658926a457256da7226da8050a099f2e580 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 26 Aug 2015 14:51:05 -0700 Subject: [PATCH 173/275] fix sourceTopicNames --- .../org/apache/kafka/streaming/processor/TopologyBuilder.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 651299b142e3a..6ee522a0dc0f3 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -107,6 +107,8 @@ public final void addSource(String name, Deserializer keyDeserializer, Deseriali for (String topic : topics) { if (sourceTopicNames.contains(topic)) throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); + + sourceTopicNames.add(topic); } nodeNames.add(name); From f5a37bdd6e9284994d5ee1b0e0479580476e7a26 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 27 Aug 2015 09:50:35 -0700 Subject: [PATCH 174/275] kstream refactored --- .../kafka/stream/topology/SlidingWindow.java | 287 ++++++++++-------- .../kafka/streaming/kstream/KStream.java | 5 +- .../kafka/streaming/kstream/Window.java | 18 +- .../kstream/internals/KStreamBranch.java | 36 ++- .../kstream/internals/KStreamFilter.java | 45 +-- .../kstream/internals/KStreamFlatMap.java | 30 +- .../internals/KStreamFlatMapValues.java | 28 +- .../kstream/internals/KStreamImpl.java | 54 ++-- .../kstream/internals/KStreamJoin.java | 162 +++++----- .../kstream/internals/KStreamMap.java | 27 +- .../kstream/internals/KStreamMapValues.java | 27 +- .../kstream/internals/KStreamPassThrough.java | 36 +++ .../kstream/internals/KStreamProcessor.java | 25 +- .../kstream/internals/KStreamSend.java | 62 ---- .../kstream/internals/KStreamWindow.java | 79 ++--- .../internals/KStreamWindowedImpl.java | 44 +++ .../streaming/processor/ProcessorContext.java | 6 +- .../internals/ProcessorContextImpl.java | 20 +- .../internals/ProcessorStateManager.java | 6 +- 19 files changed, 525 insertions(+), 472 deletions(-) create mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java create mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java index de157a4e04f8a..b2de36c6d9254 100644 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +++ b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java @@ -45,13 +45,15 @@ import java.util.LinkedList; import java.util.Map; -public class SlidingWindow extends WindowSupport implements Window { - - private final Object lock = new Object(); +public class SlidingWindow implements Window { + private String name; + private final long duration; + private final int maxCount; private final Serializer keySerializer; private final Serializer valueSerializer; private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; +<<<<<<< HEAD:stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java private KStreamContext context; private int slotNum; private String name; @@ -59,6 +61,8 @@ public class SlidingWindow extends WindowSupport implements Window { private final int maxCount; private LinkedList list = new LinkedList(); private HashMap> map = new HashMap<>(); +======= +>>>>>>> kstream refactored:stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java public SlidingWindow( String name, @@ -78,6 +82,7 @@ public SlidingWindow( } @Override +<<<<<<< HEAD:stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java public void init(KStreamContext context) { this.context = context; RestoreFuncImpl restoreFunc = new RestoreFuncImpl(); @@ -87,173 +92,203 @@ public void init(KStreamContext context) { valueList.clearDirtyValues(); } this.slotNum = restoreFunc.slotNum; +======= + public String name() { + return name; +>>>>>>> kstream refactored:stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java } @Override - public Iterator findAfter(K key, final long timestamp) { - return find(key, timestamp, timestamp + duration); + public WindowInstance build() { + return new SlidingWindowInstance(); } - @Override - public Iterator findBefore(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp); - } + public class SlidingWindowInstance extends WindowSupport implements WindowInstance { + private final Object lock = new Object(); + private ProcessorContext context; + private int slotNum; // used as a key for Kafka log compaction + private LinkedList list = new LinkedList(); + private HashMap> map = new HashMap<>(); - @Override - public Iterator find(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp + duration); - } + @Override + public void init(ProcessorContext context) { + this.context = context; + RestoreFuncImpl restoreFunc = new RestoreFuncImpl(); + context.register(this, restoreFunc); - /* - * finds items in the window between startTime and endTime (both inclusive) - */ - private Iterator find(K key, final long startTime, final long endTime) { - final ValueList values = map.get(key); - - if (values == null) { - return null; - } else { - return new FilteredIterator>(values.iterator()) { - @Override - protected V filter(Value item) { - if (startTime <= item.timestamp && item.timestamp <= endTime) - return item.value; - else - return null; - } - }; + for (ValueList valueList : map.values()) { + valueList.clearDirtyValues(); + } + this.slotNum = restoreFunc.slotNum; } - } - @Override - public void put(K key, V value, long timestamp) { - synchronized (lock) { - slotNum++; + @Override + public Iterator findAfter(K key, final long timestamp) { + return find(key, timestamp, timestamp + duration); + } + + @Override + public Iterator findBefore(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp); + } + + @Override + public Iterator find(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp + duration); + } - list.offerLast(key); + /* + * finds items in the window between startTime and endTime (both inclusive) + */ + private Iterator find(K key, final long startTime, final long endTime) { + final ValueList values = map.get(key); - ValueList values = map.get(key); if (values == null) { - values = new ValueList<>(); - map.put(key, values); + return null; + } else { + return new FilteredIterator>(values.iterator()) { + @Override + protected V filter(Value item) { + if (startTime <= item.timestamp && item.timestamp <= endTime) + return item.value; + else + return null; + } + }; } - - values.add(slotNum, value, timestamp); } - evictExcess(); - evictExpired(timestamp - duration); - } - private void evictExcess() { - while (list.size() > maxCount) { - K oldestKey = list.pollFirst(); + @Override + public void put(K key, V value, long timestamp) { + synchronized (lock) { + slotNum++; - ValueList values = map.get(oldestKey); - values.removeFirst(); + list.offerLast(key); - if (values.isEmpty()) map.remove(oldestKey); - } - } + ValueList values = map.get(key); + if (values == null) { + values = new ValueList<>(); + map.put(key, values); + } - private void evictExpired(long cutoffTime) { - while (true) { - K oldestKey = list.peekFirst(); + values.add(slotNum, value, timestamp); + } + evictExcess(); + evictExpired(timestamp - duration); + } - ValueList values = map.get(oldestKey); - Stamped oldestValue = values.first(); + private void evictExcess() { + while (list.size() > maxCount) { + K oldestKey = list.pollFirst(); - if (oldestValue.timestamp < cutoffTime) { - list.pollFirst(); + ValueList values = map.get(oldestKey); values.removeFirst(); if (values.isEmpty()) map.remove(oldestKey); - } else { - break; } } - } - @Override - public String name() { - return name; - } + private void evictExpired(long cutoffTime) { + while (true) { + K oldestKey = list.peekFirst(); - @Override - public void flush() { - IntegerSerializer intSerializer = new IntegerSerializer(); - ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); + ValueList values = map.get(oldestKey); + Stamped oldestValue = values.first(); + + if (oldestValue.timestamp < cutoffTime) { + list.pollFirst(); + values.removeFirst(); - RecordCollector collector = ((ProcessorContextImpl)context).recordCollector(); + if (values.isEmpty()) map.remove(oldestKey); + } else { + break; + } + } + } - for (Map.Entry> entry : map.entrySet()) { - ValueList values = entry.getValue(); - if (values.hasDirtyValues()) { - K key = entry.getKey(); + @Override + public String name() { + return name; + } - byte[] keyBytes = keySerializer.serialize(name, key); + @Override + public void flush() { + IntegerSerializer intSerializer = new IntegerSerializer(); + ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); - Iterator> iterator = values.dirtyValueIterator(); - while (iterator.hasNext()) { - Value dirtyValue = iterator.next(); - byte[] slot = intSerializer.serialize("", dirtyValue.slotNum); - byte[] valBytes = valueSerializer.serialize(name, dirtyValue.value); + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); - byte[] combined = new byte[8 + 4 + keyBytes.length + 4 + valBytes.length]; + for (Map.Entry> entry : map.entrySet()) { + ValueList values = entry.getValue(); + if (values.hasDirtyValues()) { + K key = entry.getKey(); - int offset = 0; - offset += putLong(combined, offset, dirtyValue.timestamp); - offset += puts(combined, offset, keyBytes); - offset += puts(combined, offset, valBytes); + byte[] keyBytes = keySerializer.serialize(name, key); - if (offset != combined.length) throw new IllegalStateException("serialized length does not match"); + Iterator> iterator = values.dirtyValueIterator(); + while (iterator.hasNext()) { + Value dirtyValue = iterator.next(); + byte[] slot = intSerializer.serialize("", dirtyValue.slotNum); + byte[] valBytes = valueSerializer.serialize(name, dirtyValue.value); - collector.send(new ProducerRecord<>(name, context.id(), slot, combined), byteArraySerializer, byteArraySerializer); + 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(); } - values.clearDirtyValues(); } } - } - @Override - public void close() { - // TODO - } + @Override + public void close() { + // TODO + } - @Override - public boolean persistent() { - // TODO: should not be persistent, right? - return false; - } + @Override + public boolean persistent() { + // TODO: should not be persistent, right? + return false; + } - private class RestoreFuncImpl implements RestoreFunc { + private class RestoreFuncImpl implements RestoreFunc { - final IntegerDeserializer intDeserializer; - int slotNum = 0; + final IntegerDeserializer intDeserializer; + int slotNum = 0; - RestoreFuncImpl() { - intDeserializer = new IntegerDeserializer(); - } + 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); + @Override + public void apply(byte[] slot, byte[] bytes) { + + slotNum = intDeserializer.deserialize("", slot); + + int offset = 0; + // timestamp + long timestamp = getLong(bytes, offset); + offset += 8; + // key + int length = getInt(bytes, offset); + offset += 4; + K key = deserialize(bytes, offset, length, name, keyDeserializer); + offset += length; + // value + length = getInt(bytes, offset); + offset += 4; + V value = deserialize(bytes, offset, length, name, valueDeserializer); + + put(key, value, timestamp); + } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java index fc23e2d6d530d..a2c855c18eb43 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.processor.Processor; +import org.apache.kafka.streaming.processor.ProcessorFactory; /** * KStream is an abstraction of a stream of key-value pairs. @@ -133,7 +134,7 @@ public interface KStream { /** * Processes all elements in this stream by applying a processor. * - * @param processor the class of Processor + * @param processorFactory the class of ProcessorFactory */ - KStream process(Processor processor); + KStream process(ProcessorFactory processorFactory); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java index d805466144d4c..b3ec0e2a108e0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java @@ -22,16 +22,22 @@ import java.util.Iterator; -public interface Window extends StateStore { +public interface Window { - void init(ProcessorContext context); + interface WindowInstance extends StateStore { - Iterator find(K key, long timestamp); + void init(ProcessorContext context); - Iterator findAfter(K key, long timestamp); + Iterator find(K key, long timestamp); - Iterator findBefore(K key, long timestamp); + Iterator findAfter(K key, long timestamp); - void put(K key, V value, long timestamp); + Iterator findBefore(K key, long timestamp); + void put(K key, V value, long timestamp); + } + + String name(); + + WindowInstance build(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java index 25189f677c994..e9d6030401c58 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java @@ -17,36 +17,34 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.ProcessorFactory; import org.apache.kafka.streaming.kstream.Predicate; -class KStreamBranch extends Processor { +class KStreamBranch implements ProcessorFactory { private final Predicate[] predicates; @SuppressWarnings("unchecked") - public KStreamBranch(String name, ProcessorMetadata config) { - super(name, config); - - if (this.metadata() == null) - throw new IllegalStateException("ProcessorMetadata should be specified."); - - this.predicates = (Predicate[]) config.value(); + public KStreamBranch(Predicate... predicates) { + this.predicates = predicates; } @Override - public void process(K key, V value) { - if (this.children().size() != this.predicates.length) - throw new KafkaException("Number of branched streams does not match the length of predicates: this should not happen."); + public Processor build() { + return new KStreamBranchProcessor(); + } - for (int i = 0; i < predicates.length; i++) { - if (predicates[i].apply(key, value)) { - // do not use forward here bu directly call process() and then break the loop - // so that no record is going to be piped to multiple streams - this.children().get(i).process(key, value); - break; + private class KStreamBranchProcessor extends KStreamProcessor { + @Override + public void process(K key, V value) { + for (int i = 0; i < predicates.length; i++) { + if (predicates[i].apply(key, value)) { + // use forward with childIndex here and then break the loop + // so that no record is going to be piped to multiple streams + context.forward(key, value, i); + break; + } } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java index b467a9d3d0fc3..a85e0181a7c76 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java @@ -19,42 +19,29 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.Predicate; -import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.ProcessorFactory; -class KStreamFilter extends Processor { +class KStreamFilter implements ProcessorFactory { - private final PredicateOut predicateOut; + private final Predicate predicate; + private final boolean filterOut; - public static final class PredicateOut { - - public final Predicate predicate; - public final boolean filterOut; - - public PredicateOut(Predicate predicate) { - this(predicate, false); - } - - public PredicateOut(Predicate predicate, boolean filterOut) { - this.predicate = predicate; - this.filterOut = filterOut; - } + public KStreamFilter(Predicate predicate, boolean filterOut) { + this.predicate = predicate; + this.filterOut = filterOut; } - @SuppressWarnings("unchecked") - public KStreamFilter(ProcessorMetadata metadata) { - super(metadata); - - if (this.metadata() == null) - throw new IllegalStateException("ProcessorMetadata should be specified."); - - this.predicateOut = (PredicateOut) metadata.value(); + @Override + public Processor build() { + return new KStreamFilterProcessor(); } - @Override - public void process(K key, V value) { - if ((!predicateOut.filterOut && predicateOut.predicate.apply(key, value)) - || (predicateOut.filterOut && !predicateOut.predicate.apply(key, value))) { - context.forward(key, value); + private class KStreamFilterProcessor extends KStreamProcessor { + @Override + public void process(K key, V value) { + if (filterOut ^ predicate.apply(key, value)) { + context.forward(key, value); + } } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java index 89543b704e986..bc834a8f66ae8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java @@ -17,30 +17,30 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorMetadata; import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.KeyValueFlatMap; +import org.apache.kafka.streaming.processor.Processor; +import org.apache.kafka.streaming.processor.ProcessorFactory; -class KStreamFlatMap extends Processor { +class KStreamFlatMap implements ProcessorFactory { private final KeyValueFlatMap mapper; - @SuppressWarnings("unchecked") - KStreamFlatMap(ProcessorMetadata metadata) { - super(metadata); - - if (this.metadata() == null) - throw new IllegalStateException("ProcessorMetadata should be specified."); - - this.mapper = (KeyValueFlatMap) metadata.value(); + KStreamFlatMap(KeyValueFlatMap mapper) { + this.mapper = mapper; } @Override - public void process(K1 key, V1 value) { - Iterable> pairs = mapper.apply(key, value); - for (KeyValue pair : pairs) { - context.forward(pair.key, pair.value); + public Processor build() { + return new KStreamFlatMapProcessor(); + } + + private class KStreamFlatMapProcessor extends KStreamProcessor { + @Override + public void process(K1 key, V1 value) { + for (KeyValue newPair : mapper.apply(key, value)) { + context.forward(newPair.key, newPair.value); + } } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java index 0df01e62358ed..5e18f188ffa08 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java @@ -19,27 +19,29 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.ProcessorFactory; -class KStreamFlatMapValues extends Processor { +class KStreamFlatMapValues implements ProcessorFactory { private final ValueMapper> mapper; @SuppressWarnings("unchecked") - KStreamFlatMapValues(String name, ProcessorMetadata config) { - super(name, config); - - if (this.metadata() == null) - throw new IllegalStateException("ProcessorMetadata should be specified."); - - this.mapper = (ValueMapper>) config.value(); + KStreamFlatMapValues(ValueMapper> mapper) { + this.mapper = mapper; } @Override - public void process(K1 key, V1 value) { - Iterable newValues = mapper.apply(value); - for (V2 v : newValues) { - forward(key, v); + public Processor build() { + return new KStreamFlatMapValuesProcessor(); + } + + private class KStreamFlatMapValuesProcessor extends KStreamProcessor { + @Override + public void process(K1 key, V1 value) { + Iterable newValues = mapper.apply(value); + for (V2 v : newValues) { + context.forward(key, v); + } } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index 14b09c52bceb3..aef2899f4c428 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.kstream.KeyValueFlatMap; -import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.ProcessorFactory; import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.kstream.KStreamWindowed; import org.apache.kafka.streaming.kstream.KeyValueMapper; @@ -30,7 +30,6 @@ import org.apache.kafka.streaming.kstream.Window; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; @@ -50,10 +49,20 @@ public class KStreamImpl implements KStream { private static final String BRANCH_NAME = "KAFKA-BRANCH-"; + private static final String BRANCHCHILD_NAME = "KAFKA-BRANCHCHILD-"; + + private static final String WINDOWED_NAME = "KAFKA-WINDOWED-"; + + public static final String JOIN_NAME = "KAFKA-JOIN-"; + + public static final String JOINOTHER_NAME = "KAFKA-JOINOTHER-"; + public static final String SOURCE_NAME = "KAFKA-SOURCE-"; public static final String SEND_NAME = "KAFKA-SEND-"; + public static final String WINDOW_NAME = "KAFKA-WINDOW-"; + public static final AtomicInteger INDEX = new AtomicInteger(1); protected TopologyBuilder topology; @@ -68,7 +77,7 @@ public KStreamImpl(TopologyBuilder topology, String name) { public KStream filter(Predicate predicate) { String name = FILTER_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFilter.class, new ProcessorMetadata("Predicate", new KStreamFilter.PredicateOut<>(predicate)), this.name); + topology.addProcessor(name, new KStreamFilter(predicate, false), this.name); return new KStreamImpl<>(topology, name); } @@ -77,7 +86,7 @@ public KStream filter(Predicate predicate) { public KStream filterOut(final Predicate predicate) { String name = FILTER_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFilter.class, new ProcessorMetadata("Predicate", new KStreamFilter.PredicateOut<>(predicate, true)), this.name); + topology.addProcessor(name, new KStreamFilter(predicate, true), this.name); return new KStreamImpl<>(topology, name); } @@ -86,7 +95,7 @@ public KStream filterOut(final Predicate predicate) { public KStream map(KeyValueMapper mapper) { String name = MAP_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamMap.class, new ProcessorMetadata("Mapper", mapper), this.name); + topology.addProcessor(name, new KStreamMap(mapper), this.name); return new KStreamImpl<>(topology, name); } @@ -95,7 +104,7 @@ public KStream map(KeyValueMapper mapper) { public KStream mapValues(ValueMapper mapper) { String name = MAPVALUES_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamMapValues.class, new ProcessorMetadata("ValueMapper", mapper), this.name); + topology.addProcessor(name, new KStreamMapValues(mapper), this.name); return new KStreamImpl<>(topology, name); } @@ -105,7 +114,7 @@ public KStream mapValues(ValueMapper mapper) { public KStream flatMap(KeyValueFlatMap mapper) { String name = FLATMAP_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFlatMap.class, new ProcessorMetadata("FlatMapper", mapper), this.name); + topology.addProcessor(name, new KStreamFlatMap(mapper), this.name); return new KStreamImpl<>(topology, name); } @@ -115,34 +124,37 @@ public KStream flatMap(KeyValueFlatMap mapper) { public KStream flatMapValues(ValueMapper> mapper) { String name = FLATMAPVALUES_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamFlatMapValues.class, new ProcessorMetadata("ValueMapper", mapper), this.name); + topology.addProcessor(name, new KStreamFlatMapValues(mapper), this.name); return new KStreamImpl<>(topology, name); } @Override public KStreamWindowed with(Window window) { - KStreamWindow windowed = new KStreamWindow<>(window); + String name = WINDOWED_NAME + INDEX.getAndIncrement(); - topology.addProcessor(windowed, processor); + topology.addProcessor(name, new KStreamWindow<>(window), this.name); - return new KStreamWindow.KStreamWindowedImpl<>(topology, windowed); + return new KStreamWindowedImpl<>(topology, name, window); } @Override @SuppressWarnings("unchecked") public KStream[] branch(Predicate... predicates) { - String name = BRANCH_NAME + INDEX.getAndIncrement(); + String branchName = BRANCH_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamBranch.class, new ProcessorMetadata("Predicates", Arrays.copyOf(predicates, predicates.length)), this.name); + topology.addProcessor(branchName, new KStreamBranch(predicates.clone()), this.name); - KStreamImpl branch = new KStreamImpl<>(topology, name); - List> avatars = new ArrayList<>(); + List> branchChildren = new ArrayList<>(); for (int i = 0; i < predicates.length; i++) { - avatars.add(branch); + String childName = BRANCHCHILD_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(childName, new KStreamPassThrough(), branchName); + + branchChildren.add(new KStreamImpl(topology, childName)); } - return (KStream[]) avatars.toArray(); + return (KStream[]) branchChildren.toArray(); } @SuppressWarnings("unchecked") @@ -154,7 +166,7 @@ public KStream through(String topic, Deserializer valDeserializer) { String sendName = SEND_NAME + INDEX.getAndIncrement(); - process(new KStreamSend(sendName, new ProcessorMetadata("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer)))); + topology.addSink(sendName, topic, keySerializer, valSerializer, this.name); String sourceName = SOURCE_NAME + INDEX.getAndIncrement(); @@ -168,15 +180,15 @@ public KStream through(String topic, public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { String name = SEND_NAME + INDEX.getAndIncrement(); - process(new KStreamSend(name, new ProcessorMetadata("Topic-Ser", new KStreamSend.TopicSer(topic, (Serializer) keySerializer, (Serializer) valSerializer)))); + topology.addSink(name, topic, keySerializer, valSerializer, this.name); } @SuppressWarnings("unchecked") @Override - public KStream process(final Processor processor) { + public KStream process(final ProcessorFactory processorFactory) { String name = PROCESSOR_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, KStreamProcessor.class, new ProcessorMetadata("Processor", processor), this.name); + topology.addProcessor(name, processorFactory, this.name); return new KStreamImpl<>(topology, name); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index 7751a7f3603f3..9880930f027e4 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -20,110 +20,112 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.ValueJoiner; -import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.kstream.Window.WindowInstance; +import org.apache.kafka.streaming.processor.ProcessorFactory; import java.util.Iterator; -class KStreamJoin extends Processor { - - private static final String JOIN_NAME = "KAFKA-JOIN"; - private static final String JOIN_OTHER_NAME = "KAFKA-JOIN-OTHER"; +class KStreamJoin implements ProcessorFactory { private static abstract class Finder { abstract Iterator find(K key, long timestamp); } - private final KStreamWindow stream1; - private final KStreamWindow stream2; - private final Finder finder1; - private final Finder finder2; + private final String windowName1; + private final String windowName2; private final ValueJoiner joiner; - final Processor processorForOtherStream; - - private ProcessorContext context; - - KStreamJoin(KStreamWindow stream1, KStreamWindow stream2, boolean prior, ValueJoiner joiner) { - super(JOIN_NAME); + private final boolean prior; - this.stream1 = stream1; - this.stream2 = stream2; - final Window window1 = stream1.window(); - final Window window2 = stream2.window(); - - if (prior) { - this.finder1 = new Finder() { - Iterator find(K key, long timestamp) { - return window1.findAfter(key, timestamp); - } - }; - this.finder2 = new Finder() { - Iterator find(K key, long timestamp) { - return window2.findBefore(key, timestamp); - } - }; - } else { - this.finder1 = new Finder() { - Iterator find(K key, long timestamp) { - return window1.find(key, timestamp); - } - }; - this.finder2 = new Finder() { - Iterator find(K key, long timestamp) { - return window2.find(key, timestamp); - } - }; + private Processor processorForOtherStream = null; + public final ProcessorFactory processorFactoryForOtherStream = new ProcessorFactory() { + @Override + public Processor build() { + return processorForOtherStream; } + }; + KStreamJoin(String windowName1, String windowName2, boolean prior, ValueJoiner joiner) { + this.windowName1 = windowName1; + this.windowName2 = windowName2; this.joiner = joiner; - - this.processorForOtherStream = processorForOther(); + this.prior = prior; } @Override - public void init(ProcessorContext context) { - this.context = context; - - // check if these two streams are joinable - if (!stream1.context().joinable(stream2.context())) - throw new IllegalStateException("Stream " + stream1.name() + " and stream " + - stream2.name() + " are not joinable."); + public Processor build() { + return new KStreamJoinProcessor(); } - @Override - public void process(K key, V1 value) { - long timestamp = context.timestamp(); - Iterator iter = finder2.find(key, timestamp); - if (iter != null) { - while (iter.hasNext()) { - doJoin(key, value, iter.next()); + private class KStreamJoinProcessor extends KStreamProcessor { + + private Finder finder1; + private Finder finder2; + + @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 WindowInstance window1 = (WindowInstance) context.getStateStore(windowName1); + final WindowInstance window2 = (WindowInstance) context.getStateStore(windowName2); + + if (prior) { + this.finder1 = new Finder() { + Iterator find(K key, long timestamp) { + return window1.findAfter(key, timestamp); + } + }; + this.finder2 = new Finder() { + Iterator find(K key, long timestamp) { + return window2.findBefore(key, timestamp); + } + }; + } else { + this.finder1 = new Finder() { + Iterator find(K key, long timestamp) { + return window1.find(key, timestamp); + } + }; + this.finder2 = new Finder() { + Iterator find(K key, long timestamp) { + return window2.find(key, timestamp); + } + }; } - } - } - private Processor processorForOther() { - return new Processor(JOIN_OTHER_NAME) { - - @SuppressWarnings("unchecked") - @Override - public void process(K key, V2 value) { - long timestamp = context.timestamp(); - Iterator iter = finder1.find(key, timestamp); - if (iter != null) { - while (iter.hasNext()) { - doJoin(key, iter.next(), value); + processorForOtherStream = new KStreamProcessor() { + @Override + public void process(K key, V2 value) { + long timestamp = context.timestamp(); + Iterator iter = finder1.find(key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + doJoin(key, iter.next(), value); + } } } - } + }; + } - @Override - public void close() { - // down stream instances are close when the primary stream is closed + @Override + public void process(K key, V1 value) { + long timestamp = context.timestamp(); + Iterator iter = finder2.find(key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + doJoin(key, value, iter.next()); + } } - }; - } + } - // TODO: use the "outer-stream" topic as the resulted join stream topic - private void doJoin(K key, V1 value1, V2 value2) { - forward(key, joiner.apply(value1, value2)); + // TODO: use the "outer-stream" topic as the resulted join stream topic + private void doJoin(K key, V1 value1, V2 value2) { + context.forward(key, joiner.apply(value1, value2)); + } } + } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java index 903fb785a7561..83fb4c11c99f6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java @@ -20,25 +20,26 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.ProcessorFactory; -class KStreamMap extends Processor { +class KStreamMap implements ProcessorFactory { private final KeyValueMapper mapper; - @SuppressWarnings("unchecked") - public KStreamMap(ProcessorMetadata metadata) { - super(metadata); - - if (this.metadata() == null) - throw new IllegalStateException("ProcessorMetadata should be specified."); - - this.mapper = (KeyValueMapper) metadata.value(); + public KStreamMap(KeyValueMapper mapper) { + this.mapper = mapper; } @Override - public void process(K1 key, V1 value) { - KeyValue newPair = mapper.apply(key, value); - context.forward(newPair.key, newPair.value); + public Processor build() { + return new KStreamMapProcessor(); + } + + private class KStreamMapProcessor extends KStreamProcessor { + @Override + public void process(K1 key, V1 value) { + KeyValue newPair = mapper.apply(key, value); + context.forward(newPair.key, newPair.value); + } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java index a8aac1285039e..667c04386c381 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java @@ -19,25 +19,26 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.ProcessorFactory; -class KStreamMapValues extends Processor { +class KStreamMapValues implements ProcessorFactory { private final ValueMapper mapper; - @SuppressWarnings("unchecked") - public KStreamMapValues(String name, ProcessorMetadata config) { - super(name, config); - - if (this.metadata() == null) - throw new IllegalStateException("ProcessorMetadata should be specified."); - - this.mapper = (ValueMapper) config.value(); + public KStreamMapValues(ValueMapper mapper) { + this.mapper = mapper; } @Override - public void process(K1 key, V1 value) { - V2 newValue = mapper.apply(value); - forward(key, newValue); + public Processor build() { + return new KStreamMapProcessor(); + } + + private class KStreamMapProcessor extends KStreamProcessor { + @Override + public void process(K1 key, V1 value) { + V2 newValue = mapper.apply(value); + context.forward(key, newValue); + } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java new file mode 100644 index 0000000000000..98528f72c7c1b --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.kstream.internals; + +import org.apache.kafka.streaming.processor.Processor; +import org.apache.kafka.streaming.processor.ProcessorFactory; + +class KStreamPassThrough implements ProcessorFactory { + + @Override + public Processor build() { + return new KStreamPassThroughProcessor(); + } + + public class KStreamPassThroughProcessor extends KStreamProcessor { + @Override + public void process(K key, V value) { + context.forward(key, value); + } + } +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java index 486b67e3c7625..a289893ae1fd0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java @@ -18,24 +18,27 @@ package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorMetadata; +import org.apache.kafka.streaming.processor.ProcessorContext; -public class KStreamProcessor extends Processor { +abstract class KStreamProcessor implements Processor { - private final Processor processor; + protected ProcessorContext context; - @SuppressWarnings("unchecked") - public KStreamProcessor(ProcessorMetadata metadata) { - super(metadata); + @Override + abstract public void process(K key, V value); - if (this.metadata() != null) - throw new IllegalStateException("ProcessorMetadata should be null."); + @Override + public void init(ProcessorContext context) { + this.context = context; + } - this.processor = (Processor) metadata.value(); + @Override + public void punctuate(long streamTime) { + // do nothing } @Override - public void process(K key, V value) { - processor.process(key, value); + public void close() { + // do nothing } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java deleted file mode 100644 index 12b5a8ffa2837..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSend.java +++ /dev/null @@ -1,62 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.kstream.internals; - -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorMetadata; -import org.apache.kafka.streaming.processor.ProcessorContext; - -class KStreamSend extends Processor { - - private ProcessorContext context; - - private TopicSer topicSerializers; - - public static final class TopicSer { - public String topic; - public Serializer keySerializer; - public Serializer valSerializer; - - public TopicSer(String topic, Serializer keySerializer, Serializer valSerializer) { - this.topic = topic; - this.keySerializer = keySerializer; - this.valSerializer = valSerializer; - } - } - - @SuppressWarnings("unchecked") - public KStreamSend(String name, ProcessorMetadata config) { - super(name, config); - - if (this.metadata() == null) - throw new IllegalStateException("ProcessorMetadata should be specified."); - - this.topicSerializers = (TopicSer) config.value(); - } - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void process(K key, V value) { - this.context.send(topicSerializers.topic, key, value, topicSerializers.keySerializer, topicSerializers.valSerializer); - } -} \ No newline at end of file diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java index 2a6628ab5859a..2260d06ade674 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java @@ -18,6 +18,7 @@ package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.streaming.processor.Processor; +import org.apache.kafka.streaming.processor.ProcessorFactory; import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.KStream; @@ -25,47 +26,11 @@ import org.apache.kafka.streaming.kstream.ValueJoiner; import org.apache.kafka.streaming.kstream.Window; -public class KStreamWindow extends Processor { - - public static final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { - - public KStreamWindow windowed; - - public KStreamWindowedImpl(TopologyBuilder topology, KStreamWindow windowed) { - super(topology, windowed); - this.windowed = windowed; - } - - @Override - public KStream join(KStreamWindowed other, ValueJoiner processor) { - return join(other, false, processor); - } - - @Override - public KStream joinPrior(KStreamWindowed other, ValueJoiner processor) { - return join(other, true, processor); - } - - private KStream join(KStreamWindowed other, boolean prior, ValueJoiner processor) { - KStreamWindow thisWindow = this.windowed; - KStreamWindow otherWindow = ((KStreamWindowedImpl) other).windowed; - - KStreamJoin join = new KStreamJoin<>(thisWindow, otherWindow, prior, processor); - - topology.addProcessor(join, thisWindow); - topology.addProcessor(join.processorForOtherStream, otherWindow); - - return new KStreamImpl<>(topology, join); - } - } - - private static final String WINDOW_NAME = "KAFKA-WINDOW"; +public class KStreamWindow implements ProcessorFactory { private final Window window; - private ProcessorContext context; KStreamWindow(Window window) { - super(WINDOW_NAME); this.window = window; } @@ -73,26 +38,34 @@ public Window window() { return window; } - public ProcessorContext context() { - return context; - } - @Override - public void init(ProcessorContext context) { - this.context = context; + public Processor build() { + return new KStreamWindowProcessor(); } - @SuppressWarnings("unchecked") - @Override - public void process(K key, V value) { - synchronized (this) { - window.put(key, value, context.timestamp()); - forward(key, value); + private class KStreamWindowProcessor extends KStreamProcessor { + + private Window.WindowInstance windowInstance; + + @Override + public void init(ProcessorContext context) { + this.context = context; + this.windowInstance = window.build(); + this.windowInstance.init(context); } - } - @Override - public void close() { - window.close(); + @SuppressWarnings("unchecked") + @Override + public void process(K key, V value) { + synchronized (this) { + windowInstance.put(key, value, context.timestamp()); + context.forward(key, value); + } + } + + @Override + public void close() { + windowInstance.close(); + } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java new file mode 100644 index 0000000000000..2e3e674260437 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java @@ -0,0 +1,44 @@ +package org.apache.kafka.streaming.kstream.internals; + +import org.apache.kafka.streaming.kstream.KStream; +import org.apache.kafka.streaming.kstream.KStreamWindowed; +import org.apache.kafka.streaming.kstream.ValueJoiner; +import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.processor.TopologyBuilder; + +/** + * Created by yasuhiro on 8/27/15. + */ +public final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { + + private final Window window; + + public KStreamWindowedImpl(TopologyBuilder topology, String name, Window window) { + super(topology, name); + this.window = window; + } + + @Override + public KStream join(KStreamWindowed other, ValueJoiner valueJoiner) { + return join(other, false, valueJoiner); + } + + @Override + public KStream joinPrior(KStreamWindowed other, ValueJoiner valueJoiner) { + return join(other, true, valueJoiner); + } + + private KStream join(KStreamWindowed other, boolean prior, ValueJoiner valueJoiner) { + String thisWindowName = this.window.name(); + String otherWindowName = ((KStreamWindowedImpl) other).window.name(); + + KStreamJoin join = new KStreamJoin<>(thisWindowName, otherWindowName, prior, valueJoiner); + + String joinName = JOIN_NAME + INDEX.getAndIncrement(); + String joinOtherName = JOINOTHER_NAME + INDEX.getAndIncrement(); + topology.addProcessor(joinName, join, this.name); + topology.addProcessor(joinOtherName, join.processorFactoryForOtherStream, ((KStreamImpl) other).name); + + return new KStreamImpl<>(topology, joinName); + } +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java index 15d2e2981eee7..db78fe52e365c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java @@ -26,7 +26,7 @@ public interface ProcessorContext { // TODO: this is better moved to a KStreamContext - boolean joinable(ProcessorContext other); + boolean joinable(); /** * Returns the partition group id @@ -84,10 +84,14 @@ public interface ProcessorContext { */ void register(StateStore store, RestoreFunc restoreFunc); + StateStore getStateStore(String name); + void schedule(Processor processor, long interval); void forward(K key, V value); + void forward(K key, V value, int childIndex); + void commit(); String topic(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index 1691632d879a0..18c5b01cd4e58 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -94,13 +94,7 @@ public RecordCollector recordCollector() { } @Override - public boolean joinable(ProcessorContext o) { - - ProcessorContextImpl other = (ProcessorContextImpl) o; - - if (this.task != other.task) - return false; - + public boolean joinable() { Set partitions = this.task.partitions(); Map> partitionsById = new HashMap<>(); int firstId = -1; @@ -171,6 +165,10 @@ public void register(StateStore store, RestoreFunc restoreFunc) { stateMgr.register(store, restoreFunc); } + public StateStore getStateStore(String name) { + return stateMgr.getStore(name); + } + @Override public String topic() { if (task.record() == null) @@ -212,6 +210,14 @@ public void forward(K key, V value) { } } + @Override + @SuppressWarnings("unchecked") + public void forward(K key, V value, int childIndex) { + ProcessorNode childNode = (ProcessorNode) task.node().children().get(childIndex); + task.node(childNode); + childNode.process(key, value); + } + @Override public void commit() { task.commitOffset(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java index 5052421e0fe9f..c659e0459d2e2 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java @@ -77,7 +77,7 @@ public void register(StateStore store, RestoreFunc restoreFunc) { // ---- register the store ---- // // check that the underlying change log topic exist or not - if (restoreConsumer.listTopics().keySet().contains(store.name())) { + if (restoreConsumer.listTopics().containsKey(store.name())) { boolean partitionNotFound = true; for (PartitionInfo partitionInfo : restoreConsumer.partitionsFor(store.name())) { if (partitionInfo.partition() == id) { @@ -139,6 +139,10 @@ public void register(StateStore store, RestoreFunc restoreFunc) { restoreConsumer.unsubscribe(storePartition); } + 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()) { From 16e760fd536638b8e268ccdba4c405157581fe7c Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 27 Aug 2015 11:47:35 -0700 Subject: [PATCH 175/275] remove ProcessorMetadata --- .../processor/ProcessorMetadata.java | 33 ------------------- 1 file changed, 33 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorMetadata.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorMetadata.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorMetadata.java deleted file mode 100644 index ba773b0f279ca..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorMetadata.java +++ /dev/null @@ -1,33 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor; - -public class ProcessorMetadata { - - private String name; - private Object value; - - public ProcessorMetadata(String name, Object value) { - this.name = name; - this.value = value; - } - - public Object value() { - return value; - } -} From 0674b742fa9e3c07db800d7bb1e0b414180b8471 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 27 Aug 2015 12:45:52 -0700 Subject: [PATCH 176/275] adress comments --- .../kafka/streaming/kstream/KStream.java | 5 +- .../streaming/kstream/SlidingWindowDef.java | 264 ++++++++++++++++++ .../kafka/streaming/kstream/Window.java | 36 +-- .../kafka/streaming/kstream/WindowDef.java | 25 ++ .../kstream/internals/KStreamImpl.java | 8 +- .../kstream/internals/KStreamJoin.java | 6 +- .../kstream/internals/KStreamWindow.java | 29 +- .../internals/KStreamWindowedImpl.java | 12 +- .../streaming/processor/TopologyBuilder.java | 4 +- .../internals/ProcessorContextImpl.java | 4 +- .../processor/internals/ProcessorNode.java | 25 +- .../processor/internals/SinkNode.java | 2 +- .../processor/internals/SourceNode.java | 13 +- .../kafka/streaming/KStreamWindowedTest.java | 6 +- .../internals/KStreamWindowedTest.java | 4 +- .../apache/kafka/test/UnlimitedWindow.java | 4 +- temp | 7 +- temp2 | 7 +- 18 files changed, 358 insertions(+), 103 deletions(-) create mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java create mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java index a2c855c18eb43..8df2131624eea 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorFactory; /** @@ -84,10 +83,10 @@ public interface KStream { /** * Creates a new windowed stream using a specified window instance. * - * @param window the instance of Window + * @param windowDef the instance of Window * @return KStream */ - KStreamWindowed with(Window window); + KStreamWindowed with(WindowDef windowDef); /** * Creates an array of streams from this stream. Each stream in the array coresponds to a predicate in diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java new file mode 100644 index 0000000000000..4fc8ca41411c8 --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java @@ -0,0 +1,264 @@ +/** + * 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.streaming.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.streaming.kstream.internals.FilteredIterator; +import org.apache.kafka.streaming.kstream.internals.WindowSupport; +import org.apache.kafka.streaming.processor.internals.ProcessorContextImpl; +import org.apache.kafka.streaming.processor.internals.RecordCollector; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.RestoreFunc; +import org.apache.kafka.streaming.processor.internals.Stamped; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; + +public class SlidingWindowDef implements WindowDef { + private String name; + private final long duration; + private final int maxCount; + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + + public SlidingWindowDef( + String name, + long duration, + int maxCount, + Serializer keySerializer, + Serializer valueSerializer, + Deserializer keyDeseriaizer, + Deserializer valueDeserializer) { + this.name = name; + this.duration = duration; + this.maxCount = maxCount; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.keyDeserializer = keyDeseriaizer; + this.valueDeserializer = valueDeserializer; + } + + @Override + public String name() { + return name; + } + + @Override + public Window build() { + return new SlidingWindow(); + } + + public class SlidingWindow extends WindowSupport implements Window { + private final Object lock = new Object(); + private ProcessorContext context; + private int slotNum; // used as a key for Kafka log compaction + private LinkedList list = new LinkedList(); + private HashMap> map = new HashMap<>(); + + @Override + public void init(ProcessorContext context) { + this.context = context; + RestoreFuncImpl restoreFunc = new RestoreFuncImpl(); + context.register(this, restoreFunc); + + for (ValueList valueList : map.values()) { + valueList.clearDirtyValues(); + } + this.slotNum = restoreFunc.slotNum; + } + + @Override + public Iterator findAfter(K key, final long timestamp) { + return find(key, timestamp, timestamp + duration); + } + + @Override + public Iterator findBefore(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp); + } + + @Override + public Iterator find(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp + duration); + } + + /* + * finds items in the window between startTime and endTime (both inclusive) + */ + private Iterator find(K key, final long startTime, final long endTime) { + final ValueList values = map.get(key); + + if (values == null) { + return null; + } else { + return new FilteredIterator>(values.iterator()) { + @Override + protected V filter(Value item) { + if (startTime <= item.timestamp && item.timestamp <= endTime) + return item.value; + else + return null; + } + }; + } + } + + @Override + public void put(K key, V value, long timestamp) { + synchronized (lock) { + slotNum++; + + list.offerLast(key); + + ValueList values = map.get(key); + if (values == null) { + values = new ValueList<>(); + map.put(key, values); + } + + values.add(slotNum, value, timestamp); + } + evictExcess(); + evictExpired(timestamp - duration); + } + + private void evictExcess() { + while (list.size() > maxCount) { + K oldestKey = list.pollFirst(); + + ValueList values = map.get(oldestKey); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } + } + + private void evictExpired(long cutoffTime) { + while (true) { + K oldestKey = list.peekFirst(); + + ValueList values = map.get(oldestKey); + Stamped oldestValue = values.first(); + + if (oldestValue.timestamp < cutoffTime) { + list.pollFirst(); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } else { + break; + } + } + } + + @Override + public String name() { + return name; + } + + @Override + public void flush() { + IntegerSerializer intSerializer = new IntegerSerializer(); + ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); + + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); + + for (Map.Entry> entry : map.entrySet()) { + ValueList values = entry.getValue(); + if (values.hasDirtyValues()) { + K key = entry.getKey(); + + byte[] keyBytes = keySerializer.serialize(name, key); + + Iterator> iterator = values.dirtyValueIterator(); + while (iterator.hasNext()) { + Value dirtyValue = iterator.next(); + byte[] slot = intSerializer.serialize("", dirtyValue.slotNum); + byte[] valBytes = valueSerializer.serialize(name, dirtyValue.value); + + byte[] combined = new byte[8 + 4 + keyBytes.length + 4 + valBytes.length]; + + int offset = 0; + offset += putLong(combined, offset, dirtyValue.timestamp); + offset += puts(combined, offset, keyBytes); + offset += puts(combined, offset, valBytes); + + if (offset != combined.length) + throw new IllegalStateException("serialized length does not match"); + + collector.send(new ProducerRecord<>(name, context.id(), slot, combined), byteArraySerializer, byteArraySerializer); + } + values.clearDirtyValues(); + } + } + } + + @Override + public void close() { + // TODO + } + + @Override + public boolean persistent() { + // TODO: should not be persistent, right? + return false; + } + + private class RestoreFuncImpl implements RestoreFunc { + + final IntegerDeserializer intDeserializer; + int slotNum = 0; + + RestoreFuncImpl() { + intDeserializer = new IntegerDeserializer(); + } + + @Override + public void apply(byte[] slot, byte[] bytes) { + + slotNum = intDeserializer.deserialize("", slot); + + int offset = 0; + // timestamp + long timestamp = getLong(bytes, offset); + offset += 8; + // key + int length = getInt(bytes, offset); + offset += 4; + K key = deserialize(bytes, offset, length, name, keyDeserializer); + offset += length; + // value + length = getInt(bytes, offset); + offset += 4; + V value = deserialize(bytes, offset, length, name, valueDeserializer); + + put(key, value, timestamp); + } + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java index b3ec0e2a108e0..5c9002b20842e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java @@ -1,20 +1,3 @@ -/** - * 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.streaming.kstream; import org.apache.kafka.streaming.processor.ProcessorContext; @@ -22,22 +5,15 @@ import java.util.Iterator; -public interface Window { - - interface WindowInstance extends StateStore { - - void init(ProcessorContext context); - - Iterator find(K key, long timestamp); +public interface Window extends StateStore { - Iterator findAfter(K key, long timestamp); + void init(ProcessorContext context); - Iterator findBefore(K key, long timestamp); + Iterator find(K key, long timestamp); - void put(K key, V value, long timestamp); - } + Iterator findAfter(K key, long timestamp); - String name(); + Iterator findBefore(K key, long timestamp); - WindowInstance build(); + void put(K key, V value, long timestamp); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java new file mode 100644 index 0000000000000..26971937eccfe --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streaming.kstream; + +public interface WindowDef { + + String name(); + + Window build(); +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index aef2899f4c428..e317338b134e6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -27,7 +27,7 @@ import org.apache.kafka.streaming.kstream.Predicate; import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.kstream.WindowDef; import java.util.ArrayList; import java.util.List; @@ -130,12 +130,12 @@ public KStream flatMapValues(ValueMapper> } @Override - public KStreamWindowed with(Window window) { + public KStreamWindowed with(WindowDef windowDef) { String name = WINDOWED_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, new KStreamWindow<>(window), this.name); + topology.addProcessor(name, new KStreamWindow<>(windowDef), this.name); - return new KStreamWindowedImpl<>(topology, name, window); + return new KStreamWindowedImpl<>(topology, name, windowDef); } @Override diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index 9880930f027e4..a8452d5ff0b64 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -20,7 +20,7 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.ValueJoiner; -import org.apache.kafka.streaming.kstream.Window.WindowInstance; +import org.apache.kafka.streaming.kstream.Window; import org.apache.kafka.streaming.processor.ProcessorFactory; import java.util.Iterator; @@ -70,8 +70,8 @@ public void init(ProcessorContext context) { if (!context.joinable()) throw new IllegalStateException("Streams are not joinable."); - final WindowInstance window1 = (WindowInstance) context.getStateStore(windowName1); - final WindowInstance window2 = (WindowInstance) context.getStateStore(windowName2); + final Window window1 = (Window) context.getStateStore(windowName1); + final Window window2 = (Window) context.getStateStore(windowName2); if (prior) { this.finder1 = new Finder() { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java index 2260d06ade674..f357613d94ce4 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java @@ -17,25 +17,22 @@ package org.apache.kafka.streaming.kstream.internals; +import org.apache.kafka.streaming.kstream.Window; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorFactory; -import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamWindowed; -import org.apache.kafka.streaming.kstream.ValueJoiner; -import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.kstream.WindowDef; public class KStreamWindow implements ProcessorFactory { - private final Window window; + private final WindowDef windowDef; - KStreamWindow(Window window) { - this.window = window; + KStreamWindow(WindowDef windowDef) { + this.windowDef = windowDef; } - public Window window() { - return window; + public WindowDef window() { + return windowDef; } @Override @@ -45,27 +42,27 @@ public Processor build() { private class KStreamWindowProcessor extends KStreamProcessor { - private Window.WindowInstance windowInstance; + private Window window; @Override public void init(ProcessorContext context) { - this.context = context; - this.windowInstance = window.build(); - this.windowInstance.init(context); + super.init(context); + this.window = windowDef.build(); + this.window.init(context); } @SuppressWarnings("unchecked") @Override public void process(K key, V value) { synchronized (this) { - windowInstance.put(key, value, context.timestamp()); + window.put(key, value, context.timestamp()); context.forward(key, value); } } @Override public void close() { - windowInstance.close(); + window.close(); } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java index 2e3e674260437..e269c3ad40bcc 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java @@ -3,7 +3,7 @@ import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KStreamWindowed; import org.apache.kafka.streaming.kstream.ValueJoiner; -import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.kstream.WindowDef; import org.apache.kafka.streaming.processor.TopologyBuilder; /** @@ -11,11 +11,11 @@ */ public final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { - private final Window window; + private final WindowDef windowDef; - public KStreamWindowedImpl(TopologyBuilder topology, String name, Window window) { + public KStreamWindowedImpl(TopologyBuilder topology, String name, WindowDef windowDef) { super(topology, name); - this.window = window; + this.windowDef = windowDef; } @Override @@ -29,8 +29,8 @@ public KStream joinPrior(KStreamWindowed other, ValueJoin } private KStream join(KStreamWindowed other, boolean prior, ValueJoiner valueJoiner) { - String thisWindowName = this.window.name(); - String otherWindowName = ((KStreamWindowedImpl) other).window.name(); + String thisWindowName = this.windowDef.name(); + String otherWindowName = ((KStreamWindowedImpl) other).windowDef.name(); KStreamJoin join = new KStreamJoin<>(thisWindowName, otherWindowName, prior, valueJoiner); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 6ee522a0dc0f3..894e0e37f0244 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -172,7 +172,7 @@ public ProcessorTopology build() { if (factory instanceof ProcessorNodeFactory) { for (String parent : ((ProcessorNodeFactory) factory).parents) { - processorMap.get(parent).chain(node); + processorMap.get(parent).addChild(node); } } else if (factory instanceof SourceNodeFactory) { for (String topic : ((SourceNodeFactory)factory).topics) { @@ -183,7 +183,7 @@ public ProcessorTopology build() { topicSinkMap.put(topic, (SinkNode) node); for (String parent : ((SinkNodeFactory) factory).parents) { - processorMap.get(parent).chain(node); + processorMap.get(parent).addChild(node); } } else { throw new IllegalStateException("unknown factory class: " + factory.getClass().getName()); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index 18c5b01cd4e58..fcff4b0c1b9b8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -204,7 +204,7 @@ public long timestamp() { @Override @SuppressWarnings("unchecked") public void forward(K key, V value) { - for (ProcessorNode childNode : (List>) task.node().children()) { + for (ProcessorNode childNode : (List>) task.node().children()) { task.node(childNode); childNode.process(key, value); } @@ -213,7 +213,7 @@ public void forward(K key, V value) { @Override @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { - ProcessorNode childNode = (ProcessorNode) task.node().children().get(childIndex); + ProcessorNode childNode = (ProcessorNode) task.node().children().get(childIndex); task.node(childNode); childNode.process(key, value); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java index 480c577754fdb..fa1dbb1e734bc 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java @@ -23,22 +23,20 @@ import java.util.ArrayList; import java.util.List; -public class ProcessorNode { +public class ProcessorNode { - private final List> children; - private final List> parents; + private final List> children; private final String name; - private final Processor processor; + private final Processor processor; public ProcessorNode(String name) { this(name, null); } - public ProcessorNode(String name, Processor processor) { + public ProcessorNode(String name, Processor processor) { this.name = name; this.processor = processor; - this.parents = new ArrayList<>(); this.children = new ArrayList<>(); } @@ -46,20 +44,11 @@ public String name() { return name; } - public Processor processor() { - return processor; - } - - public List> parents() { - return parents; - } - - public List> children() { + public List> children() { return children; } - public final void chain(ProcessorNode child) { - child.parents.add(this); + public final void addChild(ProcessorNode child) { children.add(child); } @@ -67,7 +56,7 @@ public void init(ProcessorContext context) { processor.init(context); } - public void process(K1 key, V1 value) { + public void process(K key, V value) { processor.process(key, value); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java index 0daa054657130..14bec7ea5fc65 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java @@ -24,7 +24,7 @@ import java.util.ArrayList; import java.util.List; -public class SinkNode extends ProcessorNode { +public class SinkNode extends ProcessorNode { private final String topic; private final Serializer keySerializer; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java index e86fe9c4d661d..f20afee0d3bef 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java @@ -20,11 +20,13 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.streaming.processor.ProcessorContext; -public class SourceNode extends ProcessorNode { +public class SourceNode extends ProcessorNode { public Deserializer keyDeserializer; public Deserializer valDeserializer; + private ProcessorContext context; + public SourceNode(String name, Deserializer keyDeserializer, Deserializer valDeserializer) { super(name); @@ -34,15 +36,12 @@ public SourceNode(String name, Deserializer keyDeserializer, Deserializer @Override public void init(ProcessorContext context) { - // do nothing + this.context = context; } @Override public void process(K key, V value) { - // just forward to all children - for (ProcessorNode childNode : this.children()) { - childNode.process(key, value); - } + context.forward(key, value); } @Override @@ -50,4 +49,4 @@ public void close() { // do nothing } -} \ No newline at end of file +} diff --git a/stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java index de5cd622a0d4d..99689d333c5e4 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.kstream.WindowDef; import org.apache.kafka.streaming.kstream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamBuilder; import org.apache.kafka.test.MockProcessorContext; @@ -70,8 +70,12 @@ public void testWindowedStream() { >>>>>>> compile and test passed ======= KStream stream; +<<<<<<< HEAD Window window; >>>>>>> wip +======= + WindowDef window; +>>>>>>> adress comments window = new UnlimitedWindow<>(); stream = topology.from(keyDeserializer, valDeserializer, topicName); diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java index 88e92a551a981..aaa4e3a04a1f0 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.kstream.WindowDef; import org.apache.kafka.streaming.kstream.internals.KStreamSource; import org.apache.kafka.test.MockKStreamBuilder; import org.apache.kafka.test.MockProcessorContext; @@ -46,7 +46,7 @@ public void testWindowedStream() { final int[] expectedKeys = new int[]{0, 1, 2, 3}; KStream stream; - Window window; + WindowDef window; window = new UnlimitedWindow<>(); stream = topology.from(keyDeserializer, valDeserializer, topicName); diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java index bea636167e11a..2dc4c6cbed287 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java @@ -19,14 +19,14 @@ import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.kstream.WindowDef; import org.apache.kafka.streaming.kstream.internals.FilteredIterator; import org.apache.kafka.streaming.processor.internals.Stamped; import java.util.Iterator; import java.util.LinkedList; -public class UnlimitedWindow implements Window { +public class UnlimitedWindow implements WindowDef { private LinkedList>> list = new LinkedList<>(); diff --git a/temp b/temp index abe38868a0a47..2e8567c23ef09 100644 --- a/temp +++ b/temp @@ -1,3 +1,4 @@ -clients/src/main/java/org/apache/kafka/common/utils/Utils.java: needs merge -stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java: needs merge -stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java: needs merge +stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java: needs merge +stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java: needs merge +stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java: needs merge diff --git a/temp2 b/temp2 index c5ac26d2fffd7..67423b33540a4 100644 --- a/temp2 +++ b/temp2 @@ -1,3 +1,4 @@ -clients/src/main/java/org/apache/kafka/common/utils/Utils.java -stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java -stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java +stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java +stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java From cae41cdb5bdd9064b93a21293fe88d67c64158f3 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 27 Aug 2015 12:48:15 -0700 Subject: [PATCH 177/275] comment --- .../apache/kafka/streaming/kstream/Window.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java index 5c9002b20842e..fc9582d21fc20 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java @@ -1,3 +1,20 @@ +/** + * 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.streaming.kstream; import org.apache.kafka.streaming.processor.ProcessorContext; From 4b920e766518beedfe43606796f06a4c125d206f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 26 Aug 2015 15:39:12 -0700 Subject: [PATCH 178/275] wip: commit task states --- .../processor/internals/PartitionGroup.java | 13 +++--- .../internals/ProcessorContextImpl.java | 6 ++- .../internals/ProcessorStateManager.java | 4 -- .../processor/internals/RecordQueue.java | 18 ++++---- .../processor/internals/StampedRecord.java | 1 - .../processor/internals/StreamTask.java | 42 +++++++++++++++---- .../processor/internals/StreamThread.java | 17 ++++---- 7 files changed, 65 insertions(+), 36 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java index a473c4eff0891..bcde36dbdc3b5 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java @@ -22,13 +22,13 @@ import org.apache.kafka.common.serialization.Deserializer; import java.util.Comparator; -import java.util.HashMap; import java.util.Map; import java.util.PriorityQueue; import java.util.Set; /** - * A PartitionGroup is composed from a set of partitions. + * 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 { @@ -60,13 +60,14 @@ public int compare(RecordQueue queue1, RecordQueue queue2) { /** * Get one record from the specified partition queue + * + * @return StampedRecord */ public StampedRecord getRecord(RecordQueue queue) { // get the first record from this queue. StampedRecord record = queue.poll(); // update the partition's timestamp and re-order it against other partitions. - queuesByTime.remove(queue); if (queue.size() > 0) { @@ -80,6 +81,8 @@ public StampedRecord getRecord(RecordQueue queue) { /** * Get the next partition queue that has the lowest timestamp to process + * + * @return RecordQueue */ public RecordQueue nextQueue() { // get the partition with the lowest timestamp @@ -92,7 +95,7 @@ public RecordQueue nextQueue() { } /** - * Put a timestamped record associated into its corresponding partition's queues. + * Put a timestamped record associated into its corresponding partition's queues */ public void putRecord(StampedRecord record, TopicPartition partition) { if (record.partition() != partition.partition() || !record.topic().equals(partition.topic())) @@ -143,7 +146,7 @@ public Set partitions() { */ public long timestamp() { if (queuesByTime.isEmpty()) { - return -1L; + return TimestampTracker.NOT_KNOWN; } else { return queuesByTime.peek().timestamp(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index fcff4b0c1b9b8..57d695242dd63 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -89,6 +89,10 @@ public ProcessorContextImpl(int id, this.initialized = false; } + public ProcessorStateManager stateManager() { + return this.stateMgr; + } + public RecordCollector recordCollector() { return this.collector; } @@ -220,7 +224,7 @@ public void forward(K key, V value, int childIndex) { @Override public void commit() { - task.commitOffset(); + task.commit(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java index c659e0459d2e2..810966d83d8d9 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java @@ -63,10 +63,6 @@ public File baseDir() { return this.baseDir; } - public Consumer restoreConsumer() { - return this.restoreConsumer; - } - public void register(StateStore store, RestoreFunc restoreFunc) { if (store.name().equals(CHECKPOINT_FILE_NAME)) throw new IllegalArgumentException("Illegal store name: " + CHECKPOINT_FILE_NAME); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java index 93e9420e5bd57..b628c1688e785 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java @@ -31,22 +31,24 @@ public class RecordQueue { private final SourceNode source; private final TopicPartition partition; - private final ArrayDeque fifoQueue = new ArrayDeque<>(); - private final TimestampTracker> timeTracker = new MinTimestampTracker<>(); + private final ArrayDeque fifoQueue; + private final TimestampTracker> timeTracker; private long partitionTime = TimestampTracker.NOT_KNOWN; - /** - * Creates a new instance of RecordQueue - * - * @param partition partition - * @param source source node - */ 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; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StampedRecord.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StampedRecord.java index e294c365f6346..bce29c2d954b0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StampedRecord.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StampedRecord.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; -// TODO: making this class exposed to user in the lower-level Processor public class StampedRecord extends Stamped> { public StampedRecord(ConsumerRecord record, long timestamp) { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java index 751f504c8f778..4c1b22342aa0e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -17,16 +17,21 @@ package org.apache.kafka.streaming.processor.internals; +import org.apache.kafka.clients.consumer.CommitType; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.RecordMetadata; 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.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.processor.TimestampExtractor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Collection; @@ -40,6 +45,8 @@ */ public class StreamTask { + private static final Logger log = LoggerFactory.getLogger(StreamTask.class); + private final int id; private final int maxBufferedSize; @@ -50,6 +57,8 @@ public class StreamTask { private final TimestampExtractor timestampExtractor; private final Map consumedOffsets; + private final Map producedoffsets; + private final Callback producerCallback; private boolean commitRequested = false; private StampedRecord currRecord = null; @@ -98,8 +107,22 @@ public StreamTask(int id, topology.init(this.processorContext); - // initialize the consumed offset cache + // initialize the consumed and produced offset cache this.consumedOffsets = new HashMap<>(); + this.producedoffsets = new HashMap<>(); + + this.producerCallback = new Callback() { + + @Override + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception == null) { + TopicPartition partition = new TopicPartition(metadata.topic(), metadata.partition()); + producedoffsets.put(partition, metadata.offset()); + } else { + log.error("Error sending record: ", exception); + } + } + }; } public int id() { @@ -165,14 +188,19 @@ public boolean process() { this.currNode = queue.source(); this.currNode.process(currRecord.key(), currRecord.value()); - // update the consumed offset map. + // update the consumed offset map after processing is done consumedOffsets.put(queue.partition(), currRecord.offset()); + // commit the current task state if requested during the processing if (commitRequested) { - // TODO: flush the following states atomically // 1) flush local state + ((ProcessorContextImpl) processorContext).stateManager().flush(); + // 2) commit consumed offsets + consumer.commit(consumedOffsets, CommitType.SYNC); + // 3) flush produced records in the downstream + ((ProcessorContextImpl) processorContext).recordCollector().flush(); } // we can continue processing this task as long as its @@ -209,9 +237,9 @@ public void node(ProcessorNode node) { } /** - * Request committing the current record's offset + * Request committing the current task's state */ - public void commitOffset() { + public void commit() { this.commitRequested = true; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index 93059746ff999..be973e1b30c17 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -53,9 +53,9 @@ public class StreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(StreamThread.class); - private final Consumer consumer; private final TopologyBuilder builder; private final RecordCollector collector; + private final Consumer consumer; private final Map tasks = new HashMap<>(); private final Metrics metrics; private final Time time; @@ -128,7 +128,7 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce */ @Override public synchronized void run() { - log.info("Starting a kstream thread"); + log.info("Starting a stream thread"); try { runLoop(); } catch (RuntimeException e) { @@ -140,13 +140,13 @@ public synchronized void run() { } private void shutdown() { - log.info("Shutting down a kstream thread"); + log.info("Shutting down a stream thread"); commitAll(time.milliseconds()); collector.close(); consumer.close(); removePartitions(); - log.info("kstream thread shutdown complete"); + log.info("Stream thread shutdown complete"); } /** @@ -206,8 +206,6 @@ private void maybeCommit() { } private void commitAll(long now) { - - /* Map commit = new HashMap<>(); for (ProcessorContextImpl context : tasks.values()) { context.flush(); @@ -222,7 +220,6 @@ private void commitAll(long now) { consumer.commit(commit); // TODO: can this be async? streamingMetrics.commitTime.record(now - lastCommit); } - */ } /* delete any state dirs that aren't for active contexts */ @@ -303,9 +300,9 @@ public KafkaStreamingMetrics() { this.commitTime.add(new MetricName(group, "commits-per-second"), new Rate(new Count())); this.processTime = metrics.sensor("process-time"); - this.commitTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); - this.commitTime.add(new MetricName(group, "process-time-max-ms"), new Max()); - this.commitTime.add(new MetricName(group, "process-calls-per-second"), new Rate(new Count())); + this.processTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); + this.processTime.add(new MetricName(group, "process-time-max-ms"), new Max()); + this.processTime.add(new MetricName(group, "process-calls-per-second"), new Rate(new Count())); this.windowTime = metrics.sensor("window-time"); this.windowTime.add(new MetricName(group, "window-time-avg-ms"), new Avg()); From 237415f5b14da09ad8e941eadba1d9dde3eb6c44 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 26 Aug 2015 17:06:18 -0700 Subject: [PATCH 179/275] Fix fetch / commit / clean logic --- .../processor/internals/PartitionGroup.java | 7 +- .../internals/ProcessorContextImpl.java | 2 +- .../processor/internals/StreamTask.java | 120 +++++++++-------- .../processor/internals/StreamThread.java | 126 +++++++++--------- 4 files changed, 132 insertions(+), 123 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java index bcde36dbdc3b5..2b708a3d933d2 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java @@ -86,12 +86,7 @@ public StampedRecord getRecord(RecordQueue queue) { */ public RecordQueue nextQueue() { // get the partition with the lowest timestamp - RecordQueue recordQueue = queuesByTime.peek(); - - if (recordQueue == null) - throw new KafkaException("No records have ever been added to this partition group yet."); - - return recordQueue; + return queuesByTime.peek(); } /** diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index 57d695242dd63..a4954b408f806 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -224,7 +224,7 @@ public void forward(K key, V value, int childIndex) { @Override public void commit() { - task.commit(); + task.needCommit(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java index 4c1b22342aa0e..7f48c1864c46b 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -20,12 +20,12 @@ import org.apache.kafka.clients.consumer.CommitType; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.RecordMetadata; +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.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; @@ -57,10 +57,10 @@ public class StreamTask { private final TimestampExtractor timestampExtractor; private final Map consumedOffsets; - private final Map producedoffsets; - private final Callback producerCallback; + private final RecordCollector recordCollector; private boolean commitRequested = false; + private boolean commitOffsetNeeded = false; private StampedRecord currRecord = null; private ProcessorNode currNode = null; @@ -69,15 +69,17 @@ public class StreamTask { * * @param id the ID of this task * @param consumer the instance of {@link Consumer} - * @param topology the instance of {@link ProcessorTopology} + * @param producer the instance of {@link Producer} * @param partitions the collection of assigned {@link TopicPartition} + * @param topology the instance of {@link ProcessorTopology} * @param config the {@link StreamingConfig} specified by the user */ + @SuppressWarnings("unchecked") public StreamTask(int id, - Consumer consumer, - ProcessorTopology topology, + Consumer consumer, + Producer producer, Collection partitions, - RecordCollector collector, + ProcessorTopology topology, StreamingConfig config) { this.id = id; @@ -98,31 +100,22 @@ public StreamTask(int id, this.partitionGroup = new PartitionGroup(partitionQueues); + // 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, + (Serializer) config.getConfiguredInstance(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, Serializer.class), + (Serializer) config.getConfiguredInstance(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Serializer.class)); + // initialize the topology with its own context try { - this.processorContext = new ProcessorContextImpl(id, this, config, collector, new Metrics()); + this.processorContext = new ProcessorContextImpl(id, this, config, recordCollector, new Metrics()); } catch (IOException e) { throw new KafkaException("Error while creating the state manager in processor context."); } topology.init(this.processorContext); - - // initialize the consumed and produced offset cache - this.consumedOffsets = new HashMap<>(); - this.producedoffsets = new HashMap<>(); - - this.producerCallback = new Callback() { - - @Override - public void onCompletion(RecordMetadata metadata, Exception exception) { - if (exception == null) { - TopicPartition partition = new TopicPartition(metadata.topic(), metadata.partition()); - producedoffsets.put(partition, metadata.offset()); - } else { - log.error("Error sending record: ", exception); - } - } - }; } public int id() { @@ -159,29 +152,30 @@ public void addRecords(TopicPartition partition, Iterator 0) { - readyForNextExecution = true; + commit(); } // if after processing this record, its partition queue's buffered size has been @@ -220,7 +202,7 @@ public boolean process() { long timestamp = partitionGroup.timestamp(); punctuationQueue.mayPunctuate(timestamp); - return readyForNextExecution; + return partitionGroup.numbuffered(); } } @@ -237,18 +219,46 @@ public void node(ProcessorNode node) { } /** - * Request committing the current task's state + * Commit the current task state */ public void commit() { + // 1) flush local state + ((ProcessorContextImpl) processorContext).stateManager().flush(); + + // 2) commit consumed offsets if it is dirty already + if (commitOffsetNeeded) { + consumer.commit(consumedOffsets, CommitType.SYNC); + commitOffsetNeeded = false; + } + + // 3) flush produced records in the downstream + // TODO: this will actually block on all produced records across the tasks + recordCollector.flush(); + } + + /** + * Request committing the current task's state + */ + public void needCommit() { this.commitRequested = true; } + /** + * Schedules a punctuation for the processor + * + * @param processor the processor requesting scheduler + * @param interval the interval in milliseconds + */ + public void schedule(Processor processor, long interval) { + punctuationQueue.schedule(new PunctuationSchedule(processor, interval)); + } + public void close() { this.partitionGroup.close(); this.consumedOffsets.clear(); } - protected RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { + private RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { return new RecordQueue(partition, source); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index be973e1b30c17..5807fbf9d52a8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -35,7 +35,6 @@ 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.serialization.Serializer; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -53,24 +52,24 @@ public class StreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(StreamThread.class); + private volatile boolean running; + private final TopologyBuilder builder; - private final RecordCollector collector; + private final Producer producer; private final Consumer consumer; - private final Map tasks = new HashMap<>(); - private final Metrics metrics; + private final Map tasks; private final Time time; - private final StreamingConfig config; private final File stateDir; private final long pollTimeMs; + private final long cleanTimeMs; private final long commitTimeMs; - private final long stateCleanupDelayMs; private final long totalRecordsToProcess; - private final KafkaStreamingMetrics streamingMetrics; + private final KafkaStreamingMetrics metrics; + private final StreamingConfig config; - private volatile boolean running; + private long lastClean; private long lastCommit; - private long nextStateCleaning; private long recordsProcessed; protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { @@ -93,34 +92,34 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce this.config = config; this.builder = builder; - this.streamingMetrics = new KafkaStreamingMetrics(); - // create the producer and consumer clients - Producer producer = new KafkaProducer<>(config.getProducerProperties(), + this.producer = new KafkaProducer<>(config.getProducerProperties(), new ByteArraySerializer(), new ByteArraySerializer()); - this.collector = new RecordCollector(producer, - (Serializer) config.getConfiguredInstance(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, Serializer.class), - (Serializer) config.getConfiguredInstance(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Serializer.class)); - consumer = new KafkaConsumer<>(config.getConsumerProperties(), + this.consumer = new KafkaConsumer<>(config.getConsumerProperties(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + // 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.pollTimeMs = config.getLong(StreamingConfig.POLL_MS_CONFIG); this.commitTimeMs = config.getLong(StreamingConfig.COMMIT_INTERVAL_MS_CONFIG); - this.stateCleanupDelayMs = config.getLong(StreamingConfig.STATE_CLEANUP_DELAY_MS_CONFIG); + this.cleanTimeMs = config.getLong(StreamingConfig.STATE_CLEANUP_DELAY_MS_CONFIG); this.totalRecordsToProcess = config.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); - this.running = true; + this.lastClean = 0; this.lastCommit = 0; - this.nextStateCleaning = Long.MAX_VALUE; this.recordsProcessed = 0; this.time = new SystemTime(); - this.metrics = new Metrics(); + this.metrics = new KafkaStreamingMetrics(); + + this.running = true; } /** @@ -139,31 +138,30 @@ public synchronized void run() { } } + /** + * Shutdown this streaming thread. + */ + public synchronized void close() { + running = false; + } + private void shutdown() { log.info("Shutting down a stream thread"); commitAll(time.milliseconds()); - collector.close(); + producer.close(); consumer.close(); removePartitions(); log.info("Stream thread shutdown complete"); } - /** - * Shutdown this streaming thread. - */ - public synchronized void close() { - running = false; - } - private void runLoop() { try { - boolean readyForNextExecution = false; + int totalNumBuffered = 0; while (stillRunning()) { - // try to fetch some records and put them to tasks' queues - // TODO: we may not need to poll every iteration - ConsumerRecords records = consumer.poll(readyForNextExecution ? 0 : this.pollTimeMs); + // try to fetch some records if necessary + ConsumerRecords records = consumer.poll(totalNumBuffered == 0 ? this.pollTimeMs : 0); for (StreamTask task : tasks.values()) { for (TopicPartition partition : task.partitions()) { @@ -172,13 +170,14 @@ private void runLoop() { } // try to process one record from each task - // TODO: we may want to process more than one record in each iteration + totalNumBuffered = 0; + for (StreamTask task : tasks.values()) { - readyForNextExecution = task.process(); + totalNumBuffered += task.process(); } + maybeClean(); maybeCommit(); - maybeCleanState(); } } catch (Exception e) { throw new KafkaException(e); @@ -190,49 +189,52 @@ private boolean stillRunning() { 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); + log.debug("Shutting down as we've reached the user configured limit of {} records to process.", totalRecordsToProcess); return false; } + return true; } private void maybeCommit() { long now = time.milliseconds(); + if (commitTimeMs >= 0 && lastCommit + commitTimeMs < now) { log.trace("Committing processor instances because the commit interval has elapsed."); commitAll(now); } } + /** + * Commit the states of all its tasks + * @param now + */ private void commitAll(long now) { - Map commit = new HashMap<>(); - for (ProcessorContextImpl context : tasks.values()) { - context.flush(); - commit.putAll(context.consumedOffsets()); + for (StreamTask task : tasks.values()) { + task.commit(); } - // check if commit is really needed, i.e. if all the offsets are already committed - if (consumer.commitNeeded(commit)) { - // TODO: for exactly-once we need to make sure the flush and commit - // are executed atomically whenever it is triggered by user - collector.flush(); - consumer.commit(commit); // TODO: can this be async? - streamingMetrics.commitTime.record(now - lastCommit); - } + metrics.commitTime.record(now - time.milliseconds()); + + lastCommit = now; } - /* delete any state dirs that aren't for active contexts */ - private void maybeCleanState() { + /** + * Cleanup any states of the tasks that have been removed from this thread + */ + private void maybeClean() { long now = time.milliseconds(); - if (now > nextStateCleaning) { + + if (now > lastClean) { File[] stateDirs = stateDir.listFiles(); if (stateDirs != null) { for (File dir : stateDirs) { try { Integer id = Integer.parseInt(dir.getName()); if (!tasks.keySet().contains(id)) { - log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), stateCleanupDelayMs); + log.info("Deleting obsolete state directory {} after delayed {} ms.", dir.getAbsolutePath(), cleanTimeMs); Utils.rm(dir); } } catch (NumberFormatException e) { @@ -241,7 +243,8 @@ private void maybeCleanState() { } } } - nextStateCleaning = Long.MAX_VALUE; + + lastClean = now; } } @@ -260,13 +263,13 @@ private void addPartitions(Collection assignment) { partitionsForTask.add(part); // create the task - task = new StreamTask(id, consumer, builder.build(), partitionsForTask, collector, config); + task = new StreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); tasks.put(id, task); } } - nextStateCleaning = time.milliseconds() + stateCleanupDelayMs; + lastClean = time.milliseconds() + cleanTimeMs; } private void removePartitions() { @@ -279,12 +282,14 @@ private void removePartitions() { } catch (Exception e) { throw new KafkaException(e); } - streamingMetrics.processorDestruction.record(); + metrics.processorDestruction.record(); } tasks.clear(); } private class KafkaStreamingMetrics { + final Metrics metrics; + final Sensor commitTime; final Sensor processTime; final Sensor windowTime; @@ -294,10 +299,12 @@ private class KafkaStreamingMetrics { public KafkaStreamingMetrics() { String group = "kafka-streaming"; + this.metrics = new Metrics(); + this.commitTime = metrics.sensor("commit-time"); this.commitTime.add(new MetricName(group, "commit-time-avg-ms"), new Avg()); - this.commitTime.add(new MetricName(group, "commits-time-max-ms"), new Max()); - this.commitTime.add(new MetricName(group, "commits-per-second"), new Rate(new Count())); + this.commitTime.add(new MetricName(group, "commit-time-max-ms"), new Max()); + this.commitTime.add(new MetricName(group, "commit-per-second"), new Rate(new Count())); this.processTime = metrics.sensor("process-time"); this.processTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); @@ -314,9 +321,6 @@ public KafkaStreamingMetrics() { this.processorDestruction = metrics.sensor("processor-destruction"); this.processorDestruction.add(new MetricName(group, "processor-destruction"), new Rate(new Count())); - } - } - } From 1c3a86ea31067a7e18e199fc5952e9df16b1fefb Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 27 Aug 2015 09:50:35 -0700 Subject: [PATCH 180/275] kstream refactored --- .../streaming/kstream/SlidingWindow.java | 264 ++++++++++++++++++ .../kafka/streaming/kstream/Window.java | 19 +- .../kstream/internals/KStreamImpl.java | 8 +- .../kstream/internals/KStreamJoin.java | 9 + .../internals/ProcessorContextImpl.java | 4 + 5 files changed, 294 insertions(+), 10 deletions(-) create mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java new file mode 100644 index 0000000000000..ffa366239c39f --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java @@ -0,0 +1,264 @@ +/** + * 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.streaming.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.streaming.kstream.internals.FilteredIterator; +import org.apache.kafka.streaming.kstream.internals.WindowSupport; +import org.apache.kafka.streaming.processor.internals.ProcessorContextImpl; +import org.apache.kafka.streaming.processor.internals.RecordCollector; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.RestoreFunc; +import org.apache.kafka.streaming.processor.internals.Stamped; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; + +public class SlidingWindow implements Window { + private String name; + private final long duration; + private final int maxCount; + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + + public SlidingWindow( + String name, + long duration, + int maxCount, + Serializer keySerializer, + Serializer valueSerializer, + Deserializer keyDeseriaizer, + Deserializer valueDeserializer) { + this.name = name; + this.duration = duration; + this.maxCount = maxCount; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.keyDeserializer = keyDeseriaizer; + this.valueDeserializer = valueDeserializer; + } + + @Override + public String name() { + return name; + } + + @Override + public WindowInstance build() { + return new SlidingWindowInstance(); + } + + public class SlidingWindowInstance extends WindowSupport implements WindowInstance { + private final Object lock = new Object(); + private ProcessorContext context; + private int slotNum; // used as a key for Kafka log compaction + private LinkedList list = new LinkedList(); + private HashMap> map = new HashMap<>(); + + @Override + public void init(ProcessorContext context) { + this.context = context; + RestoreFuncImpl restoreFunc = new RestoreFuncImpl(); + context.register(this, restoreFunc); + + for (ValueList valueList : map.values()) { + valueList.clearDirtyValues(); + } + this.slotNum = restoreFunc.slotNum; + } + + @Override + public Iterator findAfter(K key, final long timestamp) { + return find(key, timestamp, timestamp + duration); + } + + @Override + public Iterator findBefore(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp); + } + + @Override + public Iterator find(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp + duration); + } + + /* + * finds items in the window between startTime and endTime (both inclusive) + */ + private Iterator find(K key, final long startTime, final long endTime) { + final ValueList values = map.get(key); + + if (values == null) { + return null; + } else { + return new FilteredIterator>(values.iterator()) { + @Override + protected V filter(Value item) { + if (startTime <= item.timestamp && item.timestamp <= endTime) + return item.value; + else + return null; + } + }; + } + } + + @Override + public void put(K key, V value, long timestamp) { + synchronized (lock) { + slotNum++; + + list.offerLast(key); + + ValueList values = map.get(key); + if (values == null) { + values = new ValueList<>(); + map.put(key, values); + } + + values.add(slotNum, value, timestamp); + } + evictExcess(); + evictExpired(timestamp - duration); + } + + private void evictExcess() { + while (list.size() > maxCount) { + K oldestKey = list.pollFirst(); + + ValueList values = map.get(oldestKey); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } + } + + private void evictExpired(long cutoffTime) { + while (true) { + K oldestKey = list.peekFirst(); + + ValueList values = map.get(oldestKey); + Stamped oldestValue = values.first(); + + if (oldestValue.timestamp < cutoffTime) { + list.pollFirst(); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } else { + break; + } + } + } + + @Override + public String name() { + return name; + } + + @Override + public void flush() { + IntegerSerializer intSerializer = new IntegerSerializer(); + ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); + + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); + + for (Map.Entry> entry : map.entrySet()) { + ValueList values = entry.getValue(); + if (values.hasDirtyValues()) { + K key = entry.getKey(); + + byte[] keyBytes = keySerializer.serialize(name, key); + + Iterator> iterator = values.dirtyValueIterator(); + while (iterator.hasNext()) { + Value dirtyValue = iterator.next(); + byte[] slot = intSerializer.serialize("", dirtyValue.slotNum); + byte[] valBytes = valueSerializer.serialize(name, dirtyValue.value); + + byte[] combined = new byte[8 + 4 + keyBytes.length + 4 + valBytes.length]; + + int offset = 0; + offset += putLong(combined, offset, dirtyValue.timestamp); + offset += puts(combined, offset, keyBytes); + offset += puts(combined, offset, valBytes); + + if (offset != combined.length) + throw new IllegalStateException("serialized length does not match"); + + collector.send(new ProducerRecord<>(name, context.id(), slot, combined), byteArraySerializer, byteArraySerializer); + } + values.clearDirtyValues(); + } + } + } + + @Override + public void close() { + // TODO + } + + @Override + public boolean persistent() { + // TODO: should not be persistent, right? + return false; + } + + private class RestoreFuncImpl implements RestoreFunc { + + final IntegerDeserializer intDeserializer; + int slotNum = 0; + + RestoreFuncImpl() { + intDeserializer = new IntegerDeserializer(); + } + + @Override + public void apply(byte[] slot, byte[] bytes) { + + slotNum = intDeserializer.deserialize("", slot); + + int offset = 0; + // timestamp + long timestamp = getLong(bytes, offset); + offset += 8; + // key + int length = getInt(bytes, offset); + offset += 4; + K key = deserialize(bytes, offset, length, name, keyDeserializer); + offset += length; + // value + length = getInt(bytes, offset); + offset += 4; + V value = deserialize(bytes, offset, length, name, valueDeserializer); + + put(key, value, timestamp); + } + } + } + +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java index fc9582d21fc20..b3ec0e2a108e0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java @@ -22,15 +22,22 @@ import java.util.Iterator; -public interface Window extends StateStore { +public interface Window { - void init(ProcessorContext context); + interface WindowInstance extends StateStore { - Iterator find(K key, long timestamp); + void init(ProcessorContext context); - Iterator findAfter(K key, long timestamp); + Iterator find(K key, long timestamp); - Iterator findBefore(K key, long timestamp); + Iterator findAfter(K key, long timestamp); - void put(K key, V value, long timestamp); + Iterator findBefore(K key, long timestamp); + + void put(K key, V value, long timestamp); + } + + String name(); + + WindowInstance build(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index e317338b134e6..ba386c7c85cbe 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -19,9 +19,9 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.kstream.KeyValueFlatMap; import org.apache.kafka.streaming.processor.ProcessorFactory; import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streaming.kstream.KeyValueFlatMap; import org.apache.kafka.streaming.kstream.KStreamWindowed; import org.apache.kafka.streaming.kstream.KeyValueMapper; import org.apache.kafka.streaming.kstream.Predicate; @@ -130,12 +130,12 @@ public KStream flatMapValues(ValueMapper> } @Override - public KStreamWindowed with(WindowDef windowDef) { + public KStreamWindowed with(WindowDef window) { String name = WINDOWED_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, new KStreamWindow<>(windowDef), this.name); + topology.addProcessor(name, new KStreamWindow<>(window), this.name); - return new KStreamWindowedImpl<>(topology, name, windowDef); + return new KStreamWindowedImpl<>(topology, name, window); } @Override diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index a8452d5ff0b64..445ece58c435e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -20,7 +20,11 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.ValueJoiner; +<<<<<<< HEAD import org.apache.kafka.streaming.kstream.Window; +======= +import org.apache.kafka.streaming.kstream.Window.WindowInstance; +>>>>>>> kstream refactored import org.apache.kafka.streaming.processor.ProcessorFactory; import java.util.Iterator; @@ -70,8 +74,13 @@ public void init(ProcessorContext context) { if (!context.joinable()) throw new IllegalStateException("Streams are not joinable."); +<<<<<<< HEAD final Window window1 = (Window) context.getStateStore(windowName1); final Window window2 = (Window) context.getStateStore(windowName2); +======= + final WindowInstance window1 = (WindowInstance) context.getStateStore(windowName1); + final WindowInstance window2 = (WindowInstance) context.getStateStore(windowName2); +>>>>>>> kstream refactored if (prior) { this.finder1 = new Finder() { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index a4954b408f806..03df3e3cf5a57 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -217,7 +217,11 @@ public void forward(K key, V value) { @Override @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { +<<<<<<< HEAD ProcessorNode childNode = (ProcessorNode) task.node().children().get(childIndex); +======= + ProcessorNode childNode = (ProcessorNode) task.node().children().get(childIndex); +>>>>>>> kstream refactored task.node(childNode); childNode.process(key, value); } From 8515ec777ef67401a205d9c08bc864843efcd42f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 27 Aug 2015 13:11:18 -0700 Subject: [PATCH 181/275] rename factory to def, move some unit test files --- .../kafka/streaming/kstream/KStream.java | 6 +- .../streaming/kstream/SlidingWindowDef.java | 2 +- .../kafka/streaming/kstream/WindowDef.java | 2 +- .../kstream/internals/KStreamBranch.java | 6 +- .../kstream/internals/KStreamFilter.java | 6 +- .../kstream/internals/KStreamFlatMap.java | 6 +- .../internals/KStreamFlatMapValues.java | 6 +- .../kstream/internals/KStreamImpl.java | 7 +- .../kstream/internals/KStreamJoin.java | 18 +- .../kstream/internals/KStreamMap.java | 6 +- .../kstream/internals/KStreamMapValues.java | 6 +- .../kstream/internals/KStreamPassThrough.java | 6 +- .../kstream/internals/KStreamWindow.java | 8 +- .../internals/KStreamWindowedImpl.java | 2 +- ...rocessorFactory.java => ProcessorDef.java} | 4 +- .../streaming/processor/TopologyBuilder.java | 14 +- .../kafka/streaming/FilteredIteratorTest.java | 95 ---------- .../internals/KStreamFilterTest.java | 84 --------- .../internals/KStreamFlatMapTest.java | 80 --------- .../internals/KStreamFlatMapValuesTest.java | 77 --------- .../streaming/internals/KStreamMapTest.java | 73 -------- .../internals/KStreamSourceTest.java | 59 ------- .../streaming/internals/StreamGroupTest.java | 162 ------------------ .../internals/FilteredIteratorTest.java | 2 +- .../internals/KStreamBranchTest.java | 2 +- .../internals}/KStreamFilterTest.java | 2 +- .../internals}/KStreamFlatMapTest.java | 2 +- .../internals}/KStreamFlatMapValuesTest.java | 2 +- .../internals/KStreamJoinTest.java | 2 +- .../internals}/KStreamMapTest.java | 2 +- .../internals/KStreamMapValuesTest.java | 2 +- .../internals}/KStreamSourceTest.java | 2 +- .../internals/KStreamWindowedTest.java | 2 +- .../internals}/StreamGroupTest.java | 2 +- temp | 6 +- temp.sh | 4 + temp2 | 6 +- 37 files changed, 70 insertions(+), 703 deletions(-) rename stream/src/main/java/org/apache/kafka/streaming/processor/{ProcessorFactory.java => ProcessorDef.java} (93%) delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/FilteredIteratorTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFilterTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapValuesTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/KStreamSourceTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java rename stream/src/test/java/org/apache/kafka/streaming/{ => kstream}/internals/FilteredIteratorTest.java (98%) rename stream/src/test/java/org/apache/kafka/streaming/{ => kstream}/internals/KStreamBranchTest.java (98%) rename stream/src/test/java/org/apache/kafka/streaming/{ => kstream/internals}/KStreamFilterTest.java (98%) rename stream/src/test/java/org/apache/kafka/streaming/{ => kstream/internals}/KStreamFlatMapTest.java (98%) rename stream/src/test/java/org/apache/kafka/streaming/{ => kstream/internals}/KStreamFlatMapValuesTest.java (98%) rename stream/src/test/java/org/apache/kafka/streaming/{ => kstream}/internals/KStreamJoinTest.java (99%) rename stream/src/test/java/org/apache/kafka/streaming/{ => kstream/internals}/KStreamMapTest.java (98%) rename stream/src/test/java/org/apache/kafka/streaming/{ => kstream}/internals/KStreamMapValuesTest.java (98%) rename stream/src/test/java/org/apache/kafka/streaming/{ => kstream/internals}/KStreamSourceTest.java (98%) rename stream/src/test/java/org/apache/kafka/streaming/{ => kstream}/internals/KStreamWindowedTest.java (98%) rename stream/src/test/java/org/apache/kafka/streaming/{ => processor/internals}/StreamGroupTest.java (99%) create mode 100755 temp.sh diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java index 8df2131624eea..2530b37d0d613 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.processor.ProcessorDef; /** * KStream is an abstraction of a stream of key-value pairs. @@ -133,7 +133,7 @@ public interface KStream { /** * Processes all elements in this stream by applying a processor. * - * @param processorFactory the class of ProcessorFactory + * @param processorDef the class of ProcessorDef */ - KStream process(ProcessorFactory processorFactory); + KStream process(ProcessorDef processorDef); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java index 4fc8ca41411c8..6225fd990c9d2 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java @@ -68,7 +68,7 @@ public String name() { } @Override - public Window build() { + public Window define() { return new SlidingWindow(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java index 26971937eccfe..454987677a79e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java @@ -21,5 +21,5 @@ public interface WindowDef { String name(); - Window build(); + Window define(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java index e9d6030401c58..4ecd10cff0a29 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java @@ -18,10 +18,10 @@ package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.processor.ProcessorDef; import org.apache.kafka.streaming.kstream.Predicate; -class KStreamBranch implements ProcessorFactory { +class KStreamBranch implements ProcessorDef { private final Predicate[] predicates; @@ -31,7 +31,7 @@ public KStreamBranch(Predicate... predicates) { } @Override - public Processor build() { + public Processor define() { return new KStreamBranchProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java index a85e0181a7c76..f7b593e2fc95d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java @@ -19,9 +19,9 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.Predicate; -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.processor.ProcessorDef; -class KStreamFilter implements ProcessorFactory { +class KStreamFilter implements ProcessorDef { private final Predicate predicate; private final boolean filterOut; @@ -32,7 +32,7 @@ public KStreamFilter(Predicate predicate, boolean filterOut) { } @Override - public Processor build() { + public Processor define() { return new KStreamFilterProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java index bc834a8f66ae8..ba872b167ef19 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java @@ -20,9 +20,9 @@ import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.KeyValueFlatMap; import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.processor.ProcessorDef; -class KStreamFlatMap implements ProcessorFactory { +class KStreamFlatMap implements ProcessorDef { private final KeyValueFlatMap mapper; @@ -31,7 +31,7 @@ class KStreamFlatMap implements ProcessorFactory { } @Override - public Processor build() { + public Processor define() { return new KStreamFlatMapProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java index 5e18f188ffa08..291a18fa1b529 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java @@ -19,9 +19,9 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.processor.ProcessorDef; -class KStreamFlatMapValues implements ProcessorFactory { +class KStreamFlatMapValues implements ProcessorDef { private final ValueMapper> mapper; @@ -31,7 +31,7 @@ class KStreamFlatMapValues implements ProcessorFactory { } @Override - public Processor build() { + public Processor define() { return new KStreamFlatMapValuesProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index ba386c7c85cbe..bb293f9b87efa 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -19,7 +19,8 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.kstream.KeyValueFlatMap; +import org.apache.kafka.streaming.processor.ProcessorDef; import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.kstream.KeyValueFlatMap; import org.apache.kafka.streaming.kstream.KStreamWindowed; @@ -185,10 +186,10 @@ public void sendTo(String topic, Serializer keySerializer, Serializer valS @SuppressWarnings("unchecked") @Override - public KStream process(final ProcessorFactory processorFactory) { + public KStream process(final ProcessorDef processorDef) { String name = PROCESSOR_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, processorFactory, this.name); + topology.addProcessor(name, processorDef, this.name); return new KStreamImpl<>(topology, name); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index 445ece58c435e..e34d19a4d71fe 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -20,16 +20,13 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.ValueJoiner; -<<<<<<< HEAD import org.apache.kafka.streaming.kstream.Window; -======= import org.apache.kafka.streaming.kstream.Window.WindowInstance; ->>>>>>> kstream refactored -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.processor.ProcessorDef; import java.util.Iterator; -class KStreamJoin implements ProcessorFactory { +class KStreamJoin implements ProcessorDef { private static abstract class Finder { abstract Iterator find(K key, long timestamp); @@ -41,9 +38,9 @@ private static abstract class Finder { private final boolean prior; private Processor processorForOtherStream = null; - public final ProcessorFactory processorFactoryForOtherStream = new ProcessorFactory() { + public final ProcessorDef processorDefForOtherStream = new ProcessorDef() { @Override - public Processor build() { + public Processor define() { return processorForOtherStream; } }; @@ -56,7 +53,7 @@ public Processor build() { } @Override - public Processor build() { + public Processor define() { return new KStreamJoinProcessor(); } @@ -74,13 +71,8 @@ public void init(ProcessorContext context) { if (!context.joinable()) throw new IllegalStateException("Streams are not joinable."); -<<<<<<< HEAD final Window window1 = (Window) context.getStateStore(windowName1); final Window window2 = (Window) context.getStateStore(windowName2); -======= - final WindowInstance window1 = (WindowInstance) context.getStateStore(windowName1); - final WindowInstance window2 = (WindowInstance) context.getStateStore(windowName2); ->>>>>>> kstream refactored if (prior) { this.finder1 = new Finder() { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java index 83fb4c11c99f6..f4ad16cebaddd 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java @@ -20,9 +20,9 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.processor.ProcessorDef; -class KStreamMap implements ProcessorFactory { +class KStreamMap implements ProcessorDef { private final KeyValueMapper mapper; @@ -31,7 +31,7 @@ public KStreamMap(KeyValueMapper mapper) { } @Override - public Processor build() { + public Processor define() { return new KStreamMapProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java index 667c04386c381..f7dc8027ad1a1 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java @@ -19,9 +19,9 @@ import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.processor.ProcessorDef; -class KStreamMapValues implements ProcessorFactory { +class KStreamMapValues implements ProcessorDef { private final ValueMapper mapper; @@ -30,7 +30,7 @@ public KStreamMapValues(ValueMapper mapper) { } @Override - public Processor build() { + public Processor define() { return new KStreamMapProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java index 98528f72c7c1b..7c0863d7c733d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java @@ -18,12 +18,12 @@ package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.processor.ProcessorDef; -class KStreamPassThrough implements ProcessorFactory { +class KStreamPassThrough implements ProcessorDef { @Override - public Processor build() { + public Processor define() { return new KStreamPassThroughProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java index f357613d94ce4..b9ea3bce45c7c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java @@ -19,11 +19,11 @@ import org.apache.kafka.streaming.kstream.Window; import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorFactory; +import org.apache.kafka.streaming.processor.ProcessorDef; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.WindowDef; -public class KStreamWindow implements ProcessorFactory { +public class KStreamWindow implements ProcessorDef { private final WindowDef windowDef; @@ -36,7 +36,7 @@ public WindowDef window() { } @Override - public Processor build() { + public Processor define() { return new KStreamWindowProcessor(); } @@ -47,7 +47,7 @@ private class KStreamWindowProcessor extends KStreamProcessor { @Override public void init(ProcessorContext context) { super.init(context); - this.window = windowDef.build(); + this.window = windowDef.define(); this.window.init(context); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java index e269c3ad40bcc..62351a4e3607f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java @@ -37,7 +37,7 @@ private KStream join(KStreamWindowed other, boolean prior String joinName = JOIN_NAME + INDEX.getAndIncrement(); String joinOtherName = JOINOTHER_NAME + INDEX.getAndIncrement(); topology.addProcessor(joinName, join, this.name); - topology.addProcessor(joinOtherName, join.processorFactoryForOtherStream, ((KStreamImpl) other).name); + topology.addProcessor(joinOtherName, join.processorDefForOtherStream, ((KStreamImpl) other).name); return new KStreamImpl<>(topology, joinName); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorFactory.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorDef.java similarity index 93% rename from stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorFactory.java rename to stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorDef.java index 1d1c4816d8cce..8f1ef4df60777 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorFactory.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorDef.java @@ -17,7 +17,7 @@ package org.apache.kafka.streaming.processor; -public interface ProcessorFactory { +public interface ProcessorDef { - Processor build(); + Processor define(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 894e0e37f0244..8afacfc05f78f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -47,16 +47,16 @@ private interface NodeFactory { private class ProcessorNodeFactory implements NodeFactory { public final String[] parents; private final String name; - private final ProcessorFactory factory; + private final ProcessorDef definition; - public ProcessorNodeFactory(String name, String[] parents, ProcessorFactory factory) { + public ProcessorNodeFactory(String name, String[] parents, ProcessorDef definition) { this.name = name; this.parents = parents.clone(); - this.factory = factory; + this.definition = definition; } public ProcessorNode build() { - Processor processor = factory.build(); + Processor processor = definition.define(); return new ProcessorNode(name, processor); } } @@ -134,7 +134,7 @@ public final void addSink(String name, String topic, Serializer keySerializer, S nodeFactories.add(new SinkNodeFactory(name, parentNames, topic, keySerializer, valSerializer)); } - public final void addProcessor(String name, ProcessorFactory factory, String... parentNames) { + public final void addProcessor(String name, ProcessorDef definition, String... parentNames) { if (nodeNames.contains(name)) throw new IllegalArgumentException("Processor " + name + " is already added."); @@ -150,7 +150,7 @@ public final void addProcessor(String name, ProcessorFactory factory, String... } nodeNames.add(name); - nodeFactories.add(new ProcessorNodeFactory(name, parentNames, factory)); + nodeFactories.add(new ProcessorNodeFactory(name, parentNames, definition)); } /** @@ -186,7 +186,7 @@ public ProcessorTopology build() { processorMap.get(parent).addChild(node); } } else { - throw new IllegalStateException("unknown factory class: " + factory.getClass().getName()); + throw new IllegalStateException("unknown definition class: " + factory.getClass().getName()); } } } catch (Exception e) { diff --git a/stream/src/test/java/org/apache/kafka/streaming/FilteredIteratorTest.java b/stream/src/test/java/org/apache/kafka/streaming/FilteredIteratorTest.java deleted file mode 100644 index f2c1f9f5023eb..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/FilteredIteratorTest.java +++ /dev/null @@ -1,95 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming; - -import static org.junit.Assert.assertEquals; - -import org.apache.kafka.streaming.kstream.internals.FilteredIterator; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; - -public class FilteredIteratorTest { - - @Test - public void testFiltering() { - List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); - - Iterator filtered = new FilteredIterator(list.iterator()) { - protected String filter(Integer i) { - if (i % 3 == 0) return i.toString(); - return null; - } - }; - - List expected = Arrays.asList("3", "9", "6", "3"); - List result = new ArrayList(); - - while (filtered.hasNext()) { - result.add(filtered.next()); - } - - assertEquals(expected, result); - } - - @Test - public void testEmptySource() { - List list = new ArrayList(); - - Iterator filtered = new FilteredIterator(list.iterator()) { - protected String filter(Integer i) { - if (i % 3 == 0) return i.toString(); - return null; - } - }; - - List expected = new ArrayList(); - List result = new ArrayList(); - - while (filtered.hasNext()) { - result.add(filtered.next()); - } - - assertEquals(expected, result); - } - - @Test - public void testNoMatch() { - List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); - - Iterator filtered = new FilteredIterator(list.iterator()) { - protected String filter(Integer i) { - if (i % 7 == 0) return i.toString(); - return null; - } - }; - - List expected = new ArrayList(); - List result = new ArrayList(); - - while (filtered.hasNext()) { - result.add(filtered.next()); - } - - assertEquals(expected, result); - } - -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFilterTest.java deleted file mode 100644 index 276302b738bbf..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFilterTest.java +++ /dev/null @@ -1,84 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.internals; - -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.Predicate; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class KStreamFilterTest { - - private String topicName = "topic"; - - private KStreamBuilder topology = new MockKStreamBuilder(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - private Predicate isMultipleOfThree = new Predicate() { - @Override - public boolean apply(Integer key, String value) { - return (key % 3) == 0; - } - }; - - @Test - public void testFilter() { - final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - - KStream stream; - MockProcessor processor; - - processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); - stream.filter(isMultipleOfThree).process(processor); - - for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); - } - - assertEquals(2, processor.processed.size()); - } - - @Test - public void testFilterOut() { - final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - - KStream stream; - MockProcessor processor; - - processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); - stream.filterOut(isMultipleOfThree).process(processor); - - for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); - } - - assertEquals(5, processor.processed.size()); - } - -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapTest.java deleted file mode 100644 index 82b36308a0c8b..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapTest.java +++ /dev/null @@ -1,80 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.internals; - -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -import java.util.ArrayList; - -public class KStreamFlatMapTest { - - private String topicName = "topic"; - - private KStreamBuilder topology = new MockKStreamBuilder(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - @Test - public void testFlatMap() { - - KeyValueMapper> mapper = - new KeyValueMapper>() { - @Override - public KeyValue> apply(Integer key, String value) { - ArrayList result = new ArrayList(); - for (int i = 0; i < key; i++) { - result.add(value); - } - return KeyValue.pair(Integer.toString(key * 10), (Iterable) result); - } - }; - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - - KStream stream; - MockProcessor processor; - - processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); - stream.flatMap(mapper).process(processor); - - for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); - } - - assertEquals(6, processor.processed.size()); - - String[] expected = new String[]{"10:V1", "20:V2", "20:V2", "30:V3", "30:V3", "30:V3"}; - - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); - } - } - -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapValuesTest.java deleted file mode 100644 index 9c5863bf3ed19..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamFlatMapValuesTest.java +++ /dev/null @@ -1,77 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.internals; - -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; -import org.junit.Test; - -import java.util.ArrayList; - -import static org.junit.Assert.assertEquals; - -public class KStreamFlatMapValuesTest { - - private String topicName = "topic"; - - private KStreamBuilder topology = new MockKStreamBuilder(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - @Test - public void testFlatMapValues() { - - ValueMapper> mapper = - new ValueMapper>() { - @Override - public Iterable apply(String value) { - ArrayList result = new ArrayList(); - result.add(value.toLowerCase()); - result.add(value); - return result; - } - }; - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - - KStream stream; - MockProcessor processor; - - processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); - stream.flatMapValues(mapper).process(processor); - - for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); - } - - assertEquals(8, processor.processed.size()); - - String[] expected = new String[]{"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)); - } - } -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapTest.java deleted file mode 100644 index 7104e5efefec4..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapTest.java +++ /dev/null @@ -1,73 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.internals; - -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class KStreamMapTest { - - private String topicName = "topic"; - - private KStreamBuilder topology = new MockKStreamBuilder(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - @Test - public void testMap() { - - KeyValueMapper mapper = - new KeyValueMapper() { - @Override - public KeyValue apply(Integer key, String value) { - return KeyValue.pair(value, key); - } - }; - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - - KStream stream; - MockProcessor processor; - - processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); - stream.map(mapper).process(processor); - - for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); - } - - assertEquals(4, processor.processed.size()); - - String[] expected = new String[]{"V0:0", "V1:1", "V2:2", "V3:3"}; - - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); - } - } -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamSourceTest.java deleted file mode 100644 index 2f792f415605a..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamSourceTest.java +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.internals; - -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class KStreamSourceTest { - - private String topicName = "topic"; - - private KStreamBuilder topology = new MockKStreamBuilder(); - private StringDeserializer keyDeserializer = new StringDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - @Test - public void testKStreamSource() { - - MockProcessor processor = new MockProcessor<>(); - - KStream stream = topology.from(keyDeserializer, valDeserializer, topicName); - stream.process(processor); - - final String[] expectedKeys = new String[]{"k1", "k2", "k3"}; - final String[] expectedValues = new String[]{"v1", "v2", "v3"}; - - for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], expectedValues[i]); - } - - assertEquals(3, processor.processed.size()); - - for (int i = 0; i < expectedKeys.length; i++) { - assertEquals(expectedKeys[i] + ":" + expectedValues[i], processor.processed.get(i)); - } - } -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java deleted file mode 100644 index 496174ad52872..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/StreamGroupTest.java +++ /dev/null @@ -1,162 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.internals; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streaming.processor.TimestampExtractor; -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.streaming.processor.internals.StreamGroup; -import org.apache.kafka.test.MockProcessorContext; -import org.apache.kafka.test.MockSourceNode; -import org.junit.Test; - -import java.util.Arrays; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -public class StreamGroupTest { - - private static Serializer serializer = new IntegerSerializer(); - private static Deserializer deserializer = new IntegerDeserializer(); - - @SuppressWarnings("unchecked") - @Test - public void testAddPartition() { - - MockIngestor mockIngestor = new MockIngestor(); - - StreamGroup streamGroup = new StreamGroup( - new MockProcessorContext(serializer, deserializer), - mockIngestor, - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - if (topic.equals("topic1")) - return ((Integer) key).longValue(); - else - return ((Integer) key).longValue() / 10L + 5L; - } - }, - 3 - ); - - TopicPartition partition1 = new TopicPartition("topic1", 1); - TopicPartition partition2 = new TopicPartition("topic2", 1); - MockSourceNode source1 = new MockSourceNode(deserializer, deserializer); - MockSourceNode source2 = new MockSourceNode(deserializer, deserializer); - MockSourceNode source3 = new MockSourceNode(deserializer, deserializer); - - streamGroup.addPartition(partition1, source1); - mockIngestor.addPartitionStreamToGroup(streamGroup, partition1); - - streamGroup.addPartition(partition2, source2); - mockIngestor.addPartitionStreamToGroup(streamGroup, partition2); - - Exception exception = null; - try { - streamGroup.addPartition(partition1, source3); - } catch (Exception ex) { - exception = ex; - } - assertTrue(exception != null); - - byte[] recordValue = serializer.serialize(null, new Integer(10)); - - mockIngestor.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 1, serializer.serialize(partition1.topic(), new Integer(10)), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 2, serializer.serialize(partition1.topic(), new Integer(20)), recordValue) - )); - - mockIngestor.addRecords(partition2, records( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 1, serializer.serialize(partition1.topic(), new Integer(300)), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 2, serializer.serialize(partition1.topic(), new Integer(400)), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 3, serializer.serialize(partition1.topic(), new Integer(500)), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 4, serializer.serialize(partition1.topic(), new Integer(600)), recordValue) - )); - - streamGroup.process(); - assertEquals(source1.numReceived, 1); - assertEquals(source2.numReceived, 0); - - assertEquals(mockIngestor.paused.size(), 1); - assertTrue(mockIngestor.paused.contains(partition2)); - - mockIngestor.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 3, serializer.serialize(partition1.topic(), new Integer(30)), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 4, serializer.serialize(partition1.topic(), new Integer(40)), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 5, serializer.serialize(partition1.topic(), new Integer(50)), recordValue) - )); - - streamGroup.process(); - assertEquals(source1.numReceived, 2); - assertEquals(source2.numReceived, 0); - - assertEquals(mockIngestor.paused.size(), 2); - assertTrue(mockIngestor.paused.contains(partition1)); - assertTrue(mockIngestor.paused.contains(partition2)); - - streamGroup.process(); - assertEquals(source1.numReceived, 3); - assertEquals(source2.numReceived, 0); - - streamGroup.process(); - assertEquals(source1.numReceived, 3); - assertEquals(source2.numReceived, 1); - - assertEquals(mockIngestor.paused.size(), 1); - assertTrue(mockIngestor.paused.contains(partition2)); - - streamGroup.process(); - assertEquals(source1.numReceived, 4); - assertEquals(source2.numReceived, 1); - - assertEquals(mockIngestor.paused.size(), 1); - - streamGroup.process(); - assertEquals(source1.numReceived, 4); - assertEquals(source2.numReceived, 2); - - assertEquals(mockIngestor.paused.size(), 0); - - streamGroup.process(); - assertEquals(source1.numReceived, 5); - assertEquals(source2.numReceived, 2); - - streamGroup.process(); - assertEquals(source1.numReceived, 5); - assertEquals(source2.numReceived, 3); - - streamGroup.process(); - assertEquals(source1.numReceived, 5); - assertEquals(source2.numReceived, 4); - - assertEquals(mockIngestor.paused.size(), 0); - - streamGroup.process(); - assertEquals(source1.numReceived, 5); - assertEquals(source2.numReceived, 4); - } - - private Iterable> records(ConsumerRecord... recs) { - return Arrays.asList(recs); - } -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/FilteredIteratorTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/internals/FilteredIteratorTest.java rename to stream/src/test/java/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.java index 2bd658c826a0c..ff8ed098f0977 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/FilteredIteratorTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.internals; +package org.apache.kafka.streaming.kstream.internals; import static org.junit.Assert.assertEquals; diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/internals/KStreamBranchTest.java rename to stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java index 518d00f43d7e7..217a65f79afde 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.internals; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/stream/src/test/java/org/apache/kafka/streaming/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/KStreamFilterTest.java rename to stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java index b55946d14c75f..301a5a7fbe308 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapTest.java rename to stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java index 56471048cf3c8..4884fee7f2617 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapValuesTest.java rename to stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java index e001f7bd2a52b..48b2733cf15fd 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java similarity index 99% rename from stream/src/test/java/org/apache/kafka/streaming/internals/KStreamJoinTest.java rename to stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java index bdc2b23a82e5b..472dfd44eb7f0 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.internals; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/stream/src/test/java/org/apache/kafka/streaming/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/KStreamMapTest.java rename to stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java index 79306d9ebdd1d..73e9852b3b975 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapValuesTest.java rename to stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.java index 9cca557bc85d9..45a793fc7ab9f 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.internals; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/stream/src/test/java/org/apache/kafka/streaming/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/KStreamSourceTest.java rename to stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java index da2512b56580f..6ce4825600711 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/KStreamSourceTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.streaming.kstream.KStream; diff --git a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java rename to stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.java index aaa4e3a04a1f0..20cc445e814b4 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/internals/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.internals; +package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamGroupTest.java similarity index 99% rename from stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java rename to stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamGroupTest.java index 4cf2d99556262..659de88f03f30 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/StreamGroupTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamGroupTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming; +package org.apache.kafka.streaming.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.streaming.processor.TimestampExtractor; diff --git a/temp b/temp index 2e8567c23ef09..0c9e971a95412 100644 --- a/temp +++ b/temp @@ -1,4 +1,4 @@ -stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java: needs merge -stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java: needs merge -stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java: needs merge +stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java: needs merge +stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java: needs merge +stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java: needs merge stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java: needs merge diff --git a/temp.sh b/temp.sh new file mode 100755 index 0000000000000..74e9f056bd9b5 --- /dev/null +++ b/temp.sh @@ -0,0 +1,4 @@ +#!/bin/bash + +cat temp | awk -F":" '{print $1}' > temp2 +cat temp2 | while read in; do git add "$1"; done diff --git a/temp2 b/temp2 index 67423b33540a4..8a742c01794a1 100644 --- a/temp2 +++ b/temp2 @@ -1,4 +1,4 @@ -stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java -stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java -stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java +stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java +stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java +stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java From 372c66c09a1da2dc20c72f0ebd7410f70d740193 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 27 Aug 2015 13:41:16 -0700 Subject: [PATCH 182/275] rename define() to instance() --- .../org/apache/kafka/streaming/kstream/SlidingWindowDef.java | 2 +- .../java/org/apache/kafka/streaming/kstream/WindowDef.java | 2 +- .../kafka/streaming/kstream/internals/KStreamBranch.java | 2 +- .../kafka/streaming/kstream/internals/KStreamFilter.java | 2 +- .../kafka/streaming/kstream/internals/KStreamFlatMap.java | 2 +- .../streaming/kstream/internals/KStreamFlatMapValues.java | 2 +- .../apache/kafka/streaming/kstream/internals/KStreamJoin.java | 4 ++-- .../apache/kafka/streaming/kstream/internals/KStreamMap.java | 2 +- .../kafka/streaming/kstream/internals/KStreamMapValues.java | 2 +- .../kafka/streaming/kstream/internals/KStreamPassThrough.java | 2 +- .../kafka/streaming/kstream/internals/KStreamWindow.java | 4 ++-- .../org/apache/kafka/streaming/processor/ProcessorDef.java | 2 +- .../org/apache/kafka/streaming/processor/TopologyBuilder.java | 2 +- 13 files changed, 15 insertions(+), 15 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java index 6225fd990c9d2..43c2ea7f5410c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java @@ -68,7 +68,7 @@ public String name() { } @Override - public Window define() { + public Window instance() { return new SlidingWindow(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java index 454987677a79e..f1ab71e9622a3 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java @@ -21,5 +21,5 @@ public interface WindowDef { String name(); - Window define(); + Window instance(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java index 4ecd10cff0a29..e1407b1f44b57 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java @@ -31,7 +31,7 @@ public KStreamBranch(Predicate... predicates) { } @Override - public Processor define() { + public Processor instance() { return new KStreamBranchProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java index f7b593e2fc95d..8d62f7af85190 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java @@ -32,7 +32,7 @@ public KStreamFilter(Predicate predicate, boolean filterOut) { } @Override - public Processor define() { + public Processor instance() { return new KStreamFilterProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java index ba872b167ef19..b9a7f5dcd8285 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java @@ -31,7 +31,7 @@ class KStreamFlatMap implements ProcessorDef { } @Override - public Processor define() { + public Processor instance() { return new KStreamFlatMapProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java index 291a18fa1b529..241970c4a5323 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java @@ -31,7 +31,7 @@ class KStreamFlatMapValues implements ProcessorDef { } @Override - public Processor define() { + public Processor instance() { return new KStreamFlatMapValuesProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index e34d19a4d71fe..ce1f2ce022d66 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -40,7 +40,7 @@ private static abstract class Finder { private Processor processorForOtherStream = null; public final ProcessorDef processorDefForOtherStream = new ProcessorDef() { @Override - public Processor define() { + public Processor instance() { return processorForOtherStream; } }; @@ -53,7 +53,7 @@ public Processor define() { } @Override - public Processor define() { + public Processor instance() { return new KStreamJoinProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java index f4ad16cebaddd..2400e03e73ad8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java @@ -31,7 +31,7 @@ public KStreamMap(KeyValueMapper mapper) { } @Override - public Processor define() { + public Processor instance() { return new KStreamMapProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java index f7dc8027ad1a1..dc8267a4fa91d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java @@ -30,7 +30,7 @@ public KStreamMapValues(ValueMapper mapper) { } @Override - public Processor define() { + public Processor instance() { return new KStreamMapProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java index 7c0863d7c733d..89c3550998b12 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java @@ -23,7 +23,7 @@ class KStreamPassThrough implements ProcessorDef { @Override - public Processor define() { + public Processor instance() { return new KStreamPassThroughProcessor(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java index b9ea3bce45c7c..980fb4e08e350 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java @@ -36,7 +36,7 @@ public WindowDef window() { } @Override - public Processor define() { + public Processor instance() { return new KStreamWindowProcessor(); } @@ -47,7 +47,7 @@ private class KStreamWindowProcessor extends KStreamProcessor { @Override public void init(ProcessorContext context) { super.init(context); - this.window = windowDef.define(); + this.window = windowDef.instance(); this.window.init(context); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorDef.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorDef.java index 8f1ef4df60777..382c95733ebcd 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorDef.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorDef.java @@ -19,5 +19,5 @@ public interface ProcessorDef { - Processor define(); + Processor instance(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 8afacfc05f78f..596a4c6c49885 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -56,7 +56,7 @@ public ProcessorNodeFactory(String name, String[] parents, ProcessorDef definiti } public ProcessorNode build() { - Processor processor = definition.define(); + Processor processor = definition.instance(); return new ProcessorNode(name, processor); } } From f1070fd2a7a29e411a8497b6dcecd03891cecceb Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 27 Aug 2015 14:00:15 -0700 Subject: [PATCH 183/275] rebased --- .../kafka/streaming/kstream/Window.java | 21 +++++++------------ .../kafka/streaming/kstream/WindowDef.java | 4 ++-- .../kstream/internals/KStreamJoin.java | 1 - 3 files changed, 9 insertions(+), 17 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java index b3ec0e2a108e0..a968b1782e2a2 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java @@ -22,22 +22,15 @@ import java.util.Iterator; -public interface Window { +public interface Window extends StateStore { - interface WindowInstance extends StateStore { + void init(ProcessorContext context); - void init(ProcessorContext context); + Iterator find(K key, long timestamp); - Iterator find(K key, long timestamp); + Iterator findAfter(K key, long timestamp); - Iterator findAfter(K key, long timestamp); + Iterator findBefore(K key, long timestamp); - Iterator findBefore(K key, long timestamp); - - void put(K key, V value, long timestamp); - } - - String name(); - - WindowInstance build(); -} + void put(K key, V value, long timestamp); +} \ No newline at end of file diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java index f1ab71e9622a3..776171242f7fb 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java @@ -21,5 +21,5 @@ public interface WindowDef { String name(); - Window instance(); -} + Window build(); +} \ No newline at end of file diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index ce1f2ce022d66..2c0020b12afb8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -21,7 +21,6 @@ import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.ValueJoiner; import org.apache.kafka.streaming.kstream.Window; -import org.apache.kafka.streaming.kstream.Window.WindowInstance; import org.apache.kafka.streaming.processor.ProcessorDef; import java.util.Iterator; From 3902e17cc2f20a20c5fc9429a68bdcd048085007 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 27 Aug 2015 14:05:48 -0700 Subject: [PATCH 184/275] rebased continue --- .../streaming/kstream/SlidingWindow.java | 264 ------------------ .../kafka/streaming/kstream/WindowDef.java | 2 +- .../kstream/internals/KStreamImpl.java | 1 - 3 files changed, 1 insertion(+), 266 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java deleted file mode 100644 index ffa366239c39f..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java +++ /dev/null @@ -1,264 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.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.streaming.kstream.internals.FilteredIterator; -import org.apache.kafka.streaming.kstream.internals.WindowSupport; -import org.apache.kafka.streaming.processor.internals.ProcessorContextImpl; -import org.apache.kafka.streaming.processor.internals.RecordCollector; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RestoreFunc; -import org.apache.kafka.streaming.processor.internals.Stamped; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.Map; - -public class SlidingWindow implements Window { - private String name; - private final long duration; - private final int maxCount; - private final Serializer keySerializer; - private final Serializer valueSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; - - public SlidingWindow( - String name, - long duration, - int maxCount, - Serializer keySerializer, - Serializer valueSerializer, - Deserializer keyDeseriaizer, - Deserializer valueDeserializer) { - this.name = name; - this.duration = duration; - this.maxCount = maxCount; - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - this.keyDeserializer = keyDeseriaizer; - this.valueDeserializer = valueDeserializer; - } - - @Override - public String name() { - return name; - } - - @Override - public WindowInstance build() { - return new SlidingWindowInstance(); - } - - public class SlidingWindowInstance extends WindowSupport implements WindowInstance { - private final Object lock = new Object(); - private ProcessorContext context; - private int slotNum; // used as a key for Kafka log compaction - private LinkedList list = new LinkedList(); - private HashMap> map = new HashMap<>(); - - @Override - public void init(ProcessorContext context) { - this.context = context; - RestoreFuncImpl restoreFunc = new RestoreFuncImpl(); - context.register(this, restoreFunc); - - for (ValueList valueList : map.values()) { - valueList.clearDirtyValues(); - } - this.slotNum = restoreFunc.slotNum; - } - - @Override - public Iterator findAfter(K key, final long timestamp) { - return find(key, timestamp, timestamp + duration); - } - - @Override - public Iterator findBefore(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp); - } - - @Override - public Iterator find(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp + duration); - } - - /* - * finds items in the window between startTime and endTime (both inclusive) - */ - private Iterator find(K key, final long startTime, final long endTime) { - final ValueList values = map.get(key); - - if (values == null) { - return null; - } else { - return new FilteredIterator>(values.iterator()) { - @Override - protected V filter(Value item) { - if (startTime <= item.timestamp && item.timestamp <= endTime) - return item.value; - else - return null; - } - }; - } - } - - @Override - public void put(K key, V value, long timestamp) { - synchronized (lock) { - slotNum++; - - list.offerLast(key); - - ValueList values = map.get(key); - if (values == null) { - values = new ValueList<>(); - map.put(key, values); - } - - values.add(slotNum, value, timestamp); - } - evictExcess(); - evictExpired(timestamp - duration); - } - - private void evictExcess() { - while (list.size() > maxCount) { - K oldestKey = list.pollFirst(); - - ValueList values = map.get(oldestKey); - values.removeFirst(); - - if (values.isEmpty()) map.remove(oldestKey); - } - } - - private void evictExpired(long cutoffTime) { - while (true) { - K oldestKey = list.peekFirst(); - - ValueList values = map.get(oldestKey); - Stamped oldestValue = values.first(); - - if (oldestValue.timestamp < cutoffTime) { - list.pollFirst(); - values.removeFirst(); - - if (values.isEmpty()) map.remove(oldestKey); - } else { - break; - } - } - } - - @Override - public String name() { - return name; - } - - @Override - public void flush() { - IntegerSerializer intSerializer = new IntegerSerializer(); - ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); - - RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); - - for (Map.Entry> entry : map.entrySet()) { - ValueList values = entry.getValue(); - if (values.hasDirtyValues()) { - K key = entry.getKey(); - - byte[] keyBytes = keySerializer.serialize(name, key); - - Iterator> iterator = values.dirtyValueIterator(); - while (iterator.hasNext()) { - Value dirtyValue = iterator.next(); - byte[] slot = intSerializer.serialize("", dirtyValue.slotNum); - byte[] valBytes = valueSerializer.serialize(name, dirtyValue.value); - - byte[] combined = new byte[8 + 4 + keyBytes.length + 4 + valBytes.length]; - - int offset = 0; - offset += putLong(combined, offset, dirtyValue.timestamp); - offset += puts(combined, offset, keyBytes); - offset += puts(combined, offset, valBytes); - - if (offset != combined.length) - throw new IllegalStateException("serialized length does not match"); - - collector.send(new ProducerRecord<>(name, context.id(), slot, combined), byteArraySerializer, byteArraySerializer); - } - values.clearDirtyValues(); - } - } - } - - @Override - public void close() { - // TODO - } - - @Override - public boolean persistent() { - // TODO: should not be persistent, right? - return false; - } - - private class RestoreFuncImpl implements RestoreFunc { - - final IntegerDeserializer intDeserializer; - int slotNum = 0; - - RestoreFuncImpl() { - intDeserializer = new IntegerDeserializer(); - } - - @Override - public void apply(byte[] slot, byte[] bytes) { - - slotNum = intDeserializer.deserialize("", slot); - - int offset = 0; - // timestamp - long timestamp = getLong(bytes, offset); - offset += 8; - // key - int length = getInt(bytes, offset); - offset += 4; - K key = deserialize(bytes, offset, length, name, keyDeserializer); - offset += length; - // value - length = getInt(bytes, offset); - offset += 4; - V value = deserialize(bytes, offset, length, name, valueDeserializer); - - put(key, value, timestamp); - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java index 776171242f7fb..6ad75528ebf15 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java @@ -21,5 +21,5 @@ public interface WindowDef { String name(); - Window build(); + Window instance(); } \ No newline at end of file diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index bb293f9b87efa..4cf99cf6ed842 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.kstream.KeyValueFlatMap; import org.apache.kafka.streaming.processor.ProcessorDef; import org.apache.kafka.streaming.processor.TopologyBuilder; import org.apache.kafka.streaming.kstream.KeyValueFlatMap; From 78656335add3880d1345ba6a3efe20ef74036e00 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 27 Aug 2015 15:38:45 -0700 Subject: [PATCH 185/275] kstream test fix --- .../kstream/internals/KStreamSource.java | 35 ------ .../internals/ProcessorContextImpl.java | 4 - .../kstream/internals/KStreamBranchTest.java | 20 ++-- .../kstream/internals/KStreamFilterTest.java | 29 +++-- .../kstream/internals/KStreamFlatMapTest.java | 32 +++--- .../internals/KStreamFlatMapValuesTest.java | 18 +-- .../kstream/internals/KStreamJoinTest.java | 71 ++++++------ .../kstream/internals/KStreamMapTest.java | 18 +-- .../internals/KStreamMapValuesTest.java | 14 +-- .../internals/KStreamWindowedTest.java | 28 ++--- .../apache/kafka/test/KStreamTestDriver.java | 55 +++++++++ .../apache/kafka/test/MockKStreamBuilder.java | 26 ----- .../kafka/test/MockProcessorContext.java | 43 +++++++- ...ckProcessor.java => MockProcessorDef.java} | 37 +++++-- .../apache/kafka/test/UnlimitedWindow.java | 86 --------------- .../apache/kafka/test/UnlimitedWindowDef.java | 104 ++++++++++++++++++ temp | 10 +- temp2 | 10 +- 18 files changed, 361 insertions(+), 279 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java create mode 100644 stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java delete mode 100644 stream/src/test/java/org/apache/kafka/test/MockKStreamBuilder.java rename stream/src/test/java/org/apache/kafka/test/{MockProcessor.java => MockProcessorDef.java} (58%) delete mode 100644 stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java create mode 100644 stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java deleted file mode 100644 index e292ab4ac74a8..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamSource.java +++ /dev/null @@ -1,35 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.kstream.internals; - -import org.apache.kafka.streaming.processor.internals.SourceNode; -import org.apache.kafka.streaming.processor.TopologyBuilder; - -public class KStreamSource extends KStreamImpl { - - private SourceNode source; - - public KStreamSource(TopologyBuilder topology, SourceNode source) { - super(topology, source.name()); - this.source = source; - } - - public SourceNode source() { - return source; - } -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index 03df3e3cf5a57..a4954b408f806 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -217,11 +217,7 @@ public void forward(K key, V value) { @Override @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { -<<<<<<< HEAD ProcessorNode childNode = (ProcessorNode) task.node().children().get(childIndex); -======= - ProcessorNode childNode = (ProcessorNode) task.node().children().get(childIndex); ->>>>>>> kstream refactored task.node(childNode); childNode.process(key, value); } diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java index 217a65f79afde..89c0278f79e1a 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java @@ -22,9 +22,8 @@ import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KStreamBuilder; import org.apache.kafka.streaming.kstream.Predicate; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; import org.junit.Test; import java.lang.reflect.Array; @@ -33,15 +32,15 @@ public class KStreamBranchTest { - private String topic1 = "topic"; + private String topicName = "topic"; - private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); @SuppressWarnings("unchecked") @Test public void testKStreamBranch() { + KStreamBuilder builder = new KStreamBuilder(); Predicate isEven = new Predicate() { @Override @@ -66,21 +65,22 @@ public boolean apply(Integer key, String value) { KStream stream; KStream[] branches; - MockProcessor[] processors; + MockProcessorDef[] processors; - stream = topology.from(keyDeserializer, valDeserializer, topic1); + stream = builder.from(keyDeserializer, valDeserializer, topicName); branches = stream.branch(isEven, isMultipleOfThree, isOdd); assertEquals(3, branches.length); - processors = (MockProcessor[]) Array.newInstance(MockProcessor.class, branches.length); + processors = (MockProcessorDef[]) Array.newInstance(MockProcessorDef.class, branches.length); for (int i = 0; i < branches.length; i++) { - processors[i] = new MockProcessor<>(); + processors[i] = new MockProcessorDef<>(); branches[i].process(processors[i]); } + KStreamTestDriver driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(3, processors[0].processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java index 301a5a7fbe308..312eab4f908c8 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java @@ -22,9 +22,8 @@ import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KStreamBuilder; import org.apache.kafka.streaming.kstream.Predicate; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; import org.junit.Test; @@ -34,7 +33,6 @@ public class KStreamFilterTest { private String topicName = "topic"; - private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); @@ -47,6 +45,7 @@ public boolean apply(Integer key, String value) { @Test public void testFilter() { + KStreamBuilder builder = new KStreamBuilder(); final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; <<<<<<< HEAD @@ -71,16 +70,21 @@ public void testFilter() { KStreamSource stream; ======= KStream stream; +<<<<<<< HEAD >>>>>>> wip MockProcessor processor; >>>>>>> compile and test passed +======= + MockProcessorDef processor; +>>>>>>> kstream test fix - processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); + 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++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(2, processor.processed.size()); @@ -88,14 +92,16 @@ public void testFilter() { @Test public void testFilterOut() { + KStreamBuilder builder = new KStreamBuilder(); final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; KStream stream; - MockProcessor processor; + MockProcessorDef processor; <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD +<<<<<<< HEAD <<<<<<< HEAD KStreamTopology initializer = new MockKStreamTopology(); ======= @@ -112,11 +118,16 @@ public void testFilterOut() { ======= processor = new MockProcessor<>(); stream = topology.from(keyDeserializer, valDeserializer, topicName); +======= + processor = new MockProcessorDef<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); +>>>>>>> kstream test fix stream.filterOut(isMultipleOfThree).process(processor); >>>>>>> compile and test passed + KStreamTestDriver driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(5, processor.processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java index 4884fee7f2617..9ea49833baa28 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java @@ -22,10 +22,9 @@ import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KStreamBuilder; import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.streaming.kstream.KeyValueFlatMap; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -36,22 +35,22 @@ public class KStreamFlatMapTest { private String topicName = "topic"; - private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testFlatMap() { + KStreamBuilder builder = new KStreamBuilder(); - KeyValueMapper> mapper = - new KeyValueMapper>() { + KeyValueFlatMap mapper = + new KeyValueFlatMap() { @Override - public KeyValue> apply(Integer key, String value) { - ArrayList result = new ArrayList(); + public Iterable> apply(Integer key, String value) { + ArrayList> result = new ArrayList<>(); for (int i = 0; i < key; i++) { - result.add(value); + result.add(KeyValue.pair(Integer.toString(key), value)); } - return KeyValue.pair(Integer.toString(key * 10), (Iterable) result); + return result; } }; @@ -98,16 +97,21 @@ public KeyValue> apply(Integer key, String value) { KStreamSource stream; ======= KStream stream; +<<<<<<< HEAD >>>>>>> wip MockProcessor processor; +======= + MockProcessorDef processor; +>>>>>>> kstream test fix - processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); + processor = new MockProcessorDef<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); stream.flatMap(mapper).process(processor); >>>>>>> compile and test passed + KStreamTestDriver driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(6, processor.processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java index 48b2733cf15fd..9da50917d798e 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java @@ -22,9 +22,8 @@ import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KStreamBuilder; import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; import org.junit.Test; import java.util.ArrayList; @@ -35,12 +34,12 @@ public class KStreamFlatMapValuesTest { private String topicName = "topic"; - private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testFlatMapValues() { + KStreamBuilder builder = new KStreamBuilder(); ValueMapper> mapper = new ValueMapper>() { @@ -77,16 +76,21 @@ public Iterable apply(String value) { KStreamSource stream; ======= KStream stream; +<<<<<<< HEAD >>>>>>> wip MockProcessor processor; >>>>>>> compile and test passed +======= + MockProcessorDef processor; +>>>>>>> kstream test fix - processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); + 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++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(8, processor.processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java index 472dfd44eb7f0..5752c353ea505 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java @@ -27,11 +27,9 @@ import org.apache.kafka.streaming.kstream.KeyValueMapper; import org.apache.kafka.streaming.kstream.ValueJoiner; import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorContext; -import org.apache.kafka.test.UnlimitedWindow; +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; @@ -41,7 +39,6 @@ public class KStreamJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); @@ -85,6 +82,7 @@ public KeyValue> apply(Integer key, String value) { @Test public void testJoin() { + KStreamBuilder builder = new KStreamBuilder(); final int[] expectedKeys = new int[]{0, 1, 2, 3}; @@ -92,25 +90,24 @@ public void testJoin() { KStream stream2; KStreamWindowed windowed1; KStreamWindowed windowed2; - MockProcessor processor; + MockProcessorDef processor; String[] expected; - processor = new MockProcessor<>(); - stream1 = topology.from(keyDeserializer, valDeserializer, topic1); - stream2 = topology.from(keyDeserializer, valDeserializer, topic2); - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); + processor = new MockProcessorDef<>(); + stream1 = builder.from(keyDeserializer, valDeserializer, topic1); + stream2 = builder.from(keyDeserializer, valDeserializer, topic2); + windowed1 = stream1.with(new UnlimitedWindowDef("window1")); + windowed2 = stream2.with(new UnlimitedWindowDef("window2")); windowed1.join(windowed2, joiner).process(processor); - MockProcessorContext context = new MockProcessorContext(null, null); - topology.init(context); - context.setTime(0L); + 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++) { - ((KStreamSource) stream1).source().process(expectedKeys[i], "X" + expectedKeys[i]); + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } assertEquals(0, processor.processed.size()); @@ -118,7 +115,7 @@ public void testJoin() { // push two items to the other stream. the main stream's window has two items for (int i = 0; i < 2; i++) { - ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]); + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } assertEquals(2, processor.processed.size()); @@ -134,7 +131,7 @@ public void testJoin() { // push all items to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream1).source().process(expectedKeys[i], "X" + expectedKeys[i]); + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } assertEquals(2, processor.processed.size()); @@ -151,7 +148,7 @@ public void testJoin() { // push all items to the other stream. this should produce 6 items for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]); + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } assertEquals(6, processor.processed.size()); @@ -165,6 +162,7 @@ public void testJoin() { @Test public void testJoinPrior() { + KStreamBuilder builder = new KStreamBuilder(); final int[] expectedKeys = new int[]{0, 1, 2, 3}; @@ -172,26 +170,24 @@ public void testJoinPrior() { KStream stream2; KStreamWindowed windowed1; KStreamWindowed windowed2; - MockProcessor processor; + MockProcessorDef processor; String[] expected; - processor = new MockProcessor<>(); - stream1 = topology.from(keyDeserializer, valDeserializer, topic1); - stream2 = topology.from(keyDeserializer, valDeserializer, topic2); - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); + processor = new MockProcessorDef<>(); + stream1 = builder.from(keyDeserializer, valDeserializer, topic1); + stream2 = builder.from(keyDeserializer, valDeserializer, topic2); + windowed1 = stream1.with(new UnlimitedWindowDef("window1")); + windowed2 = stream2.with(new UnlimitedWindowDef("window2")); windowed1.joinPrior(windowed2, joiner).process(processor); - MockProcessorContext context = new MockProcessorContext(null, null); - topology.init(context); + KStreamTestDriver driver = new KStreamTestDriver(builder); // push two items to the main stream. the other stream's window is empty for (int i = 0; i < 2; i++) { - context.setTime(i); - - ((KStreamSource) stream1).source().process(expectedKeys[i], "X" + expectedKeys[i]); + driver.setTime(i); + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } assertEquals(0, processor.processed.size()); @@ -200,9 +196,8 @@ public void testJoinPrior() { // no corresponding item in the main window has a newer timestamp for (int i = 0; i < 2; i++) { - context.setTime(i + 1); - - ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]); + driver.setTime(i + 1); + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } assertEquals(0, processor.processed.size()); @@ -212,9 +207,8 @@ public void testJoinPrior() { // push all items with newer timestamps to the main stream. this should produce two items. for (int i = 0; i < expectedKeys.length; i++) { - context.setTime(i + 2); - - ((KStreamSource) stream1).source().process(expectedKeys[i], "X" + expectedKeys[i]); + driver.setTime(i + 2); + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } assertEquals(2, processor.processed.size()); @@ -231,9 +225,8 @@ public void testJoinPrior() { // push all items with older timestamps to the other stream. this should produce six items for (int i = 0; i < expectedKeys.length; i++) { - context.setTime(i); - - ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]); + driver.setTime(i); + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } assertEquals(6, processor.processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java index 73e9852b3b975..f8f89ae676f93 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java @@ -23,9 +23,8 @@ import org.apache.kafka.streaming.kstream.KStreamBuilder; import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -34,12 +33,12 @@ public class KStreamMapTest { private String topicName = "topic"; - private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testMap() { + KStreamBuilder builder = new KStreamBuilder(); KeyValueMapper mapper = new KeyValueMapper() { @@ -73,16 +72,21 @@ public KeyValue apply(Integer key, String value) { KStreamSource stream; ======= KStream stream; +<<<<<<< HEAD >>>>>>> wip MockProcessor processor; >>>>>>> compile and test passed +======= + MockProcessorDef processor; +>>>>>>> kstream test fix - processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); + 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++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(4, processor.processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.java index 45a793fc7ab9f..f48a03244cf17 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.java @@ -22,9 +22,8 @@ import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KStreamBuilder; import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -33,12 +32,12 @@ public class KStreamMapValuesTest { private String topicName = "topic"; - private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testFlatMapValues() { + KStreamBuilder builder = new KStreamBuilder(); ValueMapper mapper = new ValueMapper() { @@ -51,12 +50,13 @@ public Integer apply(String value) { final int[] expectedKeys = new int[]{1, 10, 100, 1000}; KStream stream; - MockProcessor processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); + MockProcessorDef processor = new MockProcessorDef<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); stream.mapValues(mapper).process(processor); + KStreamTestDriver driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], Integer.toString(expectedKeys[i])); + driver.process(topicName, expectedKeys[i], Integer.toString(expectedKeys[i])); } assertEquals(4, processor.processed.size()); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.java index 20cc445e814b4..dfa8f3fba3c50 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.java @@ -21,11 +21,10 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.kstream.Window; import org.apache.kafka.streaming.kstream.WindowDef; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessorContext; -import org.apache.kafka.test.UnlimitedWindow; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.UnlimitedWindowDef; import org.junit.Test; import java.util.Iterator; @@ -35,31 +34,32 @@ public class KStreamWindowedTest { private String topicName = "topic"; + private String windowName = "MyWindow"; - private KStreamBuilder topology = new MockKStreamBuilder(); private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); private StringDeserializer valDeserializer = new StringDeserializer(); @Test public void testWindowedStream() { + KStreamBuilder builder = new KStreamBuilder(); final int[] expectedKeys = new int[]{0, 1, 2, 3}; KStream stream; - WindowDef window; + WindowDef windowDef; - window = new UnlimitedWindow<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); - stream.with(window); + windowDef = new UnlimitedWindowDef<>(windowName); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.with(windowDef); - MockProcessorContext context = new MockProcessorContext(null, null); - topology.init(context); - context.setTime(0L); + KStreamTestDriver driver = new KStreamTestDriver(builder); + Window window = (Window) driver.getStateStore(windowName); + driver.setTime(0L); // two items in the window for (int i = 0; i < 2; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } assertEquals(1, countItem(window.find(0, 0L))); @@ -70,7 +70,7 @@ public void testWindowedStream() { // previous two items + all items, thus two are duplicates, in the window for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "Y" + expectedKeys[i]); + driver.process(topicName, expectedKeys[i], "Y" + expectedKeys[i]); } assertEquals(2, countItem(window.find(0, 0L))); diff --git a/stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java new file mode 100644 index 0000000000000..e026fcc67bd50 --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -0,0 +1,55 @@ +/** + * 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.streaming.kstream.KStreamBuilder; +import org.apache.kafka.streaming.processor.StateStore; +import org.apache.kafka.streaming.processor.internals.ProcessorTopology; + +public class KStreamTestDriver { + + private final ProcessorTopology topology; + private final MockProcessorContext context; + + public KStreamTestDriver(KStreamBuilder builder) { + this(builder, null, null); + } + + public KStreamTestDriver(KStreamBuilder builder, Serializer serializer, Deserializer deserializer) { + this.topology = builder.build(); + this.context = new MockProcessorContext(serializer, deserializer); + + this.topology.init(context); + } + + public void process(String topicName, Object key, Object value) { + context.node(topology.source(topicName)); + context.forward(key, value); + } + + public void setTime(long timestamp) { + context.setTime(timestamp); + } + + public StateStore getStateStore(String name) { + return context.getStateStore(name); + } + +} diff --git a/stream/src/test/java/org/apache/kafka/test/MockKStreamBuilder.java b/stream/src/test/java/org/apache/kafka/test/MockKStreamBuilder.java deleted file mode 100644 index aa3de2b7a050b..0000000000000 --- a/stream/src/test/java/org/apache/kafka/test/MockKStreamBuilder.java +++ /dev/null @@ -1,26 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.test; - -import org.apache.kafka.streaming.kstream.KStreamBuilder; - -public class MockKStreamBuilder extends KStreamBuilder { - public MockKStreamBuilder() { - super(); - } -} diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 3edae79e66092..ca928f9faaaae 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -92,11 +92,16 @@ 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.streaming.processor.internals.ProcessorNode; import java.io.File; +import java.util.HashMap; +import java.util.List; +import java.util.Map; public class MockProcessorContext implements ProcessorContext { +<<<<<<< HEAD <<<<<<< HEAD Serializer serializer; Deserializer deserializer; @@ -197,6 +202,13 @@ public void schedule(Processor processor, long interval) { ======= Serializer serializer; Deserializer deserializer; +======= + private Serializer serializer; + private Deserializer deserializer; + private ProcessorNode node; + + private Map storeMap = new HashMap<>(); +>>>>>>> kstream test fix long timestamp = -1L; @@ -210,7 +222,7 @@ public void setTime(long timestamp) { } @Override - public boolean joinable(ProcessorContext other) { + public boolean joinable() { // TODO return true; } @@ -252,7 +264,13 @@ public Metrics metrics() { @Override public void register(StateStore store, RestoreFunc func) { - throw new UnsupportedOperationException("restore() not supported."); + if (func != null) new UnsupportedOperationException("RestoreFunc not supported."); + storeMap.put(store.name(), store); + } + + @Override + public StateStore getStateStore(String name) { + return storeMap.get(name); } @Override @@ -261,8 +279,20 @@ public void schedule(Processor processor, long interval) { } @Override + @SuppressWarnings("unchecked") public void forward(K key, V value) { - throw new UnsupportedOperationException("forward() not supported"); + for (ProcessorNode childNode : (List>) node().children()) { + node(childNode); + childNode.process(key, value); + } + } + + @Override + @SuppressWarnings("unchecked") + public void forward(K key, V value, int childIndex) { + ProcessorNode childNode = (ProcessorNode) node().children().get(childIndex); + node(childNode); + childNode.process(key, value); } @Override @@ -291,4 +321,11 @@ public long timestamp() { } >>>>>>> compile and test passed + public void node(ProcessorNode node) { + this.node = node; + } + + public ProcessorNode node() { + return this.node; + } } diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorDef.java similarity index 58% rename from stream/src/test/java/org/apache/kafka/test/MockProcessor.java rename to stream/src/test/java/org/apache/kafka/test/MockProcessorDef.java index 81dbf49a5fe87..472f91918a48c 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessor.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorDef.java @@ -18,24 +18,41 @@ package org.apache.kafka.test; import org.apache.kafka.streaming.processor.Processor; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.processor.ProcessorDef; import java.util.ArrayList; -public class MockProcessor extends Processor { +public class MockProcessorDef implements ProcessorDef { + public final ArrayList processed = new ArrayList<>(); public final ArrayList punctuated = new ArrayList<>(); - public MockProcessor() { - super("MOCK"); + public Processor instance() { + return new MockProcessor(); } - @Override - public void process(K1 key, V1 value) { - processed.add(key + ":" + value); - } + public class MockProcessor implements Processor { + + @Override + public void init(ProcessorContext context) { + // do nothing + } + + @Override + public void process(K key, V value) { + processed.add(key + ":" + value); + } + + @Override + public void punctuate(long streamTime) { + punctuated.add(streamTime); + } + + @Override + public void close() { + // do nothing + } - @Override - public void punctuate(long streamTime) { - punctuated.add(streamTime); } } diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java deleted file mode 100644 index 2dc4c6cbed287..0000000000000 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindow.java +++ /dev/null @@ -1,86 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.test; - -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.WindowDef; -import org.apache.kafka.streaming.kstream.internals.FilteredIterator; -import org.apache.kafka.streaming.processor.internals.Stamped; - -import java.util.Iterator; -import java.util.LinkedList; - -public class UnlimitedWindow implements WindowDef { - - private LinkedList>> list = new LinkedList<>(); - - @Override - public void init(ProcessorContext context) { - // do nothing - } - - @Override - public Iterator find(final K key, long timestamp) { - return find(key, Long.MIN_VALUE, timestamp); - } - - @Override - public Iterator findAfter(final K key, long timestamp) { - return find(key, timestamp, Long.MAX_VALUE); - } - - @Override - public Iterator findBefore(final K key, long timestamp) { - return find(key, Long.MIN_VALUE, Long.MAX_VALUE); - } - - private Iterator find(final K key, final long startTime, final long endTime) { - return new FilteredIterator>>(list.iterator()) { - protected V filter(Stamped> item) { - if (item.value.key.equals(key) && startTime <= item.timestamp && item.timestamp <= endTime) - return item.value.value; - else - return null; - } - }; - } - - @Override - public void put(K key, V value, long timestamp) { - list.add(new Stamped<>(KeyValue.pair(key, value), timestamp)); - } - - @Override - public String name() { - return null; - } - - @Override - public void flush() { - } - - @Override - public void close() { - } - - @Override - public boolean persistent() { - return false; - } -} diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java new file mode 100644 index 0000000000000..c6d929725c65f --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.test; + +import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.kstream.KeyValue; +import org.apache.kafka.streaming.kstream.WindowDef; +import org.apache.kafka.streaming.kstream.internals.FilteredIterator; +import org.apache.kafka.streaming.processor.internals.Stamped; + +import java.util.Iterator; +import java.util.LinkedList; + +public class UnlimitedWindowDef implements WindowDef { + + private final String name; + + public UnlimitedWindowDef(String name) { + this.name = name; + } + + public String name() { + return name; + } + + public Window instance() { + return new UnlimitedWindow(); + } + + public class UnlimitedWindow implements Window { + + private LinkedList>> list = new LinkedList<>(); + + @Override + public void init (ProcessorContext context){ + context.register(this, null); + } + + @Override + public Iterator find(final K key, long timestamp){ + return find(key, Long.MIN_VALUE, timestamp); + } + + @Override + public Iterator findAfter(final K key, long timestamp){ + return find(key, timestamp, Long.MAX_VALUE); + } + + @Override + public Iterator findBefore(final K key, long timestamp){ + return find(key, Long.MIN_VALUE, Long.MAX_VALUE); + } + + private Iterator find(final K key, final long startTime, final long endTime){ + return new FilteredIterator>>(list.iterator()) { + protected V filter(Stamped> item) { + if (item.value.key.equals(key) && startTime <= item.timestamp && item.timestamp <= endTime) + return item.value.value; + else + return null; + } + }; + } + + @Override + public void put (K key, V value,long timestamp){ + list.add(new Stamped<>(KeyValue.pair(key, value), timestamp)); + } + + @Override + public String name () { + return null; + } + + @Override + public void flush () { + } + + @Override + public void close () { + } + + @Override + public boolean persistent () { + return false; + } + } +} diff --git a/temp b/temp index 0c9e971a95412..a5d7e05387188 100644 --- a/temp +++ b/temp @@ -1,4 +1,6 @@ -stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java: needs merge -stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java: needs merge -stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java: needs merge -stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java: needs merge +stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java: needs merge +stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java: needs merge +stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java: needs merge +stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java: needs merge +stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java: needs merge +stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java: needs merge diff --git a/temp2 b/temp2 index 8a742c01794a1..9fb64018d3abe 100644 --- a/temp2 +++ b/temp2 @@ -1,4 +1,6 @@ -stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java -stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java -stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java -stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java +stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java +stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java +stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java +stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java +stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java +stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java From 33febf9c484727083cf27b75ce0bde81066dadf4 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 27 Aug 2015 16:38:54 -0700 Subject: [PATCH 186/275] fix StreamTaskTest --- .../kafka/clients/consumer/MockConsumer.java | 19 +- .../kafka/streaming/StreamingConfig.java | 12 +- .../streaming/examples/ProcessorJob.java | 101 ++++++++++ .../streaming/examples/SimpleProcessJob.java | 69 ------- .../examples/StatefulProcessJob.java | 90 --------- .../kstream/internals/KStreamImpl.java | 2 - .../kstream/internals/KStreamJoin.java | 37 +++- .../streaming/processor/ProcessorContext.java | 3 - .../internals/ProcessorContextImpl.java | 30 +-- .../processor/internals/StreamTask.java | 2 +- .../processor/internals/StreamTaskTest.java | 184 ++++++++++++++++++ .../kafka/test/MockProcessorContext.java | 7 - .../org/apache/kafka/test/MockSourceNode.java | 16 +- 13 files changed, 353 insertions(+), 219 deletions(-) create mode 100644 stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java create mode 100644 stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index e33f120923046..2b17fa681e217 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -22,6 +22,7 @@ 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.Map.Entry; @@ -39,12 +40,14 @@ public class MockConsumer implements Consumer { private final Map> partitions; private final SubscriptionState subscriptions; private Map>> records; + private Set paused; private boolean closed; public MockConsumer(OffsetResetStrategy offsetResetStrategy) { this.subscriptions = new SubscriptionState(offsetResetStrategy); - this.partitions = new HashMap>(); - this.records = new HashMap>>(); + this.partitions = new HashMap<>(); + this.records = new HashMap<>(); + this.paused = new HashSet<>(); this.closed = false; } @@ -194,14 +197,18 @@ public synchronized void updatePartitions(String topic, List part @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 @@ -215,6 +222,10 @@ public void wakeup() { } + public Set paused() { + return paused; + } + private void ensureNotClosed() { if (this.closed) throw new IllegalStateException("This consumer has already been closed."); diff --git a/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java b/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java index 5b59197388dc3..773a2e3c2b35a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java +++ b/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java @@ -100,12 +100,12 @@ public class StreamingConfig extends AbstractConfig { Type.LONG, 100, Importance.LOW, - POLL_MS_DOC) + POLL_MS_DOC) .define(NUM_STREAM_THREADS_CONFIG, - Type.INT, - 1, - Importance.LOW, - NUM_STREAM_THREADS_DOC) + Type.INT, + 1, + Importance.LOW, + NUM_STREAM_THREADS_DOC) .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, Type.INT, 1000, @@ -115,7 +115,7 @@ public class StreamingConfig extends AbstractConfig { Type.LONG, 60000, Importance.LOW, - STATE_CLEANUP_DELAY_MS_DOC) + STATE_CLEANUP_DELAY_MS_DOC) .define(TOTAL_RECORDS_TO_PROCESS, Type.LONG, -1L, diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java new file mode 100644 index 0000000000000..c7defec9115db --- /dev/null +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java @@ -0,0 +1,101 @@ +/** + * 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.streaming.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.streaming.KafkaStreaming; +import org.apache.kafka.streaming.processor.Processor; +import org.apache.kafka.streaming.processor.ProcessorDef; +import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streaming.StreamingConfig; +import org.apache.kafka.streaming.state.Entry; +import org.apache.kafka.streaming.state.InMemoryKeyValueStore; +import org.apache.kafka.streaming.state.KeyValueIterator; +import org.apache.kafka.streaming.state.KeyValueStore; + +import java.util.Properties; + +public class ProcessorJob { + + private static class MyProcessorDef implements ProcessorDef { + + @Override + public Processor instance() { + return new Processor() { + private ProcessorContext context; + private KeyValueStore kvStore; + + @Override + public void init(ProcessorContext context) { + this.context = context; + this.context.schedule(this, 1000); + this.kvStore = new InMemoryKeyValueStore<>("local-state", context); + } + + @Override + public void process(String key, Integer value) { + Integer oldValue = this.kvStore.get(key); + if (oldValue == null) { + this.kvStore.put(key, value); + } else { + int newValue = oldValue + value; + this.kvStore.put(key, newValue); + } + + context.commit(); + } + + @Override + public void punctuate(long streamTime) { + KeyValueIterator iter = this.kvStore.all(); + + while (iter.hasNext()) { + Entry entry = iter.next(); + + System.out.println("[" + entry.key() + ", " + entry.value() + "]"); + + context.forward(entry.key(), entry.value()); + } + } + + @Override + public void close() { + this.kvStore.close(); + } + }; + } + } + + public static void main(String[] args) throws Exception { + StreamingConfig config = new StreamingConfig(new Properties()); + TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); + + builder.addProcessor("PROCESS", new MyProcessorDef(), null, "SOURCE"); + + builder.addSink("SINK", "topic-sink", new StringSerializer(), new IntegerSerializer(), "PROCESS"); + + KafkaStreaming streaming = new KafkaStreaming(builder, config); + streaming.run(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java deleted file mode 100644 index 22d6425e7aadf..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/SimpleProcessJob.java +++ /dev/null @@ -1,69 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.examples; - -import org.apache.kafka.streaming.KafkaStreaming; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.StreamingConfig; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; - -import java.util.Properties; - -public class SimpleProcessJob { - - private static class MyProcessor extends Processor { - private ProcessorContext context; - - public MyProcessor(String name) { - super(name); - } - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void process(String key, Integer value) { - System.out.println("[" + key + ", " + value + "]"); - - context.commit(); - - context.send("topic-dest", key, value); - } - - @Override - public void close() { - // do nothing - } - } - - public static void main(String[] args) throws Exception { - StreamingConfig config = new StreamingConfig(new Properties()); - TopologyBuilder builder = new TopologyBuilder(); - - builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); - builder.addProcessor("PROCESS", MyProcessor.class, null, "SOURCE"); - - KafkaStreaming streaming = new KafkaStreaming(builder, config); - streaming.run(); - } -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java deleted file mode 100644 index 6a01551b4972a..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/StatefulProcessJob.java +++ /dev/null @@ -1,90 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.examples; - -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.KafkaStreaming; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.StreamingConfig; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.state.Entry; -import org.apache.kafka.streaming.state.InMemoryKeyValueStore; -import org.apache.kafka.streaming.state.KeyValueIterator; -import org.apache.kafka.streaming.state.KeyValueStore; - -import java.util.Properties; - -public class StatefulProcessJob { - - private static class MyProcessor extends Processor { - private ProcessorContext context; - private KeyValueStore kvStore; - - public MyProcessor(String name) { - super(name); - } - - @Override - public void init(ProcessorContext context) { - this.context = context; - this.context.schedule(this, 1000); - - this.kvStore = new InMemoryKeyValueStore<>("local-state", context); - } - - @Override - public void process(String key, Integer value) { - Integer oldValue = this.kvStore.get(key); - if (oldValue == null) { - this.kvStore.put(key, value); - } else { - int newValue = oldValue + value; - this.kvStore.put(key, newValue); - } - - context.commit(); - } - - @Override - public void punctuate(long streamTime) { - KeyValueIterator iter = this.kvStore.all(); - while (iter.hasNext()) { - Entry entry = iter.next(); - System.out.println("[" + entry.key() + ", " + entry.value() + "]"); - } - } - - @Override - public void close() { - this.kvStore.close(); - } - } - - public static void main(String[] args) throws Exception { - StreamingConfig config = new StreamingConfig(new Properties()); - TopologyBuilder builder = new TopologyBuilder(); - - builder.addSource("SOURCE", new StringDeserializer(), new IntegerDeserializer(), "topic-source"); - builder.addProcessor("PROCESS", MyProcessor.class, null, "SOURCE"); - - KafkaStreaming streaming = new KafkaStreaming(builder, config); - streaming.run(); - } -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index 4cf99cf6ed842..bf7456e5236df 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -61,8 +61,6 @@ public class KStreamImpl implements KStream { public static final String SEND_NAME = "KAFKA-SEND-"; - public static final String WINDOW_NAME = "KAFKA-WINDOW-"; - public static final AtomicInteger INDEX = new AtomicInteger(1); protected TopologyBuilder topology; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index 2c0020b12afb8..c2451c5a17e73 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -17,13 +17,20 @@ package org.apache.kafka.streaming.kstream.internals; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.ValueJoiner; import org.apache.kafka.streaming.kstream.Window; import org.apache.kafka.streaming.processor.ProcessorDef; +import org.apache.kafka.streaming.processor.internals.ProcessorContextImpl; +import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; class KStreamJoin implements ProcessorDef { @@ -67,7 +74,7 @@ public void init(ProcessorContext context) { super.init(context); // check if these two streams are joinable - if (!context.joinable()) + if (!joinable()) throw new IllegalStateException("Streams are not joinable."); final Window window1 = (Window) context.getStateStore(windowName1); @@ -122,6 +129,34 @@ public void process(K key, V1 value) { } } + private boolean joinable() { + Set partitions = ((ProcessorContextImpl) this.context).task().partitions(); + Map> partitionsById = new HashMap<>(); + int firstId = -1; + for (TopicPartition partition : partitions) { + if (!partitionsById.containsKey(partition.partition())) { + partitionsById.put(partition.partition(), new ArrayList<>()); + } + partitionsById.get(partition.partition()).add(partition.topic()); + + if (firstId < 0) + firstId = partition.partition(); + } + + List topics = partitionsById.get(firstId); + for (List topicsPerPartition : partitionsById.values()) { + if (topics.size() != topicsPerPartition.size()) + return false; + + for (String topic : topicsPerPartition) { + if (!topics.contains(topic)) + return false; + } + } + + return true; + } + // TODO: use the "outer-stream" topic as the resulted join stream topic private void doJoin(K key, V1 value1, V2 value2) { context.forward(key, joiner.apply(value1, value2)); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java index db78fe52e365c..eb0784195f896 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java @@ -25,9 +25,6 @@ public interface ProcessorContext { - // TODO: this is better moved to a KStreamContext - boolean joinable(); - /** * Returns the partition group id * diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index a4954b408f806..240c9ee773bff 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -97,34 +97,7 @@ public RecordCollector recordCollector() { return this.collector; } - @Override - public boolean joinable() { - Set partitions = this.task.partitions(); - Map> partitionsById = new HashMap<>(); - int firstId = -1; - for (TopicPartition partition : partitions) { - if (!partitionsById.containsKey(partition.partition())) { - partitionsById.put(partition.partition(), new ArrayList()); - } - partitionsById.get(partition.partition()).add(partition.topic()); - - if (firstId < 0) - firstId = partition.partition(); - } - - List topics = partitionsById.get(firstId); - for (List topicsPerPartition : partitionsById.values()) { - if (topics.size() != topicsPerPartition.size()) - return false; - - for (String topic : topicsPerPartition) { - if (!topics.contains(topic)) - return false; - } - } - - return true; - } + public StreamTask task() { return this.task; } @Override public int id() { @@ -169,6 +142,7 @@ public void register(StateStore store, RestoreFunc restoreFunc) { stateMgr.register(store, restoreFunc); } + @Override public StateStore getStateStore(String name) { return stateMgr.getStore(name); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java index 7f48c1864c46b..2c06ebf32117b 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -156,7 +156,7 @@ public void addRecords(TopicPartition partition, Iterator this.maxBufferedSize) { consumer.pause(partition); } } diff --git a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java new file mode 100644 index 0000000000000..a14e9dad1ca4c --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java @@ -0,0 +1,184 @@ +/** + * 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.streaming.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.streaming.StreamingConfig; +import org.apache.kafka.test.MockSourceNode; +import org.junit.Test; +import org.junit.Before; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class StreamTaskTest { + + private final Serializer intSerializer = new IntegerSerializer(); + private final Deserializer intDeserializer = new IntegerDeserializer(); + private final Serializer bytesSerializer = new ByteArraySerializer(); + + private final TopicPartition partition1 = new TopicPartition("topic1", 1); + private final TopicPartition partition2 = new TopicPartition("topic2", 1); + private final HashSet partitions = new HashSet<>(Arrays.asList(partition1, partition2)); + + private final MockSourceNode source1 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final MockSourceNode source2 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final ProcessorTopology topology = new ProcessorTopology( + Arrays.asList(source1, source2), + new HashMap(){{ + put("topic1", source1); + put("topic2", source2); + }}, + Collections.emptyMap() + ); + + 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.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + }}); + + private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + private final MockProducer producer = new MockProducer<>(false, bytesSerializer, bytesSerializer); + private final StreamTask task = new StreamTask(0, consumer, producer, partitions, topology, config); + + @Before + public void setup() { + consumer.subscribe(partition1, partition2); + } + + @SuppressWarnings("unchecked") + @Test + public void testProcessOrder() { + byte[] recordValue = intSerializer.serialize(null, 10); + + task.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 1, intSerializer.serialize(partition1.topic(), 10), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 2, intSerializer.serialize(partition1.topic(), 20), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 3, intSerializer.serialize(partition1.topic(), 30), recordValue) + )); + + task.addRecords(partition2, records( + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 1, intSerializer.serialize(partition1.topic(), 25), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 2, intSerializer.serialize(partition1.topic(), 35), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 3, intSerializer.serialize(partition1.topic(), 45), recordValue) + )); + + assertEquals(task.process(), 5); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 0); + + assertEquals(task.process(), 4); + assertEquals(source1.numReceived, 2); + assertEquals(source2.numReceived, 0); + + 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); + } + + @SuppressWarnings("unchecked") + @Test + public void testPauseResume() { + byte[] recordValue = intSerializer.serialize(null, 10); + + task.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 1, intSerializer.serialize(partition1.topic(), 10), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 2, intSerializer.serialize(partition1.topic(), 20), recordValue) + )); + + task.addRecords(partition2, records( + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 3, intSerializer.serialize(partition1.topic(), 35), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 4, intSerializer.serialize(partition1.topic(), 45), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 5, intSerializer.serialize(partition1.topic(), 55), recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 6, intSerializer.serialize(partition1.topic(), 65), 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(), 3, intSerializer.serialize(partition1.topic(), 30), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 4, intSerializer.serialize(partition1.topic(), 40), recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 5, intSerializer.serialize(partition1.topic(), 50), recordValue) + )); + + assertEquals(consumer.paused().size(), 2); + assertTrue(consumer.paused().contains(partition1)); + assertTrue(consumer.paused().contains(partition2)); + + assertEquals(task.process(), 7); + assertEquals(source1.numReceived, 2); + assertEquals(source2.numReceived, 0); + + assertEquals(consumer.paused().size(), 1); + assertTrue(consumer.paused().contains(partition2)); + + assertEquals(task.process(), 6); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 0); + + assertEquals(consumer.paused().size(), 1); + assertTrue(consumer.paused().contains(partition2)); + + assertEquals(task.process(), 5); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 1); + + assertEquals(consumer.paused().size(), 0); + } + + private Iterator> records(ConsumerRecord... recs) { + return Arrays.asList(recs).iterator(); + } +} diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java index ca928f9faaaae..b27aacb27bfbc 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -221,13 +221,6 @@ public void setTime(long timestamp) { this.timestamp = timestamp; } - @Override - public boolean joinable() { - // TODO - return true; - } - - @Override public int id() { return -1; } diff --git a/stream/src/test/java/org/apache/kafka/test/MockSourceNode.java b/stream/src/test/java/org/apache/kafka/test/MockSourceNode.java index cd29a317c4ff9..9b6e4b01b6128 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockSourceNode.java +++ b/stream/src/test/java/org/apache/kafka/test/MockSourceNode.java @@ -17,25 +17,25 @@ package org.apache.kafka.test; -import org.apache.kafka.streaming.processor.internals.SourceNode; + import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streaming.processor.internals.SourceNode; import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicInteger; public class MockSourceNode extends SourceNode { - public Deserializer keyDeserializer; - public Deserializer valDeserializer; + public static final String NAME = "MOCK-SOURCE-"; + + public static final AtomicInteger INDEX = new AtomicInteger(1); public int numReceived = 0; public ArrayList keys = new ArrayList<>(); public ArrayList values = new ArrayList<>(); - public MockSourceNode(Deserializer keyDeserializer, Deserializer valDeserializer) { - super(keyDeserializer, valDeserializer); - - this.keyDeserializer = keyDeserializer; - this.valDeserializer = valDeserializer; + public MockSourceNode(Deserializer keyDeserializer, Deserializer valDeserializer) { + super(NAME + INDEX.getAndIncrement(), keyDeserializer, valDeserializer); } @Override From 98f525a68237d7a47e8b5a096547ed46c719b9d1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 27 Aug 2015 17:11:07 -0700 Subject: [PATCH 187/275] minor fixes, now compiles --- .../kafka/streaming/StreamingConfig.java | 27 +++++++++++------- .../kstream/internals/KStreamJoin.java | 2 +- .../internals/ProcessorContextImpl.java | 7 +---- .../processor/internals/StreamTask.java | 4 +-- .../processor/internals/StreamThread.java | 4 +-- .../processor/internals/StreamTaskTest.java | 20 +++++-------- .../kafka/test/MockTimestampExtractor.java | 28 +++++++++++++++++++ 7 files changed, 58 insertions(+), 34 deletions(-) create mode 100644 stream/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java b/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java index 773a2e3c2b35a..888145412f0a7 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java +++ b/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java @@ -17,6 +17,7 @@ package org.apache.kafka.streaming; +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; @@ -80,8 +81,10 @@ public class StreamingConfig extends AbstractConfig { /** value.deserializer */ public static final String VALUE_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; - - + /** + * bootstrap.servers + */ + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; private static final String SYSTEM_TEMP_DIRECTORY = System.getProperty("java.io.tmpdir"); @@ -145,28 +148,32 @@ public class StreamingConfig extends AbstractConfig { .define(TIMESTAMP_EXTRACTOR_CLASS_CONFIG, Type.CLASS, Importance.HIGH, - TIMESTAMP_EXTRACTOR_CLASS_DOC); + TIMESTAMP_EXTRACTOR_CLASS_DOC) + .define(BOOTSTRAP_SERVERS_CONFIG, + Type.STRING, + Importance.HIGH, + CommonClientConfigs.BOOSTRAP_SERVERS_DOC); } public StreamingConfig(Map props) { super(CONFIG, props); } - public Properties getConsumerProperties() { - Properties props = new Properties(); + public Map getConsumerConfigs() { + Map props = this.originals(); // set consumer default property values - props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - props.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); return props; } - public Properties getProducerProperties() { - Properties props = new Properties(); + public Map getProducerConfigs() { + Map props = this.originals(); // set producer default property values - props.setProperty(ProducerConfig.LINGER_MS_CONFIG, "100"); + props.put(ProducerConfig.LINGER_MS_CONFIG, "100"); return props; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index c2451c5a17e73..733e6d1a39c72 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -135,7 +135,7 @@ private boolean joinable() { int firstId = -1; for (TopicPartition partition : partitions) { if (!partitionsById.containsKey(partition.partition())) { - partitionsById.put(partition.partition(), new ArrayList<>()); + partitionsById.put(partition.partition(), new ArrayList()); } partitionsById.get(partition.partition()).add(partition.topic()); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index 240c9ee773bff..f6f0216e248f6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -20,7 +20,6 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.KafkaConsumer; 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.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; @@ -37,11 +36,7 @@ import java.io.File; import java.io.IOException; -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 { @@ -79,7 +74,7 @@ public ProcessorContextImpl(int id, File stateFile = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG), Integer.toString(id)); Consumer restoreConsumer = new KafkaConsumer<>( - config.getConsumerProperties(), + config.getConsumerConfigs(), null /* no callback for restore consumer */, new ByteArrayDeserializer(), new ByteArrayDeserializer()); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java index 2c06ebf32117b..6a5b868fade10 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -105,8 +105,8 @@ public StreamTask(int id, // create the record recordCollector that maintains the produced offsets this.recordCollector = new RecordCollector(producer, - (Serializer) config.getConfiguredInstance(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, Serializer.class), - (Serializer) config.getConfiguredInstance(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Serializer.class)); + (Serializer) config.getConfiguredInstance(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class), + (Serializer) config.getConfiguredInstance(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class)); // initialize the topology with its own context try { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index 5807fbf9d52a8..791b5cb1c295c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -93,11 +93,11 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce this.builder = builder; // create the producer and consumer clients - this.producer = new KafkaProducer<>(config.getProducerProperties(), + this.producer = new KafkaProducer<>(config.getProducerConfigs(), new ByteArraySerializer(), new ByteArraySerializer()); - this.consumer = new KafkaConsumer<>(config.getConsumerProperties(), + this.consumer = new KafkaConsumer<>(config.getConsumerConfigs(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); diff --git a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java index a14e9dad1ca4c..c69c2841b04c7 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java @@ -69,6 +69,7 @@ public class StreamTaskTest { 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"); }}); @@ -103,8 +104,8 @@ public void testProcessOrder() { assertEquals(source2.numReceived, 0); assertEquals(task.process(), 4); - assertEquals(source1.numReceived, 2); - assertEquals(source2.numReceived, 0); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 1); assertEquals(task.process(), 3); assertEquals(source1.numReceived, 2); @@ -158,21 +159,14 @@ public void testPauseResume() { assertTrue(consumer.paused().contains(partition2)); assertEquals(task.process(), 7); - assertEquals(source1.numReceived, 2); - assertEquals(source2.numReceived, 0); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 1); assertEquals(consumer.paused().size(), 1); - assertTrue(consumer.paused().contains(partition2)); + assertTrue(consumer.paused().contains(partition1)); assertEquals(task.process(), 6); - assertEquals(source1.numReceived, 3); - assertEquals(source2.numReceived, 0); - - assertEquals(consumer.paused().size(), 1); - assertTrue(consumer.paused().contains(partition2)); - - assertEquals(task.process(), 5); - assertEquals(source1.numReceived, 3); + assertEquals(source1.numReceived, 2); assertEquals(source2.numReceived, 1); assertEquals(consumer.paused().size(), 0); diff --git a/stream/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java b/stream/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java new file mode 100644 index 0000000000000..331f01533edbd --- /dev/null +++ b/stream/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.test; + +import org.apache.kafka.streaming.processor.TimestampExtractor; + +public class MockTimestampExtractor implements TimestampExtractor { + + @Override + public long extract(String topic, Object key, Object value) { + return ((Integer) key).longValue(); + } +} From 50c27797aa6bfcee6d972ea606747f91290e753d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 27 Aug 2015 18:46:10 -0700 Subject: [PATCH 188/275] Address comments --- .../streaming/processor/ProcessorContext.java | 5 +++ .../internals/ProcessorContextImpl.java | 34 +++++++++++++++++++ .../kafka/test/MockProcessorContext.java | 5 +++ 3 files changed, 44 insertions(+) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java index eb0784195f896..06e219ae45304 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java @@ -74,6 +74,11 @@ public interface ProcessorContext { */ Metrics metrics(); + /** + * Check if this process's incoming streams are joinable + */ + boolean joinable(); + /** * Registers and possibly restores the specified storage engine. * diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index f6f0216e248f6..de1af645cc452 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.KafkaConsumer; 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.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; @@ -36,7 +37,11 @@ import java.io.File; import java.io.IOException; +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 { @@ -94,6 +99,35 @@ public RecordCollector recordCollector() { public StreamTask task() { return this.task; } + @Override + public boolean joinable() { + Set partitions = this.task.partitions(); + Map> partitionsById = new HashMap<>(); + int firstId = -1; + for (TopicPartition partition : partitions) { + if (!partitionsById.containsKey(partition.partition())) { + partitionsById.put(partition.partition(), new ArrayList()); + } + partitionsById.get(partition.partition()).add(partition.topic()); + + if (firstId < 0) + firstId = partition.partition(); + } + + List topics = partitionsById.get(firstId); + for (List topicsPerPartition : partitionsById.values()) { + if (topics.size() != topicsPerPartition.size()) + return false; + + for (String topic : topicsPerPartition) { + if (!topics.contains(topic)) + return false; + } + } + + return true; + } + @Override public int id() { return id; diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java index b27aacb27bfbc..6755be93041fe 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -225,6 +225,11 @@ public int id() { return -1; } + @Override + public boolean joinable() { + return true; + } + @Override public Serializer keySerializer() { return serializer; From 93e36e0978c0fd9f4be4f404d49ac40ed3d6e07c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 28 Aug 2015 08:38:53 -0700 Subject: [PATCH 189/275] imports --- .../apache/kafka/streaming/kstream/internals/KStreamJoin.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index 733e6d1a39c72..688cb65f9fc2a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -18,10 +18,10 @@ package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.kstream.ValueJoiner; import org.apache.kafka.streaming.kstream.Window; +import org.apache.kafka.streaming.processor.Processor; +import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.processor.ProcessorDef; import org.apache.kafka.streaming.processor.internals.ProcessorContextImpl; From 9b7f66f810560f9c526c55b57a9fdf8799154e5c Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 28 Aug 2015 09:28:02 -0700 Subject: [PATCH 190/275] unify KeyValueMapper and KeyValueFlatMap --- .../kafka/streaming/examples/KStreamJob.java | 2 +- .../kafka/streaming/kstream/KStream.java | 6 ++--- .../streaming/kstream/KeyValueFlatMap.java | 23 ------------------- .../streaming/kstream/KeyValueMapper.java | 4 ++-- .../kstream/internals/KStreamFlatMap.java | 6 ++--- .../kstream/internals/KStreamImpl.java | 8 +++---- .../kstream/internals/KStreamMap.java | 4 ++-- .../kstream/internals/KStreamFlatMapTest.java | 6 ++--- .../kstream/internals/KStreamJoinTest.java | 8 +++---- .../kstream/internals/KStreamMapTest.java | 4 ++-- 10 files changed, 24 insertions(+), 47 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueFlatMap.java diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java index 9246998c5964d..22554a306406b 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java @@ -42,7 +42,7 @@ public static void main(String[] args) throws Exception { KStream stream1 = builder.from(new StringDeserializer(), new StringDeserializer(), "topic1"); KStream stream2 = - stream1.map(new KeyValueMapper() { + stream1.map(new KeyValueMapper>() { @Override public KeyValue apply(String key, String value) { return new KeyValue<>(key, new Integer(value)); diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java index 2530b37d0d613..d30dce96c7fc1 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java @@ -50,7 +50,7 @@ public interface KStream { * @param the value type of the new stream * @return KStream */ - KStream map(KeyValueMapper mapper); + KStream map(KeyValueMapper> mapper); /** * Creates a new stream by transforming valuesa by a mapper to all values of this stream @@ -69,7 +69,7 @@ public interface KStream { * @param the value type of the new stream * @return KStream */ - KStream flatMap(KeyValueFlatMap mapper); + KStream flatMap(KeyValueMapper>> mapper); /** * Creates a new stream by applying a mapper to all values of this stream and using the values in the resulting Iterable @@ -78,7 +78,7 @@ public interface KStream { * @param the value type of the new stream * @return KStream */ - KStream flatMapValues(ValueMapper> processor); + KStream flatMapValues(ValueMapper> processor); /** * Creates a new windowed stream using a specified window instance. diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueFlatMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueFlatMap.java deleted file mode 100644 index 16c96dc21310d..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueFlatMap.java +++ /dev/null @@ -1,23 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.kstream; - -public interface KeyValueFlatMap { - - Iterable> apply(K1 key, V1 value); -} \ No newline at end of file diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueMapper.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueMapper.java index 10e05d69e114b..9624550f7f7ae 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueMapper.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueMapper.java @@ -17,7 +17,7 @@ package org.apache.kafka.streaming.kstream; -public interface KeyValueMapper { +public interface KeyValueMapper { - KeyValue apply(K1 key, V1 value); + R apply(K key, V value); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java index b9a7f5dcd8285..d3f13bb659984 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java @@ -18,15 +18,15 @@ package org.apache.kafka.streaming.kstream.internals; import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueFlatMap; +import org.apache.kafka.streaming.kstream.KeyValueMapper; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorDef; class KStreamFlatMap implements ProcessorDef { - private final KeyValueFlatMap mapper; + private final KeyValueMapper>> mapper; - KStreamFlatMap(KeyValueFlatMap mapper) { + KStreamFlatMap(KeyValueMapper>> mapper) { this.mapper = mapper; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index bf7456e5236df..661b4584c018b 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -19,9 +19,9 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.processor.ProcessorDef; import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.kstream.KeyValueFlatMap; import org.apache.kafka.streaming.kstream.KStreamWindowed; import org.apache.kafka.streaming.kstream.KeyValueMapper; import org.apache.kafka.streaming.kstream.Predicate; @@ -90,7 +90,7 @@ public KStream filterOut(final Predicate predicate) { } @Override - public KStream map(KeyValueMapper mapper) { + public KStream map(KeyValueMapper> mapper) { String name = MAP_NAME + INDEX.getAndIncrement(); topology.addProcessor(name, new KStreamMap(mapper), this.name); @@ -109,7 +109,7 @@ public KStream mapValues(ValueMapper mapper) { @SuppressWarnings("unchecked") @Override - public KStream flatMap(KeyValueFlatMap mapper) { + public KStream flatMap(KeyValueMapper>> mapper) { String name = FLATMAP_NAME + INDEX.getAndIncrement(); topology.addProcessor(name, new KStreamFlatMap(mapper), this.name); @@ -119,7 +119,7 @@ public KStream flatMap(KeyValueFlatMap mapper) { @SuppressWarnings("unchecked") @Override - public KStream flatMapValues(ValueMapper> mapper) { + public KStream flatMapValues(ValueMapper> mapper) { String name = FLATMAPVALUES_NAME + INDEX.getAndIncrement(); topology.addProcessor(name, new KStreamFlatMapValues(mapper), this.name); diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java index 2400e03e73ad8..f6a1a8d9352f8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java @@ -24,9 +24,9 @@ class KStreamMap implements ProcessorDef { - private final KeyValueMapper mapper; + private final KeyValueMapper> mapper; - public KStreamMap(KeyValueMapper mapper) { + public KStreamMap(KeyValueMapper> mapper) { this.mapper = mapper; } diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java index 9ea49833baa28..ec1637c08a9a2 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java @@ -22,7 +22,7 @@ import org.apache.kafka.streaming.kstream.KStream; import org.apache.kafka.streaming.kstream.KStreamBuilder; import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueFlatMap; +import org.apache.kafka.streaming.kstream.KeyValueMapper; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorDef; import org.junit.Test; @@ -42,8 +42,8 @@ public class KStreamFlatMapTest { public void testFlatMap() { KStreamBuilder builder = new KStreamBuilder(); - KeyValueFlatMap mapper = - new KeyValueFlatMap() { + KeyValueMapper>> mapper = + new KeyValueMapper>>() { @Override public Iterable> apply(Integer key, String value) { ArrayList> result = new ArrayList<>(); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java index 5752c353ea505..acb305a4565ed 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java @@ -63,16 +63,16 @@ public Iterable apply(String value) { } }; - private KeyValueMapper keyValueMapper = - new KeyValueMapper() { + private KeyValueMapper> keyValueMapper = + new KeyValueMapper>() { @Override public KeyValue apply(Integer key, String value) { return KeyValue.pair(key, value); } }; - KeyValueMapper> keyValueMapper2 = - new KeyValueMapper>() { + KeyValueMapper>> keyValueMapper2 = + new KeyValueMapper>>() { @Override public KeyValue> apply(Integer key, String value) { return KeyValue.pair(key, (Iterable) Utils.mkSet(value)); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java index f8f89ae676f93..b5bb8c5a364a4 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java @@ -40,8 +40,8 @@ public class KStreamMapTest { public void testMap() { KStreamBuilder builder = new KStreamBuilder(); - KeyValueMapper mapper = - new KeyValueMapper() { + KeyValueMapper> mapper = + new KeyValueMapper>() { @Override public KeyValue apply(Integer key, String value) { return KeyValue.pair(value, key); From e4d7309e635107c8a5818f54444dbd9f26be7230 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 28 Aug 2015 09:43:02 -0700 Subject: [PATCH 191/275] make testing work --- checkstyle/import-control.xml | 4 +- .../kafka/streaming/StreamingConfig.java | 1 - .../internals/KStreamWindowedImpl.java | 20 ++++++++-- .../streaming/processor/TopologyBuilder.java | 2 +- .../internals/ProcessorContextImpl.java | 4 +- .../internals/ProcessorTopology.java | 1 - .../processor/internals/SinkNode.java | 5 +-- .../streaming/state/MeteredKeyValueStore.java | 2 +- .../internals/FilteredIteratorTest.java | 1 - .../processor/internals/StreamTaskTest.java | 38 ++++++++++--------- .../apache/kafka/test/UnlimitedWindowDef.java | 20 +++++----- 11 files changed, 56 insertions(+), 42 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 16036d6e0a32e..12357f069a078 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -108,12 +108,12 @@ - + + - diff --git a/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java b/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java index 888145412f0a7..cad0b6dfec3bb 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java +++ b/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.config.ConfigDef.Type; import java.util.Map; -import java.util.Properties; public class StreamingConfig extends AbstractConfig { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java index 62351a4e3607f..ec1e2c10f1f67 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java @@ -1,3 +1,20 @@ +/** + * 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.streaming.kstream.internals; import org.apache.kafka.streaming.kstream.KStream; @@ -6,9 +23,6 @@ import org.apache.kafka.streaming.kstream.WindowDef; import org.apache.kafka.streaming.processor.TopologyBuilder; -/** - * Created by yasuhiro on 8/27/15. - */ public final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { private final WindowDef windowDef; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 596a4c6c49885..8bf8ee8c152d5 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -175,7 +175,7 @@ public ProcessorTopology build() { processorMap.get(parent).addChild(node); } } else if (factory instanceof SourceNodeFactory) { - for (String topic : ((SourceNodeFactory)factory).topics) { + for (String topic : ((SourceNodeFactory) factory).topics) { topicSourceMap.put(topic, (SourceNode) node); } } else if (factory instanceof SinkNodeFactory) { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index de1af645cc452..d4fc4e63a3beb 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -97,7 +97,9 @@ public RecordCollector recordCollector() { return this.collector; } - public StreamTask task() { return this.task; } + public StreamTask task() { + return this.task; + } @Override public boolean joinable() { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java index bf4ddd796ece8..9ebbe5a5702d8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java @@ -20,7 +20,6 @@ import org.apache.kafka.streaming.processor.ProcessorContext; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java index 14bec7ea5fc65..412824c517578 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java @@ -21,9 +21,6 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streaming.processor.ProcessorContext; -import java.util.ArrayList; -import java.util.List; - public class SinkNode extends ProcessorNode { private final String topic; @@ -48,7 +45,7 @@ public void init(ProcessorContext context) { @Override public void process(K key, V value) { // send to all the registered topics - RecordCollector collector = ((ProcessorContextImpl)context).recordCollector(); + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java b/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java index 93f7c093a7107..805de51c8acab 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java +++ b/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java @@ -213,7 +213,7 @@ public void flush() { } private void logChange() { - RecordCollector collector = ((ProcessorContextImpl)context).recordCollector(); + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); Serializer keySerializer = (Serializer) context.keySerializer(); Serializer valueSerializer = (Serializer) context.valueSerializer(); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.java index ff8ed098f0977..6c71742403d1a 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; -import org.apache.kafka.streaming.kstream.internals.FilteredIterator; import org.junit.Test; import java.util.ArrayList; diff --git a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java index c69c2841b04c7..c269d47bc2d49 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java @@ -52,26 +52,30 @@ public class StreamTaskTest { private final TopicPartition partition2 = new TopicPartition("topic2", 1); private final HashSet partitions = new HashSet<>(Arrays.asList(partition1, partition2)); - private final MockSourceNode source1 = new MockSourceNode<>(intDeserializer, intDeserializer); - private final MockSourceNode source2 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final MockSourceNode source1 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final MockSourceNode source2 = new MockSourceNode<>(intDeserializer, intDeserializer); private final ProcessorTopology topology = new ProcessorTopology( - Arrays.asList(source1, source2), - new HashMap(){{ - put("topic1", source1); - put("topic2", source2); - }}, - Collections.emptyMap() + Arrays.asList((ProcessorNode) source1, (ProcessorNode) source2), + new HashMap() { + { + put("topic1", source1); + put("topic2", source2); + } + }, + Collections.emptyMap() ); - 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 StreamingConfig config = new StreamingConfig(new Properties() { + { + setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); + setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); + setProperty(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + } + }); private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); private final MockProducer producer = new MockProducer<>(false, bytesSerializer, bytesSerializer); diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java index c6d929725c65f..b20a4b7ae2ede 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java @@ -48,26 +48,26 @@ public class UnlimitedWindow implements Window { private LinkedList>> list = new LinkedList<>(); @Override - public void init (ProcessorContext context){ + public void init(ProcessorContext context) { context.register(this, null); } @Override - public Iterator find(final K key, long timestamp){ + public Iterator find(final K key, long timestamp) { return find(key, Long.MIN_VALUE, timestamp); } @Override - public Iterator findAfter(final K key, long timestamp){ + public Iterator findAfter(final K key, long timestamp) { return find(key, timestamp, Long.MAX_VALUE); } @Override - public Iterator findBefore(final K key, long timestamp){ + public Iterator findBefore(final K key, long timestamp) { return find(key, Long.MIN_VALUE, Long.MAX_VALUE); } - private Iterator find(final K key, final long startTime, final long endTime){ + private Iterator find(final K key, final long startTime, final long endTime) { return new FilteredIterator>>(list.iterator()) { protected V filter(Stamped> item) { if (item.value.key.equals(key) && startTime <= item.timestamp && item.timestamp <= endTime) @@ -79,25 +79,25 @@ protected V filter(Stamped> item) { } @Override - public void put (K key, V value,long timestamp){ + public void put(K key, V value, long timestamp) { list.add(new Stamped<>(KeyValue.pair(key, value), timestamp)); } @Override - public String name () { + public String name() { return null; } @Override - public void flush () { + public void flush() { } @Override - public void close () { + public void close() { } @Override - public boolean persistent () { + public boolean persistent() { return false; } } From fd0e805895b5dc0169e4d9a4f24832213c4a94c9 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 28 Aug 2015 10:08:32 -0700 Subject: [PATCH 192/275] disallow SinkNode.addChild() --- .../kafka/streaming/processor/internals/ProcessorNode.java | 6 +++--- .../kafka/streaming/processor/internals/SinkNode.java | 5 +++++ 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java index fa1dbb1e734bc..feee522158011 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java @@ -40,15 +40,15 @@ public ProcessorNode(String name, Processor processor) { this.children = new ArrayList<>(); } - public String name() { + public final String name() { return name; } - public List> children() { + public final List> children() { return children; } - public final void addChild(ProcessorNode child) { + public void addChild(ProcessorNode child) { children.add(child); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java index 412824c517578..9a6e3b57917b7 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java @@ -37,6 +37,11 @@ public SinkNode(String name, String topic, Serializer keySerializer, Serializ this.valSerializer = valSerializer; } + @Override + public void addChild(ProcessorNode child) { + throw new UnsupportedOperationException("sink node does not allow addChild"); + } + @Override public void init(ProcessorContext context) { this.context = context; From 0c48d538355860b25505a0d67229f9ffd4921a66 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 28 Aug 2015 11:30:21 -0700 Subject: [PATCH 193/275] cleanup KStreamJoin --- .../kstream/internals/KStreamJoin.java | 37 +------------------ 1 file changed, 1 insertion(+), 36 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index 688cb65f9fc2a..91ab5fef36ef9 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -17,20 +17,13 @@ package org.apache.kafka.streaming.kstream.internals; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streaming.kstream.ValueJoiner; import org.apache.kafka.streaming.kstream.Window; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; import org.apache.kafka.streaming.processor.ProcessorDef; -import org.apache.kafka.streaming.processor.internals.ProcessorContextImpl; -import java.util.ArrayList; -import java.util.HashMap; import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; class KStreamJoin implements ProcessorDef { @@ -74,7 +67,7 @@ public void init(ProcessorContext context) { super.init(context); // check if these two streams are joinable - if (!joinable()) + if (!context.joinable()) throw new IllegalStateException("Streams are not joinable."); final Window window1 = (Window) context.getStateStore(windowName1); @@ -129,34 +122,6 @@ public void process(K key, V1 value) { } } - private boolean joinable() { - Set partitions = ((ProcessorContextImpl) this.context).task().partitions(); - Map> partitionsById = new HashMap<>(); - int firstId = -1; - for (TopicPartition partition : partitions) { - if (!partitionsById.containsKey(partition.partition())) { - partitionsById.put(partition.partition(), new ArrayList()); - } - partitionsById.get(partition.partition()).add(partition.topic()); - - if (firstId < 0) - firstId = partition.partition(); - } - - List topics = partitionsById.get(firstId); - for (List topicsPerPartition : partitionsById.values()) { - if (topics.size() != topicsPerPartition.size()) - return false; - - for (String topic : topicsPerPartition) { - if (!topics.contains(topic)) - return false; - } - } - - return true; - } - // TODO: use the "outer-stream" topic as the resulted join stream topic private void doJoin(K key, V1 value1, V2 value2) { context.forward(key, joiner.apply(value1, value2)); From ead02e5291d24a0c24ff72d49924150d9f6cca0f Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 28 Aug 2015 11:54:46 -0700 Subject: [PATCH 194/275] fix KStreamBranchTest --- .../kafka/streaming/kstream/internals/KStreamImpl.java | 9 ++++----- .../streaming/kstream/internals/KStreamBranchTest.java | 6 +++--- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index 661b4584c018b..a24d092e3ea3f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -29,8 +29,7 @@ import org.apache.kafka.streaming.kstream.ValueMapper; import org.apache.kafka.streaming.kstream.WindowDef; -import java.util.ArrayList; -import java.util.List; +import java.lang.reflect.Array; import java.util.concurrent.atomic.AtomicInteger; public class KStreamImpl implements KStream { @@ -143,16 +142,16 @@ public KStream[] branch(Predicate... predicates) { topology.addProcessor(branchName, new KStreamBranch(predicates.clone()), this.name); - List> branchChildren = new ArrayList<>(); + KStream[] branchChildren = (KStream[]) Array.newInstance(KStream.class, predicates.length); for (int i = 0; i < predicates.length; i++) { String childName = BRANCHCHILD_NAME + INDEX.getAndIncrement(); topology.addProcessor(childName, new KStreamPassThrough(), branchName); - branchChildren.add(new KStreamImpl(topology, childName)); + branchChildren[i] = new KStreamImpl(topology, childName); } - return (KStream[]) branchChildren.toArray(); + return branchChildren; } @SuppressWarnings("unchecked") diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java index 89c0278f79e1a..ad54b14ec6ee4 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java @@ -61,7 +61,7 @@ public boolean apply(Integer key, String value) { } }; - final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6}; KStream stream; KStream[] branches; @@ -84,7 +84,7 @@ public boolean apply(Integer key, String value) { } assertEquals(3, processors[0].processed.size()); - assertEquals(2, processors[1].processed.size()); - assertEquals(4, processors[2].processed.size()); + assertEquals(1, processors[1].processed.size()); + assertEquals(2, processors[2].processed.size()); } } From 26a5d17c017b30dea7889714318eb8dd10678ac8 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 28 Aug 2015 12:25:06 -0700 Subject: [PATCH 195/275] Let KafkaProcess not extending from Runnable to be started and closed from the same caller --- .../org/apache/kafka/common/utils/Utils.java | 23 +---- .../kafka/streaming/KafkaStreaming.java | 89 +++++++------------ .../kafka/streaming/examples/KStreamJob.java | 2 +- .../streaming/examples/ProcessorJob.java | 2 +- .../streaming/processor/RestoreFunc.java | 5 +- 5 files changed, 40 insertions(+), 81 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 9893e42ecac34..aaf194b76fd0a 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -535,28 +535,9 @@ public static ByteBuffer ensureCapacity(ByteBuffer existingBuffer, int newLength * @return Set */ public static HashSet mkSet(T... elems) { - HashSet set = new HashSet(); - for (T e : elems) set.add(e); - return set; + return new HashSet<>(Arrays.asList(elems)); } - - /** - * Makes a string of a comma separated list of collection elements - * @param collection the collection - * @param the type of collection elements - * @return String - */ - public static String mkString(Collection collection) { - StringBuilder sb = new StringBuilder(); - int count = collection.size(); - for (E elem : collection) { - sb.append(elem.toString()); - count--; - if (count > 0) sb.append(", "); - } - return sb.toString(); - } - + /** * Recursively delete the given file/directory and any subfiles (if any exist) * diff --git a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java index 35d1b751a02e3..be27ae83acc7d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java +++ b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java @@ -52,21 +52,17 @@ * * */ -public class KafkaStreaming implements Runnable { +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 STOPPING = 2; - private static final int STOPPED = 3; + private static final int STOPPED = 2; private int state = CREATED; - private final Object lock = new Object(); private final StreamThread[] threads; - private final File stateDir; - public KafkaStreaming(TopologyBuilder builder, StreamingConfig config) throws Exception { if (config.getClass(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG) == null) @@ -76,72 +72,51 @@ public KafkaStreaming(TopologyBuilder builder, StreamingConfig config) throws Ex for (int i = 0; i < this.threads.length; i++) { this.threads[i] = new StreamThread(builder, config); } - - this.stateDir = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG)); } /** - * Execute the stream processors + * Start the stream process by starting all its threads */ - public void run() { - synchronized (lock) { - log.info("Starting container"); - if (state == CREATED) { - if (!stateDir.exists() && !stateDir.mkdirs()) - throw new IllegalArgumentException("Failed to create state directory: " + stateDir.getAbsolutePath()); - - for (StreamThread thread : threads) thread.start(); - log.info("Start-up complete"); - } else { - throw new IllegalStateException("This container was already started"); - } - - state = RUNNING; - while (state == RUNNING) { - try { - lock.wait(); - } catch (InterruptedException ex) { - Thread.interrupted(); - } - } - - if (state == STOPPING) { - log.info("Shutting down the container"); + public synchronized void start() { + log.debug("Starting Kafka Stream process"); + + if (state == CREATED) { + for (StreamThread thread : threads) + thread.start(); + } else { + throw new IllegalStateException("This process was already started."); + } - for (StreamThread thread : threads) - thread.close(); + state = RUNNING; - for (StreamThread thread : threads) { - try { - thread.join(); - } catch (InterruptedException ex) { - Thread.interrupted(); - } - } - state = STOPPED; - lock.notifyAll(); - log.info("Shutdown complete"); - } - } + log.info("Started Kafka Stream process"); } /** - * Shutdown this streaming instance. + * Shutdown this stream process by signaling the threads to stop, + * wait for them to join and clean up the process instance. */ - public void close() { - synchronized (lock) { - if (state == CREATED || state == RUNNING) { - state = STOPPING; - lock.notifyAll(); - } - while (state == STOPPING) { + 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 { - lock.wait(); + thread.join(); } catch (InterruptedException ex) { Thread.interrupted(); } } + } else { + throw new IllegalStateException("This process has not started yet."); } - } + state = STOPPED; + + log.info("Stopped Kafka Stream process"); + } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java index 22554a306406b..d478c87725a49 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java @@ -73,6 +73,6 @@ public boolean apply(String key, Integer value) { streams[1].sendTo("topic3", stringSerializer, intSerializer); KafkaStreaming kstream = new KafkaStreaming(builder, config); - kstream.run(); + kstream.start(); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java index c7defec9115db..459cd2eb1c0cf 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java @@ -96,6 +96,6 @@ public static void main(String[] args) throws Exception { builder.addSink("SINK", "topic-sink", new StringSerializer(), new IntegerSerializer(), "PROCESS"); KafkaStreaming streaming = new KafkaStreaming(builder, config); - streaming.run(); + streaming.start(); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/RestoreFunc.java b/stream/src/main/java/org/apache/kafka/streaming/processor/RestoreFunc.java index 877c6a49188e6..9de8ab1ae0a37 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/RestoreFunc.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/RestoreFunc.java @@ -17,7 +17,10 @@ package org.apache.kafka.streaming.processor; -// TODO: this should be removed once we move to Java 8 +/** + * 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); From b9dc1a5bcc7895c20bbc5338aa6e3e594d86e6b2 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 28 Aug 2015 13:58:46 -0700 Subject: [PATCH 196/275] fix context.forward() using a stack of nodes --- .../streaming/kstream/KStreamWindowed.java | 14 ---- .../kstream/internals/KStreamImpl.java | 4 +- .../kstream/internals/KStreamJoin.java | 79 +++++++------------ .../internals/KStreamWindowedImpl.java | 23 +++--- .../processor/internals/PartitionGroup.java | 2 +- .../internals/ProcessorContextImpl.java | 28 ++++++- .../kstream/internals/KStreamFlatMapTest.java | 4 +- .../kstream/internals/KStreamJoinTest.java | 78 ------------------ .../apache/kafka/test/KStreamTestDriver.java | 8 +- .../kafka/test/MockProcessorContext.java | 29 +++++-- .../apache/kafka/test/UnlimitedWindowDef.java | 2 +- 11 files changed, 99 insertions(+), 172 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamWindowed.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamWindowed.java index 517d63bc45ad2..07d2f136fde9b 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamWindowed.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamWindowed.java @@ -35,18 +35,4 @@ public interface KStreamWindowed extends KStream { */ KStream join(KStreamWindowed other, ValueJoiner joiner); - /** - * 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 if - * the element from the other stream has an older timestamp. - * The resulting values are computed by applying a joiner. - * - * @param other the other windowed stream - * @param joiner the instance ValueJoiner - * @param the value type of the other stream - * @param the value type of the new stream - * @return KStream - */ - KStream joinPrior(KStreamWindowed other, ValueJoiner joiner); - } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index a24d092e3ea3f..28f023f8f4a72 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -52,10 +52,12 @@ public class KStreamImpl implements KStream { private static final String WINDOWED_NAME = "KAFKA-WINDOWED-"; - public static final String JOIN_NAME = "KAFKA-JOIN-"; + 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 String SEND_NAME = "KAFKA-SEND-"; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index 91ab5fef36ef9..fee1f479390af 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -34,7 +34,6 @@ private static abstract class Finder { private final String windowName1; private final String windowName2; private final ValueJoiner joiner; - private final boolean prior; private Processor processorForOtherStream = null; public final ProcessorDef processorDefForOtherStream = new ProcessorDef() { @@ -44,22 +43,39 @@ public Processor instance() { } }; - KStreamJoin(String windowName1, String windowName2, boolean prior, ValueJoiner joiner) { + KStreamJoin(String windowName1, String windowName2, ValueJoiner joiner) { this.windowName1 = windowName1; this.windowName2 = windowName2; this.joiner = joiner; - this.prior = prior; } @Override public Processor instance() { - return new KStreamJoinProcessor(); + // create a processor instance for the other stream + processorForOtherStream = new KStreamJoinProcessor(windowName1) { + @Override + protected void doJoin(K key, V2 value2, V1 value1) { + context.forward(key, joiner.apply(value1, value2)); + } + }; + + // create a processor instance for the primary stream + return new KStreamJoinProcessor(windowName2) { + @Override + protected void doJoin(K key, V1 value1, V2 value2) { + context.forward(key, joiner.apply(value1, value2)); + } + }; } - private class KStreamJoinProcessor extends KStreamProcessor { + private abstract class KStreamJoinProcessor extends KStreamProcessor { - private Finder finder1; - private Finder finder2; + private final String windowName; + protected Finder finder; + + public KStreamJoinProcessor(String windowName) { + this.windowName = windowName; + } @SuppressWarnings("unchecked") @Override @@ -70,51 +86,19 @@ public void init(ProcessorContext context) { if (!context.joinable()) throw new IllegalStateException("Streams are not joinable."); - final Window window1 = (Window) context.getStateStore(windowName1); - final Window window2 = (Window) context.getStateStore(windowName2); - - if (prior) { - this.finder1 = new Finder() { - Iterator find(K key, long timestamp) { - return window1.findAfter(key, timestamp); - } - }; - this.finder2 = new Finder() { - Iterator find(K key, long timestamp) { - return window2.findBefore(key, timestamp); - } - }; - } else { - this.finder1 = new Finder() { - Iterator find(K key, long timestamp) { - return window1.find(key, timestamp); - } - }; - this.finder2 = new Finder() { - Iterator find(K key, long timestamp) { - return window2.find(key, timestamp); - } - }; - } + final Window window = (Window) context.getStateStore(windowName); - processorForOtherStream = new KStreamProcessor() { - @Override - public void process(K key, V2 value) { - long timestamp = context.timestamp(); - Iterator iter = finder1.find(key, timestamp); - if (iter != null) { - while (iter.hasNext()) { - doJoin(key, iter.next(), value); - } - } + this.finder = new Finder() { + Iterator find(K key, long timestamp) { + return window.find(key, timestamp); } }; } @Override - public void process(K key, V1 value) { + public void process(K key, T1 value) { long timestamp = context.timestamp(); - Iterator iter = finder2.find(key, timestamp); + Iterator iter = finder.find(key, timestamp); if (iter != null) { while (iter.hasNext()) { doJoin(key, value, iter.next()); @@ -122,10 +106,7 @@ public void process(K key, V1 value) { } } - // TODO: use the "outer-stream" topic as the resulted join stream topic - private void doJoin(K key, V1 value1, V2 value2) { - context.forward(key, joiner.apply(value1, value2)); - } + abstract protected void doJoin(K key, T1 value1, T2 value2); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java index ec1e2c10f1f67..7bbf45ee5e2c1 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java @@ -34,25 +34,20 @@ public KStreamWindowedImpl(TopologyBuilder topology, String name, WindowDef KStream join(KStreamWindowed other, ValueJoiner valueJoiner) { - return join(other, false, valueJoiner); - } - - @Override - public KStream joinPrior(KStreamWindowed other, ValueJoiner valueJoiner) { - return join(other, true, valueJoiner); - } - - private KStream join(KStreamWindowed other, boolean prior, ValueJoiner valueJoiner) { String thisWindowName = this.windowDef.name(); String otherWindowName = ((KStreamWindowedImpl) other).windowDef.name(); - KStreamJoin join = new KStreamJoin<>(thisWindowName, otherWindowName, prior, valueJoiner); - - String joinName = JOIN_NAME + INDEX.getAndIncrement(); + String joinThisName = JOINTHIS_NAME + INDEX.getAndIncrement(); String joinOtherName = JOINOTHER_NAME + INDEX.getAndIncrement(); - topology.addProcessor(joinName, join, this.name); + String joinMergeName = JOINMERGE_NAME + INDEX.getAndIncrement(); + + KStreamJoin join = new KStreamJoin<>(thisWindowName, otherWindowName, valueJoiner); + KStreamPassThrough joinMerge = new KStreamPassThrough<>(); + + topology.addProcessor(joinThisName, join, this.name); topology.addProcessor(joinOtherName, join.processorDefForOtherStream, ((KStreamImpl) other).name); + topology.addProcessor(joinMergeName, joinMerge, joinThisName, joinOtherName); - return new KStreamImpl<>(topology, joinName); + return new KStreamImpl<>(topology, joinMergeName); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java index 2b708a3d933d2..fc5213047ec9d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java @@ -40,7 +40,7 @@ public class PartitionGroup { private int totalBuffered; public PartitionGroup(Map partitionQueues) { - this.queuesByTime = new PriorityQueue<>(new Comparator() { + this.queuesByTime = new PriorityQueue<>(partitionQueues.size(), new Comparator() { @Override public int compare(RecordQueue queue1, RecordQueue queue2) { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java index d4fc4e63a3beb..47fd04552a8d0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java @@ -37,6 +37,7 @@ import java.io.File; import java.io.IOException; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -52,6 +53,7 @@ public class ProcessorContextImpl implements ProcessorContext { private final Metrics metrics; private final RecordCollector collector; private final ProcessorStateManager stateMgr; + private final ArrayDeque nodeStack = new ArrayDeque(); private final Serializer keySerializer; private final Serializer valSerializer; @@ -214,8 +216,12 @@ public long timestamp() { @SuppressWarnings("unchecked") public void forward(K key, V value) { for (ProcessorNode childNode : (List>) task.node().children()) { - task.node(childNode); - childNode.process(key, value); + pushNode(childNode); + try { + childNode.process(key, value); + } finally { + popNode(); + } } } @@ -223,8 +229,22 @@ public void forward(K key, V value) { @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { ProcessorNode childNode = (ProcessorNode) task.node().children().get(childIndex); - task.node(childNode); - childNode.process(key, value); + pushNode(childNode); + try { + childNode.process(key, value); + } finally { + popNode(); + } + } + + private void pushNode(ProcessorNode node) { + nodeStack.push(node); + task.node(node); + } + + private void popNode() { + nodeStack.pop(); + task.node(nodeStack.peek()); } @Override diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java index ec1637c08a9a2..032edca59337b 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java @@ -48,7 +48,7 @@ public void testFlatMap() { public Iterable> apply(Integer key, String value) { ArrayList> result = new ArrayList<>(); for (int i = 0; i < key; i++) { - result.add(KeyValue.pair(Integer.toString(key), value)); + result.add(KeyValue.pair(Integer.toString(key * 10 + i), value)); } return result; } @@ -116,7 +116,7 @@ public Iterable> apply(Integer key, String value) { assertEquals(6, processor.processed.size()); - String[] expected = new String[]{"10:V1", "20:V2", "20:V2", "30:V3", "30:V3", "30:V3"}; + String[] expected = new String[]{"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)); diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java index acb305a4565ed..3374a033b6b2b 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java @@ -160,83 +160,5 @@ public void testJoin() { } } - @Test - public void testJoinPrior() { - KStreamBuilder builder = new KStreamBuilder(); - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - - KStream stream1; - KStream stream2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessorDef processor; - String[] expected; - - processor = new MockProcessorDef<>(); - stream1 = builder.from(keyDeserializer, valDeserializer, topic1); - stream2 = builder.from(keyDeserializer, valDeserializer, topic2); - windowed1 = stream1.with(new UnlimitedWindowDef("window1")); - windowed2 = stream2.with(new UnlimitedWindowDef("window2")); - - windowed1.joinPrior(windowed2, joiner).process(processor); - - KStreamTestDriver driver = new KStreamTestDriver(builder); - - // push two items to the main stream. the other stream's window is empty - - for (int i = 0; i < 2; i++) { - driver.setTime(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 - // no corresponding item in the main window has a newer timestamp - - for (int i = 0; i < 2; i++) { - driver.setTime(i + 1); - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } - - assertEquals(0, processor.processed.size()); - - processor.processed.clear(); - - // push all items with newer timestamps to the main stream. this should produce two items. - - for (int i = 0; i < expectedKeys.length; i++) { - driver.setTime(i + 2); - 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 with older timestamps to the other stream. this should produce six items - for (int i = 0; i < expectedKeys.length; i++) { - driver.setTime(i); - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } - - assertEquals(6, processor.processed.size()); - - expected = new String[]{"0:X0+Y0", "0:X0+Y0", "1:X1+Y1", "1:X1+Y1", "2:X2+Y2", "3:X3+Y3"}; - - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); - } - } - // TODO: test for joinability } diff --git a/stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index e026fcc67bd50..9f95bca83a057 100644 --- a/stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -40,8 +40,12 @@ public KStreamTestDriver(KStreamBuilder builder, Serializer serializer, Deser } public void process(String topicName, Object key, Object value) { - context.node(topology.source(topicName)); - context.forward(key, value); + context.pushNode(topology.source(topicName)); + try { + context.forward(key, value); + } finally { + context.popNode(); + } } public void setTime(long timestamp) { diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 6755be93041fe..8246cdae3d53b 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -95,6 +95,7 @@ import org.apache.kafka.streaming.processor.internals.ProcessorNode; import java.io.File; +import java.util.ArrayDeque; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -206,6 +207,7 @@ public void schedule(Processor processor, long interval) { private Serializer serializer; private Deserializer deserializer; private ProcessorNode node; + private final ArrayDeque nodeStack = new ArrayDeque(); private Map storeMap = new HashMap<>(); >>>>>>> kstream test fix @@ -280,8 +282,12 @@ public void schedule(Processor processor, long interval) { @SuppressWarnings("unchecked") public void forward(K key, V value) { for (ProcessorNode childNode : (List>) node().children()) { - node(childNode); - childNode.process(key, value); + pushNode(childNode); + try { + childNode.process(key, value); + } finally { + popNode(); + } } } @@ -289,8 +295,12 @@ public void forward(K key, V value) { @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { ProcessorNode childNode = (ProcessorNode) node().children().get(childIndex); - node(childNode); - childNode.process(key, value); + pushNode(childNode); + try { + childNode.process(key, value); + } finally { + popNode(); + } } @Override @@ -319,11 +329,18 @@ public long timestamp() { } >>>>>>> compile and test passed - public void node(ProcessorNode node) { + public void pushNode(ProcessorNode node) { + nodeStack.push(node); this.node = node; } - public ProcessorNode node() { + public ProcessorNode popNode() { + ProcessorNode node = nodeStack.pop(); + this.node = nodeStack.peek(); + return node; + } + + private ProcessorNode node() { return this.node; } } diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java index b20a4b7ae2ede..4b1b01ffff8db 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java +++ b/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java @@ -85,7 +85,7 @@ public void put(K key, V value, long timestamp) { @Override public String name() { - return null; + return name; } @Override From 9df86f1bb3d72805e05efcd00c3e6b979e11ff25 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 28 Aug 2015 15:37:29 -0700 Subject: [PATCH 197/275] simplified join processor --- .../kafka/streaming/KafkaStreaming.java | 4 +- .../kstream/internals/KStreamJoin.java | 57 +++++++------------ .../internals/KStreamWindowedImpl.java | 11 ++-- 3 files changed, 27 insertions(+), 45 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java index be27ae83acc7d..c1454aa916585 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java +++ b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java @@ -22,8 +22,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; - /** * 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. @@ -118,5 +116,5 @@ public synchronized void close() { state = STOPPED; log.info("Stopped Kafka Stream process"); - } + } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index fee1f479390af..18ad1871dccea 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -31,47 +31,23 @@ private static abstract class Finder { abstract Iterator find(K key, long timestamp); } - private final String windowName1; - private final String windowName2; + private final String windowName; private final ValueJoiner joiner; - private Processor processorForOtherStream = null; - public final ProcessorDef processorDefForOtherStream = new ProcessorDef() { - @Override - public Processor instance() { - return processorForOtherStream; - } - }; - - KStreamJoin(String windowName1, String windowName2, ValueJoiner joiner) { - this.windowName1 = windowName1; - this.windowName2 = windowName2; + KStreamJoin(String windowName, ValueJoiner joiner) { + this.windowName = windowName; this.joiner = joiner; } @Override public Processor instance() { - // create a processor instance for the other stream - processorForOtherStream = new KStreamJoinProcessor(windowName1) { - @Override - protected void doJoin(K key, V2 value2, V1 value1) { - context.forward(key, joiner.apply(value1, value2)); - } - }; - - // create a processor instance for the primary stream - return new KStreamJoinProcessor(windowName2) { - @Override - protected void doJoin(K key, V1 value1, V2 value2) { - context.forward(key, joiner.apply(value1, value2)); - } - }; + return new KStreamJoinProcessor(windowName); } - private abstract class KStreamJoinProcessor extends KStreamProcessor { + private class KStreamJoinProcessor extends KStreamProcessor { private final String windowName; - protected Finder finder; + protected Finder finder; public KStreamJoinProcessor(String windowName) { this.windowName = windowName; @@ -86,27 +62,34 @@ public void init(ProcessorContext context) { if (!context.joinable()) throw new IllegalStateException("Streams are not joinable."); - final Window window = (Window) context.getStateStore(windowName); + final Window window = (Window) context.getStateStore(windowName); - this.finder = new Finder() { - Iterator find(K key, long timestamp) { + this.finder = new Finder() { + Iterator find(K key, long timestamp) { return window.find(key, timestamp); } }; } @Override - public void process(K key, T1 value) { + public void process(K key, V1 value) { long timestamp = context.timestamp(); - Iterator iter = finder.find(key, timestamp); + Iterator iter = finder.find(key, timestamp); if (iter != null) { while (iter.hasNext()) { - doJoin(key, value, iter.next()); + context.forward(key, joiner.apply(value, iter.next())); } } } + } - abstract protected void doJoin(K key, T1 value1, T2 value2); + public static ValueJoiner reserveJoiner(final ValueJoiner joiner) { + return new ValueJoiner() { + @Override + public R apply(T2 value2, T1 value1) { + return joiner.apply(value1, value2); + } + }; } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java index 7bbf45ee5e2c1..de2173b5f6e1f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java @@ -37,15 +37,16 @@ public KStream join(KStreamWindowed other, ValueJoiner) other).windowDef.name(); + KStreamJoin joinThis = new KStreamJoin<>(otherWindowName, valueJoiner); + KStreamJoin joinOther = new KStreamJoin<>(thisWindowName, KStreamJoin.reserveJoiner(valueJoiner)); + KStreamPassThrough joinMerge = new KStreamPassThrough<>(); + String joinThisName = JOINTHIS_NAME + INDEX.getAndIncrement(); String joinOtherName = JOINOTHER_NAME + INDEX.getAndIncrement(); String joinMergeName = JOINMERGE_NAME + INDEX.getAndIncrement(); - KStreamJoin join = new KStreamJoin<>(thisWindowName, otherWindowName, valueJoiner); - KStreamPassThrough joinMerge = new KStreamPassThrough<>(); - - topology.addProcessor(joinThisName, join, this.name); - topology.addProcessor(joinOtherName, join.processorDefForOtherStream, ((KStreamImpl) other).name); + topology.addProcessor(joinThisName, joinThis, this.name); + topology.addProcessor(joinOtherName, joinOther, ((KStreamImpl) other).name); topology.addProcessor(joinMergeName, joinMerge, joinThisName, joinOtherName); return new KStreamImpl<>(topology, joinMergeName); From 5198b02b4d93fc7e245d5ff1a9972402a80c2fe5 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 28 Aug 2015 15:45:41 -0700 Subject: [PATCH 198/275] fix MinTimestampTrackerTest --- .../internals/MinTimestampTrackerTest.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java index 74be3f43122df..aa00fe3a3ea36 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java @@ -48,7 +48,7 @@ public void testTracking() { assertEquals(100L, tracker.get()); // remove 100 - tracker.addElement((Stamped) elems[removalIndex++]); + tracker.removeElement((Stamped) elems[removalIndex++]); assertEquals(101L, tracker.get()); // add 102 @@ -68,24 +68,24 @@ public void testTracking() { assertEquals(98L, tracker.get()); // remove 101 - tracker.addElement((Stamped) elems[removalIndex++]); + tracker.removeElement((Stamped) elems[removalIndex++]); assertEquals(98L, tracker.get()); // remove 102 - tracker.addElement((Stamped) elems[removalIndex++]); + tracker.removeElement((Stamped) elems[removalIndex++]); assertEquals(98L, tracker.get()); // remove 98 - tracker.addElement((Stamped) elems[removalIndex++]); + tracker.removeElement((Stamped) elems[removalIndex++]); assertEquals(99L, tracker.get()); // remove 99 - tracker.addElement((Stamped) elems[removalIndex++]); + tracker.removeElement((Stamped) elems[removalIndex++]); assertEquals(100L, tracker.get()); // remove 100 - tracker.addElement((Stamped) elems[removalIndex++]); - assertEquals(-1L, tracker.get()); + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(TimestampTracker.NOT_KNOWN, tracker.get()); assertEquals(insertionIndex, removalIndex); } From cd481a881aedb98048fbd10e6278b9ab5514f081 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 28 Aug 2015 16:18:10 -0700 Subject: [PATCH 199/275] OS comments --- .../org/apache/kafka/streaming/processor/internals/Stamped.java | 1 - 1 file changed, 1 deletion(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java index dec23759afc12..978aedf53b914 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java @@ -35,5 +35,4 @@ public int compareTo(Object other) { else if (timestamp > otherTimestamp) return 1; return 0; } - } From 0106d3f00672232a474a84e848944a2f183fc4a9 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 1 Sep 2015 13:06:20 -0700 Subject: [PATCH 200/275] fix typo --- .../apache/kafka/streaming/kstream/internals/KStreamJoin.java | 2 +- .../kafka/streaming/kstream/internals/KStreamWindowedImpl.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java index 18ad1871dccea..9529f813ac55e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java @@ -83,7 +83,7 @@ public void process(K key, V1 value) { } } - public static ValueJoiner reserveJoiner(final ValueJoiner joiner) { + public static ValueJoiner reverseJoiner(final ValueJoiner joiner) { return new ValueJoiner() { @Override public R apply(T2 value2, T1 value1) { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java index de2173b5f6e1f..69a108495b0bd 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java @@ -38,7 +38,7 @@ public KStream join(KStreamWindowed other, ValueJoiner) other).windowDef.name(); KStreamJoin joinThis = new KStreamJoin<>(otherWindowName, valueJoiner); - KStreamJoin joinOther = new KStreamJoin<>(thisWindowName, KStreamJoin.reserveJoiner(valueJoiner)); + KStreamJoin joinOther = new KStreamJoin<>(thisWindowName, KStreamJoin.reverseJoiner(valueJoiner)); KStreamPassThrough joinMerge = new KStreamPassThrough<>(); String joinThisName = JOINTHIS_NAME + INDEX.getAndIncrement(); From 20fecf951f7432cad38d7600c9f5d07073e5c66c Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 1 Sep 2015 15:55:50 -0700 Subject: [PATCH 201/275] get source topic names from builer and subscribe topics --- .../kafka/streaming/examples/KStreamJob.java | 23 +++++++++++++++++-- .../streaming/kstream/SlidingWindowDef.java | 2 +- .../streaming/processor/TopologyBuilder.java | 5 ++++ .../processor/internals/StreamThread.java | 4 ++++ 4 files changed, 31 insertions(+), 3 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java index d478c87725a49..4b1b63c975c76 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java @@ -27,13 +27,23 @@ import org.apache.kafka.streaming.kstream.KeyValue; import org.apache.kafka.streaming.kstream.KeyValueMapper; import org.apache.kafka.streaming.kstream.Predicate; +import org.apache.kafka.streaming.processor.TimestampExtractor; -import java.util.Properties; +import java.util.HashMap; +import java.util.Map; public class KStreamJob { public static void main(String[] args) throws Exception { - StreamingConfig config = new StreamingConfig(new Properties()); + Map props = new HashMap<>(); + props.put("bootstrap.servers", "localhost:9092"); + props.put("key.serializer", StringSerializer.class); + props.put("value.serializer", StringSerializer.class); + props.put("key.deserializer", StringDeserializer.class); + props.put("value.deserializer", StringDeserializer.class); + props.put("timestamp.extractor", WallclockTimestampExtractor.class); + StreamingConfig config = new StreamingConfig(props); + KStreamBuilder builder = new KStreamBuilder(); StringSerializer stringSerializer = new StringSerializer(); @@ -75,4 +85,13 @@ public boolean apply(String key, Integer value) { KafkaStreaming kstream = new KafkaStreaming(builder, config); kstream.start(); } + + public static class WallclockTimestampExtractor implements TimestampExtractor { + @Override + public long extract(String topic, Object key, Object value) { + return System.currentTimeMillis(); + } + } + + } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java index 43c2ea7f5410c..346a67df55f89 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java @@ -37,7 +37,7 @@ import java.util.Map; public class SlidingWindowDef implements WindowDef { - private String name; + private final String name; private final long duration; private final int maxCount; private final Serializer keySerializer; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 8bf8ee8c152d5..9e154e375c347 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -26,6 +26,7 @@ import org.apache.kafka.streaming.processor.internals.SourceNode; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -195,4 +196,8 @@ public ProcessorTopology build() { return new ProcessorTopology(processorNodes, topicSourceMap, topicSinkMap); } + + public Set sourceTopics() { + return Collections.unmodifiableSet(sourceTopicNames); + } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index 791b5cb1c295c..cd1bcb9c530ac 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -159,6 +159,10 @@ private void runLoop() { try { int totalNumBuffered = 0; + for (String topic : builder.sourceTopics()) { + consumer.subscribe(topic); + } + while (stillRunning()) { // try to fetch some records if necessary ConsumerRecords records = consumer.poll(totalNumBuffered == 0 ? this.pollTimeMs : 0); From 957ed18874a43edba43adea3f14ef2bb4f870957 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 1 Sep 2015 16:55:50 -0700 Subject: [PATCH 202/275] update comments --- .../kafka/streaming/KafkaStreaming.java | 24 ++++++++++++------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java index c1454aa916585..4e4d8a3e9cc40 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java +++ b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java @@ -26,8 +26,9 @@ * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and * sends output to zero or more output topics. *

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

* This streaming instance will co-ordinate with any other instances (whether in this same process, on other processes @@ -40,13 +41,20 @@ *

* A simple example might look like this: *

- *    Properties props = new Properties();
+ *    Map<String, Object> props = new HashMap<>();
  *    props.put("bootstrap.servers", "localhost:4242");
- *    properties config = new properties(props);
- *    config.processor(ExampleStreamProcessor.class);
- *    config.serialization(new StringSerializer(), new StringDeserializer());
- *    KafkaStreaming container = new KafkaStreaming(new MyKStreamTopology(), config);
- *    container.run();
+ *    props.put("key.deserializer", StringDeserializer.class);
+ *    props.put("value.deserializer", StringDeserializer.class);
+ *    props.put("key.serializer", StringSerializer.class);
+ *    props.put("value.serializer", IntegerSerializer.class);
+ *    props.put("timestamp.extractor", MyTimestampExtractor.class);
+ *    StreamingConfig config = new StreamingConfig(props);
+ *
+ *    KStreamBuilder builder = new KStreamBuilder();
+ *    builder.from("topic1").mapValue(value -> value.length()).sendTo("topic2");
+ *
+ *    KafkaStreaming streaming = new KafkaStreaming(builder, config);
+ *    streaming.start();
  * 
* */ From 6dfa33968157ed7d14158f4333bcc2a93b04904c Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 2 Sep 2015 10:04:46 -0700 Subject: [PATCH 203/275] unnecessary check --- .../main/java/org/apache/kafka/streaming/KafkaStreaming.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java index 4e4d8a3e9cc40..921c1be88288a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java +++ b/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java @@ -71,9 +71,6 @@ public class KafkaStreaming { private final StreamThread[] threads; public KafkaStreaming(TopologyBuilder builder, StreamingConfig config) throws Exception { - if (config.getClass(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG) == null) - throw new NullPointerException("timestamp extractor is missing"); - 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); From c4e92f54a1383c880be754caa997ee7d248b2cb6 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 2 Sep 2015 12:10:42 -0700 Subject: [PATCH 204/275] make the close method work --- .../streaming/processor/internals/StreamThread.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index cd1bcb9c530ac..2f6cf2c98424c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -47,12 +47,13 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; public class StreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(StreamThread.class); - private volatile boolean running; + private final AtomicBoolean running; private final TopologyBuilder builder; private final Producer producer; @@ -119,7 +120,7 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce this.metrics = new KafkaStreamingMetrics(); - this.running = true; + this.running = new AtomicBoolean(true); } /** @@ -141,8 +142,8 @@ public synchronized void run() { /** * Shutdown this streaming thread. */ - public synchronized void close() { - running = false; + public void close() { + running.set(false); } private void shutdown() { @@ -189,7 +190,7 @@ private void runLoop() { } private boolean stillRunning() { - if (!running) { + if (!running.get()) { log.debug("Shutting down at user request."); return false; } From e821386e3003be496c6d4f61f35f7545273cd454 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 2 Sep 2015 12:53:02 -0700 Subject: [PATCH 205/275] remove sync from StreamThread.run() --- .../kafka/streaming/processor/internals/StreamThread.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index 2f6cf2c98424c..c0eb5b1d82554 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -127,7 +127,7 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce * Execute the stream processors */ @Override - public synchronized void run() { + public void run() { log.info("Starting a stream thread"); try { runLoop(); From e01c82296ccd3ad3b082951247086851f76d0837 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 2 Sep 2015 13:23:33 -0700 Subject: [PATCH 206/275] make sure Processor.close() is called at the end of task --- .../processor/internals/ProcessorTopology.java | 12 ++++++------ .../streaming/processor/internals/StreamTask.java | 3 +++ 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java index 9ebbe5a5702d8..867da349e2126 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java @@ -26,14 +26,14 @@ public class ProcessorTopology { - private List processors; + private List processorNodes; private Map sourceByTopics; private Map sinkByTopics; - public ProcessorTopology(List processors, + public ProcessorTopology(List processorNodes, Map sourceByTopics, Map sinkByTopics) { - this.processors = processors; + this.processorNodes = processorNodes; this.sourceByTopics = sourceByTopics; this.sinkByTopics = sinkByTopics; } @@ -67,18 +67,18 @@ public Collection sinks() { * such that parents are always initialized before children */ public void init(ProcessorContext context) { - for (ProcessorNode node : processors) { + for (ProcessorNode node : processorNodes) { node.init(context); } } public final void close() { // close the processors - for (ProcessorNode node : processors) { + for (ProcessorNode node : processorNodes) { node.close(); } - processors.clear(); + processorNodes.clear(); sourceByTopics.clear(); sinkByTopics.clear(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java index 6a5b868fade10..9e8ef35c1f8e8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -55,6 +55,7 @@ public class StreamTask { private final PunctuationQueue punctuationQueue; private final ProcessorContext processorContext; private final TimestampExtractor timestampExtractor; + private final ProcessorTopology topology; private final Map consumedOffsets; private final RecordCollector recordCollector; @@ -87,6 +88,7 @@ public StreamTask(int id, this.punctuationQueue = new PunctuationQueue(); this.maxBufferedSize = config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); this.timestampExtractor = config.getConfiguredInstance(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); + this.topology = topology; // create queues for each assigned partition and associate them // to corresponding source nodes in the processor topology @@ -256,6 +258,7 @@ public void schedule(Processor processor, long interval) { public void close() { this.partitionGroup.close(); this.consumedOffsets.clear(); + this.topology.close(); } private RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { From 998a60bad3b786fa0a52289ce17b82faf234b834 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 2 Sep 2015 14:21:24 -0700 Subject: [PATCH 207/275] no need for recordcollector to know default serializers --- .../streaming/processor/internals/RecordCollector.java | 10 +--------- .../streaming/processor/internals/StreamTask.java | 5 +---- 2 files changed, 2 insertions(+), 13 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollector.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollector.java index 1be19aebcd729..67925b2e0ad82 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollector.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollector.java @@ -45,19 +45,11 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } } }; - private final Serializer keySerializer; - private final Serializer valueSerializer; - public RecordCollector(Producer producer, Serializer keySerializer, Serializer valueSerializer) { + public RecordCollector(Producer producer) { this.producer = producer; this.offsets = new HashMap<>(); - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - } - - public void send(ProducerRecord record) { - send(record, this.keySerializer, this.valueSerializer); } public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java index 9e8ef35c1f8e8..634e0aa8f0d44 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -25,7 +25,6 @@ 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.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; @@ -106,9 +105,7 @@ public StreamTask(int id, this.consumedOffsets = new HashMap<>(); // create the record recordCollector that maintains the produced offsets - this.recordCollector = new RecordCollector(producer, - (Serializer) config.getConfiguredInstance(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class), - (Serializer) config.getConfiguredInstance(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class)); + this.recordCollector = new RecordCollector(producer); // initialize the topology with its own context try { From 6d3010e5770341ab3adb4ebaeb90468ef21de728 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 3 Sep 2015 12:07:13 -0700 Subject: [PATCH 208/275] enable default SerDe, optimize record inserts --- .../kafka/streaming/examples/KStreamJob.java | 14 ++--- .../kafka/streaming/kstream/KStream.java | 7 +++ .../streaming/kstream/KStreamBuilder.java | 16 +++++- .../kstream/internals/KStreamImpl.java | 28 +++++----- .../streaming/processor/TopologyBuilder.java | 9 ++++ .../processor/internals/PartitionGroup.java | 54 +++++++------------ .../processor/internals/RecordQueue.java | 24 +++++++-- .../processor/internals/SinkNode.java | 7 ++- .../processor/internals/SourceNode.java | 18 +++++-- .../processor/internals/StreamTask.java | 37 +++---------- .../processor/internals/StreamThread.java | 2 +- .../processor/internals/StreamTaskTest.java | 5 +- 12 files changed, 121 insertions(+), 100 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java index 4b1b63c975c76..a5af1248b04a9 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java +++ b/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java @@ -38,7 +38,7 @@ public static void main(String[] args) throws Exception { Map props = new HashMap<>(); props.put("bootstrap.servers", "localhost:9092"); props.put("key.serializer", StringSerializer.class); - props.put("value.serializer", StringSerializer.class); + props.put("value.serializer", IntegerSerializer.class); props.put("key.deserializer", StringDeserializer.class); props.put("value.deserializer", StringDeserializer.class); props.put("timestamp.extractor", WallclockTimestampExtractor.class); @@ -46,10 +46,7 @@ public static void main(String[] args) throws Exception { KStreamBuilder builder = new KStreamBuilder(); - StringSerializer stringSerializer = new StringSerializer(); - IntegerSerializer intSerializer = new IntegerSerializer(); - - KStream stream1 = builder.from(new StringDeserializer(), new StringDeserializer(), "topic1"); + KStream stream1 = builder.from("topic1"); KStream stream2 = stream1.map(new KeyValueMapper>() { @@ -68,7 +65,7 @@ public boolean apply(String key, Integer value) { new Predicate() { @Override public boolean apply(String key, Integer value) { - return true; + return (value % 2) == 0; } }, new Predicate() { @@ -79,8 +76,8 @@ public boolean apply(String key, Integer value) { } ); - streams[0].sendTo("topic2", stringSerializer, intSerializer); - streams[1].sendTo("topic3", stringSerializer, intSerializer); + streams[0].sendTo("topic2"); + streams[1].sendTo("topic3"); KafkaStreaming kstream = new KafkaStreaming(builder, config); kstream.start(); @@ -93,5 +90,4 @@ public long extract(String topic, Object key, Object value) { } } - } diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java index d30dce96c7fc1..b86752a4fffec 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java @@ -119,6 +119,13 @@ public interface KStream { */ KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + /** + * Sends key-value to a topic using default serializers specified in the config. + * + * @param topic the topic name + */ + void sendTo(String topic); + /** * Sends key-value to a topic. * diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java index 83777aea9185f..b2707d6dd9bd6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java @@ -30,6 +30,21 @@ public KStreamBuilder() { super(); } + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * The default deserializers specified in the config are used. + * + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(String... topics) { + String name = KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.getAndIncrement(); + + addSource(name, topics); + + return new KStreamImpl<>(this, name); + } + /** * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. * @@ -40,7 +55,6 @@ public KStreamBuilder() { * @param topics the topic names, if empty default to all the topics in the config * @return KStream */ - @SuppressWarnings("unchecked") public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { String name = KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.getAndIncrement(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java index 28f023f8f4a72..d485dcbd97240 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java @@ -76,7 +76,7 @@ public KStreamImpl(TopologyBuilder topology, String name) { public KStream filter(Predicate predicate) { String name = FILTER_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, new KStreamFilter(predicate, false), this.name); + topology.addProcessor(name, new KStreamFilter<>(predicate, false), this.name); return new KStreamImpl<>(topology, name); } @@ -85,7 +85,7 @@ public KStream filter(Predicate predicate) { public KStream filterOut(final Predicate predicate) { String name = FILTER_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, new KStreamFilter(predicate, true), this.name); + topology.addProcessor(name, new KStreamFilter<>(predicate, true), this.name); return new KStreamImpl<>(topology, name); } @@ -94,7 +94,7 @@ public KStream filterOut(final Predicate predicate) { public KStream map(KeyValueMapper> mapper) { String name = MAP_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, new KStreamMap(mapper), this.name); + topology.addProcessor(name, new KStreamMap<>(mapper), this.name); return new KStreamImpl<>(topology, name); } @@ -103,27 +103,25 @@ public KStream map(KeyValueMapper> mappe public KStream mapValues(ValueMapper mapper) { String name = MAPVALUES_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, new KStreamMapValues(mapper), this.name); + topology.addProcessor(name, new KStreamMapValues<>(mapper), this.name); return new KStreamImpl<>(topology, name); } - @SuppressWarnings("unchecked") @Override public KStream flatMap(KeyValueMapper>> mapper) { String name = FLATMAP_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, new KStreamFlatMap(mapper), this.name); + topology.addProcessor(name, new KStreamFlatMap<>(mapper), this.name); return new KStreamImpl<>(topology, name); } - @SuppressWarnings("unchecked") @Override public KStream flatMapValues(ValueMapper> mapper) { String name = FLATMAPVALUES_NAME + INDEX.getAndIncrement(); - topology.addProcessor(name, new KStreamFlatMapValues(mapper), this.name); + topology.addProcessor(name, new KStreamFlatMapValues<>(mapper), this.name); return new KStreamImpl<>(topology, name); } @@ -148,15 +146,14 @@ public KStream[] branch(Predicate... predicates) { for (int i = 0; i < predicates.length; i++) { String childName = BRANCHCHILD_NAME + INDEX.getAndIncrement(); - topology.addProcessor(childName, new KStreamPassThrough(), branchName); + topology.addProcessor(childName, new KStreamPassThrough(), branchName); - branchChildren[i] = new KStreamImpl(topology, childName); + branchChildren[i] = new KStreamImpl<>(topology, childName); } return branchChildren; } - @SuppressWarnings("unchecked") @Override public KStream through(String topic, Serializer keySerializer, @@ -175,14 +172,19 @@ public KStream through(String topic, } @Override - @SuppressWarnings("unchecked") + public void sendTo(String topic) { + String name = SEND_NAME + INDEX.getAndIncrement(); + + topology.addSink(name, topic, this.name); + } + + @Override public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { String name = SEND_NAME + INDEX.getAndIncrement(); topology.addSink(name, topic, keySerializer, valSerializer, this.name); } - @SuppressWarnings("unchecked") @Override public KStream process(final ProcessorDef processorDef) { String name = PROCESSOR_NAME + INDEX.getAndIncrement(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 9e154e375c347..4f7e5598daf6f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -101,6 +101,10 @@ public ProcessorNode build() { public TopologyBuilder() {} + public final void addSource(String name, String... topics) { + addSource(name, (Deserializer) null, (Deserializer) null, topics); + } + public final void addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { if (nodeNames.contains(name)) throw new IllegalArgumentException("Processor " + name + " is already added."); @@ -116,7 +120,12 @@ public final void addSource(String name, Deserializer keyDeserializer, Deseriali nodeFactories.add(new SourceNodeFactory(name, topics, keyDeserializer, valDeserializer)); } + public final void addSink(String name, String topic, String... parentNames) { + addSink(name, topic, (Serializer) null, (Serializer) null, parentNames); + } + public final void addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, String... parentNames) { + if (nodeNames.contains(name)) throw new IllegalArgumentException("Processor " + name + " is already added."); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java index fc5213047ec9d..baeb8f84e683d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java @@ -17,9 +17,10 @@ package org.apache.kafka.streaming.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.common.serialization.Deserializer; +import org.apache.kafka.streaming.processor.TimestampExtractor; import java.util.Comparator; import java.util.Map; @@ -36,10 +37,12 @@ public class PartitionGroup { private final PriorityQueue queuesByTime; + private final TimestampExtractor timestampExtractor; + // since task is thread-safe, we do not need to synchronize on local variables private int totalBuffered; - public PartitionGroup(Map partitionQueues) { + public PartitionGroup(Map partitionQueues, TimestampExtractor timestampExtractor) { this.queuesByTime = new PriorityQueue<>(partitionQueues.size(), new Comparator() { @Override @@ -55,6 +58,8 @@ public int compare(RecordQueue queue1, RecordQueue queue2) { this.partitionQueues = partitionQueues; + this.timestampExtractor = timestampExtractor; + this.totalBuffered = 0; } @@ -90,45 +95,26 @@ public RecordQueue nextQueue() { } /** - * Put a timestamped record associated into its corresponding partition's queues + * Adds raw records to this partition group + * + * @param partition the partition + * @param rawRecords the raw records + * @return the queue size for the partition */ - public void putRecord(StampedRecord record, TopicPartition partition) { - if (record.partition() != partition.partition() || !record.topic().equals(partition.topic())) - throw new KafkaException("The specified partition is different from the record's associated partition."); - + public int addRawRecords(TopicPartition partition, Iterable> rawRecords) { RecordQueue recordQueue = partitionQueues.get(partition); - if (recordQueue == null) - throw new KafkaException("Record's partition does not belong to this partition-group."); - - boolean wasEmpty = recordQueue.isEmpty(); - - recordQueue.add(record); - - totalBuffered++; + 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 (wasEmpty) { + if (oldSize == 0 && newSize > 0) { queuesByTime.offer(recordQueue); } - } - public Deserializer keyDeserializer(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.source().keyDeserializer; - } - - public Deserializer valDeserializer(TopicPartition partition) { - RecordQueue recordQueue = partitionQueues.get(partition); - - if (recordQueue == null) - throw new KafkaException("Record's partition does not belong to this partition-group."); + totalBuffered += newSize - oldSize; - return recordQueue.source().valDeserializer; + return newSize; } public Set partitions() { @@ -147,7 +133,7 @@ public long timestamp() { } } - public int numbuffered(TopicPartition partition) { + public int numBuffered(TopicPartition partition) { RecordQueue recordQueue = partitionQueues.get(partition); if (recordQueue == null) @@ -156,7 +142,7 @@ public int numbuffered(TopicPartition partition) { return recordQueue.size(); } - public int numbuffered() { + public int numBuffered() { return totalBuffered; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java index b628c1688e785..421a39635e087 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streaming.processor.TimestampExtractor; import java.util.ArrayDeque; @@ -63,13 +64,26 @@ public TopicPartition partition() { } /** - * Add a {@link StampedRecord} into the queue + * Add a batch of {@link ConsumerRecord} into the queue * - * @param record StampedRecord + * @param rawRecords the raw records + * @param timestampExtractor TimestampExtractor + * @return the size of this queue */ - public void add(StampedRecord record) { - fifoQueue.addLast(record); - timeTracker.addElement(record); + public int addRawRecords(Iterable> rawRecords, TimestampExtractor timestampExtractor) { + for (ConsumerRecord rawRecord : rawRecords) { + // deserialize the raw record, extract the timestamp and put into the queue + Object key = source.deserializeKey(rawRecord.topic(), rawRecord.key()); + Object value = source.deserializeValue(rawRecord.topic(), rawRecord.value()); + long timestamp = timestampExtractor.extract(rawRecord.topic(), key, value); + + StampedRecord record = new StampedRecord(new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), key, value), timestamp); + + fifoQueue.addLast(record); + timeTracker.addElement(record); + } + + return size(); } /** diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java index 9a6e3b57917b7..fe02914189bfa 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java @@ -24,8 +24,8 @@ public class SinkNode extends ProcessorNode { private final String topic; - private final Serializer keySerializer; - private final Serializer valSerializer; + private Serializer keySerializer; + private Serializer valSerializer; private ProcessorContext context; @@ -42,9 +42,12 @@ public void addChild(ProcessorNode child) { throw new UnsupportedOperationException("sink node does not allow addChild"); } + @SuppressWarnings("unchecked") @Override public void init(ProcessorContext context) { this.context = context; + if (this.keySerializer == null) this.keySerializer = (Serializer) context.keySerializer(); + if (this.valSerializer == null) this.valSerializer = (Serializer) context.valueSerializer(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java index f20afee0d3bef..e56dd002ea03f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java @@ -22,9 +22,8 @@ public class SourceNode extends ProcessorNode { - public Deserializer keyDeserializer; - public Deserializer valDeserializer; - + private Deserializer keyDeserializer; + private Deserializer valDeserializer; private ProcessorContext context; public SourceNode(String name, Deserializer keyDeserializer, Deserializer valDeserializer) { @@ -34,9 +33,22 @@ public SourceNode(String name, Deserializer keyDeserializer, Deserializer this.valDeserializer = valDeserializer; } + public K deserializeKey(String topic, byte[] data) { + return keyDeserializer.deserialize(topic, data); + } + + public V deserializeValue(String topic, byte[] data) { + return valDeserializer.deserialize(topic, data); + } + + @SuppressWarnings("unchecked") @Override public void init(ProcessorContext context) { this.context = context; + + // if serializers are null, get the default ones from the context + if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer) context.keyDeserializer(); + if (this.valDeserializer == null) this.valDeserializer = (Deserializer) context.valueDeserializer(); } @Override diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java index 634e0aa8f0d44..d5d7296efa736 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java @@ -24,7 +24,6 @@ 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.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.Processor; import org.apache.kafka.streaming.processor.ProcessorContext; @@ -35,7 +34,6 @@ import java.io.IOException; import java.util.Collection; import java.util.HashMap; -import java.util.Iterator; import java.util.Map; import java.util.Set; @@ -53,7 +51,6 @@ public class StreamTask { private final PartitionGroup partitionGroup; private final PunctuationQueue punctuationQueue; private final ProcessorContext processorContext; - private final TimestampExtractor timestampExtractor; private final ProcessorTopology topology; private final Map consumedOffsets; @@ -86,7 +83,6 @@ public StreamTask(int id, this.consumer = consumer; this.punctuationQueue = new PunctuationQueue(); this.maxBufferedSize = config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); - this.timestampExtractor = config.getConfiguredInstance(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); this.topology = topology; // create queues for each assigned partition and associate them @@ -99,7 +95,8 @@ public StreamTask(int id, partitionQueues.put(partition, queue); } - this.partitionGroup = new PartitionGroup(partitionQueues); + 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<>(); @@ -129,33 +126,15 @@ public Set partitions() { * Adds records to queues * * @param partition the partition - * @param iterator the iterator of records + * @param records the records */ @SuppressWarnings("unchecked") - public void addRecords(TopicPartition partition, Iterator> iterator) { - - // get deserializers for this partition - Deserializer keyDeserializer = partitionGroup.keyDeserializer(partition); - Deserializer valDeserializer = partitionGroup.valDeserializer(partition); - - while (iterator.hasNext()) { - - ConsumerRecord rawRecord = iterator.next(); - - // deserialize the raw record, extract the timestamp and put into the queue - Object key = keyDeserializer.deserialize(rawRecord.topic(), rawRecord.key()); - Object value = valDeserializer.deserialize(rawRecord.topic(), rawRecord.value()); - long timestamp = timestampExtractor.extract(rawRecord.topic(), key, value); - - StampedRecord stampedRecord = new StampedRecord(new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), key, value), timestamp); - - partitionGroup.putRecord(stampedRecord, partition); - } - + public void addRecords(TopicPartition partition, Iterable> records) { + int queueSize = partitionGroup.addRawRecords(partition, records); // if after adding these records, its partition queue's buffered size has been // increased beyond the threshold, we can then pause the consumption for this partition - if (partitionGroup.numbuffered(partition) > this.maxBufferedSize) { + if (queueSize > this.maxBufferedSize) { consumer.pause(partition); } } @@ -192,7 +171,7 @@ public int process() { // if after processing this record, its partition queue's buffered size has been // decreased to the threshold, we can then resume the consumption on this partition - if (partitionGroup.numbuffered(queue.partition()) == this.maxBufferedSize) { + if (partitionGroup.numBuffered(queue.partition()) == this.maxBufferedSize) { consumer.resume(queue.partition()); } @@ -201,7 +180,7 @@ public int process() { long timestamp = partitionGroup.timestamp(); punctuationQueue.mayPunctuate(timestamp); - return partitionGroup.numbuffered(); + return partitionGroup.numBuffered(); } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index c0eb5b1d82554..98950b5bb84bb 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -170,7 +170,7 @@ private void runLoop() { for (StreamTask task : tasks.values()) { for (TopicPartition partition : task.partitions()) { - task.addRecords(partition, records.records(partition).iterator()); + task.addRecords(partition, records.records(partition)); } } diff --git a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java index c269d47bc2d49..11f55db17a887 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java +++ b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java @@ -36,7 +36,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.Properties; import static org.junit.Assert.assertEquals; @@ -176,7 +175,7 @@ public void testPauseResume() { assertEquals(consumer.paused().size(), 0); } - private Iterator> records(ConsumerRecord... recs) { - return Arrays.asList(recs).iterator(); + private Iterable> records(ConsumerRecord... recs) { + return Arrays.asList(recs); } } From bcc14649dc3393d212aea3b70246f4250e77a8fc Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 3 Sep 2015 14:10:02 -0700 Subject: [PATCH 209/275] logging exceptions --- .../internals/ProcessorTopology.java | 11 ++++- .../processor/internals/StreamThread.java | 45 +++++++++++++++---- 2 files changed, 47 insertions(+), 9 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java index 867da349e2126..51819794ab275 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java @@ -73,13 +73,22 @@ public void init(ProcessorContext context) { } public final void close() { + RuntimeException exception = null; + // close the processors + // make sure close() is called for each node even when there is a RuntimeException for (ProcessorNode node : processorNodes) { - node.close(); + try { + node.close(); + } catch (RuntimeException e) { + exception = e; + } } processorNodes.clear(); sourceByTopics.clear(); sinkByTopics.clear(); + + if (exception != null) throw exception; } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index 98950b5bb84bb..1eb91ee69a045 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -148,11 +148,30 @@ public void close() { private void shutdown() { log.info("Shutting down a stream thread"); - commitAll(time.milliseconds()); - producer.close(); - consumer.close(); - removePartitions(); + // Exceptions should not prevent this call from going through all shutdown steps. + + try { + commitAll(time.milliseconds()); + } catch (Throwable e) { + // already logged in commitAll() + } + try { + producer.close(); + } catch (Throwable e) { + log.error("failed to close producer: ", e); + } + try { + consumer.close(); + } catch (Throwable e) { + log.error("failed to close consumer: ", e); + } + try { + removePartitions(); + } catch (Throwable e) { + // already logged in removePartition() + } + log.info("Stream thread shutdown complete"); } @@ -218,7 +237,12 @@ private void maybeCommit() { */ private void commitAll(long now) { for (StreamTask task : tasks.values()) { - task.commit(); + try { + task.commit(); + } catch (Exception e) { + log.error("failed to commit: ", e); + throw e; + } } metrics.commitTime.record(now - time.milliseconds()); @@ -268,8 +292,12 @@ private void addPartitions(Collection assignment) { partitionsForTask.add(part); // create the task - task = new StreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); - + try { + task = new StreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); + } catch (Exception e) { + log.error("failed to create a task: ", e); + throw e; + } tasks.put(id, task); } } @@ -285,7 +313,8 @@ private void removePartitions() { try { task.close(); } catch (Exception e) { - throw new KafkaException(e); + log.error("failed to close a task: ", e); + throw e; } metrics.processorDestruction.record(); } From 93baa6c8553eed210146c7789b45f8aed9ccab71 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 3 Sep 2015 14:53:44 -0700 Subject: [PATCH 210/275] add thread id and task id to error messages --- .../processor/internals/StreamThread.java | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index 1eb91ee69a045..12ee62ebf2a2e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -48,10 +48,12 @@ import java.util.HashSet; 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 final AtomicInteger nextThreadNumber = new AtomicInteger(1); private final AtomicBoolean running; @@ -88,7 +90,7 @@ public void onPartitionsRevoked(Consumer consumer, Collection assignment) { try { task = new StreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); } catch (Exception e) { - log.error("failed to create a task: ", e); + log.error("failed to create a task #" + id + " in thread [" + this.getName() + "]: ", e); throw e; } tasks.put(id, task); @@ -313,7 +315,7 @@ private void removePartitions() { try { task.close(); } catch (Exception e) { - log.error("failed to close a task: ", e); + log.error("failed to close a task #" + task.id() + " in thread [" + this.getName() + "]: ", e); throw e; } metrics.processorDestruction.record(); From 7cae4d051666d49ec7ce19fa11d9c0311acf3c92 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 3 Sep 2015 14:59:38 -0700 Subject: [PATCH 211/275] capitalize the first letters of messages --- .../streaming/processor/internals/StreamThread.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java index 12ee62ebf2a2e..fbe6ab3513c38 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java @@ -53,7 +53,7 @@ public class StreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(StreamThread.class); - private static final AtomicInteger nextThreadNumber = new AtomicInteger(1); + private static AtomicInteger nextThreadNumber = new AtomicInteger(1); private final AtomicBoolean running; @@ -161,12 +161,12 @@ private void shutdown() { try { producer.close(); } catch (Throwable e) { - log.error("failed to close producer in thread [" + this.getName() + "]: ", 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); + log.error("Failed to close consumer in thread [" + this.getName() + "]: ", e); } try { removePartitions(); @@ -242,7 +242,7 @@ private void commitAll(long now) { try { task.commit(); } catch (Exception e) { - log.error("failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + log.error("Failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); throw e; } } @@ -297,7 +297,7 @@ private void addPartitions(Collection assignment) { try { task = new StreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); } catch (Exception e) { - log.error("failed to create a task #" + id + " in thread [" + this.getName() + "]: ", e); + log.error("Failed to create a task #" + id + " in thread [" + this.getName() + "]: ", e); throw e; } tasks.put(id, task); @@ -315,7 +315,7 @@ private void removePartitions() { try { task.close(); } catch (Exception e) { - log.error("failed to close a task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + log.error("Failed to close a task #" + task.id() + " in thread [" + this.getName() + "]: ", e); throw e; } metrics.processorDestruction.record(); From 03271ff5c12028bb7c0dc0bb5b299d1a46c6a659 Mon Sep 17 00:00:00 2001 From: Randall Hauch Date: Wed, 2 Sep 2015 17:36:52 -0500 Subject: [PATCH 212/275] TopologyBuilder methods return the builder to allow methods to be chained together. Since the methods returned void before, this has no other impact on existing code. --- .../streaming/processor/TopologyBuilder.java | 37 +++++++++++-------- 1 file changed, 21 insertions(+), 16 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index 4f7e5598daf6f..fd081b95fec8c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -17,14 +17,6 @@ package org.apache.kafka.streaming.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.streaming.processor.internals.ProcessorNode; -import org.apache.kafka.streaming.processor.internals.ProcessorTopology; -import org.apache.kafka.streaming.processor.internals.SinkNode; -import org.apache.kafka.streaming.processor.internals.SourceNode; - import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -33,6 +25,14 @@ import java.util.Map; import java.util.Set; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streaming.processor.internals.ProcessorNode; +import org.apache.kafka.streaming.processor.internals.ProcessorTopology; +import org.apache.kafka.streaming.processor.internals.SinkNode; +import org.apache.kafka.streaming.processor.internals.SourceNode; + public class TopologyBuilder { // list of node factories in a topological order @@ -56,6 +56,7 @@ public ProcessorNodeFactory(String name, String[] parents, ProcessorDef definiti this.definition = definition; } + @Override public ProcessorNode build() { Processor processor = definition.instance(); return new ProcessorNode(name, processor); @@ -75,6 +76,7 @@ private SourceNodeFactory(String name, String[] topics, Deserializer keyDeserial this.valDeserializer = valDeserializer; } + @Override public ProcessorNode build() { return new SourceNode(name, keyDeserializer, valDeserializer); } @@ -94,6 +96,7 @@ private SinkNodeFactory(String name, String[] parents, String topic, Serializer this.keySerializer = keySerializer; this.valSerializer = valSerializer; } + @Override public ProcessorNode build() { return new SinkNode(name, topic, keySerializer, valSerializer); } @@ -101,11 +104,11 @@ public ProcessorNode build() { public TopologyBuilder() {} - public final void addSource(String name, String... topics) { - addSource(name, (Deserializer) null, (Deserializer) null, topics); + public final TopologyBuilder addSource(String name, String... topics) { + return addSource(name, (Deserializer) null, (Deserializer) null, topics); } - public final void addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + public final TopologyBuilder addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { if (nodeNames.contains(name)) throw new IllegalArgumentException("Processor " + name + " is already added."); @@ -118,14 +121,14 @@ public final void addSource(String name, Deserializer keyDeserializer, Deseriali nodeNames.add(name); nodeFactories.add(new SourceNodeFactory(name, topics, keyDeserializer, valDeserializer)); + return this; } - public final void addSink(String name, String topic, String... parentNames) { - addSink(name, topic, (Serializer) null, (Serializer) null, parentNames); + public final TopologyBuilder addSink(String name, String topic, String... parentNames) { + return addSink(name, topic, (Serializer) null, (Serializer) null, parentNames); } - public final void addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, String... parentNames) { - + public final TopologyBuilder addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, String... parentNames) { if (nodeNames.contains(name)) throw new IllegalArgumentException("Processor " + name + " is already added."); @@ -142,9 +145,10 @@ public final void addSink(String name, String topic, Serializer keySerializer, S nodeNames.add(name); nodeFactories.add(new SinkNodeFactory(name, parentNames, topic, keySerializer, valSerializer)); + return this; } - public final void addProcessor(String name, ProcessorDef definition, String... parentNames) { + public final TopologyBuilder addProcessor(String name, ProcessorDef definition, String... parentNames) { if (nodeNames.contains(name)) throw new IllegalArgumentException("Processor " + name + " is already added."); @@ -161,6 +165,7 @@ public final void addProcessor(String name, ProcessorDef definition, String... p nodeNames.add(name); nodeFactories.add(new ProcessorNodeFactory(name, parentNames, definition)); + return this; } /** From b9489dca6601b0426ed1eea1e601cfd92c2d7235 Mon Sep 17 00:00:00 2001 From: Randall Hauch Date: Fri, 4 Sep 2015 09:44:34 -0500 Subject: [PATCH 213/275] Added JavaDoc to the public methods in the TopologyBuilder class. --- .../streaming/processor/TopologyBuilder.java | 100 ++++++++++++++++-- 1 file changed, 91 insertions(+), 9 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index fd081b95fec8c..eeb4041e90629 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -17,22 +17,33 @@ package org.apache.kafka.streaming.processor; -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; - import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streaming.KafkaStreaming; +import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.internals.ProcessorNode; import org.apache.kafka.streaming.processor.internals.ProcessorTopology; import org.apache.kafka.streaming.processor.internals.SinkNode; import org.apache.kafka.streaming.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 @@ -102,12 +113,41 @@ public ProcessorNode build() { } } + /** + * 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 IllegalArgumentException("Processor " + name + " is already added."); @@ -124,10 +164,36 @@ public final TopologyBuilder addSource(String name, Deserializer keyDeserializer 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 IllegalArgumentException("Processor " + name + " is already added."); @@ -148,6 +214,15 @@ public final TopologyBuilder addSink(String name, String topic, Serializer keySe 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 IllegalArgumentException("Processor " + name + " is already added."); @@ -169,7 +244,10 @@ public final TopologyBuilder addProcessor(String name, ProcessorDef definition, } /** - * Build the topology by creating the processors + * 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() { @@ -211,6 +289,10 @@ public ProcessorTopology build() { return new ProcessorTopology(processorNodes, topicSourceMap, topicSinkMap); } + /** + * Get the names of topics that are to be consumed by the source nodes created by this builder. + * @return the unmodifiable set of topic names used by source nodes, which changes as new sources are added; never null + */ public Set sourceTopics() { return Collections.unmodifiableSet(sourceTopicNames); } From 0614d90435cf2416d25e9699a240e4b4a3aa98d2 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 4 Sep 2015 12:16:39 -0700 Subject: [PATCH 214/275] fix unused imports --- .../streaming/processor/TopologyBuilder.java | 48 +++++++++---------- 1 file changed, 23 insertions(+), 25 deletions(-) diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java index eeb4041e90629..538e1088fe6b6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java @@ -20,8 +20,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.KafkaStreaming; -import org.apache.kafka.streaming.StreamingConfig; import org.apache.kafka.streaming.processor.internals.ProcessorNode; import org.apache.kafka.streaming.processor.internals.ProcessorTopology; import org.apache.kafka.streaming.processor.internals.SinkNode; @@ -41,8 +39,8 @@ * 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}. + * to construct an acyclic graph of these nodes, and the builder is then passed into a new {@link org.apache.kafka.streaming.KafkaStreaming} instance + * that will then {@link org.apache.kafka.streaming.KafkaStreaming#start() begin consuming, processing, and producing messages}. */ public class TopologyBuilder { @@ -120,10 +118,10 @@ 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}. - * + * The source will use the {@link org.apache.kafka.streaming.StreamingConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} and + * {@link org.apache.kafka.streaming.StreamingConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the + * {@link org.apache.kafka.streaming.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 @@ -136,15 +134,15 @@ public final TopologyBuilder addSource(String name, String... 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} + * should use the {@link org.apache.kafka.streaming.StreamingConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} specified in the + * {@link org.apache.kafka.streaming.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} + * should use the {@link org.apache.kafka.streaming.StreamingConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the + * {@link org.apache.kafka.streaming.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 */ @@ -166,10 +164,10 @@ public final TopologyBuilder addSource(String name, Deserializer keyDeserializer /** * 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}. - * + * The sink will use the {@link org.apache.kafka.streaming.StreamingConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} and + * {@link org.apache.kafka.streaming.StreamingConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * {@link org.apache.kafka.streaming.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 @@ -181,15 +179,15 @@ public final TopologyBuilder addSink(String name, String topic, String... parent /** * 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} + * should use the {@link org.apache.kafka.streaming.StreamingConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} specified in the + * {@link org.apache.kafka.streaming.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} + * should use the {@link org.apache.kafka.streaming.StreamingConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * {@link org.apache.kafka.streaming.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 @@ -245,9 +243,9 @@ public final TopologyBuilder addProcessor(String name, ProcessorDef definition, /** * 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) + * {@link org.apache.kafka.streaming.KafkaStreaming#KafkaStreaming(TopologyBuilder, org.apache.kafka.streaming.StreamingConfig)} constructor. + * + * @see org.apache.kafka.streaming.KafkaStreaming#KafkaStreaming(TopologyBuilder, org.apache.kafka.streaming.StreamingConfig) */ @SuppressWarnings("unchecked") public ProcessorTopology build() { From 74455f29f21776a94af80e2e316ab8a39ac15543 Mon Sep 17 00:00:00 2001 From: Randall Hauch Date: Thu, 3 Sep 2015 14:34:52 -0500 Subject: [PATCH 215/275] Modified the gradle configuration for the project to include the test JAR when uploading artifacts into Maven. Without these minor changes, the stream project's test JAR is never uploaded into Maven. --- build.gradle | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/build.gradle b/build.gradle index 081d79a1012a8..134e5feb7d4a1 100644 --- a/build.gradle +++ b/build.gradle @@ -515,6 +515,14 @@ project(':tools') { dependsOn 'copyDependantLibs' } + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + checkstyle { configFile = new File(rootDir, "checkstyle/checkstyle.xml") } From 0da9a16fb241c4f30e2e08e7a212c01f817278e0 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 9 Sep 2015 10:38:45 -0700 Subject: [PATCH 216/275] fix ProcessorJob; rename streaming to streams --- bin/kafka-run-class.sh | 5 + build.gradle | 6 +- checkstyle/import-control.xml | 4 +- .../kafka/clients/consumer/KafkaConsumer.java | 3 + .../kafka/clients/producer/KafkaProducer.java | 2 + .../kafka/common/config/AbstractConfig.java | 4 + kafka-work.iws | 177 +++++++++++------- settings.gradle | 4 +- .../apache/kafka/streams}/KafkaStreaming.java | 6 +- .../kafka/streams}/StreamingConfig.java | 12 +- .../kafka/streams}/examples/KStreamJob.java | 37 ++-- .../kafka/streams}/examples/ProcessorJob.java | 53 +++--- .../examples/WallclockTimestampExtractor.java | 27 +++ .../kafka/streams}/kstream/KStream.java | 4 +- .../streams}/kstream/KStreamBuilder.java | 6 +- .../streams}/kstream/KStreamWindowed.java | 2 +- .../kafka/streams}/kstream/KeyValue.java | 2 +- .../streams}/kstream/KeyValueMapper.java | 2 +- .../kafka/streams}/kstream/Predicate.java | 2 +- .../streams}/kstream/SlidingWindowDef.java | 16 +- .../kafka/streams}/kstream/ValueJoiner.java | 2 +- .../kafka/streams}/kstream/ValueMapper.java | 2 +- .../apache/kafka/streams}/kstream/Window.java | 6 +- .../kafka/streams}/kstream/WindowDef.java | 2 +- .../kstream/internals/FilteredIterator.java | 2 +- .../kstream/internals/KStreamBranch.java | 8 +- .../kstream/internals/KStreamFilter.java | 8 +- .../kstream/internals/KStreamFlatMap.java | 10 +- .../internals/KStreamFlatMapValues.java | 8 +- .../kstream/internals/KStreamImpl.java | 20 +- .../kstream/internals/KStreamJoin.java | 12 +- .../kstream/internals/KStreamMap.java | 10 +- .../kstream/internals/KStreamMapValues.java | 8 +- .../kstream/internals/KStreamPassThrough.java | 6 +- .../kstream/internals/KStreamProcessor.java | 6 +- .../kstream/internals/KStreamWindow.java | 12 +- .../internals/KStreamWindowedImpl.java | 12 +- .../kstream/internals/WindowSupport.java | 159 ++++++++++++++++ .../kafka/streams}/processor/Processor.java | 2 +- .../streams}/processor/ProcessorContext.java | 2 +- .../streams}/processor/ProcessorDef.java | 2 +- .../kafka/streams}/processor/RestoreFunc.java | 2 +- .../kafka/streams}/processor/StateStore.java | 2 +- .../processor/TimestampExtractor.java | 2 +- .../streams}/processor/TopologyBuilder.java | 48 ++--- .../internals/MinTimestampTracker.java | 12 +- .../processor/internals/PartitionGroup.java | 12 +- .../internals/ProcessorContextImpl.java | 22 ++- .../processor/internals/ProcessorNode.java | 10 +- .../internals/ProcessorStateManager.java | 8 +- .../internals/ProcessorTopology.java | 11 +- .../processor/internals/PunctuationQueue.java | 19 +- .../internals/PunctuationSchedule.java | 12 +- .../processor/internals/RecordCollector.java | 2 +- .../processor/internals/RecordQueue.java | 4 +- .../processor/internals/SinkNode.java | 4 +- .../processor/internals/SourceNode.java | 4 +- .../streams}/processor/internals/Stamped.java | 2 +- .../processor/internals/StampedRecord.java | 7 +- .../processor/internals/StreamTask.java | 29 ++- .../processor/internals/StreamThread.java | 21 ++- .../processor/internals/TimestampTracker.java | 2 +- .../apache/kafka/streams}/state/Entry.java | 2 +- .../streams}/state/InMemoryKeyValueStore.java | 4 +- .../streams}/state/KeyValueIterator.java | 2 +- .../kafka/streams}/state/KeyValueStore.java | 4 +- .../streams}/state/MeteredKeyValueStore.java | 16 +- .../streams}/state/OffsetCheckpoint.java | 2 +- .../streams}/state/RocksDBKeyValueStore.java | 4 +- .../internals/FilteredIteratorTest.java | 2 +- .../kstream/internals/KStreamBranchTest.java | 8 +- .../kstream/internals/KStreamFilterTest.java | 8 +- .../kstream/internals/KStreamFlatMapTest.java | 10 +- .../internals/KStreamFlatMapValuesTest.java | 8 +- .../kstream/internals/KStreamJoinTest.java | 16 +- .../kstream/internals/KStreamMapTest.java | 10 +- .../internals/KStreamMapValuesTest.java | 8 +- .../internals/KStreamWindowedTest.java | 10 +- .../internals/MinTimestampTrackerTest.java | 4 +- .../processor/internals/StreamTaskTest.java | 4 +- .../apache/kafka/test/KStreamTestDriver.java | 6 +- .../kafka/test/MockProcessorContext.java | 9 +- .../apache/kafka/test/MockProcessorDef.java | 6 +- .../org/apache/kafka/test/MockSourceNode.java | 2 +- .../kafka/test/MockTimestampExtractor.java | 2 +- .../apache/kafka/test/UnlimitedWindowDef.java | 12 +- temp | 12 +- temp2 | 12 +- 88 files changed, 709 insertions(+), 382 deletions(-) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/KafkaStreaming.java (96%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/StreamingConfig.java (93%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/examples/KStreamJob.java (69%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/examples/ProcessorJob.java (63%) create mode 100644 streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/KStream.java (98%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/KStreamBuilder.java (93%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/KStreamWindowed.java (97%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/KeyValue.java (96%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/KeyValueMapper.java (95%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/Predicate.java (95%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/SlidingWindowDef.java (94%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/ValueJoiner.java (95%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/ValueMapper.java (95%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/Window.java (87%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/WindowDef.java (95%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/FilteredIterator.java (96%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamBranch.java (88%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamFilter.java (86%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamFlatMap.java (84%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamFlatMapValues.java (87%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamImpl.java (92%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamJoin.java (89%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamMap.java (83%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamMapValues.java (85%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamPassThrough.java (87%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamProcessor.java (87%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamWindow.java (84%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/kstream/internals/KStreamWindowedImpl.java (86%) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/Processor.java (97%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/ProcessorContext.java (98%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/ProcessorDef.java (94%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/RestoreFunc.java (95%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/StateStore.java (97%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/TimestampExtractor.java (96%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/TopologyBuilder.java (83%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/MinTimestampTracker.java (82%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/PartitionGroup.java (90%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/ProcessorContextImpl.java (91%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/ProcessorNode.java (87%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/ProcessorStateManager.java (97%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/ProcessorTopology.java (91%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/PunctuationQueue.java (74%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/PunctuationSchedule.java (74%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/RecordCollector.java (98%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/RecordQueue.java (97%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/SinkNode.java (95%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/SourceNode.java (94%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/Stamped.java (95%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/StampedRecord.java (89%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/StreamTask.java (90%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/StreamThread.java (94%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/processor/internals/TimestampTracker.java (96%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/state/Entry.java (96%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/state/InMemoryKeyValueStore.java (97%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/state/KeyValueIterator.java (95%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/state/KeyValueStore.java (96%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/state/MeteredKeyValueStore.java (95%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/state/OffsetCheckpoint.java (99%) rename {stream/src/main/java/org/apache/kafka/streaming => streams/src/main/java/org/apache/kafka/streams}/state/RocksDBKeyValueStore.java (98%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/kstream/internals/FilteredIteratorTest.java (98%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/kstream/internals/KStreamBranchTest.java (93%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/kstream/internals/KStreamFilterTest.java (95%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/kstream/internals/KStreamFlatMapTest.java (93%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/kstream/internals/KStreamFlatMapValuesTest.java (94%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/kstream/internals/KStreamJoinTest.java (92%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/kstream/internals/KStreamMapTest.java (92%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/kstream/internals/KStreamMapValuesTest.java (91%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/kstream/internals/KStreamWindowedTest.java (91%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/processor/internals/MinTimestampTrackerTest.java (96%) rename {stream/src/test/java/org/apache/kafka/streaming => streams/src/test/java/org/apache/kafka/streams}/processor/internals/StreamTaskTest.java (98%) rename {stream => streams}/src/test/java/org/apache/kafka/test/KStreamTestDriver.java (90%) rename {stream => streams}/src/test/java/org/apache/kafka/test/MockProcessorContext.java (95%) rename {stream => streams}/src/test/java/org/apache/kafka/test/MockProcessorDef.java (90%) rename {stream => streams}/src/test/java/org/apache/kafka/test/MockSourceNode.java (96%) rename {stream => streams}/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java (94%) rename {stream => streams}/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java (89%) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index dd37df4cc0765..d8a111edb1481 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -56,6 +56,11 @@ do CLASSPATH=$CLASSPATH:$file done +for file in $base_dir/stream/build/libs/kafka-streams*.jar; +do + CLASSPATH=$CLASSPATH:$file +done + for file in $base_dir/tools/build/libs/kafka-tools*.jar; do CLASSPATH=$CLASSPATH:$file diff --git a/build.gradle b/build.gradle index 134e5feb7d4a1..3de9101e7cde6 100644 --- a/build.gradle +++ b/build.gradle @@ -529,9 +529,9 @@ project(':tools') { test.dependsOn('checkstyleMain', 'checkstyleTest') } -project(':stream') { +project(':streams') { apply plugin: 'checkstyle' - archivesBaseName = "kafka-stream" + archivesBaseName = "kafka-streams" dependencies { compile project(':clients') @@ -555,7 +555,7 @@ project(':stream') { } javadoc { - include "**/org/apache/kafka/stream/*" + include "**/org/apache/kafka/streams/*" } tasks.create(name: "copyDependantLibs", type: Copy) { diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 12357f069a078..2c5df0bec16e1 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -108,14 +108,14 @@ - + - + diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 19ef6ebead3dc..f99cfb9ee41ae 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.consumer.internals.Fetcher; import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; @@ -546,6 +547,7 @@ private KafkaConsumer(ConsumerConfig config, Deserializer.class); this.keyDeserializer.configure(config.originals(), true); } else { + config.ignore(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG); this.keyDeserializer = keyDeserializer; } if (valueDeserializer == null) { @@ -553,6 +555,7 @@ private KafkaConsumer(ConsumerConfig config, Deserializer.class); this.valueDeserializer.configure(config.originals(), false); } else { + config.ignore(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG); this.valueDeserializer = valueDeserializer; } this.fetcher = new Fetcher(this.client, diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 804d569498396..3a783ec4164a0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -259,6 +259,7 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial Serializer.class); this.keySerializer.configure(config.originals(), true); } else { + config.ignore(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); this.keySerializer = keySerializer; } if (valueSerializer == null) { @@ -266,6 +267,7 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial Serializer.class); this.valueSerializer.configure(config.originals(), false); } else { + config.ignore(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); this.valueSerializer = valueSerializer; } config.logUnused(); diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index 12a19274ffb56..2961e09d7d1f1 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -62,6 +62,10 @@ protected Object get(String key) { return values.get(key); } + public void ignore(String key) { + used.add(key); + } + public Short getShort(String key) { return (Short) get(key); } diff --git a/kafka-work.iws b/kafka-work.iws index 7c61e8a0889e2..70668de07a654 100644 --- a/kafka-work.iws +++ b/kafka-work.iws @@ -2,46 +2,93 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - + + @@ -127,8 +174,8 @@ - - + + @@ -144,11 +191,11 @@ - + - - + + @@ -158,7 +205,7 @@ - + @@ -168,7 +215,7 @@ - + @@ -184,11 +231,11 @@ - + - - + + @@ -281,10 +328,10 @@ @@ -938,7 +985,7 @@ - + @@ -972,8 +1019,8 @@ - - + + @@ -986,42 +1033,42 @@ - + - + - + - + - + - + - + - + - + - - + + diff --git a/settings.gradle b/settings.gradle index 3ff9144eca2fe..357305ba23e8c 100644 --- a/settings.gradle +++ b/settings.gradle @@ -14,5 +14,5 @@ // limitations under the License. apply from: file('scala.gradle') -include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'stream', 'log4j-appender', - 'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file' +include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'streams', 'log4j-appender', + 'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file' \ No newline at end of file diff --git a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java rename to streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java index 921c1be88288a..9dbc52ca0de71 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/KafkaStreaming.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.streaming; +package org.apache.kafka.streams; -import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.processor.internals.StreamThread; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.processor.internals.StreamThread; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java similarity index 93% rename from stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java rename to streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java index cad0b6dfec3bb..3d2a1a8e5f666 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/StreamingConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming; +package org.apache.kafka.streams; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -165,6 +165,11 @@ public Map getConsumerConfigs() { 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; } @@ -174,6 +179,11 @@ public Map getProducerConfigs() { // 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; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java similarity index 69% rename from stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java rename to streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java index a5af1248b04a9..c7c7bbb47f6e6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/KStreamJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java @@ -15,19 +15,18 @@ * limitations under the License. */ -package org.apache.kafka.streaming.examples; +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.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.StreamingConfig; -import org.apache.kafka.streaming.KafkaStreaming; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.kstream.Predicate; -import org.apache.kafka.streaming.processor.TimestampExtractor; +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.HashMap; import java.util.Map; @@ -36,12 +35,12 @@ public class KStreamJob { public static void main(String[] args) throws Exception { Map props = new HashMap<>(); - props.put("bootstrap.servers", "localhost:9092"); - props.put("key.serializer", StringSerializer.class); - props.put("value.serializer", IntegerSerializer.class); - props.put("key.deserializer", StringDeserializer.class); - props.put("value.deserializer", StringDeserializer.class); - props.put("timestamp.extractor", WallclockTimestampExtractor.class); + 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(); @@ -82,12 +81,4 @@ public boolean apply(String key, Integer value) { KafkaStreaming kstream = new KafkaStreaming(builder, config); kstream.start(); } - - public static class WallclockTimestampExtractor implements TimestampExtractor { - @Override - public long extract(String topic, Object key, Object value) { - return System.currentTimeMillis(); - } - } - } diff --git a/stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java similarity index 63% rename from stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java rename to streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java index 459cd2eb1c0cf..fb0a1a0f181df 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/examples/ProcessorJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java @@ -15,32 +15,33 @@ * limitations under the License. */ -package org.apache.kafka.streaming.examples; +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.streaming.KafkaStreaming; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorDef; -import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.StreamingConfig; -import org.apache.kafka.streaming.state.Entry; -import org.apache.kafka.streaming.state.InMemoryKeyValueStore; -import org.apache.kafka.streaming.state.KeyValueIterator; -import org.apache.kafka.streaming.state.KeyValueStore; - -import java.util.Properties; +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.HashMap; +import java.util.Map; public class ProcessorJob { private static class MyProcessorDef implements ProcessorDef { @Override - public Processor instance() { - return new Processor() { + public Processor instance() { + return new Processor() { private ProcessorContext context; private KeyValueStore kvStore; @@ -52,13 +53,13 @@ public void init(ProcessorContext context) { } @Override - public void process(String key, Integer value) { + 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, value); - } else { - int newValue = oldValue + value; this.kvStore.put(key, newValue); + } else { + this.kvStore.put(key, oldValue + newValue); } context.commit(); @@ -86,12 +87,20 @@ public void close() { } public static void main(String[] args) throws Exception { - StreamingConfig config = new StreamingConfig(new Properties()); + Map props = new HashMap<>(); + 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 IntegerDeserializer(), "topic-source"); + builder.addSource("SOURCE", new StringDeserializer(), new StringDeserializer(), "topic-source"); - builder.addProcessor("PROCESS", new MyProcessorDef(), null, "SOURCE"); + builder.addProcessor("PROCESS", new MyProcessorDef(), "SOURCE"); builder.addSink("SINK", "topic-sink", new StringSerializer(), new IntegerSerializer(), "PROCESS"); diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java new file mode 100644 index 0000000000000..8b39946bdf09c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.examples; + +import org.apache.kafka.streams.processor.TimestampExtractor; + +public class WallclockTimestampExtractor implements TimestampExtractor { + @Override + public long extract(String topic, Object key, Object value) { + return System.currentTimeMillis(); + } +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index b86752a4fffec..ca2f809065883 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.processor.ProcessorDef; +import org.apache.kafka.streams.processor.ProcessorDef; /** * KStream is an abstraction of a stream of key-value pairs. diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java similarity index 93% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java index b2707d6dd9bd6..2d4dcc72f8057 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.streaming.kstream.internals.KStreamImpl; -import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streams.kstream.internals.KStreamImpl; +import org.apache.kafka.streams.processor.TopologyBuilder; /** * KStreamBuilder is the class to create KStream instances. diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamWindowed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamWindowed.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java index 07d2f136fde9b..4d73128839f21 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KStreamWindowed.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; /** * KStreamWindowed is an abstraction of a stream of key-value pairs with a window. diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValue.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValue.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValue.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/KeyValue.java index bd67064bec914..f633f6e337532 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValue.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; public class KeyValue { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueMapper.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java index 9624550f7f7ae..62b07f646c812 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/KeyValueMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; public interface KeyValueMapper { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/Predicate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/Predicate.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java index 8c3b6db39c364..9cdb3bc6caa56 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/Predicate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; public interface Predicate { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowDef.java similarity index 94% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowDef.java index 346a67df55f89..51937fb2db330 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindowDef.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowDef.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.ByteArraySerializer; @@ -23,13 +23,13 @@ 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.streaming.kstream.internals.FilteredIterator; -import org.apache.kafka.streaming.kstream.internals.WindowSupport; -import org.apache.kafka.streaming.processor.internals.ProcessorContextImpl; -import org.apache.kafka.streaming.processor.internals.RecordCollector; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RestoreFunc; -import org.apache.kafka.streaming.processor.internals.Stamped; +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.HashMap; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/ValueJoiner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/ValueJoiner.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java index cc89062369529..93fc359bcaa35 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/ValueJoiner.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; public interface ValueJoiner { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/ValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/ValueMapper.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java index 1592d4a09ae62..a32423d04926c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/ValueMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; public interface ValueMapper { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java similarity index 87% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index a968b1782e2a2..a1456f6c421b6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.StateStore; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java index 6ad75528ebf15..bbc5979300b79 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/WindowDef.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream; +package org.apache.kafka.streams.kstream; public interface WindowDef { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/FilteredIterator.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FilteredIterator.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/FilteredIterator.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/FilteredIterator.java index 9ffa04a40d0c0..54d44f01b8723 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/FilteredIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FilteredIterator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java similarity index 88% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java index e1407b1f44b57..473adcd363daf 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamBranch.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorDef; -import org.apache.kafka.streaming.kstream.Predicate; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorDef; +import org.apache.kafka.streams.kstream.Predicate; class KStreamBranch implements ProcessorDef { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java similarity index 86% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java index 8d62f7af85190..181f9f647919e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.kstream.Predicate; -import org.apache.kafka.streaming.processor.ProcessorDef; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.streams.processor.ProcessorDef; class KStreamFilter implements ProcessorDef { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java similarity index 84% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java index d3f13bb659984..c367146c66ab0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorDef; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorDef; class KStreamFlatMap implements ProcessorDef { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java similarity index 87% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java index 241970c4a5323..a56d1d625c34e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.processor.ProcessorDef; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.ProcessorDef; class KStreamFlatMapValues implements ProcessorDef { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java similarity index 92% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index d485dcbd97240..7750568d0db30 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -15,19 +15,19 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +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.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.processor.ProcessorDef; -import org.apache.kafka.streaming.processor.TopologyBuilder; -import org.apache.kafka.streaming.kstream.KStreamWindowed; -import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.kstream.Predicate; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.kstream.WindowDef; +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; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java similarity index 89% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java index 9529f813ac55e..f5d96007007f3 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.kstream.ValueJoiner; -import org.apache.kafka.streaming.kstream.Window; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.ProcessorDef; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.Window; +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; diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java similarity index 83% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java index f6a1a8d9352f8..eff5cfe4f0b57 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.processor.ProcessorDef; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.processor.ProcessorDef; class KStreamMap implements ProcessorDef { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java similarity index 85% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java index dc8267a4fa91d..664acb90352d1 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.processor.ProcessorDef; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.ProcessorDef; class KStreamMapValues implements ProcessorDef { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java similarity index 87% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java index 89c3550998b12..71c2427a9a51a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorDef; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorDef; class KStreamPassThrough implements ProcessorDef { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamProcessor.java similarity index 87% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamProcessor.java index a289893ae1fd0..a128ac6953ea6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamProcessor.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; abstract class KStreamProcessor implements Processor { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java similarity index 84% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java index 980fb4e08e350..e15c82894a254 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.kstream.Window; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorDef; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.kstream.WindowDef; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorDef; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.kstream.WindowDef; public class KStreamWindow implements ProcessorDef { diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java similarity index 86% rename from stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java index 69a108495b0bd..a208af6cca137 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamWindowed; -import org.apache.kafka.streaming.kstream.ValueJoiner; -import org.apache.kafka.streaming.kstream.WindowDef; -import org.apache.kafka.streaming.processor.TopologyBuilder; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamWindowed; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.WindowDef; +import org.apache.kafka.streams.processor.TopologyBuilder; public final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java new file mode 100644 index 0000000000000..b54bcc998c539 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.processor.internals.Stamped; + +import java.util.Iterator; + +public class WindowSupport { + + public static class ValueList { + Value head = null; + Value tail = null; + Value dirty = null; + + public void add(int slotNum, V value, long timestamp) { + Value v = new Value<>(slotNum, value, timestamp); + if (tail != null) { + tail.next = v; + } else { + head = v; + } + tail = v; + if (dirty == null) dirty = v; + } + + public Value first() { + return head; + } + + public void removeFirst() { + if (head != null) { + if (head == tail) tail = null; + head = head.next; + } + } + + public boolean isEmpty() { + return head == null; + } + + public boolean hasDirtyValues() { + return dirty != null; + } + + public void clearDirtyValues() { + dirty = null; + } + + public Iterator> iterator() { + return new ValueListIterator(head); + } + + public Iterator> dirtyValueIterator() { + return new ValueListIterator(dirty); + } + + } + + private static class ValueListIterator implements Iterator> { + + Value ptr; + + ValueListIterator(Value start) { + ptr = start; + } + + @Override + public boolean hasNext() { + return ptr != null; + } + + @Override + public Value next() { + Value value = ptr; + if (value != null) ptr = value.next; + return value; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + } + + public static class Value extends Stamped { + public final int slotNum; + private Value next = null; + + Value(int slotNum, V value, long timestamp) { + super(value, timestamp); + this.slotNum = slotNum; + } + } + + + public static long getLong(byte[] bytes, int offset) { + long value = 0; + for (int i = 0; i < 8; i++) { + value = (value << 8) | bytes[offset + i]; + } + return value; + } + + public static int getInt(byte[] bytes, int offset) { + int value = 0; + for (int i = 0; i < 4; i++) { + value = (value << 8) | bytes[offset + i]; + } + return value; + } + + public static int putLong(byte[] bytes, int offset, long value) { + for (int i = 7; i >= 0; i--) { + bytes[offset + i] = (byte) (value & 0xFF); + value = value >> 8; + } + return 8; + } + + public static int putInt(byte[] bytes, int offset, int value) { + for (int i = 3; i >= 0; i--) { + bytes[offset + i] = (byte) (value & 0xFF); + value = value >> 8; + } + return 4; + } + + public static int puts(byte[] bytes, int offset, byte[] value) { + offset += putInt(bytes, offset, value.length); + System.arraycopy(bytes, offset, value, 0, value.length); + return 4 + value.length; + } + + + public static T deserialize(byte[] bytes, int offset, int length, String topic, Deserializer deserializer) { + byte[] buf = new byte[length]; + System.arraycopy(bytes, offset, buf, 0, length); + return deserializer.deserialize(topic, buf); + } + +} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java rename to streams/src/main/java/org/apache/kafka/streams/processor/Processor.java index 57a618b2649f0..40a6ce4f396eb 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/Processor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java @@ -65,7 +65,7 @@ public interface ProcessorContext { * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streams.processor; public interface Processor { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java rename to streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index 06e219ae45304..aa0b4e1d644c4 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streams.processor; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorDef.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorDef.java similarity index 94% rename from stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorDef.java rename to streams/src/main/java/org/apache/kafka/streams/processor/ProcessorDef.java index 382c95733ebcd..99f029989127c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/ProcessorDef.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorDef.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streams.processor; public interface ProcessorDef { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/RestoreFunc.java b/streams/src/main/java/org/apache/kafka/streams/processor/RestoreFunc.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/processor/RestoreFunc.java rename to streams/src/main/java/org/apache/kafka/streams/processor/RestoreFunc.java index 9de8ab1ae0a37..883147e2b5386 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/RestoreFunc.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/RestoreFunc.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streams.processor; /** * Restoration logic for log-backed state stores upon restart, diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/StateStore.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/streaming/processor/StateStore.java rename to streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java index 752037c88258a..38afe9bacad67 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/StateStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streams.processor; /** * A storage engine for managing state maintained by a stream processor. diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/streaming/processor/TimestampExtractor.java rename to streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java index 5ab69677d8cb1..6bb0c4f4a55ca 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TimestampExtractor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +package org.apache.kafka.streams.processor; /** * An interface that allows the KStream framework to extract a timestamp from a key-value pair diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java similarity index 83% rename from stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java rename to streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index 538e1088fe6b6..39c1b68a53910 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -15,15 +15,17 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor; +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.streaming.processor.internals.ProcessorNode; -import org.apache.kafka.streaming.processor.internals.ProcessorTopology; -import org.apache.kafka.streaming.processor.internals.SinkNode; -import org.apache.kafka.streaming.processor.internals.SourceNode; +import org.apache.kafka.streams.KafkaStreaming; +import org.apache.kafka.streams.StreamingConfig; +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; @@ -39,8 +41,8 @@ * 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 org.apache.kafka.streaming.KafkaStreaming} instance - * that will then {@link org.apache.kafka.streaming.KafkaStreaming#start() begin consuming, processing, and producing messages}. + * 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 { @@ -118,9 +120,9 @@ 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 org.apache.kafka.streaming.StreamingConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} and - * {@link org.apache.kafka.streaming.StreamingConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the - * {@link org.apache.kafka.streaming.StreamingConfig streaming configuration}. + * 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}. @@ -138,11 +140,11 @@ public final TopologyBuilder addSource(String name, String... topics) { * @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 org.apache.kafka.streaming.StreamingConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} specified in the - * {@link org.apache.kafka.streaming.StreamingConfig streaming configuration} + * 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 org.apache.kafka.streaming.StreamingConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the - * {@link org.apache.kafka.streaming.StreamingConfig streaming configuration} + * 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 */ @@ -164,9 +166,9 @@ public final TopologyBuilder addSource(String name, Deserializer keyDeserializer /** * 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 org.apache.kafka.streaming.StreamingConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} and - * {@link org.apache.kafka.streaming.StreamingConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the - * {@link org.apache.kafka.streaming.StreamingConfig streaming configuration}. + * 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 @@ -183,11 +185,11 @@ public final TopologyBuilder addSink(String name, String topic, String... parent * @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 org.apache.kafka.streaming.StreamingConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} specified in the - * {@link org.apache.kafka.streaming.StreamingConfig streaming configuration} + * 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 org.apache.kafka.streaming.StreamingConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the - * {@link org.apache.kafka.streaming.StreamingConfig streaming configuration} + * 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 @@ -243,9 +245,9 @@ public final TopologyBuilder addProcessor(String name, ProcessorDef definition, /** * Build the topology. This is typically called automatically when passing this builder into the - * {@link org.apache.kafka.streaming.KafkaStreaming#KafkaStreaming(TopologyBuilder, org.apache.kafka.streaming.StreamingConfig)} constructor. + * {@link KafkaStreaming#KafkaStreaming(TopologyBuilder, StreamingConfig)} constructor. * - * @see org.apache.kafka.streaming.KafkaStreaming#KafkaStreaming(TopologyBuilder, org.apache.kafka.streaming.StreamingConfig) + * @see KafkaStreaming#KafkaStreaming(TopologyBuilder, StreamingConfig) */ @SuppressWarnings("unchecked") public ProcessorTopology build() { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java similarity index 82% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java index c9bc580ece19b..717df2c54abcd 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; import java.util.LinkedList; @@ -27,6 +27,11 @@ public class MinTimestampTracker implements TimestampTracker { private final LinkedList> descendingSubsequence = new LinkedList<>(); + // in the case that incoming traffic is very small, the records maybe put and polled + // within a single iteration, in this case we need to remember the last polled + // record's timestamp + private long lastKnownTime = NOT_KNOWN; + public void addElement(Stamped elem) { if (elem == null) throw new NullPointerException(); @@ -41,6 +46,9 @@ public void addElement(Stamped elem) { public void removeElement(Stamped elem) { if (elem != null && descendingSubsequence.peekFirst() == elem) descendingSubsequence.removeFirst(); + + if (descendingSubsequence.isEmpty()) + lastKnownTime = elem.timestamp; } public int size() { @@ -51,7 +59,7 @@ public long get() { Stamped stamped = descendingSubsequence.peekFirst(); if (stamped == null) - return TimestampTracker.NOT_KNOWN; + return lastKnownTime; else return stamped.timestamp; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java similarity index 90% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index baeb8f84e683d..bd9f6e284cfae 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +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.streaming.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.TimestampExtractor; import java.util.Comparator; import java.util.Map; @@ -127,7 +127,13 @@ public Set partitions() { */ public long timestamp() { if (queuesByTime.isEmpty()) { - return TimestampTracker.NOT_KNOWN; + // 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(); } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java similarity index 91% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 47fd04552a8d0..108b9ec031b47 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -25,11 +25,11 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.StreamingConfig; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.StateStore; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.RestoreFunc; +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.Processor; +import org.apache.kafka.streams.processor.RestoreFunc; import org.slf4j.Logger; @@ -80,6 +80,8 @@ public ProcessorContextImpl(int id, File stateFile = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG), Integer.toString(id)); + log.info("Creating restoration consumer client for stream task [" + task.id() + "]"); + Consumer restoreConsumer = new KafkaConsumer<>( config.getConsumerConfigs(), null /* no callback for restore consumer */, @@ -254,6 +256,12 @@ public void commit() { @Override public void schedule(Processor processor, long interval) { - task.schedule(processor, interval); + // find the processor node corresponding to this processor + for (ProcessorNode node : task.topology().processors()) { + if (node.processor().equals(processor)) { + task.schedule(node, interval); + break; + } + } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java similarity index 87% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index feee522158011..9127c3f170e63 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; import java.util.ArrayList; import java.util.List; @@ -44,6 +44,10 @@ public final String name() { return name; } + public final Processor processor() { + return processor; + } + public final List> children() { return children; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 810966d83d8d9..42446d280858e 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +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.streaming.processor.RestoreFunc; -import org.apache.kafka.streaming.processor.StateStore; +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.streaming.state.OffsetCheckpoint; +import org.apache.kafka.streams.state.OffsetCheckpoint; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java similarity index 91% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java index 51819794ab275..72b68c89db04b 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/ProcessorTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorContext; import java.util.Collection; import java.util.List; @@ -62,9 +62,12 @@ public Collection sinks() { return sinkByTopics.values(); } + public List processors() { + return processorNodes; + } + /** - * Initialize the processors following the DAG reverse ordering - * such that parents are always initialized before children + * Initialize the processors */ public void init(ProcessorContext context) { for (ProcessorNode node : processorNodes) { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java similarity index 74% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationQueue.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java index 68eb1bdfba7a8..58113f7d0ff43 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.common.KafkaException; import java.util.PriorityQueue; @@ -23,6 +25,12 @@ public class PunctuationQueue { private PriorityQueue pq = new PriorityQueue<>(); + private final StreamTask task; + + public PunctuationQueue(StreamTask task) { + this.task = task; + } + public void schedule(PunctuationSchedule sched) { synchronized (pq) { pq.add(sched); @@ -41,7 +49,14 @@ public void mayPunctuate(long streamTime) { while (top != null && top.timestamp <= streamTime) { PunctuationSchedule sched = top; pq.poll(); - sched.processor().punctuate(streamTime); + + // record the current node + if (task.node() != null) + throw new KafkaException("This should not happen"); + + ProcessorNode scheduled = sched.node(); + task.node(scheduled); + scheduled.processor().punctuate(streamTime); pq.add(sched.next()); top = pq.peek(); diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java similarity index 74% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java index 0d55876d10580..af1df5a7ab1cd 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java @@ -15,20 +15,18 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.streaming.processor.Processor; - -public class PunctuationSchedule extends Stamped { +public class PunctuationSchedule extends Stamped { final long interval; - public PunctuationSchedule(Processor processor, long interval) { - super(processor, System.currentTimeMillis() + interval); + public PunctuationSchedule(ProcessorNode node, long interval) { + super(node, System.currentTimeMillis() + interval); this.interval = interval; } - public Processor processor() { + public ProcessorNode node() { return value; } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollector.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollector.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java index 67925b2e0ad82..ad2f647a2ac3d 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordCollector.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index 421a39635e087..738ef147e11b6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streaming.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.TimestampExtractor; import java.util.ArrayDeque; diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index fe02914189bfa..e2d881c9cdea8 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +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.streaming.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorContext; public class SinkNode extends ProcessorNode { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java similarity index 94% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index e56dd002ea03f..fa4afafbbdf27 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorContext; public class SourceNode extends ProcessorNode { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java index 978aedf53b914..4e44667397d06 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/Stamped.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; public class Stamped implements Comparable { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StampedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java similarity index 89% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/StampedRecord.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java index bce29c2d954b0..febd9389fd44c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StampedRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -44,4 +44,9 @@ public Object value() { public long offset() { return value.offset(); } + + @Override + public String toString() { + return value.toString() + ", timestamp = " + timestamp; + } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java similarity index 90% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index d5d7296efa736..0687b5914bb6a 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.CommitType; import org.apache.kafka.clients.consumer.Consumer; @@ -24,10 +24,9 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.streaming.StreamingConfig; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.TimestampExtractor; +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; @@ -81,7 +80,7 @@ public StreamTask(int id, this.id = id; this.consumer = consumer; - this.punctuationQueue = new PunctuationQueue(); + this.punctuationQueue = new PunctuationQueue(this); this.maxBufferedSize = config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); this.topology = topology; @@ -111,7 +110,8 @@ public StreamTask(int id, throw new KafkaException("Error while creating the state manager in processor context."); } - topology.init(this.processorContext); + // initialize the task by initializing all its processor nodes in the topology + this.topology.init(this.processorContext); } public int id() { @@ -158,8 +158,13 @@ public int process() { // by passing to the source node of the topology this.currRecord = partitionGroup.getRecord(queue); this.currNode = queue.source(); + + 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(queue.partition(), currRecord.offset()); commitOffsetNeeded = true; @@ -196,6 +201,10 @@ public void node(ProcessorNode node) { this.currNode = node; } + public ProcessorTopology topology() { + return this.topology; + } + /** * Commit the current task state */ @@ -224,11 +233,11 @@ public void needCommit() { /** * Schedules a punctuation for the processor * - * @param processor the processor requesting scheduler + * @param node the processor node requesting scheduler * @param interval the interval in milliseconds */ - public void schedule(Processor processor, long interval) { - punctuationQueue.schedule(new PunctuationSchedule(processor, interval)); + public void schedule(ProcessorNode node, long interval) { + punctuationQueue.schedule(new PunctuationSchedule(node, interval)); } public void close() { diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java similarity index 94% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index fbe6ab3513c38..dc12b6fa61efa 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.streaming.StreamingConfig; +import org.apache.kafka.streams.StreamingConfig; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.KafkaException; @@ -38,7 +38,7 @@ 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.streaming.processor.TopologyBuilder; +import org.apache.kafka.streams.processor.TopologyBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,10 +96,14 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce this.builder = builder; // create the producer and consumer clients + log.info("Creating producer client for stream thread [" + this.getName() + "]"); + this.producer = new KafkaProducer<>(config.getProducerConfigs(), new ByteArraySerializer(), new ByteArraySerializer()); + log.info("Creating consumer client for stream thread [" + this.getName() + "]"); + this.consumer = new KafkaConsumer<>(config.getConsumerConfigs(), rebalanceCallback, new ByteArrayDeserializer(), @@ -110,6 +114,7 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce // 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); @@ -130,7 +135,8 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce */ @Override public void run() { - log.info("Starting a stream thread [" + this.getName() + "]"); + log.info("Starting stream thread [" + this.getName() + "]"); + try { runLoop(); } catch (RuntimeException e) { @@ -149,10 +155,9 @@ public void close() { } private void shutdown() { - log.info("Shutting down a stream thread [" + this.getName() + "]"); + log.info("Shutting down stream thread [" + this.getName() + "]"); // Exceptions should not prevent this call from going through all shutdown steps. - try { commitAll(time.milliseconds()); } catch (Throwable e) { @@ -269,8 +274,8 @@ private void maybeClean() { Utils.rm(dir); } } catch (NumberFormatException e) { - log.warn("Deleting unknown directory in state directory {}.", dir.getAbsolutePath()); - Utils.rm(dir); + // there may be some unknown files that sits in the same directory, + // we should ignore these files instead trying to delete them as well } } } diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java index fdd79dc8dfdbc..d8a012a1bdcba 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/TimestampTracker.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; /** * TimestampTracker is a helper class for a sliding window implementation. diff --git a/stream/src/main/java/org/apache/kafka/streaming/state/Entry.java b/streams/src/main/java/org/apache/kafka/streams/state/Entry.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/streaming/state/Entry.java rename to streams/src/main/java/org/apache/kafka/streams/state/Entry.java index 244e07f5b095b..183b691c1acf6 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/state/Entry.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/Entry.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.state; +package org.apache.kafka.streams.state; public class Entry { diff --git a/stream/src/main/java/org/apache/kafka/streaming/state/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java similarity index 97% rename from stream/src/main/java/org/apache/kafka/streaming/state/InMemoryKeyValueStore.java rename to streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java index 9b194e8883b5d..c24f1d07c502b 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/state/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.streaming.state; +package org.apache.kafka.streams.state; -import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; diff --git a/stream/src/main/java/org/apache/kafka/streaming/state/KeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/state/KeyValueIterator.java rename to streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java index 6f5c43a2505a6..0fbd4ae536f1f 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/state/KeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.kafka.streaming.state; +package org.apache.kafka.streams.state; import java.io.Closeable; import java.util.Iterator; diff --git a/stream/src/main/java/org/apache/kafka/streaming/state/KeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java similarity index 96% rename from stream/src/main/java/org/apache/kafka/streaming/state/KeyValueStore.java rename to streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java index 73518170f188a..83fe459863f61 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/state/KeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java @@ -17,9 +17,9 @@ * under the License. */ -package org.apache.kafka.streaming.state; +package org.apache.kafka.streams.state; -import org.apache.kafka.streaming.processor.StateStore; +import org.apache.kafka.streams.processor.StateStore; import java.util.List; diff --git a/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java similarity index 95% rename from stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java rename to streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java index 805de51c8acab..d781fdd7b688b 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/state/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.streaming.state; +package org.apache.kafka.streams.state; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RestoreFunc; +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; @@ -31,8 +31,8 @@ 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.streaming.processor.internals.ProcessorContextImpl; -import org.apache.kafka.streaming.processor.internals.RecordCollector; +import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; +import org.apache.kafka.streams.processor.internals.RecordCollector; import java.util.HashSet; import java.util.List; @@ -66,6 +66,7 @@ public MeteredKeyValueStore(final String name, final KeyValueStore 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"); @@ -74,7 +75,6 @@ public MeteredKeyValueStore(final String name, final KeyValueStore inner, this.rangeTime = createSensor(name, "range"); this.flushTime = createSensor(name, "flush"); this.restoreTime = createSensor(name, "restore"); - this.metrics = context.metrics(); this.topic = name; this.partition = context.id(); @@ -87,8 +87,8 @@ public MeteredKeyValueStore(final String name, final KeyValueStore inner, // register and possibly restore the state from the logs long startNs = time.nanoseconds(); try { - final Deserializer keyDeserializer = (Deserializer) context.keySerializer(); - final Deserializer valDeserializer = (Deserializer) context.valueSerializer(); + final Deserializer keyDeserializer = (Deserializer) context.keyDeserializer(); + final Deserializer valDeserializer = (Deserializer) context.valueDeserializer(); context.register(this, new RestoreFunc() { @Override diff --git a/stream/src/main/java/org/apache/kafka/streaming/state/OffsetCheckpoint.java b/streams/src/main/java/org/apache/kafka/streams/state/OffsetCheckpoint.java similarity index 99% rename from stream/src/main/java/org/apache/kafka/streaming/state/OffsetCheckpoint.java rename to streams/src/main/java/org/apache/kafka/streams/state/OffsetCheckpoint.java index 68bc35bc42f6f..e04de68afde8c 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/state/OffsetCheckpoint.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/OffsetCheckpoint.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.state; +package org.apache.kafka.streams.state; import org.apache.kafka.common.TopicPartition; diff --git a/stream/src/main/java/org/apache/kafka/streaming/state/RocksDBKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java similarity index 98% rename from stream/src/main/java/org/apache/kafka/streaming/state/RocksDBKeyValueStore.java rename to streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java index 2a5e3a0811eef..1ae5aaf18fbd0 100644 --- a/stream/src/main/java/org/apache/kafka/streaming/state/RocksDBKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.kafka.streaming.state; +package org.apache.kafka.streams.state; -import org.apache.kafka.streaming.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.SystemTime; diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FilteredIteratorTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.java rename to streams/src/test/java/org/apache/kafka/streams/kstream/internals/FilteredIteratorTest.java index 6c71742403d1a..405c7c93407e6 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FilteredIteratorTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +package org.apache.kafka.streams.kstream.internals; import static org.junit.Assert.assertEquals; diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java similarity index 93% rename from stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java rename to streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index ad54b14ec6ee4..c18ddfe3ca041 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +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.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.Predicate; +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; diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java similarity index 95% rename from stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java rename to streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index 312eab4f908c8..d381f3d7b009d 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +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.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.Predicate; +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; diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java similarity index 93% rename from stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java rename to streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index 032edca59337b..a66e0ef9aac10 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +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.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; +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; diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java similarity index 94% rename from stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java rename to streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index 9da50917d798e..4822f607f976d 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +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.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.ValueMapper; +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; diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java similarity index 92% rename from stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java rename to streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java index 3374a033b6b2b..7dea8e08863d2 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +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.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.KStreamWindowed; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.kstream.ValueJoiner; -import org.apache.kafka.streaming.kstream.ValueMapper; +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; diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java similarity index 92% rename from stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java rename to streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java index b5bb8c5a364a4..3b4abf9d3f408 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +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.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; +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; diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java similarity index 91% rename from stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.java rename to streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java index f48a03244cf17..6df5496134c95 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +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.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.ValueMapper; +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; diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java similarity index 91% rename from stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.java rename to streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java index dfa8f3fba3c50..48a9fc3d279e1 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.kafka.streaming.kstream.internals; +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.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.Window; -import org.apache.kafka.streaming.kstream.WindowDef; +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; diff --git a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java similarity index 96% rename from stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java rename to streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java index aa00fe3a3ea36..0a1f95c29e0a8 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; import static org.junit.Assert.assertEquals; @@ -85,7 +85,7 @@ public void testTracking() { // remove 100 tracker.removeElement((Stamped) elems[removalIndex++]); - assertEquals(TimestampTracker.NOT_KNOWN, tracker.get()); + assertEquals(100L, tracker.get()); assertEquals(insertionIndex, removalIndex); } diff --git a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java similarity index 98% rename from stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java rename to streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 11f55db17a887..270123edc4678 100644 --- a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streaming.processor.internals; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.MockConsumer; @@ -27,7 +27,7 @@ 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.streaming.StreamingConfig; +import org.apache.kafka.streams.StreamingConfig; import org.apache.kafka.test.MockSourceNode; import org.junit.Test; import org.junit.Before; diff --git a/stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java similarity index 90% rename from stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java rename to streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index 9f95bca83a057..7306ec4a57c60 100644 --- a/stream/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -19,9 +19,9 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.processor.StateStore; -import org.apache.kafka.streaming.processor.internals.ProcessorTopology; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; public class KStreamTestDriver { diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java similarity index 95% rename from stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java rename to streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 8246cdae3d53b..06cb829c9e70a 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -17,6 +17,7 @@ package org.apache.kafka.test; +<<<<<<< HEAD:stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java <<<<<<< HEAD <<<<<<< HEAD <<<<<<< HEAD @@ -89,10 +90,16 @@ import org.apache.kafka.streaming.processor.RestoreFunc; import org.apache.kafka.streaming.processor.StateStore; >>>>>>> Some package and class renaming, fix KafkaSource constrcution at builder +======= +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.RestoreFunc; +import org.apache.kafka.streams.processor.StateStore; +>>>>>>> fix ProcessorJob; rename streaming to streams:streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java 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.streaming.processor.internals.ProcessorNode; +import org.apache.kafka.streams.processor.internals.ProcessorNode; import java.io.File; import java.util.ArrayDeque; diff --git a/stream/src/test/java/org/apache/kafka/test/MockProcessorDef.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java similarity index 90% rename from stream/src/test/java/org/apache/kafka/test/MockProcessorDef.java rename to streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java index 472f91918a48c..918b4683317f9 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockProcessorDef.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java @@ -17,9 +17,9 @@ package org.apache.kafka.test; -import org.apache.kafka.streaming.processor.Processor; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.ProcessorDef; +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; diff --git a/stream/src/test/java/org/apache/kafka/test/MockSourceNode.java b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java similarity index 96% rename from stream/src/test/java/org/apache/kafka/test/MockSourceNode.java rename to streams/src/test/java/org/apache/kafka/test/MockSourceNode.java index 9b6e4b01b6128..dd842ebc447fe 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockSourceNode.java +++ b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.streaming.processor.internals.SourceNode; +import org.apache.kafka.streams.processor.internals.SourceNode; import java.util.ArrayList; import java.util.concurrent.atomic.AtomicInteger; diff --git a/stream/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java similarity index 94% rename from stream/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java rename to streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java index 331f01533edbd..0d854fb1f6d67 100644 --- a/stream/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java +++ b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java @@ -17,7 +17,7 @@ package org.apache.kafka.test; -import org.apache.kafka.streaming.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.TimestampExtractor; public class MockTimestampExtractor implements TimestampExtractor { diff --git a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java similarity index 89% rename from stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java rename to streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java index 4b1b01ffff8db..45eb367ace59c 100644 --- a/stream/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java +++ b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java @@ -17,12 +17,12 @@ package org.apache.kafka.test; -import org.apache.kafka.streaming.kstream.Window; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.WindowDef; -import org.apache.kafka.streaming.kstream.internals.FilteredIterator; -import org.apache.kafka.streaming.processor.internals.Stamped; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.WindowDef; +import org.apache.kafka.streams.kstream.internals.FilteredIterator; +import org.apache.kafka.streams.processor.internals.Stamped; import java.util.Iterator; import java.util.LinkedList; diff --git a/temp b/temp index a5d7e05387188..a743ad0965b15 100644 --- a/temp +++ b/temp @@ -1,6 +1,6 @@ -stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java: needs merge -stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java: needs merge -stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java: needs merge -stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java: needs merge -stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java: needs merge -stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java: needs merge +build.gradle: needs merge +settings.gradle: needs merge +stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java: needs merge +stream/src/main/java/org/apache/kafka/streaming/kstream/internals/WindowSupport.java: needs merge +streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java: needs merge +streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java: needs merge diff --git a/temp2 b/temp2 index 9fb64018d3abe..c4c6bdeea2c36 100644 --- a/temp2 +++ b/temp2 @@ -1,6 +1,6 @@ -stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.java -stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.java -stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.java -stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.java -stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java -stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java +build.gradle +settings.gradle +stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java +stream/src/main/java/org/apache/kafka/streaming/kstream/internals/WindowSupport.java +streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java +streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java From 5f442adb658d46ee12f7f801c5133f83e73cf703 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 9 Sep 2015 12:25:52 -0700 Subject: [PATCH 217/275] remove unused imports --- .../org/apache/kafka/streams/processor/TopologyBuilder.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index 39c1b68a53910..93cb4765937c3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -20,8 +20,6 @@ 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.KafkaStreaming; -import org.apache.kafka.streams.StreamingConfig; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.SinkNode; From a78a9b6f7fbf32070f0b6e29f2643432431b9172 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 9 Sep 2015 13:49:17 -0700 Subject: [PATCH 218/275] address comments --- .../kafka/clients/consumer/MockConsumer.java | 2 +- .../org/apache/kafka/common/utils/Utils.java | 8 +++--- .../apache/kafka/streams/KafkaStreaming.java | 27 ++++++++++--------- .../processor/internals/RecordQueue.java | 3 ++- .../processor/internals/StreamThread.java | 2 +- .../kstream/internals/KStreamFlatMapTest.java | 4 +-- .../internals/KStreamFlatMapValuesTest.java | 4 +-- .../internals/KStreamMapValuesTest.java | 4 +-- .../kafka/test/MockProcessorContext.java | 3 ++- .../apache/kafka/test/MockProcessorDef.java | 5 ++-- .../org/apache/kafka/test/MockSourceNode.java | 5 ++-- .../apache/kafka/test/UnlimitedWindowDef.java | 2 +- 12 files changed, 37 insertions(+), 32 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 2b17fa681e217..5f2ec6d386881 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -223,7 +223,7 @@ public void wakeup() { } public Set paused() { - return paused; + return Collections.unmodifiableSet(new HashSet<>(paused)); } private void ensureNotClosed() { diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index aaf194b76fd0a..107d4fe2b70b2 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -543,14 +543,14 @@ public static HashSet mkSet(T... elems) { * * @param file The root file at which to begin deleting */ - public static void rm(File file) { + 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) - rm(f); + delete(f); } file.delete(); } else { @@ -562,12 +562,12 @@ public static void rm(File file) { ======= /** - * Returns an empty collection if this list is empty + * Returns an empty collection if this list is null * @param other * @return */ public static List safe(List other) { - return other == null ? Collections.EMPTY_LIST : other; + return other == null ? Collections.emptyList() : other; } >>>>>>> Refactor Processor and KStream APIs } diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java index 9dbc52ca0de71..f3392d4ef0fc8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java @@ -22,6 +22,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.List; + /** * 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. @@ -68,12 +71,12 @@ public class KafkaStreaming { private static final int STOPPED = 2; private int state = CREATED; - private final StreamThread[] threads; + private final List 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); + this.threads = new ArrayList<>(config.getInt(StreamingConfig.NUM_STREAM_THREADS_CONFIG)); + for (int i = 0; i < this.threads.size(); i++) { + this.threads.add(new StreamThread(builder, config)); } } @@ -86,13 +89,13 @@ public synchronized void start() { 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."); } - - state = RUNNING; - - log.info("Started Kafka Stream process"); } /** @@ -114,12 +117,12 @@ public synchronized void close() { Thread.interrupted(); } } + + state = STOPPED; + + log.info("Stopped Kafka Stream process"); } else { throw new IllegalStateException("This process has not started yet."); } - - state = STOPPED; - - log.info("Stopped Kafka Stream process"); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index 738ef147e11b6..a4893e75d5c80 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -94,7 +94,8 @@ public int addRawRecords(Iterable> rawRecords, Ti public StampedRecord poll() { StampedRecord elem = fifoQueue.pollFirst(); - if (elem == null) return null; + if (elem == null) + return null; timeTracker.removeElement(elem); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index dc12b6fa61efa..a673db7e1620f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -271,7 +271,7 @@ private void maybeClean() { Integer id = Integer.parseInt(dir.getName()); if (!tasks.containsKey(id)) { log.info("Deleting obsolete state directory {} after delayed {} ms.", dir.getAbsolutePath(), cleanTimeMs); - Utils.rm(dir); + Utils.delete(dir); } } catch (NumberFormatException e) { // there may be some unknown files that sits in the same directory, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index a66e0ef9aac10..a92766112152f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -54,7 +54,7 @@ public Iterable> apply(Integer key, String value) { } }; - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = {0, 1, 2, 3}; <<<<<<< HEAD <<<<<<< HEAD @@ -116,7 +116,7 @@ public Iterable> apply(Integer key, String value) { assertEquals(6, processor.processed.size()); - String[] expected = new String[]{"10:V1", "20:V2", "21:V2", "30:V3", "31:V3", "32:V3"}; + 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)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index 4822f607f976d..a67ea68c8e68a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -52,7 +52,7 @@ public Iterable apply(String value) { } }; - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = {0, 1, 2, 3}; <<<<<<< HEAD <<<<<<< HEAD @@ -95,7 +95,7 @@ public Iterable apply(String value) { assertEquals(8, processor.processed.size()); - String[] expected = new String[]{"0:v0", "0:V0", "1:v1", "1:V1", "2:v2", "2:V2", "3:v3", "3:V3"}; + 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)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java index 6df5496134c95..b6507fe2f4dd6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java @@ -47,7 +47,7 @@ public Integer apply(String value) { } }; - final int[] expectedKeys = new int[]{1, 10, 100, 1000}; + final int[] expectedKeys = {1, 10, 100, 1000}; KStream stream; MockProcessorDef processor = new MockProcessorDef<>(); @@ -61,7 +61,7 @@ public Integer apply(String value) { assertEquals(4, processor.processed.size()); - String[] expected = new String[]{"1:1", "10:2", "100:3", "1000:4"}; + String[] expected = {"1:1", "10:2", "100:3", "1000:4"}; for (int i = 0; i < expected.length; i++) { assertEquals(expected[i], processor.processed.get(i)); diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 06cb829c9e70a..09e50368956e1 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -103,6 +103,7 @@ import java.io.File; import java.util.ArrayDeque; +import java.util.Deque; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -214,7 +215,7 @@ public void schedule(Processor processor, long interval) { private Serializer serializer; private Deserializer deserializer; private ProcessorNode node; - private final ArrayDeque nodeStack = new ArrayDeque(); + private final Deque nodeStack = new ArrayDeque(); private Map storeMap = new HashMap<>(); >>>>>>> kstream test fix diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java index 918b4683317f9..64f7080801c80 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java @@ -22,11 +22,12 @@ import org.apache.kafka.streams.processor.ProcessorDef; import java.util.ArrayList; +import java.util.List; public class MockProcessorDef implements ProcessorDef { - public final ArrayList processed = new ArrayList<>(); - public final ArrayList punctuated = new ArrayList<>(); + public final List processed = new ArrayList<>(); + public final List punctuated = new ArrayList<>(); public Processor instance() { return new MockProcessor(); diff --git a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java index dd842ebc447fe..cf0202e187554 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java +++ b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java @@ -27,12 +27,11 @@ public class MockSourceNode extends SourceNode { public static final String NAME = "MOCK-SOURCE-"; - public static final AtomicInteger INDEX = new AtomicInteger(1); public int numReceived = 0; - public ArrayList keys = new ArrayList<>(); - public ArrayList values = new ArrayList<>(); + public final ArrayList keys = new ArrayList<>(); + public final ArrayList values = new ArrayList<>(); public MockSourceNode(Deserializer keyDeserializer, Deserializer valDeserializer) { super(NAME + INDEX.getAndIncrement(), keyDeserializer, valDeserializer); diff --git a/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java index 45eb367ace59c..b5a3b3cb9915d 100644 --- a/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java +++ b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java @@ -45,7 +45,7 @@ public Window instance() { public class UnlimitedWindow implements Window { - private LinkedList>> list = new LinkedList<>(); + private final LinkedList>> list = new LinkedList<>(); @Override public void init(ProcessorContext context) { From ef0ff2e3af3f569040f157a4ca653dbba1cff1ca Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 9 Sep 2015 14:14:55 -0700 Subject: [PATCH 219/275] revert some changes --- .../java/org/apache/kafka/streams/KafkaStreaming.java | 11 ++++------- .../org/apache/kafka/test/MockProcessorContext.java | 3 +-- .../java/org/apache/kafka/test/MockProcessorDef.java | 5 ++--- 3 files changed, 7 insertions(+), 12 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java index f3392d4ef0fc8..b943752081967 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java @@ -22,9 +22,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.List; - /** * 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. @@ -71,12 +68,12 @@ public class KafkaStreaming { private static final int STOPPED = 2; private int state = CREATED; - private final List threads; + private final StreamThread[] threads; public KafkaStreaming(TopologyBuilder builder, StreamingConfig config) throws Exception { - this.threads = new ArrayList<>(config.getInt(StreamingConfig.NUM_STREAM_THREADS_CONFIG)); - for (int i = 0; i < this.threads.size(); i++) { - this.threads.add(new StreamThread(builder, config)); + 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); } } diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 09e50368956e1..06cb829c9e70a 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -103,7 +103,6 @@ import java.io.File; import java.util.ArrayDeque; -import java.util.Deque; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -215,7 +214,7 @@ public void schedule(Processor processor, long interval) { private Serializer serializer; private Deserializer deserializer; private ProcessorNode node; - private final Deque nodeStack = new ArrayDeque(); + private final ArrayDeque nodeStack = new ArrayDeque(); private Map storeMap = new HashMap<>(); >>>>>>> kstream test fix diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java index 64f7080801c80..918b4683317f9 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java @@ -22,12 +22,11 @@ import org.apache.kafka.streams.processor.ProcessorDef; import java.util.ArrayList; -import java.util.List; public class MockProcessorDef implements ProcessorDef { - public final List processed = new ArrayList<>(); - public final List punctuated = new ArrayList<>(); + public final ArrayList processed = new ArrayList<>(); + public final ArrayList punctuated = new ArrayList<>(); public Processor instance() { return new MockProcessor(); From d7501e05bba2f93322808eda3636c4199218cab2 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 9 Sep 2015 16:34:18 -0700 Subject: [PATCH 220/275] rebased on trunk --- .../org/apache/kafka/common/utils/Utils.java | 7 +- kafka-work.iws | 507 ++++++++---- .../apache/kafka/stream/KStreamContext.java | 252 ------ .../apache/kafka/stream/KafkaStreaming.java | 493 ------------ .../kafka/stream/examples/MapKStreamJob.java | 118 --- .../stream/examples/PrintKStreamJob.java | 102 --- .../stream/examples/StatefulKStreamJob.java | 147 ---- .../stream/internal/KStreamContextImpl.java | 477 ----------- .../kafka/stream/internal/ProcessorNode.java | 51 -- .../kafka/stream/internal/Receiver.java | 27 - .../kafka/stream/internal/RecordQueue.java | 114 --- .../kafka/stream/internal/StreamGroup.java | 311 -------- .../kafka/stream/internals/StreamGroup.java | 284 ------- .../kafka/stream/topology/SlidingWindow.java | 295 ------- .../topology/internal/KStreamBranch.java | 131 --- .../topology/internal/KStreamFilter.java | 55 -- .../topology/internal/KStreamFlatMap.java | 69 -- .../internal/KStreamFlatMapValues.java | 56 -- .../stream/topology/internal/KStreamImpl.java | 325 -------- .../stream/topology/internal/KStreamJoin.java | 153 ---- .../stream/topology/internal/KStreamMap.java | 67 -- .../topology/internal/KStreamMapValues.java | 54 -- .../topology/internal/KStreamSource.java | 130 --- .../internal/KStreamWindowedImpl.java | 97 --- .../topology/internals/KStreamThread.java | 335 -------- .../topology/internals/WindowSupport.java | 159 ---- .../kstream/internals/SlidingWindow.java | 147 ---- .../processor/internals/StreamGroup.java | 285 ------- .../kafka/streaming/KStreamBranchTest.java | 182 ----- .../kafka/streaming/KStreamJoinTest.java | 744 ------------------ .../kafka/streaming/KStreamMapValuesTest.java | 150 ---- .../kafka/streaming/KStreamWindowedTest.java | 120 --- .../kstream/internals/KStreamSourceTest.java | 84 -- .../processor/internals/StreamGroupTest.java | 204 ----- .../kafka/streams/processor/Processor.java | 51 -- .../internals/ProcessorContextImpl.java | 1 - .../internals/ProcessorStateManager.java | 7 +- .../processor/internals/StreamThread.java | 10 +- .../kstream/internals/KStreamFilterTest.java | 51 +- 39 files changed, 349 insertions(+), 6503 deletions(-) delete mode 100644 stream/src/main/java/org/apache/kafka/stream/KStreamContext.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java delete mode 100644 stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/kstream/internals/SlidingWindow.java delete mode 100644 stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java delete mode 100644 stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamGroupTest.java diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 107d4fe2b70b2..fa7c92f08502c 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -512,7 +512,6 @@ public static String readFileAsString(String path) throws IOException { } /** -<<<<<<< HEAD * Check if the given ByteBuffer capacity * @param existingBuffer ByteBuffer capacity to check * @param newLength new length for the ByteBuffer. @@ -528,7 +527,7 @@ public static ByteBuffer ensureCapacity(ByteBuffer existingBuffer, int newLength return existingBuffer; } -======= + /* * Creates a set * @param elems the elements * @param the type of element @@ -557,9 +556,6 @@ public static void delete(File file) { file.delete(); } } -<<<<<<< HEAD ->>>>>>> removing io.confluent imports: wip -======= /** * Returns an empty collection if this list is null @@ -569,5 +565,4 @@ public static void delete(File file) { public static List safe(List other) { return other == null ? Collections.emptyList() : other; } ->>>>>>> Refactor Processor and KStream APIs } diff --git a/kafka-work.iws b/kafka-work.iws index 70668de07a654..8f4e45d4be93f 100644 --- a/kafka-work.iws +++ b/kafka-work.iws @@ -2,93 +2,45 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @@ -171,91 +123,114 @@ - - + + - - - + + + + + - - + + - - - + + + + + - - + + - - + + - - + + - - + + + + + - - + + - - + + - - + + - + - - + + - - + + + + + - - + + - - + + + + + + + + + + + + + + + - - + + - + @@ -281,11 +256,6 @@ @@ -343,10 +318,10 @@ - @@ -380,6 +355,7 @@ + @@ -412,6 +388,54 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + - @@ -843,32 +866,32 @@ - + - - - - + + - + + + @@ -920,10 +943,74 @@ - + - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @@ -942,7 +1029,6 @@ - @@ -950,7 +1036,6 @@ - @@ -958,7 +1043,6 @@ - @@ -966,9 +1050,6 @@ - - - @@ -976,7 +1057,6 @@ - @@ -984,9 +1064,7 @@ - - - + @@ -1011,23 +1089,15 @@ - + - - - - - - - - - + @@ -1036,7 +1106,7 @@ - + @@ -1057,18 +1127,119 @@ + + + + + + + + + + + + + + + + + + + + + + + + - + + + + + + + + + + + - + - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java b/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java deleted file mode 100644 index ec5aefff170d4..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/KStreamContext.java +++ /dev/null @@ -1,252 +0,0 @@ -<<<<<<< HEAD -package org.apache.kafka.stream; - -import io.confluent.streaming.kv.internals.RestoreFunc; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -======= -/** - * 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.stream; - -import org.apache.kafka.clients.processor.RecordCollector; -import org.apache.kafka.clients.processor.RestoreFunc; -import org.apache.kafka.clients.processor.StateStore; -import org.apache.kafka.clients.processor.internals.StreamingConfig; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -<<<<<<< HEAD -import org.apache.kafka.stream.topology.Processor; ->>>>>>> added missing files -======= ->>>>>>> wip - -import java.io.File; -import java.util.Map; - -/** - * KStreamContext is access to the system resources for a stream processing job. - * An instance of KStreamContext is created for each partition group. - */ -public interface KStreamContext { - -<<<<<<< HEAD - /** - * 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 a RecordCollector - * @return RecordCollector - */ - RecordCollector recordCollector(); - - /** - * Returns an application context registered to {@link StreamingConfig}. - * @return an application context - */ - Map getContext(); - - /** - * Returns the state directory for the partition. - * @return the state directory - */ - File stateDir(); - - /** - * Returns Metrics instance - * @return Metrics - */ - Metrics metrics(); - - /** - * Registers and possibly restores the specified storage engine. - * @param store the storage engine - */ -<<<<<<< HEAD - void restore(StateStore store, RestoreFunc restoreFunc); - - /** -<<<<<<< HEAD - * Registers the specified storage enging. - * @param store the storage engine - */ - void register(StateStore store); -======= - void register(StateStore store, RestoreFunc restoreFunc); ->>>>>>> remove restore function - - /** - * Ensures that the context is in the initialization phase where KStream topology can be constructed -<<<<<<< HEAD -======= - * Flush the local state of this context ->>>>>>> new api model -======= - * - * Flush the local state of this context ->>>>>>> new api model - */ - void flush(); - - - void send(String topic, Object key, Object value); - - void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); - -<<<<<<< HEAD -<<<<<<< HEAD - void schedule(Processor processor, long interval); -======= - PunctuationScheduler getPunctuationScheduler(Processor processor); ->>>>>>> new api model -======= - void schedule(Processor processor, long interval); ->>>>>>> removed ProcessorContext - - void commit(); - - String topic(); - - int partition(); - - long offset(); - - long timestamp(); -======= - /** - * 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 a RecordCollector - * - * @return RecordCollector - */ - RecordCollector recordCollector(); - - /** - * Returns an application context registered to {@link StreamingConfig}. - * - * @return an application context - */ - Map getContext(); - - /** - * Returns the state directory for the partition. - * - * @return the state directory - */ - File stateDir(); - - /** - * Returns Metrics instance - * - * @return Metrics - */ - Metrics metrics(); - - /** - * Registers and possibly restores the specified storage engine. - * - * @param store the storage engine - */ - void register(StateStore store, RestoreFunc restoreFunc); - - /** - * Flush the local state of this context - */ - void flush(); - - void send(String topic, Object key, Object value); - - void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); - - void schedule(Processor processor, long interval); - - void commit(); - - String topic(); - - int partition(); - - long offset(); - - long timestamp(); ->>>>>>> added missing files - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java b/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java deleted file mode 100644 index a49efc4109ddc..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/KafkaStreaming.java +++ /dev/null @@ -1,493 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream; - -<<<<<<< HEAD -import io.confluent.streaming.internal.KStreamThread; -import io.confluent.streaming.internal.ProcessorConfig; -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.internal.IngestorImpl; -<<<<<<< HEAD -import io.confluent.streaming.internal.StreamSynchronizer; -<<<<<<< HEAD -======= -import io.confluent.streaming.internal.StreamGroup; ->>>>>>> remove SyncGroup from user facing APIs -import io.confluent.streaming.util.ParallelExecutor; -======= ->>>>>>> removed some generics -import io.confluent.streaming.util.Util; -import org.apache.kafka.clients.consumer.*; -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.errors.InterruptException; -======= ->>>>>>> added KStreamThread -======= -import io.confluent.streaming.internal.TopologyAnalyzer; ->>>>>>> new api model -======= ->>>>>>> wip -======= -import io.confluent.streaming.internal.TopologyAnalyzer; ->>>>>>> new api model -======= ->>>>>>> wip -======= ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.stream.internals.ProcessorConfig; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.internals.KStreamThread; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Set; - -/** - * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and - * sends output to zero or more output topics. - *

- * This processing is defined by extending the {@link KStreamTopology} abstract class to specify the transformation operator build. The - * {@link KafkaStreaming} instance will be responsible for the lifecycle of these processors. It will instantiate and - * start one or more of these processors to process the Kafka partitions assigned to this particular instance. - *

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

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

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

- *    Properties props = new Properties();
- *    props.put("bootstrap.servers", "localhost:4242");
- *    StreamingConfig config = new StreamingConfig(props);
- *    config.processor(ExampleStreamProcessor.class);
- *    config.serialization(new StringSerializer(), new StringDeserializer());
- *    KafkaStreaming container = new KafkaStreaming(new MyKStreamTopology(), config);
- *    container.run();
- * 
- * - */ -public class KafkaStreaming implements Runnable { - - private static final Logger log = LoggerFactory.getLogger(KafkaStreaming.class); - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - private final Class jobClass; - private final Set topics; -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - private final Map> syncGroups = new HashMap<>(); - private final ArrayList> streamSynchronizers = new ArrayList<>(); -======= - private final Map> streamSynchronizersForPartition = new HashMap<>(); -======= ->>>>>>> remove unused member variable - private final ArrayList streamGroups = new ArrayList<>(); ->>>>>>> remove SyncGroup from user facing APIs - private final ParallelExecutor parallelExecutor; -======= - private final Map> streamSynchronizersForPartition = new HashMap<>(); ->>>>>>> removed some generics - private final Map kstreamContexts = new HashMap<>(); - protected final Producer producer; - protected final Consumer consumer; - private final IngestorImpl ingestor; - private final StreamingConfig streamingConfig; -======= ->>>>>>> added KStreamThread -======= - // - // Container State Transition - // - // run() startShutdown() shutdown() - // CREATED --------> RUNNING ----------------> STOPPING -----------> STOPPED - // | ^ - // | startShutdown() | - // +--------------------------------------------+ - // -======= - // Container States -<<<<<<< HEAD ->>>>>>> fixed comments - private final int CREATED = 0; - private final int RUNNING = 1; - private final int STOPPING = 2; - private final int STOPPED = 3; -======= - private static final int CREATED = 0; - private static final int RUNNING = 1; - private static final int STOPPING = 2; - private static final int STOPPED = 3; ->>>>>>> compile and test passed - private int state = CREATED; - ->>>>>>> removed Coordinator - private final ProcessorConfig config; - private final Object lock = new Object(); - private final KStreamThread[] threads; - private final Set topics; - - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - @SuppressWarnings("unchecked") - protected KafkaStreaming(Class jobClass, - StreamingConfig config, - Producer producer, - Consumer consumer) { - this.jobClass = jobClass; - this.producer = producer == null? new KafkaProducer<>(config.config(), new ByteArraySerializer(), new ByteArraySerializer()): producer; - this.consumer = consumer == null? new KafkaConsumer<>(config.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()): consumer; - this.streamingConfig = config; - this.metrics = new Metrics(); - this.streamingMetrics = new KafkaStreamingMetrics(); - this.config = new ProcessorConfig(config.config()); -<<<<<<< HEAD -<<<<<<< HEAD - this.ingestor = -<<<<<<< HEAD - new IngestorImpl<>(this.consumer, - (Deserializer) config.keyDeserializer(), - (Deserializer) config.valueDeserializer(), - this.config.pollTimeMs); -======= - new IngestorImpl(this.consumer, - (Deserializer) config.keyDeserializer(), - (Deserializer) config.valueDeserializer(), - this.config.pollTimeMs); ->>>>>>> removed some generics -======= - this.ingestor = new IngestorImpl(this.consumer, this.config.pollTimeMs); ->>>>>>> clean up ingestor and stream synchronizer -======= - this.ingestor = new IngestorImpl(this.consumer); ->>>>>>> use poll(0) for non-blocking poll - this.running = true; - this.lastCommit = 0; - this.nextStateCleaning = Long.MAX_VALUE; - this.recordsProcessed = 0; - this.time = new SystemTime(); - this.parallelExecutor = new ParallelExecutor(this.config.numStreamThreads); -======= - public KafkaStreaming(Class jobClass, StreamingConfig streamingConfig) { ->>>>>>> added KStreamThread -======= - public KafkaStreaming(KStreamTopology topology, StreamingConfig streamingConfig) { ->>>>>>> wip -======= - public KafkaStreaming(KStreamTopology topology, StreamingConfig streamingConfig) { ->>>>>>> wip - - if (streamingConfig.timestampExtractor() == null) - throw new NullPointerException("timestamp extractor is missing"); - - this.config = new ProcessorConfig(streamingConfig.config()); -<<<<<<< HEAD -<<<<<<< HEAD - this.topics = topology.topics(); - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { -======= - for (StreamGroup streamGroup : streamGroups) { ->>>>>>> remove SyncGroup from user facing APIs - try { - streamGroup.close(); - } - catch(Exception e) { - log.error("Error while closing stream synchronizers: ", e); -======= - for (Map.Entry> entry : streamSynchronizersForPartition.entrySet()) { - for (StreamSynchronizer streamSynchronizer : entry.getValue()) { - try { - streamSynchronizer.close(); - } - catch(Exception e) { - log.error("Error while closing stream synchronizers: ", e); - } ->>>>>>> removed some generics -======= - Coordinator coordinator = new Coordinator() { - @Override - public void commit() { - throw new UnsupportedOperationException(); - } -======= - - TopologyAnalyzer topologyAnalyzer = new TopologyAnalyzer(jobClass); - - this.topics = topologyAnalyzer.topics; ->>>>>>> new api model - - @Override - public void shutdown() { - startShutdown(); ->>>>>>> added KStreamThread - } - }; -======= - this.topics = topology.topics(); ->>>>>>> wip - -<<<<<<< HEAD - producer.close(); - consumer.close(); - parallelExecutor.shutdown(); -<<<<<<< HEAD -<<<<<<< HEAD - syncGroups.clear(); - streamSynchronizers.clear(); -======= - streamSynchronizersForPartition.clear(); -======= ->>>>>>> remove unused member variable - streamGroups.clear(); ->>>>>>> remove SyncGroup from user facing APIs - shutdownComplete.countDown(); - log.info("Shut down complete"); -======= -======= ->>>>>>> removed Coordinator - Metrics metrics = new Metrics(); - - // TODO: Fix this after the threading model is decided (also fix KStreamThread) - this.threads = new KStreamThread[1]; -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - threads[0] = new KStreamThread(jobClass, topics, streamingConfig, coordinator, metrics); ->>>>>>> added KStreamThread -======= - threads[0] = new KStreamThread(jobClass, topics, streamingConfig, metrics); ->>>>>>> removed Coordinator -======= - threads[0] = new KStreamThread(topology, topics, streamingConfig, metrics); ->>>>>>> wip -======= - threads[0] = new KStreamThread(topology, topics, streamingConfig, metrics); ->>>>>>> wip - } - - /** - * Execute the stream processors - */ -<<<<<<< HEAD - public synchronized void close() { - running = false; - try { - shutdownComplete.await(); - } catch (InterruptedException e) { - throw new InterruptException(e); - } - } - - private void runLoop() { - try { - boolean readyForNextExecution = false; - - while (stillRunning()) { - ingestor.poll(readyForNextExecution ? 0 : this.config.pollTimeMs); - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - parallelExecutor.execute(streamSynchronizers, status); -======= - parallelExecutor.execute(streamSynchronizers); ->>>>>>> clean up ingestor and stream synchronizer -======= - parallelExecutor.execute(streamGroups); ->>>>>>> remove SyncGroup from user facing APIs -======= - readyForNextExecution = parallelExecutor.execute(streamGroups); ->>>>>>> use poll(0) for non-blocking poll - -======= - for (Map.Entry> entry : streamSynchronizersForPartition.entrySet()) { - for (StreamSynchronizer streamSynchronizer : entry.getValue()) { - streamSynchronizer.process(); - pollRequired = pollRequired || streamSynchronizer.requiresPoll(); - } - } ->>>>>>> removed some generics - maybeCommit(); - maybeCleanState(); -======= - public void run() { - synchronized (lock) { - log.info("Starting container"); - if (state == CREATED) { - if (!config.stateDir.exists() && !config.stateDir.mkdirs()) - throw new IllegalArgumentException("Failed to create state directory: " + config.stateDir.getAbsolutePath()); - - for (KStreamThread thread : threads) thread.start(); - log.info("Start-up complete"); - } else { - throw new IllegalStateException("This container was already started"); ->>>>>>> added KStreamThread - } - -<<<<<<< HEAD -<<<<<<< HEAD - private void commitAll(long now) { - Map commit = new HashMap<>(); - for (KStreamContextImpl context : kstreamContexts.values()) { - context.flush(); - // check co-ordinator - } -<<<<<<< HEAD -<<<<<<< HEAD - for (StreamSynchronizer streamSynchronizer : streamSynchronizers) { -======= - for (StreamGroup streamGroup : streamGroups) { ->>>>>>> remove SyncGroup from user facing APIs - try { - commit.putAll(streamGroup.consumedOffsets()); - } - catch(Exception e) { - log.error("Error while closing processor: ", e); -======= - for (Map.Entry> entry : streamSynchronizersForPartition.entrySet()) { - for (StreamSynchronizer streamSynchronizer : entry.getValue()) { - try { - commit.putAll(streamSynchronizer.consumedOffsets()); - } - catch(Exception e) { - log.error("Error while closing processor: ", e); - } ->>>>>>> removed some generics - } - } - - // check if commit is really needed, i.e. if all the offsets are already committed - boolean commitNeeded = false; - for (TopicPartition tp : commit.keySet()) { - if (consumer.committed(tp) != commit.get(tp)) { - commitNeeded = true; - break; -======= - while (!stopping) { -======= - state = RUNNING; - while (state == RUNNING) { ->>>>>>> removed Coordinator - try { - lock.wait(); - } catch (InterruptedException ex) { - Thread.interrupted(); - } ->>>>>>> added KStreamThread - } - - if (state == STOPPING) { - log.info("Shutting down the container"); - - for (KStreamThread thread : threads) - thread.close(); - - for (KStreamThread thread : threads) { - try { - thread.join(); - } catch (InterruptedException ex) { - Thread.interrupted(); - } -<<<<<<< HEAD - }; - - kstreamContext = - new KStreamContextImpl(id, job, topics, ingestor, producer, coordinator, streamingConfig, config, metrics); - - kstreamContexts.put(id, kstreamContext); - - try { - kstreamContext.init(); - } - catch (Exception e) { - throw new KafkaException(e); - } - -<<<<<<< HEAD -<<<<<<< HEAD - Collection syncGroups = kstreamContext.syncGroups(); - this.syncGroups.put(id, syncGroups); - for (SyncGroup syncGroup : syncGroups) { - streamSynchronizers.add(syncGroup.streamSynchronizer); -======= - Collection streamGroups = kstreamContext.streamSynchronizers(); - for (StreamGroup streamGroup : streamGroups) { - streamGroups.add(streamGroup); ->>>>>>> remove SyncGroup from user facing APIs - } -======= - streamSynchronizersForPartition.put(id, kstreamContext.streamSynchronizers()); ->>>>>>> removed some generics -======= - } - state = STOPPED; - lock.notifyAll(); - log.info("Shutdown complete"); ->>>>>>> added KStreamThread - } - } -<<<<<<< HEAD -======= - - nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; ->>>>>>> wip - } - - /** - * Shutdown this streaming instance. - */ - public void close() { - synchronized (lock) { - if (state == CREATED || state == RUNNING) { - state = STOPPING; - lock.notifyAll(); - } - while (state == STOPPING) { - try { - lock.wait(); - } catch (InterruptedException ex) { - Thread.interrupted(); - } - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java deleted file mode 100644 index 87c728b1aa3b4..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java +++ /dev/null @@ -1,118 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.examples; - -<<<<<<< HEAD:src/main/java/io/confluent/streaming/examples/MapKStreamJob.java -import io.confluent.streaming.KStream; -<<<<<<< HEAD -<<<<<<< HEAD -======= -import org.apache.kafka.stream.KStream; -<<<<<<< HEAD ->>>>>>> first re-org:stream/src/main/java/org/apache/kafka/stream/examples/MapKStreamJob.java -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; -import io.confluent.streaming.KStreamJob; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.KafkaStreaming; -import io.confluent.streaming.KeyValue; -import io.confluent.streaming.KeyValueMapper; -import io.confluent.streaming.Predicate; -import io.confluent.streaming.StreamingConfig; -======= ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.clients.processor.KafkaStreaming; -import org.apache.kafka.clients.processor.internals.StreamingConfig; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.KeyValue; -import org.apache.kafka.stream.topology.KeyValueMapper; -import org.apache.kafka.stream.topology.Predicate; - -import java.util.Properties; - -public class MapKStreamJob extends KStreamTopology { - -<<<<<<< HEAD - @SuppressWarnings("unchecked") - @Override -<<<<<<< HEAD -<<<<<<< HEAD - public void topology() { -======= - public void init(KStreamInitializer context) { ->>>>>>> new api model -======= - public void topology() { ->>>>>>> wip -======= - @SuppressWarnings("unchecked") - @Override -<<<<<<< HEAD - public void topology() { ->>>>>>> compile and test passed -======= - public void build() { ->>>>>>> wip - - // With overridden de-serializer - KStream stream1 = from(new StringDeserializer(), new StringDeserializer(), "topic1"); - - stream1.map(new KeyValueMapper() { - @Override - public KeyValue apply(String key, String value) { - return new KeyValue<>(key, new Integer(value)); - } - }).filter(new Predicate() { - @Override - public boolean apply(String key, Integer value) { - return true; - } - }).sendTo("topic2"); - - // Without overriden de-serialzier - KStream stream2 = (KStream) from("topic2"); - - KStream[] streams = stream2.branch( - new Predicate() { - @Override - public boolean apply(String key, Integer value) { - return true; - } - }, - new Predicate() { - @Override - public boolean apply(String key, Integer value) { - return true; - } - } - ); - - streams[0].sendTo("topic3"); - streams[1].sendTo("topic4"); - } - - public static void main(String[] args) { - KafkaStreaming kstream = new KafkaStreaming(new MapKStreamJob(), new StreamingConfig(new Properties())); - kstream.run(); - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java deleted file mode 100644 index 51216c0597529..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/examples/PrintKStreamJob.java +++ /dev/null @@ -1,102 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.examples; - -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.PTopology; -import org.apache.kafka.clients.processor.ProcessorContext; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.clients.processor.KafkaStreaming; -import org.apache.kafka.clients.processor.internals.StreamingConfig; - -import java.util.Properties; - -public class PrintKStreamJob { - - private static class MyProcessor extends KafkaProcessor { - ProcessorContext context; - - public MyProcessor(String name) { - super(name); - } - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void process(String key, Integer value) { - System.out.println("[" + key + ", " + value + "]"); - - context.commit(); - - context.send("topic-dest", key, value); - } - - @Override - public void close() { - // do nothing - } - } - -<<<<<<< HEAD - @SuppressWarnings("unchecked") - @Override - public void topology() { from("topic").process(new MyProcessor()); } - -<<<<<<< HEAD - public static void main(String[] args) { -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -======= ->>>>>>> fix examples - KafkaStreaming streaming = new KafkaStreaming( - new SingleProcessorTopology(PrintKStreamJob.class, args), - new StreamingConfig(new Properties()) - ); - streaming.run(); -<<<<<<< HEAD -======= - KafkaStreaming kstream = new KafkaStreaming(new PrintKStreamJob(), new StreamingConfig(new Properties())); -======= - KafkaStreaming kstream = new KafkaStreaming(new PrintKStreamJob(args), new StreamingConfig(new Properties())); ->>>>>>> fix examples - kstream.run(); ->>>>>>> wip -======= ->>>>>>> fix examples - } -======= -======= ->>>>>>> wip - public static void main(String[] args) { - PTopology topology = new PTopology(); - topology.addProcessor(new MyProcessor("processor"), new StringDeserializer(), new IntegerDeserializer(), "topic-source"); - topology.build(); - - KafkaStreaming streaming = new KafkaStreaming( - new PrintKStreamJob(), - new StreamingConfig(new Properties()) - ); - streaming.run(); - } ->>>>>>> compile and test passed -} diff --git a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java b/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java deleted file mode 100644 index 5d4f7c38cfabc..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/examples/StatefulKStreamJob.java +++ /dev/null @@ -1,147 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.examples; - -import org.apache.kafka.stream.KStream; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.clients.processor.KafkaStreaming; -import org.apache.kafka.clients.processor.internals.StreamingConfig; -import org.apache.kafka.stream.state.Entry; -import org.apache.kafka.stream.state.InMemoryKeyValueStore; -import org.apache.kafka.stream.state.KeyValueIterator; -import org.apache.kafka.stream.state.KeyValueStore; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.Processor; - -import java.util.Properties; - -public class StatefulKStreamJob extends KStreamTopology { - - private class MyProcessor implements Processor { - private KStreamContext context; - private KeyValueStore kvStore; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - this.kvStore = new InMemoryKeyValueStore<>("local-state", context); -<<<<<<< HEAD - this.kvStore.restore(); // call restore inside processor.init -======= - this.kvStore = new InMemoryKeyValueStore<>("local-state", context.kstreamContext()); -<<<<<<< HEAD - this.kvStore.restore(); // call restore inside processor.bind ->>>>>>> new api model -======= -======= - this.kvStore = new InMemoryKeyValueStore<>("local-state", context); ->>>>>>> removed ProcessorContext - this.kvStore.restore(); // call restore inside processor.init ->>>>>>> fix examples -======= ->>>>>>> remove restore function - } -======= -======= - public StatefulKStreamJob(String... topics) { - super(topics); - } - ->>>>>>> SingleProcessorTopology implements Processor - @Override - public void init(KStreamContext context) { - this.context = context; - this.context.schedule(this, 1000); ->>>>>>> compile and test passed -======= - @Override - public void init(KStreamContext context) { - this.context = context; - this.context.schedule(this, 1000); - - this.kvStore = new InMemoryKeyValueStore<>("local-state", context); - } ->>>>>>> Address Yasu's comments - - @Override - public void process(String key, Integer value) { - Integer oldValue = this.kvStore.get(key); - if (oldValue == null) { - this.kvStore.put(key, value); - } else { - int newValue = oldValue + value; - this.kvStore.put(key, newValue); - } - - context.commit(); - } - - @Override - public void punctuate(long streamTime) { - KeyValueIterator iter = this.kvStore.all(); - while (iter.hasNext()) { - Entry entry = iter.next(); - System.out.println("[" + entry.key() + ", " + entry.value() + "]"); - } - } - - @Override - public void close() { - // do nothing - } - } - - @SuppressWarnings("unchecked") - @Override - public void build() { ((KStream) from("topic")).process(new MyProcessor()); } - -<<<<<<< HEAD - public static void main(String[] args) { -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -======= ->>>>>>> fix examples - KafkaStreaming streaming = new KafkaStreaming( - new SingleProcessorTopology(StatefulKStreamJob.class, args), - new StreamingConfig(new Properties()) - ); - streaming.run(); -<<<<<<< HEAD -======= - KafkaStreaming kstream = new KafkaStreaming(new StatefulKStreamJob(), new StreamingConfig(new Properties())); -======= - KafkaStreaming kstream = new KafkaStreaming(new StatefulKStreamJob(args), new StreamingConfig(new Properties())); ->>>>>>> fix examples - kstream.run(); ->>>>>>> wip -======= ->>>>>>> fix examples - } -======= - public static void main(String[] args) { - KafkaStreaming streaming = new KafkaStreaming( - new StatefulKStreamJob(), - new StreamingConfig(new Properties()) - ); - streaming.run(); - } ->>>>>>> compile and test passed -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java b/stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java deleted file mode 100644 index 7da86701b0e55..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/KStreamContextImpl.java +++ /dev/null @@ -1,477 +0,0 @@ -package org.apache.kafka.stream.internal; - -import io.confluent.streaming.KStreamException; -import io.confluent.streaming.Processor; -<<<<<<< HEAD -<<<<<<< HEAD -======= -import io.confluent.streaming.PunctuationScheduler; ->>>>>>> new api model -======= ->>>>>>> removed ProcessorContext -import io.confluent.streaming.RecordCollector; -import io.confluent.streaming.StateStore; -import io.confluent.streaming.StreamingConfig; -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.TimestampExtractor; -import io.confluent.streaming.kv.internals.RestoreFunc; -import io.confluent.streaming.util.Util; -<<<<<<< HEAD -import org.apache.kafka.clients.consumer.Consumer; -<<<<<<< HEAD -======= -======= ->>>>>>> wip -import org.apache.kafka.clients.consumer.KafkaConsumer; -<<<<<<< HEAD -<<<<<<< HEAD -import org.apache.kafka.clients.producer.Producer; ->>>>>>> wip -======= -import org.apache.kafka.clients.producer.ProducerRecord; ->>>>>>> new api model -======= -import org.apache.kafka.clients.producer.ProducerRecord; ->>>>>>> new api model -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.Map; -import java.util.Set; - -/** - * Created by yasuhiro on 6/19/15. - */ -public class KStreamContextImpl implements KStreamContext { - - private static final Logger log = LoggerFactory.getLogger(KStreamContextImpl.class); - - public final int id; - final StreamGroup streamGroup; - final Ingestor ingestor; - - private final RecordCollectorImpl collector; - private final HashMap> sourceStreams = new HashMap<>(); - private final HashMap partitioningInfos = new HashMap<>(); - private final TimestampExtractor timestampExtractor; - private final StreamingConfig streamingConfig; - private final ProcessorConfig processorConfig; - private final Metrics metrics; - private final ProcessorStateManager stateMgr; - - private boolean initialized = false; - - @SuppressWarnings("unchecked") - public KStreamContextImpl(int id, - Ingestor ingestor, - RecordCollectorImpl collector, - StreamingConfig streamingConfig, - ProcessorConfig processorConfig, - Metrics metrics) { - this.id = id; - this.ingestor = ingestor; - this.collector = collector; - this.streamingConfig = streamingConfig; - this.processorConfig = processorConfig; - this.timestampExtractor = this.streamingConfig.timestampExtractor(); -<<<<<<< HEAD - if (this.timestampExtractor == null) throw new NullPointerException("timestamp extractor is missing"); - - this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id)), - new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer())); - this.stateDir = this.stateMgr.baseDir(); -======= - this.stateMgr = new ProcessorStateManager(id, new File(processorConfig.stateDir, Integer.toString(id))); ->>>>>>> close stream groups in context - this.metrics = metrics; - this.streamGroup = new StreamGroup(this, this.ingestor, new TimeBasedChooser(), this.timestampExtractor, this.processorConfig.bufferedRecordsPerPartition); - } - - @Override - public int id() { - return id; - } - - @Override - public Serializer keySerializer() { - return streamingConfig.keySerializer(); - } - - @Override - public Serializer valueSerializer() { - return streamingConfig.valueSerializer(); - } - - @Override - public Deserializer keyDeserializer() { - return streamingConfig.keyDeserializer(); - } - - @Override - public Deserializer valueDeserializer() { - return streamingConfig.valueDeserializer(); - } - - @Override -<<<<<<< HEAD -<<<<<<< HEAD - public KStream from(String... topics) { - return from(streamGroup(getNextGroupName()), null, null, topics); - } - - @Override - public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - return from(streamGroup(getNextGroupName()), keyDeserializer, valDeserializer, topics); - } - - private String getNextGroupName() { - return "StreamGroup-" + STREAM_GROUP_INDEX.getAndIncrement(); - } - - @SuppressWarnings("unchecked") - private KStream from(StreamGroup streamGroup, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { - ensureInitialization(); - - if (streamGroup == null) throw new IllegalArgumentException("unspecified stream group"); - - Set fromTopics; - - synchronized (this) { - // if topics not specified, use all the topics be default -<<<<<<< HEAD - if (topics == null) { - fromTopics = ingestor.topics(); -======= - if (topics == null || topics.length == 0) { - fromTopics = this.topics; ->>>>>>> fix vararg in KStreamContextImpl.from() - } else { - fromTopics = Collections.unmodifiableSet(Util.mkSet(topics)); - } - - // iterate over the topics and check if the stream has already been created for them - for (String topic : fromTopics) { - if (!ingestor.topics().contains(topic)) - throw new IllegalArgumentException("topic not subscribed: " + topic); - - if (sourceStreams.containsKey(topic)) - throw new IllegalArgumentException("another stream created with the same topic " + topic); - } - - // create stream metadata - Map topicPartitionInfos = new HashMap<>(); - for (String topic : fromTopics) { - PartitioningInfo partitioningInfo = this.partitioningInfos.get(topic); - - if (partitioningInfo == null) { - partitioningInfo = new PartitioningInfo(ingestor.numPartitions(topic)); - this.partitioningInfos.put(topic, partitioningInfo); - } - - topicPartitionInfos.put(topic, partitioningInfo); - } - KStreamMetadata streamMetadata = new KStreamMetadata(streamGroup, topicPartitionInfos); - - // override the deserializer classes if specified - KStreamSource stream = new KStreamSource<>( - streamMetadata, - this, - (Deserializer) (keyDeserializer == null ? keyDeserializer() : keyDeserializer), - (Deserializer) (valDeserializer == null ? valueDeserializer() : valDeserializer) - ); - - // update source stream map - for (String topic : fromTopics) { - sourceStreams.put(topic, stream); - - TopicPartition partition = new TopicPartition(topic, id); - streamGroup.addPartition(partition, stream); - } - - return stream; - } - } - - @Override -======= ->>>>>>> new api model -======= ->>>>>>> new api model - public RecordCollector recordCollector() { - return collector; - } - - @Override - public Map getContext() { - return streamingConfig.context(); - } - - @Override - public File stateDir() { - return stateMgr.baseDir(); - } - - @Override - public Metrics metrics() { - return metrics; - } - - @Override -<<<<<<< HEAD - public void restore(StateStore store, RestoreFunc restoreFunc) { - ensureInitialization(); - - stateMgr.registerAndRestore(store, restoreFunc); -======= - public void register(StateStore store, RestoreFunc restoreFunc) { - ensureInitialization(); - - stateMgr.register(store, restoreFunc); ->>>>>>> new api model - } - - public void ensureInitialization() { - if (!initialized) - throw new IllegalStateException("context initialization is already finished"); - } - - @Override - public void flush() { - stateMgr.flush(); - } - - @Override - public String topic() { -<<<<<<< HEAD -<<<<<<< HEAD - if (streamGroup.record() == null) - throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); - - return streamGroup.record().topic(); -======= - if (this.streamGroup.record() == null) - throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); - - return this.streamGroup.record().topic(); ->>>>>>> new api model -======= - if (streamGroup.record() == null) - throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); - - return streamGroup.record().topic(); ->>>>>>> removed ProcessorContext - } - - @Override - public int partition() { -<<<<<<< HEAD -<<<<<<< HEAD - if (streamGroup.record() == null) - throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); - -<<<<<<< HEAD -<<<<<<< HEAD - stateMgr.restore(store, restoreFunc); -======= - if (this.streamGroup.record() == null) -======= - if (streamGroup.record() == null) ->>>>>>> removed ProcessorContext - throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); - - return streamGroup.record().partition(); - } - - @Override - public long offset() { - if (this.streamGroup.record() == null) - throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); - - return this.streamGroup.record().offset(); ->>>>>>> new api model - } - - @Override - public long timestamp() { - if (streamGroup.record() == null) - throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); - -<<<<<<< HEAD -<<<<<<< HEAD - stateMgr.register(store); -======= - return this.streamGroup.record().partition(); ->>>>>>> new api model -======= - return streamGroup.record().partition(); ->>>>>>> removed ProcessorContext - } - - @Override - public long offset() { - if (this.streamGroup.record() == null) - throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); - - return this.streamGroup.record().offset(); - } - - @Override - public long timestamp() { - if (streamGroup.record() == null) - throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); - - return streamGroup.record().timestamp; - } - - @Override - public void send(String topic, Object key, Object value) { - collector.send(new ProducerRecord<>(topic, key, value)); - } - - @Override -======= - return this.streamGroup.record().timestamp; -======= - return streamGroup.record().timestamp; ->>>>>>> removed ProcessorContext - } - - @Override - public void send(String topic, Object key, Object value) { - collector.send(new ProducerRecord<>(topic, key, value)); - } - - @Override ->>>>>>> new api model - public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { - if (keySerializer == null || valSerializer == null) - throw new IllegalStateException("key and value serializers must be specified"); - - collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); -<<<<<<< HEAD - } - - @Override - public void commit() { - streamGroup.commitOffset(); - } - - @Override - public void schedule(Processor processor, long interval) { - streamGroup.schedule(processor, interval); - } - - void init(Collection> streams) throws IOException { -======= - } - - @Override - public void commit() { - streamGroup.commitOffset(); - } - - @Override - public void schedule(Processor processor, long interval) { - streamGroup.schedule(processor, interval); - } - -<<<<<<< HEAD - public void init(Collection> streams) throws IOException { ->>>>>>> new api model -======= - void init(Collection> streams) throws IOException { ->>>>>>> removed ProcessorContext - stateMgr.init(); - - for (KStreamSource stream: streams) { - KStreamMetadata metadata = linkStreamToTopics(stream); - - stream.bind(this, metadata); - } - - // add partition -> stream group mappings to the ingestor - for (Map.Entry> entry : sourceStreams.entrySet()) { - TopicPartition partition = new TopicPartition(entry.getKey(), id); - ingestor.addPartitionStreamToGroup(streamGroup, partition); - } - - if (!ingestor.topics().equals(sourceStreams.keySet())) { - LinkedList unusedTopics = new LinkedList<>(); - for (String topic : ingestor.topics()) { - if (!sourceStreams.containsKey(topic)) - unusedTopics.add(topic); - } - throw new KStreamException("unused topics: " + Util.mkString(unusedTopics)); - } - - initialized = true; - } - - private KStreamMetadata linkStreamToTopics(KStreamSource stream) { - ensureInitialization(); - - Set fromTopics; - - synchronized (this) { - // if topics not specified, use all the topics be default - if (stream.topics == null || stream.topics.length == 0) { - fromTopics = ingestor.topics(); - } else { - fromTopics = Collections.unmodifiableSet(Util.mkSet(stream.topics)); - } - - // iterate over the topics and check if the stream has already been created for them - for (String topic : fromTopics) { - if (!ingestor.topics().contains(topic)) - throw new IllegalArgumentException("topic not subscribed: " + topic); - - if (sourceStreams.containsKey(topic)) - throw new IllegalArgumentException("another stream created with the same topic " + topic); - } - - // create stream metadata - Map topicPartitionInfos = new HashMap<>(); - for (String topic : fromTopics) { - PartitioningInfo partitioningInfo = this.partitioningInfos.get(topic); - - if (partitioningInfo == null) { - partitioningInfo = new PartitioningInfo(ingestor.numPartitions(topic)); - this.partitioningInfos.put(topic, partitioningInfo); - } - - topicPartitionInfos.put(topic, partitioningInfo); - } - - // update source stream map - for (String topic : fromTopics) { - sourceStreams.put(topic, stream); - - TopicPartition partition = new TopicPartition(topic, id); - streamGroup.addPartition(partition, stream); - } - - return new KStreamMetadata(topicPartitionInfos); - } - } - - public Map consumedOffsets() { - return streamGroup.consumedOffsets(); - } - - public void close() throws Exception { - stateMgr.close(collector.offsets()); - streamGroup.close(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java b/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java deleted file mode 100644 index 67e16d4738eaf..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/ProcessorNode.java +++ /dev/null @@ -1,51 +0,0 @@ -package org.apache.kafka.stream.internal; - -import org.apache.kafka.stream.topology.Processor; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; - -/** - * Created by yasuhiro on 7/31/15. - */ -public class ProcessorNode implements Receiver { - - private final Processor processor; - private KStreamContext context; - - public ProcessorNode(Processor processor) { - this.processor = processor; - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); - - this.context = context; -<<<<<<< HEAD -<<<<<<< HEAD - processor.init(context); - } - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { -<<<<<<< HEAD -======= - processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); -======= - processor.init(context); ->>>>>>> removed ProcessorContext - } - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp, long streamTime) { ->>>>>>> wip -======= ->>>>>>> remove streamTime from Receiver - processor.process((K) key, (V) value); - } - @Override - public void close() { - processor.close(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java b/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java deleted file mode 100644 index 680488f01f395..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/Receiver.java +++ /dev/null @@ -1,27 +0,0 @@ -package org.apache.kafka.stream.internal; - -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.topology.internal.KStreamMetadata; - -/** - * Created by yasuhiro on 6/17/15. - */ -public interface Receiver { - - void bind(KStreamContext context, KStreamMetadata metadata); - -<<<<<<< HEAD -<<<<<<< HEAD - void receive(Object key, Object value, long timestamp); - - void close(); -======= - void receive(Object key, Object value, long timestamp, long streamTime); ->>>>>>> new api model -======= - void receive(Object key, Object value, long timestamp); ->>>>>>> remove streamTime from Receiver - - void close(); - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java b/stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java deleted file mode 100644 index 787c16fec1727..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/RecordQueue.java +++ /dev/null @@ -1,114 +0,0 @@ -package org.apache.kafka.stream.internal; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.stream.topology.internal.KStreamSource; -import org.apache.kafka.stream.util.TimestampTracker; - -import java.util.ArrayDeque; - -/** - * RecordQueue is a queue of {@link StampedRecord} (ConsumerRecord + timestamp). It is intended to be used in - * {@link StreamGroup}. - */ -public class RecordQueue { - - private final ArrayDeque queue = new ArrayDeque<>(); - public final KStreamSource stream; - private final TopicPartition partition; - private TimestampTracker> timestampTracker; - private long offset; - -<<<<<<< HEAD -<<<<<<< HEAD - public RecordQueue(TopicPartition partition, KStreamSource source, TimestampTracker> timestampTracker) { -======= - /** - * Creates a new instance of RecordQueue - * @param partition partition -<<<<<<< HEAD - * @param receiver the receiver of the stream of this partition -======= - * @param stream the instance of KStreamImpl that receives records ->>>>>>> more fix - * @param timestampTracker TimestampTracker - */ -<<<<<<< HEAD - public RecordQueue(TopicPartition partition, Receiver receiver, TimestampTracker> timestampTracker) { ->>>>>>> javadoc -======= - public RecordQueue(TopicPartition partition, KStreamImpl stream, TimestampTracker> timestampTracker) { ->>>>>>> extend sendTo and through with ser-de overriden enabled -======= - public RecordQueue(TopicPartition partition, KStreamSource stream, TimestampTracker> timestampTracker) { ->>>>>>> do not clone kstream context - this.partition = partition; - this.stream = stream; - this.timestampTracker = timestampTracker; - } - - /** - * Returns the partition with which this queue is associated - * @return TopicPartition - */ - public TopicPartition partition() { - return partition; - } - - /** - * Adds a StampedRecord to the queue - * @param record StampedRecord - */ - public void add(StampedRecord record) { - queue.addLast(record); - timestampTracker.addStampedElement(record); - offset = record.offset(); - } - - /** - * Returns the next record fro the queue - * @return StampedRecord - */ - public StampedRecord next() { - StampedRecord elem = queue.pollFirst(); - - if (elem == null) return null; - - timestampTracker.removeStampedElement(elem); - - return elem; - } - - /** - * Returns the highest offset in the queue - * @return offset - */ - public long offset() { - return offset; - } - - /** - * Returns the number of records in the queue - * @return the number of records - */ - public int size() { - return queue.size(); - } - - /** - * Tests if the queue is empty - * @return true if the queue is empty, otherwise false - */ - public boolean isEmpty() { - return queue.isEmpty(); - } - - /** - * Returns a timestamp tracked by the TimestampTracker - * @return timestamp - */ - public long trackedTimestamp() { - return timestampTracker.get(); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java b/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java deleted file mode 100644 index ef03c493c651b..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internal/StreamGroup.java +++ /dev/null @@ -1,311 +0,0 @@ -package org.apache.kafka.stream.internal; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.stream.topology.Processor; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.stream.Chooser; -import org.apache.kafka.stream.TimestampExtractor; -import org.apache.kafka.stream.topology.internal.KStreamSource; -import org.apache.kafka.stream.util.MinTimestampTracker; -import org.apache.kafka.stream.util.ParallelExecutor; - -import java.util.ArrayDeque; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; - -/** - * A StreamGroup is composed of multiple streams from different topics that need to be synchronized. - */ -<<<<<<< HEAD:src/main/java/io/confluent/streaming/internal/StreamSynchronizer.java -<<<<<<< HEAD -public class StreamSynchronizer implements ParallelExecutor.Task { - - public final String name; - private final Ingestor ingestor; - private final Chooser chooser; - private final TimestampExtractor timestampExtractor; - private final Map> stash = new HashMap<>(); -======= -public class StreamSynchronizer implements SyncGroup { -======= -public class StreamGroup implements ParallelExecutor.Task { ->>>>>>> remove SyncGroup from user facing APIs:src/main/java/io/confluent/streaming/internal/StreamGroup.java - - private final KStreamContext context; - private final Ingestor ingestor; - private final Chooser chooser; - private final TimestampExtractor timestampExtractor; - private final Map stash = new HashMap<>(); ->>>>>>> removed some generics - private final int desiredUnprocessed; - - // TODO: merge stash, consumedOffset, and newRecordBuffer into sth. like partition metadata - private final Map consumedOffsets; - private final PunctuationQueue punctuationQueue = new PunctuationQueue(); - private final ArrayDeque> newRecordBuffer = new ArrayDeque<>(); - - private long streamTime = -1; - private boolean commitRequested = false; - private StampedRecord currRecord = null; - private volatile int buffered = 0; - - /** - * Creates StreamGroup - * @param context the task context - * @param ingestor the instance of {@link Ingestor} - * @param chooser the instance of {@link Chooser} - * @param timestampExtractor the instance of {@link TimestampExtractor} - * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic - */ - public StreamGroup(KStreamContext context, - Ingestor ingestor, - Chooser chooser, - TimestampExtractor timestampExtractor, - int desiredUnprocessedPerPartition) { - this.context = context; - this.ingestor = ingestor; - this.chooser = chooser; - this.timestampExtractor = timestampExtractor; - this.desiredUnprocessed = desiredUnprocessedPerPartition; - this.consumedOffsets = new HashMap<>(); -<<<<<<< HEAD -======= - } - -<<<<<<< HEAD -<<<<<<< HEAD - public String name() { - return name; ->>>>>>> removed some generics - } - -======= ->>>>>>> new api model -======= ->>>>>>> new api model - public StampedRecord record() { return currRecord; } - - /** - * Merges a stream group into this group - */ - public void mergeStreamGroup(StreamGroup other) { - // check these groups have the same ingestor - if (ingestor != other.ingestor) - throw new IllegalArgumentException("groups with different ingestors cannot be merged"); - - // check these group have the same chooser and time extractor types - if (!this.chooser.getClass().equals(other.chooser.getClass())) - throw new IllegalArgumentException("groups with different type of choosers cannot be merged"); - - if (!this.timestampExtractor.getClass().equals(other.timestampExtractor.getClass())) - throw new IllegalArgumentException("groups with different type of time extractors cannot be merged"); - - // add all the other's groups partitions - this.stash.putAll(other.stash); - } - - /** - * Adds a partition and its receiver to this stream synchronizer - * @param partition the partition - * @param stream the instance of KStreamImpl - */ - @SuppressWarnings("unchecked") - public void addPartition(TopicPartition partition, KStreamSource stream) { - synchronized (this) { - RecordQueue recordQueue = stash.get(partition); - - if (recordQueue == null) { - stash.put(partition, createRecordQueue(partition, stream)); - } else { - throw new IllegalStateException("duplicate partition"); - } - } - } - -<<<<<<< HEAD -<<<<<<< HEAD - public void addRecords(TopicPartition partition, Iterator> iterator) { -======= -======= - /** - * Adds records - * @param partition the partition - * @param iterator the iterator of records - */ ->>>>>>> javadoc - @SuppressWarnings("unchecked") -<<<<<<< HEAD - public void addRecords(TopicPartition partition, Iterator> iterator) { ->>>>>>> removed some generics -======= - public void addRecords(TopicPartition partition, Iterator> iterator) { ->>>>>>> allow deserializer override at KStream construction - synchronized (this) { - newRecordBuffer.addLast(new NewRecords(partition, iterator)); - } - } - - @SuppressWarnings("unchecked") - private void ingestNewRecords() { - for (NewRecords newRecords : newRecordBuffer) { - TopicPartition partition = newRecords.partition; -<<<<<<< HEAD - Iterator> iterator = newRecords.iterator; -======= - Iterator> iterator = newRecords.iterator; ->>>>>>> allow deserializer override at KStream construction - - RecordQueue recordQueue = stash.get(partition); - if (recordQueue != null) { - boolean wasEmpty = recordQueue.isEmpty(); - - while (iterator.hasNext()) { -<<<<<<< HEAD - ConsumerRecord record = iterator.next(); - long timestamp = timestampExtractor.extract(record.topic(), record.key(), record.value()); -<<<<<<< HEAD - recordQueue.add(new StampedRecord<>(record, timestamp)); -======= - recordQueue.add(new StampedRecord(record, timestamp)); ->>>>>>> removed some generics -======= - ConsumerRecord record = iterator.next(); - - // deserialize the raw record, extract the timestamp and put into the queue - Deserializer keyDeserializer = recordQueue.stream.keyDeserializer(); - Deserializer valDeserializer = recordQueue.stream.valueDeserializer(); - - Object key = keyDeserializer.deserialize(record.topic(), record.key()); - Object value = valDeserializer.deserialize(record.topic(), record.value()); - ConsumerRecord deserializedRecord = new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), key, value); - - long timestamp = timestampExtractor.extract(record.topic(), key, value); - recordQueue.add(new StampedRecord(deserializedRecord, timestamp)); ->>>>>>> allow deserializer override at KStream construction - buffered++; - } - - int queueSize = recordQueue.size(); - if (wasEmpty && queueSize > 0) chooser.add(recordQueue); - - // if we have buffered enough for this partition, pause - if (queueSize >= this.desiredUnprocessed) { - ingestor.pause(partition); - } - } - } - newRecordBuffer.clear(); - } - - /** - * Schedules a punctuation for the processor - * @param processor the processor requesting scheduler - * @param interval the interval in milliseconds - */ - public void schedule(Processor processor, long interval) { - punctuationQueue.schedule(new PunctuationSchedule(processor, interval)); - } - - /** - * Processes one record - */ - @SuppressWarnings("unchecked") - @Override - public boolean process() { - synchronized (this) { - boolean readyForNextExecution = false; - ingestNewRecords(); - - RecordQueue recordQueue = chooser.next(); - if (recordQueue == null) { - return false; - } - - if (recordQueue.size() == 0) throw new IllegalStateException("empty record queue"); - - if (recordQueue.size() == this.desiredUnprocessed) { - ingestor.unpause(recordQueue.partition(), recordQueue.offset()); - } - - long trackedTimestamp = recordQueue.trackedTimestamp(); - currRecord = recordQueue.next(); - - if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - - recordQueue.stream.receive(currRecord.key(), currRecord.value(), currRecord.timestamp); - consumedOffsets.put(recordQueue.partition(), currRecord.offset()); - - // TODO: local state flush and downstream producer flush - // need to be done altogether with offset commit atomically - if (commitRequested) { - // flush local state - context.flush(); - - // flush produced records in the downstream - context.recordCollector().flush(); - - // commit consumed offsets - ingestor.commit(consumedOffsets()); - } - - if (commitRequested) ingestor.commit(Collections.singletonMap( - new TopicPartition(currRecord.topic(), currRecord.partition()), - currRecord.offset())); - - if (recordQueue.size() > 0) { - readyForNextExecution = true; - chooser.add(recordQueue); - } - - - buffered--; - currRecord = null; - - punctuationQueue.mayPunctuate(streamTime); - - return readyForNextExecution; - } - } - - /** - * Returns consumed offsets - * @return the map of partition to consumed offset - */ - public Map consumedOffsets() { - return this.consumedOffsets; - } - - /** - * Request committing the current record's offset - */ - public void commitOffset() { - this.commitRequested = true; - } - - public int buffered() { - return buffered; - } - - public void close() { - chooser.close(); - stash.clear(); - } - - protected RecordQueue createRecordQueue(TopicPartition partition, KStreamSource stream) { - return new RecordQueue(partition, stream, new MinTimestampTracker<>()); - } - - private static class NewRecords { - final TopicPartition partition; - final Iterator> iterator; - - NewRecords(TopicPartition partition, Iterator> iterator) { - this.partition = partition; - this.iterator = iterator; - } - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java deleted file mode 100644 index 5f7b52c0824e6..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/internals/StreamGroup.java +++ /dev/null @@ -1,284 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.internals; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.stream.topology.Processor; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.stream.Chooser; -import org.apache.kafka.stream.TimestampExtractor; -import org.apache.kafka.stream.topology.internals.KStreamSource; -import org.apache.kafka.stream.util.MinTimestampTracker; -import org.apache.kafka.stream.util.ParallelExecutor; - -import java.util.ArrayDeque; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; - -/** - * A StreamGroup is composed of multiple streams from different topics that need to be synchronized. - */ -public class StreamGroup implements ParallelExecutor.Task { - - private final KStreamContext context; - private final Ingestor ingestor; - private final Chooser chooser; - private final TimestampExtractor timestampExtractor; - private final Map stash = new HashMap<>(); - - private final int desiredUnprocessed; - - // TODO: merge stash, consumedOffset, and newRecordBuffer into sth. like partition metadata - private final Map consumedOffsets; - private final PunctuationQueue punctuationQueue = new PunctuationQueue(); - private final ArrayDeque newRecordBuffer = new ArrayDeque<>(); - - private long streamTime = -1; - private boolean commitRequested = false; - private StampedRecord currRecord = null; - private volatile int buffered = 0; - - /** - * Creates StreamGroup - * - * @param context the task context - * @param ingestor the instance of {@link Ingestor} - * @param chooser the instance of {@link Chooser} - * @param timestampExtractor the instance of {@link TimestampExtractor} - * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic - */ - public StreamGroup(KStreamContext context, - Ingestor ingestor, - Chooser chooser, - TimestampExtractor timestampExtractor, - int desiredUnprocessedPerPartition) { - this.context = context; - this.ingestor = ingestor; - this.chooser = chooser; - this.timestampExtractor = timestampExtractor; - this.desiredUnprocessed = desiredUnprocessedPerPartition; - this.consumedOffsets = new HashMap<>(); - } - - public StampedRecord record() { - return currRecord; - } - - /** - * Merges a stream group into this group - */ - public void mergeStreamGroup(StreamGroup other) { - // check these groups have the same ingestor - if (ingestor != other.ingestor) - throw new IllegalArgumentException("groups with different ingestors cannot be merged"); - - // check these group have the same chooser and time extractor types - if (!this.chooser.getClass().equals(other.chooser.getClass())) - throw new IllegalArgumentException("groups with different type of choosers cannot be merged"); - - if (!this.timestampExtractor.getClass().equals(other.timestampExtractor.getClass())) - throw new IllegalArgumentException("groups with different type of time extractors cannot be merged"); - - // add all the other's groups partitions - this.stash.putAll(other.stash); - } - - /** - * Adds a partition and its receiver to this stream synchronizer - * - * @param partition the partition - * @param stream the instance of KStreamImpl - */ - @SuppressWarnings("unchecked") - public void addPartition(TopicPartition partition, KStreamSource stream) { - synchronized (this) { - RecordQueue recordQueue = stash.get(partition); - - if (recordQueue == null) { - stash.put(partition, createRecordQueue(partition, stream)); - } else { - throw new IllegalStateException("duplicate partition"); - } - } - } - - /** - * Adds records - * - * @param partition the partition - * @param iterator the iterator of records - */ - @SuppressWarnings("unchecked") - public void addRecords(TopicPartition partition, Iterator> iterator) { - synchronized (this) { - newRecordBuffer.addLast(new NewRecords(partition, iterator)); - } - } - - @SuppressWarnings("unchecked") - private void ingestNewRecords() { - for (NewRecords newRecords : newRecordBuffer) { - TopicPartition partition = newRecords.partition; - Iterator> iterator = newRecords.iterator; - - RecordQueue recordQueue = stash.get(partition); - if (recordQueue != null) { - boolean wasEmpty = recordQueue.isEmpty(); - - while (iterator.hasNext()) { - ConsumerRecord record = iterator.next(); - - // deserialize the raw record, extract the timestamp and put into the queue - Deserializer keyDeserializer = recordQueue.stream.keyDeserializer(); - Deserializer valDeserializer = recordQueue.stream.valueDeserializer(); - - Object key = keyDeserializer.deserialize(record.topic(), record.key()); - Object value = valDeserializer.deserialize(record.topic(), record.value()); - ConsumerRecord deserializedRecord = new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), key, value); - - long timestamp = timestampExtractor.extract(record.topic(), key, value); - recordQueue.add(new StampedRecord(deserializedRecord, timestamp)); - buffered++; - } - - int queueSize = recordQueue.size(); - if (wasEmpty && queueSize > 0) chooser.add(recordQueue); - - // if we have buffered enough for this partition, pause - if (queueSize >= this.desiredUnprocessed) { - ingestor.pause(partition); - } - } - } - newRecordBuffer.clear(); - } - - /** - * Schedules a punctuation for the processor - * - * @param processor the processor requesting scheduler - * @param interval the interval in milliseconds - */ - public void schedule(Processor processor, long interval) { - punctuationQueue.schedule(new PunctuationSchedule(processor, interval)); - } - - /** - * Processes one record - */ - @SuppressWarnings("unchecked") - @Override - public boolean process() { - synchronized (this) { - boolean readyForNextExecution = false; - ingestNewRecords(); - - RecordQueue recordQueue = chooser.next(); - if (recordQueue == null) { - return false; - } - - if (recordQueue.size() == 0) throw new IllegalStateException("empty record queue"); - - if (recordQueue.size() == this.desiredUnprocessed) { - ingestor.unpause(recordQueue.partition(), recordQueue.offset()); - } - - long trackedTimestamp = recordQueue.trackedTimestamp(); - currRecord = recordQueue.next(); - - if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - - recordQueue.stream.receive(currRecord.key(), currRecord.value(), currRecord.timestamp); - consumedOffsets.put(recordQueue.partition(), currRecord.offset()); - - // TODO: local state flush and downstream producer flush - // need to be done altogether with offset commit atomically - if (commitRequested) { - // flush local state - context.flush(); - - // flush produced records in the downstream - context.recordCollector().flush(); - - // commit consumed offsets - ingestor.commit(consumedOffsets()); - } - - if (commitRequested) ingestor.commit(Collections.singletonMap( - new TopicPartition(currRecord.topic(), currRecord.partition()), - currRecord.offset())); - - if (recordQueue.size() > 0) { - readyForNextExecution = true; - chooser.add(recordQueue); - } - - buffered--; - currRecord = null; - - punctuationQueue.mayPunctuate(streamTime); - - return readyForNextExecution; - } - } - - /** - * Returns consumed offsets - * - * @return the map of partition to consumed offset - */ - public Map consumedOffsets() { - return this.consumedOffsets; - } - - /** - * Request committing the current record's offset - */ - public void commitOffset() { - this.commitRequested = true; - } - - public int buffered() { - return buffered; - } - - public void close() { - for (RecordQueue queue : stash.values()) queue.stream.close(); - stash.clear(); - chooser.close(); - } - - protected RecordQueue createRecordQueue(TopicPartition partition, KStreamSource stream) { - return new RecordQueue(partition, stream, new MinTimestampTracker>()); - } - - private static class NewRecords { - final TopicPartition partition; - final Iterator> iterator; - - NewRecords(TopicPartition partition, Iterator> iterator) { - this.partition = partition; - this.iterator = iterator; - } - } -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java deleted file mode 100644 index b2de36c6d9254..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java +++ /dev/null @@ -1,295 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology; - -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; -<<<<<<< HEAD:stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.RecordCollector; -import org.apache.kafka.stream.RestoreFunc; -import org.apache.kafka.stream.topology.internals.WindowSupport; -import org.apache.kafka.stream.util.FilteredIterator; -import org.apache.kafka.clients.processor.internals.Stamped; -======= -import org.apache.kafka.streaming.kstream.internals.FilteredIterator; -import org.apache.kafka.streaming.kstream.internals.WindowSupport; -import org.apache.kafka.streaming.processor.internals.ProcessorContextImpl; -import org.apache.kafka.streaming.processor.internals.RecordCollector; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RestoreFunc; -import org.apache.kafka.streaming.processor.internals.Stamped; ->>>>>>> record current Node and current Record in StreamTask, add ProcessorFactory:stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java - -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.Map; - -public class SlidingWindow implements Window { - private String name; - private final long duration; - private final int maxCount; - private final Serializer keySerializer; - private final Serializer valueSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; -<<<<<<< HEAD:stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java - private KStreamContext context; - private int slotNum; - private String name; - private final long duration; - private final int maxCount; - private LinkedList list = new LinkedList(); - private HashMap> map = new HashMap<>(); -======= ->>>>>>> kstream refactored:stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java - - public SlidingWindow( - String name, - long duration, - int maxCount, - Serializer keySerializer, - Serializer valueSerializer, - Deserializer keyDeseriaizer, - Deserializer valueDeserializer) { - this.name = name; - this.duration = duration; - this.maxCount = maxCount; - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - this.keyDeserializer = keyDeseriaizer; - this.valueDeserializer = valueDeserializer; - } - - @Override -<<<<<<< HEAD:stream/src/main/java/org/apache/kafka/stream/topology/SlidingWindow.java - public void init(KStreamContext context) { - this.context = context; - RestoreFuncImpl restoreFunc = new RestoreFuncImpl(); - context.register(this, restoreFunc); - - for (ValueList valueList : map.values()) { - valueList.clearDirtyValues(); - } - this.slotNum = restoreFunc.slotNum; -======= - public String name() { - return name; ->>>>>>> kstream refactored:stream/src/main/java/org/apache/kafka/streaming/kstream/SlidingWindow.java - } - - @Override - public WindowInstance build() { - return new SlidingWindowInstance(); - } - - public class SlidingWindowInstance extends WindowSupport implements WindowInstance { - private final Object lock = new Object(); - private ProcessorContext context; - private int slotNum; // used as a key for Kafka log compaction - private LinkedList list = new LinkedList(); - private HashMap> map = new HashMap<>(); - - @Override - public void init(ProcessorContext context) { - this.context = context; - RestoreFuncImpl restoreFunc = new RestoreFuncImpl(); - context.register(this, restoreFunc); - - for (ValueList valueList : map.values()) { - valueList.clearDirtyValues(); - } - this.slotNum = restoreFunc.slotNum; - } - - @Override - public Iterator findAfter(K key, final long timestamp) { - return find(key, timestamp, timestamp + duration); - } - - @Override - public Iterator findBefore(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp); - } - - @Override - public Iterator find(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp + duration); - } - - /* - * finds items in the window between startTime and endTime (both inclusive) - */ - private Iterator find(K key, final long startTime, final long endTime) { - final ValueList values = map.get(key); - - if (values == null) { - return null; - } else { - return new FilteredIterator>(values.iterator()) { - @Override - protected V filter(Value item) { - if (startTime <= item.timestamp && item.timestamp <= endTime) - return item.value; - else - return null; - } - }; - } - } - - @Override - public void put(K key, V value, long timestamp) { - synchronized (lock) { - slotNum++; - - list.offerLast(key); - - ValueList values = map.get(key); - if (values == null) { - values = new ValueList<>(); - map.put(key, values); - } - - values.add(slotNum, value, timestamp); - } - evictExcess(); - evictExpired(timestamp - duration); - } - - private void evictExcess() { - while (list.size() > maxCount) { - K oldestKey = list.pollFirst(); - - ValueList values = map.get(oldestKey); - values.removeFirst(); - - if (values.isEmpty()) map.remove(oldestKey); - } - } - - private void evictExpired(long cutoffTime) { - while (true) { - K oldestKey = list.peekFirst(); - - ValueList values = map.get(oldestKey); - Stamped oldestValue = values.first(); - - if (oldestValue.timestamp < cutoffTime) { - list.pollFirst(); - values.removeFirst(); - - if (values.isEmpty()) map.remove(oldestKey); - } else { - break; - } - } - } - - @Override - public String name() { - return name; - } - - @Override - public void flush() { - IntegerSerializer intSerializer = new IntegerSerializer(); - ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); - - RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); - - for (Map.Entry> entry : map.entrySet()) { - ValueList values = entry.getValue(); - if (values.hasDirtyValues()) { - K key = entry.getKey(); - - byte[] keyBytes = keySerializer.serialize(name, key); - - Iterator> iterator = values.dirtyValueIterator(); - while (iterator.hasNext()) { - Value dirtyValue = iterator.next(); - byte[] slot = intSerializer.serialize("", dirtyValue.slotNum); - byte[] valBytes = valueSerializer.serialize(name, dirtyValue.value); - - byte[] combined = new byte[8 + 4 + keyBytes.length + 4 + valBytes.length]; - - int offset = 0; - offset += putLong(combined, offset, dirtyValue.timestamp); - offset += puts(combined, offset, keyBytes); - offset += puts(combined, offset, valBytes); - - if (offset != combined.length) - throw new IllegalStateException("serialized length does not match"); - - collector.send(new ProducerRecord<>(name, context.id(), slot, combined), byteArraySerializer, byteArraySerializer); - } - values.clearDirtyValues(); - } - } - } - - @Override - public void close() { - // TODO - } - - @Override - public boolean persistent() { - // TODO: should not be persistent, right? - return false; - } - - private class RestoreFuncImpl implements RestoreFunc { - - final IntegerDeserializer intDeserializer; - int slotNum = 0; - - RestoreFuncImpl() { - intDeserializer = new IntegerDeserializer(); - } - - @Override - public void apply(byte[] slot, byte[] bytes) { - - slotNum = intDeserializer.deserialize("", slot); - - int offset = 0; - // timestamp - long timestamp = getLong(bytes, offset); - offset += 8; - // key - int length = getInt(bytes, offset); - offset += 4; - K key = deserialize(bytes, offset, length, name, keyDeserializer); - offset += length; - // value - length = getInt(bytes, offset); - offset += 4; - V value = deserialize(bytes, offset, length, name, valueDeserializer); - - put(key, value, timestamp); - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java deleted file mode 100644 index 7828b67fc22c3..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamBranch.java +++ /dev/null @@ -1,131 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamContext; -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.Predicate; -======= -import org.apache.kafka.clients.processor.ProcessorContext; -======= -import org.apache.kafka.stream.KStreamContext; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.internal.Receiver; -import org.apache.kafka.stream.topology.KStreamTopology; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.topology.Predicate; - -import java.lang.reflect.Array; -import java.util.Arrays; - -/** - * Created by yasuhiro on 6/18/15. - */ -class KStreamBranch implements Receiver { - - private final Predicate[] predicates; - final KStreamSource[] branches; - - @SuppressWarnings("unchecked") -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamBranch(Predicate[] predicates, KStreamTopology topology) { - this.predicates = Arrays.copyOf(predicates, predicates.length); - this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); - for (int i = 0; i < branches.length; i++) { - branches[i] = new KStreamSource<>(null, topology); -======= - KStreamBranch(Predicate[] predicates, KStreamInitializer initializer) { -======= - KStreamBranch(Predicate[] predicates, KStreamTopology initializer) { ->>>>>>> wip - this.predicates = Arrays.copyOf(predicates, predicates.length); - this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); - for (int i = 0; i < branches.length; i++) { - branches[i] = new KStreamSource<>(null, initializer); ->>>>>>> new api model -======= - KStreamBranch(Predicate[] predicates, KStreamTopology topology) { - this.predicates = Arrays.copyOf(predicates, predicates.length); - this.branches = (KStreamSource[]) Array.newInstance(KStreamSource.class, predicates.length); - for (int i = 0; i < branches.length; i++) { - branches[i] = new KStreamSource<>(null, topology); ->>>>>>> fix parameter name - } - } - - @Override -<<<<<<< HEAD -<<<<<<< HEAD - public void bind(KStreamContext context, KStreamMetadata metadata) { -<<<<<<< HEAD -<<<<<<< HEAD - for (KStreamSource branch : branches) { - branch.bind(context, metadata); -======= - for (KStreamSource stream : branches) { - stream.bind(context, metadata); ->>>>>>> new api model -======= -======= - public void bind(ProcessorContext context, KStreamMetadata metadata) { ->>>>>>> removing io.confluent imports: wip -======= - public void bind(KStreamContext context, KStreamMetadata metadata) { ->>>>>>> removing io.confluent imports: wip - for (KStreamSource branch : branches) { - branch.bind(context, metadata); ->>>>>>> wip - } - } - - @SuppressWarnings("unchecked") - @Override -<<<<<<< HEAD -<<<<<<< HEAD - public void receive(Object key, Object value, long timestamp) { - for (int i = 0; i < predicates.length; i++) { - Predicate predicate = predicates[i]; - if (predicate.apply((K)key, (V)value)) { - branches[i].receive(key, value, timestamp); - return; - } - } - } - - @Override - public void close() { - for (KStreamSource branch : branches) { - branch.close(); -======= - public void receive(Object key, Object value, long timestamp, long streamTime) { -======= - public void receive(Object key, Object value, long timestamp) { ->>>>>>> remove streamTime from Receiver - for (int i = 0; i < predicates.length; i++) { - Predicate predicate = predicates[i]; - if (predicate.apply((K)key, (V)value)) { - branches[i].receive(key, value, timestamp); - return; - } ->>>>>>> new api model - } - } - - @Override - public void close() { - for (KStreamSource branch : branches) { - branch.close(); - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java deleted file mode 100644 index ead6b56f7a4d6..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFilter.java +++ /dev/null @@ -1,55 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.Predicate; -======= -import org.apache.kafka.stream.topology.KStreamTopology; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.topology.Predicate; - -/** - * Created by yasuhiro on 6/17/15. - */ -class KStreamFilter extends KStreamImpl { - - private final Predicate predicate; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamFilter(Predicate predicate, KStreamTopology topology) { - super(topology); -======= - KStreamFilter(Predicate predicate, KStreamInitializer initializer) { -======= - KStreamFilter(Predicate predicate, KStreamTopology initializer) { ->>>>>>> wip - super(initializer); ->>>>>>> new api model -======= - KStreamFilter(Predicate predicate, KStreamTopology topology) { - super(topology); ->>>>>>> fix parameter name - this.predicate = predicate; - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized(this) { - if (predicate.apply((K)key, (V)value)) { - forward(key, value, timestamp); - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java deleted file mode 100644 index 63c30ce9b0e78..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMap.java +++ /dev/null @@ -1,69 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamContext; -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.KeyValueMapper; -import io.confluent.streaming.KeyValue; -======= -import org.apache.kafka.clients.processor.ProcessorContext; -======= -import org.apache.kafka.stream.KStreamContext; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.KeyValue; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.topology.KeyValueMapper; - -/** - * Created by yasuhiro on 6/17/15. - */ -class KStreamFlatMap extends KStreamImpl { - - private final KeyValueMapper, K1, V1> mapper; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology topology) { - super(topology); -======= - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamInitializer initializer) { -======= - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology initializer) { ->>>>>>> wip - super(initializer); ->>>>>>> new api model -======= - KStreamFlatMap(KeyValueMapper, K1, V1> mapper, KStreamTopology topology) { - super(topology); ->>>>>>> fix parameter name - this.mapper = mapper; - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - super.bind(context, KStreamMetadata.unjoinable()); - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized(this) { - KeyValue> newPair = mapper.apply((K1)key, (V1)value); - for (V v : newPair.value) { - forward(newPair.key, v, timestamp); - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java deleted file mode 100644 index b18ac9c080430..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamFlatMapValues.java +++ /dev/null @@ -1,56 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.ValueMapper; -======= -import org.apache.kafka.stream.topology.KStreamTopology; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.topology.ValueMapper; - -/** - * Created by yasuhiro on 6/17/15. - */ -class KStreamFlatMapValues extends KStreamImpl { - - private final ValueMapper, V1> mapper; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology topology) { - super(topology); -======= - KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamInitializer initializer) { -======= - KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology initializer) { ->>>>>>> wip - super(initializer); ->>>>>>> new api model -======= - KStreamFlatMapValues(ValueMapper, V1> mapper, KStreamTopology topology) { - super(topology); ->>>>>>> fix parameter name - this.mapper = mapper; - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized(this) { - Iterable newValues = mapper.apply((V1)value); - for (V v : newValues) { - forward(key, v, timestamp); - } - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java deleted file mode 100644 index 8b4462ae01b1c..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamImpl.java +++ /dev/null @@ -1,325 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -import org.apache.kafka.stream.topology.Processor; -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.KStream; -<<<<<<< HEAD -import io.confluent.streaming.KStreamContext; -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.KStreamWindowed; -import io.confluent.streaming.KeyValueMapper; -import io.confluent.streaming.Predicate; -import io.confluent.streaming.Processor; -import io.confluent.streaming.Transformer; -import io.confluent.streaming.ValueMapper; -import io.confluent.streaming.Window; -======= ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.stream.internal.ProcessorNode; -import org.apache.kafka.stream.internal.Receiver; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.KStreamWindowed; -import org.apache.kafka.stream.topology.KeyValueMapper; -import org.apache.kafka.stream.topology.Predicate; -import org.apache.kafka.stream.topology.Transformer; -import org.apache.kafka.stream.topology.ValueMapper; -import org.apache.kafka.stream.topology.Window; - -import java.util.ArrayList; - -/** - * Created by yasuhiro on 6/17/15. - */ -abstract class KStreamImpl implements KStream, Receiver { - - private final ArrayList nextReceivers = new ArrayList<>(1); -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - protected KStreamTopology topology; - protected KStreamContext context; - protected KStreamMetadata metadata; - - protected KStreamImpl(KStreamTopology topology) { - this.topology = topology; - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); - this.context = context; - this.metadata = metadata; - int numReceivers = nextReceivers.size(); - for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).bind(context, metadata); - } - } - - @Override - public void close() { - int numReceivers = nextReceivers.size(); - for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).close(); -======= -======= - protected KStreamTopology initializer; -======= - protected KStreamTopology topology; ->>>>>>> removed ProcessorContext - protected KStreamContext context; ->>>>>>> wip - protected KStreamMetadata metadata; - - protected KStreamImpl(KStreamTopology topology) { - this.topology = topology; - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - if (this.context != null) throw new IllegalStateException("kstream topology is already bound"); - this.context = context; - this.metadata = metadata; - int numReceivers = nextReceivers.size(); - for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).bind(context, metadata); ->>>>>>> new api model - } - } - - @Override - public void close() { - int numReceivers = nextReceivers.size(); - for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).close(); - } - } - - @Override - public KStream filter(Predicate predicate) { -<<<<<<< HEAD -<<<<<<< HEAD - return chain(new KStreamFilter(predicate, topology)); -======= - return chain(new KStreamFilter(predicate, initializer)); ->>>>>>> new api model -======= - return chain(new KStreamFilter(predicate, topology)); ->>>>>>> removed ProcessorContext - } - - @Override - public KStream filterOut(final Predicate predicate) { - return filter(new Predicate() { - public boolean apply(K key, V value) { - return !predicate.apply(key, value); - } - }); - } - - @Override - public KStream map(KeyValueMapper mapper) { -<<<<<<< HEAD -<<<<<<< HEAD - return chain(new KStreamMap(mapper, topology)); -======= - return chain(new KStreamMap(mapper, initializer)); ->>>>>>> new api model -======= - return chain(new KStreamMap(mapper, topology)); ->>>>>>> removed ProcessorContext - } - - @Override - public KStream mapValues(ValueMapper mapper) { -<<<<<<< HEAD -<<<<<<< HEAD - return chain(new KStreamMapValues(mapper, topology)); -======= - return chain(new KStreamMapValues(mapper, initializer)); ->>>>>>> new api model -======= - return chain(new KStreamMapValues(mapper, topology)); ->>>>>>> removed ProcessorContext - } - - @Override - public KStream flatMap(KeyValueMapper, K, V> mapper) { -<<<<<<< HEAD -<<<<<<< HEAD - return chain(new KStreamFlatMap(mapper, topology)); -======= - return chain(new KStreamFlatMap(mapper, initializer)); ->>>>>>> new api model -======= - return chain(new KStreamFlatMap(mapper, topology)); ->>>>>>> removed ProcessorContext - } - - @Override - public KStream flatMapValues(ValueMapper, V> mapper) { -<<<<<<< HEAD -<<<<<<< HEAD - return chain(new KStreamFlatMapValues(mapper, topology)); -======= - return chain(new KStreamFlatMapValues(mapper, initializer)); ->>>>>>> new api model -======= - return chain(new KStreamFlatMapValues(mapper, topology)); ->>>>>>> removed ProcessorContext - } - - @Override - public KStreamWindowed with(Window window) { -<<<<<<< HEAD -<<<<<<< HEAD - return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, topology)); -======= - return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, initializer)); ->>>>>>> new api model -======= - return (KStreamWindowed)chain(new KStreamWindowedImpl<>(window, topology)); ->>>>>>> removed ProcessorContext - } - - @Override - public KStream[] branch(Predicate... predicates) { -<<<<<<< HEAD -<<<<<<< HEAD - KStreamBranch branch = new KStreamBranch<>(predicates, topology); -======= - KStreamBranch branch = new KStreamBranch<>(predicates, initializer); ->>>>>>> new api model -======= - KStreamBranch branch = new KStreamBranch<>(predicates, topology); ->>>>>>> removed ProcessorContext - registerReceiver(branch); - return branch.branches; - } - - @SuppressWarnings("unchecked") - @Override - public KStream through(String topic) { - return through(topic, null, null, null, null); - } - - @SuppressWarnings("unchecked") - @Override - public KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { - process(this.getSendProcessor(topic, keySerializer, valSerializer)); -<<<<<<< HEAD -<<<<<<< HEAD - return topology.from(keyDeserializer, valDeserializer, topic); -======= - return initializer.from(keyDeserializer, valDeserializer, topic); ->>>>>>> new api model -======= - return topology.from(keyDeserializer, valDeserializer, topic); ->>>>>>> removed ProcessorContext - } - - @Override - public void sendTo(String topic) { - process(this.getSendProcessor(topic, null, null)); - } - - @Override - public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { - process(this.getSendProcessor(topic, keySerializer, valSerializer)); - } - - @SuppressWarnings("unchecked") - private Processor getSendProcessor(final String sendTopic, final Serializer keySerializer, final Serializer valSerializer) { - return new Processor() { - private KStreamContext context; - - @Override - public void init(KStreamContext context) { - this.context = context; - } - @Override - public void process(K key, V value) { -<<<<<<< HEAD -<<<<<<< HEAD - this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); -======= - this.processorContext.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); ->>>>>>> wip -======= - this.context.send(sendTopic, key, value, (Serializer) keySerializer, (Serializer) valSerializer); ->>>>>>> removed ProcessorContext - } - @Override - public void punctuate(long streamTime) {} - @Override - public void close() {} - }; - } - - @SuppressWarnings("unchecked") - @Override -<<<<<<< HEAD -<<<<<<< HEAD - public void process(Processor processor) { - registerReceiver(new ProcessorNode<>(processor)); - } - - @SuppressWarnings("unchecked") - @Override - public KStream transform(Transformer transformer) { - return chain(new KStreamTransform<>(transformer, topology)); -======= - public void process(final Processor processor) { -<<<<<<< HEAD - Receiver receiver = new Receiver() { - public void bind(KStreamContext context, KStreamMetadata metadata) { - processor.init(new ProcessorContextImpl(context, context.getPunctuationScheduler(processor))); - } - public void receive(Object key, Object value, long timestamp, long streamTime) { - processor.process((K) key, (V) value); - } - }; - registerReceiver(receiver); ->>>>>>> new api model -======= -======= - public void process(Processor processor) { ->>>>>>> KStream.tranform method for generalized transformation - registerReceiver(new ProcessorNode<>(processor)); ->>>>>>> wip - } - - @SuppressWarnings("unchecked") - @Override - public KStream transform(Transformer transformer) { - return chain(new KStreamTransform<>(transformer, topology)); - } - - void registerReceiver(Receiver receiver) { - nextReceivers.add(receiver); - } - - protected void forward(Object key, Object value, long timestamp) { - int numReceivers = nextReceivers.size(); - for (int i = 0; i < numReceivers; i++) { - nextReceivers.get(i).receive(key, value, timestamp); - } - } - - protected KStream chain(KStreamImpl kstream) { - synchronized(this) { - nextReceivers.add(kstream); - return kstream; - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java deleted file mode 100644 index d2058e8cb9dd2..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamJoin.java +++ /dev/null @@ -1,153 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamContext; -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.NotCopartitionedException; -import io.confluent.streaming.ValueJoiner; -import io.confluent.streaming.Window; -======= -import org.apache.kafka.clients.processor.ProcessorContext; -======= -import org.apache.kafka.stream.KStreamContext; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.NotCopartitionedException; -import org.apache.kafka.stream.internal.Receiver; -import org.apache.kafka.stream.topology.KStreamTopology; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.topology.ValueJoiner; -import org.apache.kafka.stream.topology.Window; - -import java.util.Iterator; - -/** - * Created by yasuhiro on 6/17/15. - */ -class KStreamJoin extends KStreamImpl { - - private static abstract class Finder { - abstract Iterator find(K key, long timestamp); - } - - private final Finder finder1; - private final Finder finder2; - private final ValueJoiner joiner; - final Receiver receiverForOtherStream; - private KStreamMetadata thisMetadata; - private KStreamMetadata otherMetadata; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology topology) { - super(topology); -======= - KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamInitializer initializer) { -======= - KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology initializer) { ->>>>>>> wip - super(initializer); ->>>>>>> new api model -======= - KStreamJoin(KStreamWindowedImpl stream1, KStreamWindowedImpl stream2, boolean prior, ValueJoiner joiner, KStreamTopology topology) { - super(topology); ->>>>>>> fix parameter name - - final Window window1 = stream1.window; - final Window window2 = stream2.window; - - if (prior) { - this.finder1 = new Finder() { - Iterator find(K key, long timestamp) { - return window1.findAfter(key, timestamp); - } - }; - this.finder2 = new Finder() { - Iterator find(K key, long timestamp) { - return window2.findBefore(key, timestamp); - } - }; - } - else { - this.finder1 = new Finder() { - Iterator find(K key, long timestamp) { - return window1.find(key, timestamp); - } - }; - this.finder2 = new Finder() { - Iterator find(K key, long timestamp) { - return window2.find(key, timestamp); - } - }; - } - - this.joiner = joiner; - - this.receiverForOtherStream = getReceiverForOther(); - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - super.bind(context, metadata); - - thisMetadata = metadata; - if (otherMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) throw new NotCopartitionedException(); - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - Iterator iter = finder2.find((K)key, timestamp); - if (iter != null) { - while (iter.hasNext()) { - doJoin((K)key, (V1)value, iter.next(), timestamp); - } - } - } - - private Receiver getReceiverForOther() { - return new Receiver() { - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - otherMetadata = metadata; - if (thisMetadata != null && !thisMetadata.isJoinCompatibleWith(otherMetadata)) throw new NotCopartitionedException(); - } -<<<<<<< HEAD -<<<<<<< HEAD -======= - ->>>>>>> new api model -======= ->>>>>>> wip - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value2, long timestamp) { - Iterator iter = finder1.find((K)key, timestamp); - if (iter != null) { - while (iter.hasNext()) { - doJoin((K)key, iter.next(), (V2)value2, timestamp); - } - } - } - @Override - public void close() { - // down stream instances are close when the primary stream is closed - } - }; - } - - // TODO: use the "outer-stream" topic as the resulted join stream topic - private void doJoin(K key, V1 value1, V2 value2, long timestamp) { - forward(key, joiner.apply(value1, value2), timestamp); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java deleted file mode 100644 index be73101fe7ba8..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMap.java +++ /dev/null @@ -1,67 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamContext; -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.KeyValueMapper; -import io.confluent.streaming.KeyValue; -======= -import org.apache.kafka.clients.processor.ProcessorContext; -======= -import org.apache.kafka.stream.KStreamContext; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.KeyValue; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.topology.KeyValueMapper; - -/** - * Created by yasuhiro on 6/17/15. - */ -class KStreamMap extends KStreamImpl { - - private final KeyValueMapper mapper; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamMap(KeyValueMapper mapper, KStreamTopology topology) { - super(topology); -======= - KStreamMap(KeyValueMapper mapper, KStreamInitializer initializer) { -======= - KStreamMap(KeyValueMapper mapper, KStreamTopology initializer) { ->>>>>>> wip - super(initializer); ->>>>>>> new api model -======= - KStreamMap(KeyValueMapper mapper, KStreamTopology topology) { - super(topology); ->>>>>>> fix parameter name - this.mapper = mapper; - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - super.bind(context, KStreamMetadata.unjoinable()); - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - KeyValue newPair = mapper.apply((K1)key, (V1)value); - forward(newPair.key, newPair.value, timestamp); - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java deleted file mode 100644 index 28df6b3837f69..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamMapValues.java +++ /dev/null @@ -1,54 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.ValueMapper; -======= -import org.apache.kafka.stream.topology.KStreamTopology; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.topology.ValueMapper; - -/** - * Created by yasuhiro on 6/17/15. - */ -class KStreamMapValues extends KStreamImpl { - - private final ValueMapper mapper; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamMapValues(ValueMapper mapper, KStreamTopology topology) { - super(topology); -======= - KStreamMapValues(ValueMapper mapper, KStreamInitializer initializer) { -======= - KStreamMapValues(ValueMapper mapper, KStreamTopology initializer) { ->>>>>>> wip - super(initializer); ->>>>>>> new api model -======= - KStreamMapValues(ValueMapper mapper, KStreamTopology topology) { - super(topology); ->>>>>>> fix parameter name - this.mapper = mapper; - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized (this) { - V newValue = mapper.apply((V1)value); - forward(key, newValue, timestamp); - } - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java deleted file mode 100644 index 670f1ae316d27..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamSource.java +++ /dev/null @@ -1,130 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamTopology; -<<<<<<< HEAD -======= -======= -import io.confluent.streaming.KStreamContext; ->>>>>>> wip -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= ->>>>>>> wip -======= -import org.apache.kafka.clients.processor.ProcessorContext; ->>>>>>> removing io.confluent imports: wip -======= -import org.apache.kafka.stream.KStreamContext; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.stream.topology.KStreamTopology; - -import java.util.Arrays; -import java.util.HashSet; -import java.util.Set; - -/** - * Created by yasuhiro on 6/17/15. - */ -public class KStreamSource extends KStreamImpl { - - private Deserializer keyDeserializer; - private Deserializer valueDeserializer; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - String[] topics; -======= - public String[] topics; ->>>>>>> removing io.confluent imports: wip - - public KStreamSource(String[] topics, KStreamTopology topology) { - this(topics, null, null, topology); - } -======= - private Deserializer keyDeserializer; - private Deserializer valueDeserializer; ->>>>>>> wip - - public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology topology) { - super(topology); -======= - final String[] topics; -======= - String[] topics; ->>>>>>> wip - - public KStreamSource(String[] topics, KStreamTopology topology) { - this(topics, null, null, topology); - } - -<<<<<<< HEAD - public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology initializer) { - super(initializer); ->>>>>>> new api model -======= - public KStreamSource(String[] topics, Deserializer keyDeserializer, Deserializer valueDeserializer, KStreamTopology topology) { - super(topology); ->>>>>>> fix parameter name - this.topics = topics; - this.keyDeserializer = keyDeserializer; - this.valueDeserializer = valueDeserializer; - } - - @SuppressWarnings("unchecked") -<<<<<<< HEAD -======= - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - if (keyDeserializer == null) keyDeserializer = (Deserializer) context.keyDeserializer(); - if (valueDeserializer == null) valueDeserializer = (Deserializer) context.valueDeserializer(); - - super.bind(context, metadata); - } - ->>>>>>> wip - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - if (keyDeserializer == null) keyDeserializer = (Deserializer) context.keyDeserializer(); - if (valueDeserializer == null) valueDeserializer = (Deserializer) context.valueDeserializer(); - - super.bind(context, metadata); - } - - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized(this) { - // KStream needs to forward the topic name since it is directly from the Kafka source - forward(key, value, timestamp); - } - } - - public Deserializer keyDeserializer() { - return keyDeserializer; - } - - public Deserializer valueDeserializer() { - return valueDeserializer; - } - -<<<<<<< HEAD -<<<<<<< HEAD -======= ->>>>>>> wip - public Set topics() { - return new HashSet<>(Arrays.asList(topics)); - } - -<<<<<<< HEAD -======= ->>>>>>> wip -======= ->>>>>>> wip -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java b/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java deleted file mode 100644 index e097bfb11b490..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internal/KStreamWindowedImpl.java +++ /dev/null @@ -1,97 +0,0 @@ -package org.apache.kafka.stream.topology.internal; - - -import org.apache.kafka.stream.KStreamContext; -import org.apache.kafka.stream.KStream; -<<<<<<< HEAD -import io.confluent.streaming.KStreamContext; -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.KStreamWindowed; -import io.confluent.streaming.ValueJoiner; -import io.confluent.streaming.Window; -======= -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.KStreamWindowed; ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.stream.topology.ValueJoiner; -import org.apache.kafka.stream.topology.Window; - -/** - * Created by yasuhiro on 6/18/15. - */ -public class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { - - final Window window; - -<<<<<<< HEAD -<<<<<<< HEAD - KStreamWindowedImpl(Window window, KStreamTopology initializer) { -======= - KStreamWindowedImpl(Window window, KStreamInitializer initializer) { ->>>>>>> new api model -======= - KStreamWindowedImpl(Window window, KStreamTopology initializer) { ->>>>>>> wip - super(initializer); - this.window = window; - } - - @Override - public void bind(KStreamContext context, KStreamMetadata metadata) { - super.bind(context, metadata); - window.init(context); - } - - @SuppressWarnings("unchecked") - @Override - public void receive(Object key, Object value, long timestamp) { - synchronized(this) { - window.put((K)key, (V)value, timestamp); - // KStreamWindowed needs to forward the topic name since it may receive directly from KStreamSource - forward(key, value, timestamp); - } - } - - @Override - public void close() { - window.close(); - super.close(); - } - - @Override - public KStream join(KStreamWindowed other, ValueJoiner processor) { - return join(other, false, processor); - } - - @Override - public KStream joinPrior(KStreamWindowed other, ValueJoiner processor) { - return join(other, true, processor); - } - - private KStream join(KStreamWindowed other, boolean prior, ValueJoiner processor) { - - KStreamWindowedImpl otherImpl = (KStreamWindowedImpl) other; - -<<<<<<< HEAD -<<<<<<< HEAD - KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, topology); -======= - KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, initializer); ->>>>>>> new api model -======= - KStreamJoin stream = new KStreamJoin<>(this, otherImpl, prior, processor, topology); ->>>>>>> removed ProcessorContext - otherImpl.registerReceiver(stream.receiverForOtherStream); - - return chain(stream); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java deleted file mode 100644 index fbb91b92c83e7..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/KStreamThread.java +++ /dev/null @@ -1,335 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.KStreamJob; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.StreamingConfig; -import io.confluent.streaming.util.ParallelExecutor; -import io.confluent.streaming.util.Util; -======= ->>>>>>> removing io.confluent imports: wip -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; -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.serialization.Serializer; -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.stream.StreamingConfig; -import org.apache.kafka.stream.internals.IngestorImpl; -import org.apache.kafka.stream.internals.KStreamContextImpl; -import org.apache.kafka.stream.internals.ProcessorConfig; -import org.apache.kafka.stream.internals.RecordCollectorImpl; -import org.apache.kafka.stream.internals.StreamGroup; -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.util.ParallelExecutor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -public class KStreamThread extends Thread { - - private static final Logger log = LoggerFactory.getLogger(KStreamThread.class); - - private final KStreamTopology topology; - private final ArrayList streamGroups = new ArrayList<>(); - private final ParallelExecutor parallelExecutor; - private final Map kstreamContexts = new HashMap<>(); - private final IngestorImpl ingestor; - private final RecordCollectorImpl collector; - private final StreamingConfig streamingConfig; - private final ProcessorConfig config; - private final Metrics metrics; - private final KafkaStreamingMetrics streamingMetrics; - private final Time time; - private volatile boolean running; - private long lastCommit; - private long nextStateCleaning; - private long recordsProcessed; - - protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { - @Override - public void onPartitionsAssigned(Consumer consumer, Collection assignment) { - ingestor.init(); - addPartitions(assignment); - } - - @Override - public void onPartitionsRevoked(Consumer consumer, Collection assignment) { - commitAll(time.milliseconds()); - removePartitions(); - ingestor.clear(); - } - }; - - @SuppressWarnings("unchecked") - public KStreamThread(KStreamTopology topology, Set topics, StreamingConfig streamingConfig, Metrics metrics) { - super(); - this.config = new ProcessorConfig(streamingConfig.config()); - this.topology = topology; - this.streamingConfig = streamingConfig; - this.metrics = metrics; - this.streamingMetrics = new KafkaStreamingMetrics(); - - Consumer consumer = new KafkaConsumer<>(streamingConfig.config(), rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); - this.ingestor = new IngestorImpl(consumer, topics); - - Producer producer = new KafkaProducer<>(streamingConfig.config(), new ByteArraySerializer(), new ByteArraySerializer()); - this.collector = new RecordCollectorImpl(producer, (Serializer) streamingConfig.keySerializer(), (Serializer) streamingConfig.valueSerializer()); - - this.running = true; - this.lastCommit = 0; - this.nextStateCleaning = Long.MAX_VALUE; - this.recordsProcessed = 0; - this.time = new SystemTime(); - - // TODO: Fix this after the threading model is decided (also fix KafkaStreaming) - this.parallelExecutor = new ParallelExecutor(this.config.numStreamThreads); - } - - /** - * Execute the stream processors - */ - public synchronized void run() { - log.info("Starting a kstream thread"); - try { - ingestor.open(); - runLoop(); - } catch (RuntimeException e) { - log.error("Uncaught error during processing: ", e); - throw e; - } finally { - shutdown(); - } - } - - private void shutdown() { - log.info("Shutting down a kstream thread"); - commitAll(time.milliseconds()); - - collector.close(); - ingestor.close(); - parallelExecutor.shutdown(); - removePartitions(); - log.info("kstream thread shutdown complete"); - } - - /** - * Shutdown this streaming thread. - */ - public synchronized void close() { - running = false; - } - - private void runLoop() { - try { - boolean readyForNextExecution = false; - - while (stillRunning()) { - ingestor.poll(readyForNextExecution ? 0 : this.config.pollTimeMs); - - readyForNextExecution = parallelExecutor.execute(streamGroups); - - maybeCommit(); - maybeCleanState(); - } - } catch (Exception e) { - throw new KafkaException(e); - } - } - - private boolean stillRunning() { - if (!running) { - log.debug("Shutting down at user request."); - return false; - } - if (config.totalRecordsToProcess >= 0 && recordsProcessed >= config.totalRecordsToProcess) { - log.debug("Shutting down as we've reached the user-configured limit of {} records to process.", config.totalRecordsToProcess); - return false; - } - return true; - } - - private void maybeCommit() { - long now = time.milliseconds(); - if (config.commitTimeMs >= 0 && lastCommit + config.commitTimeMs < now) { - log.trace("Committing processor instances because the commit interval has elapsed."); - commitAll(now); - } - } - - private void commitAll(long now) { - Map commit = new HashMap<>(); - for (KStreamContextImpl context : kstreamContexts.values()) { - context.flush(); - commit.putAll(context.consumedOffsets()); - } - - // check if commit is really needed, i.e. if all the offsets are already committed - if (ingestor.commitNeeded(commit)) { - // TODO: for exactly-once we need to make sure the flush and commit - // are executed atomically whenever it is triggered by user - collector.flush(); - ingestor.commit(commit); // TODO: can this be async? - streamingMetrics.commitTime.record(now - lastCommit); - } - } - - /* delete any state dirs that aren't for active contexts */ - private void maybeCleanState() { - long now = time.milliseconds(); - if (now > nextStateCleaning) { - File[] stateDirs = config.stateDir.listFiles(); - if (stateDirs != null) { - for (File dir : stateDirs) { - try { - Integer id = Integer.parseInt(dir.getName()); - if (!kstreamContexts.keySet().contains(id)) { - log.info("Deleting obsolete state directory {} after {} delay ms.", dir.getAbsolutePath(), config.stateCleanupDelay); - Utils.rm(dir); - } - } catch (NumberFormatException e) { - log.warn("Deleting unknown directory in state directory {}.", dir.getAbsolutePath()); - Utils.rm(dir); - } - } - } - nextStateCleaning = Long.MAX_VALUE; - } - } - - private void addPartitions(Collection assignment) { - HashSet partitions = new HashSet<>(assignment); - -<<<<<<< HEAD -<<<<<<< HEAD - ingestor.init(); -======= - Consumer restoreConsumer = - new KafkaConsumer<>(streamingConfig.config(), null, new ByteArrayDeserializer(), new ByteArrayDeserializer()); ->>>>>>> close stream groups in context - - for (TopicPartition partition : partitions) { - final Integer id = partition.partition(); // TODO: switch this to the group id - KStreamContextImpl context = kstreamContexts.get(id); - if (context == null) { - try { - context = new KStreamContextImpl(id, ingestor, collector, streamingConfig, config, metrics); - context.init(topology.sourceStreams()); - -======= - for (TopicPartition partition : partitions) { - final Integer id = partition.partition(); // TODO: switch this to the group id - KStreamContextImpl context = kstreamContexts.get(id); - if (context == null) { - try { - context = new KStreamContextImpl(id, ingestor, collector, streamingConfig, config, metrics); - context.init(topology.sourceStreams()); - ->>>>>>> new api model - kstreamContexts.put(id, context); - } catch (Exception e) { - throw new KafkaException(e); - } - - streamGroups.add(context.streamGroup); - } - } - - nextStateCleaning = time.milliseconds() + config.stateCleanupDelay; - } - - private void removePartitions() { - for (KStreamContextImpl context : kstreamContexts.values()) { - log.info("Removing task context {}", context.id()); - try { - context.close(); - } catch (Exception e) { - throw new KafkaException(e); - } - streamingMetrics.processorDestruction.record(); - } - streamGroups.clear(); - } - - private class KafkaStreamingMetrics { - final Sensor commitTime; - final Sensor processTime; - final Sensor windowTime; - final Sensor processorCreation; - final Sensor processorDestruction; - - public KafkaStreamingMetrics() { - String group = "kafka-streaming"; - - this.commitTime = metrics.sensor("commit-time"); - this.commitTime.add(new MetricName(group, "commit-time-avg-ms"), new Avg()); - this.commitTime.add(new MetricName(group, "commits-time-max-ms"), new Max()); - this.commitTime.add(new MetricName(group, "commits-per-second"), new Rate(new Count())); - - this.processTime = metrics.sensor("process-time"); - this.commitTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); - this.commitTime.add(new MetricName(group, "process-time-max-ms"), new Max()); - this.commitTime.add(new MetricName(group, "process-calls-per-second"), new Rate(new Count())); - - this.windowTime = metrics.sensor("window-time"); - this.windowTime.add(new MetricName(group, "window-time-avg-ms"), new Avg()); - this.windowTime.add(new MetricName(group, "window-time-max-ms"), new Max()); - this.windowTime.add(new MetricName(group, "window-calls-per-second"), new Rate(new Count())); - - this.processorCreation = metrics.sensor("processor-creation"); - this.processorCreation.add(new MetricName(group, "processor-creation"), new Rate(new Count())); - - this.processorDestruction = metrics.sensor("processor-destruction"); - this.processorDestruction.add(new MetricName(group, "processor-destruction"), new Rate(new Count())); - - } - - } - -} diff --git a/stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java b/stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java deleted file mode 100644 index 5ce2c7b75e363..0000000000000 --- a/stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java +++ /dev/null @@ -1,159 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.stream.topology.internals; - -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.stream.util.Stamped; - -import java.util.Iterator; - -public class WindowSupport { - - public static class ValueList { - Value head = null; - Value tail = null; - Value dirty = null; - - public void add(int slotNum, V value, long timestamp) { - Value v = new Value<>(slotNum, value, timestamp); - if (tail != null) { - tail.next = v; - } else { - head = v; - } - tail = v; - if (dirty == null) dirty = v; - } - - public Value first() { - return head; - } - - public void removeFirst() { - if (head != null) { - if (head == tail) tail = null; - head = head.next; - } - } - - public boolean isEmpty() { - return head == null; - } - - public boolean hasDirtyValues() { - return dirty != null; - } - - public void clearDirtyValues() { - dirty = null; - } - - public Iterator> iterator() { - return new ValueListIterator(head); - } - - public Iterator> dirtyValueIterator() { - return new ValueListIterator(dirty); - } - - } - - private static class ValueListIterator implements Iterator> { - - Value ptr; - - ValueListIterator(Value start) { - ptr = start; - } - - @Override - public boolean hasNext() { - return ptr != null; - } - - @Override - public Value next() { - Value value = ptr; - if (value != null) ptr = value.next; - return value; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - - } - - public static class Value extends Stamped { - public final int slotNum; - private Value next = null; - - Value(int slotNum, V value, long timestamp) { - super(value, timestamp); - this.slotNum = slotNum; - } - } - - - public static long getLong(byte[] bytes, int offset) { - long value = 0; - for (int i = 0; i < 8; i++) { - value = (value << 8) | bytes[offset + i]; - } - return value; - } - - public static int getInt(byte[] bytes, int offset) { - int value = 0; - for (int i = 0; i < 4; i++) { - value = (value << 8) | bytes[offset + i]; - } - return value; - } - - public static int putLong(byte[] bytes, int offset, long value) { - for (int i = 7; i >= 0; i--) { - bytes[offset + i] = (byte) (value & 0xFF); - value = value >> 8; - } - return 8; - } - - public static int putInt(byte[] bytes, int offset, int value) { - for (int i = 3; i >= 0; i--) { - bytes[offset + i] = (byte) (value & 0xFF); - value = value >> 8; - } - return 4; - } - - public static int puts(byte[] bytes, int offset, byte[] value) { - offset += putInt(bytes, offset, value.length); - System.arraycopy(bytes, offset, value, 0, value.length); - return 4 + value.length; - } - - - public static T deserialize(byte[] bytes, int offset, int length, String topic, Deserializer deserializer) { - byte[] buf = new byte[length]; - System.arraycopy(bytes, offset, buf, 0, length); - return deserializer.deserialize(topic, buf); - } - -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/SlidingWindow.java b/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/SlidingWindow.java deleted file mode 100644 index 2ed323aaf419b..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/kstream/internals/SlidingWindow.java +++ /dev/null @@ -1,147 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.kstream.internals; - -import org.apache.kafka.streaming.kstream.Window; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.internals.Stamped; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; - -public class SlidingWindow implements Window { - - private String name; - private final long duration; - private final int maxCount; - private LinkedList list = new LinkedList(); - private HashMap>> map = new HashMap>>(); - - public SlidingWindow(String name, long duration, int maxCount) { - this.name = name; - this.duration = duration; - this.maxCount = maxCount; - } - - @Override - public void init(ProcessorContext context) { - } - - @Override - public Iterator findAfter(K key, final long timestamp) { - return find(key, timestamp, timestamp + duration); - } - - @Override - public Iterator findBefore(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp); - } - - @Override - public Iterator find(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp + duration); - } - - /* - * finds items in the window between startTime and endTime (both inclusive) - */ - private Iterator find(K key, final long startTime, final long endTime) { - final LinkedList> values = map.get(key); - - if (values == null) { - return null; - } else { - return new FilteredIterator>(values.iterator()) { - @Override - protected V filter(Stamped item) { - if (startTime <= item.timestamp && item.timestamp <= endTime) - return item.value; - else - return null; - } - }; - } - } - - @Override - public void put(K key, V value, long timestamp) { - list.offerLast(key); - - LinkedList> values = map.get(key); - if (values == null) { - values = new LinkedList>(); - map.put(key, values); - } - - values.offerLast(new Stamped(value, timestamp)); - - evictExcess(); - evictExpired(timestamp - duration); - } - - private void evictExcess() { - while (list.size() > maxCount) { - K oldestKey = list.pollFirst(); - - LinkedList> values = map.get(oldestKey); - values.removeFirst(); - - if (values.isEmpty()) map.remove(oldestKey); - } - } - - private void evictExpired(long cutoffTime) { - while (true) { - K oldestKey = list.peekFirst(); - - LinkedList> values = map.get(oldestKey); - Stamped oldestValue = values.peekFirst(); - - 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() { - // TODO - } - - @Override - public void close() { - // TODO - } - - @Override - public boolean persistent() { - // TODO: should not be persistent, right? - return false; - } -} diff --git a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java b/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java deleted file mode 100644 index af1954b23d1ac..0000000000000 --- a/stream/src/main/java/org/apache/kafka/streaming/processor/internals/StreamGroup.java +++ /dev/null @@ -1,285 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor.internals; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streaming.processor.Punctuator; -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.TimestampExtractor; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.Deserializer; - -import java.util.ArrayDeque; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.PriorityQueue; -import java.util.Set; - -/** - * A StreamGroup is composed of multiple streams from different topics that need to be synchronized. - */ -public class StreamGroup { - - private final Ingestor ingestor; - private final ProcessorContext context; - private final TimestampExtractor timestampExtractor; - - private final Map queuesPerPartition = new HashMap<>(); - private final PriorityQueue queuesByTime = new PriorityQueue<>(new Comparator() { - 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; - } - }); - - private final int desiredUnprocessed; - - // TODO: merge queuesPerPartition, consumedOffset, and newRecordBuffer into sth. like partition metadata - private final Map consumedOffsets; - private final PunctuationQueue punctuationQueue = new PunctuationQueue(); - private final ArrayDeque newRecordBuffer = new ArrayDeque<>(); - - private long streamTime = -1; - private boolean commitRequested = false; - private StampedRecord currRecord = null; - private ProcessorNode currNode = null; - private volatile int buffered = 0; - - /** - * Creates StreamGroup - * - * @param context the task context - * @param ingestor the instance of {@link Ingestor} - * @param timestampExtractor the instance of {@link TimestampExtractor} - * @param desiredUnprocessedPerPartition the target number of records kept in a queue for each topic - */ - public StreamGroup(ProcessorContext context, - Ingestor ingestor, - TimestampExtractor timestampExtractor, - int desiredUnprocessedPerPartition) { - this.context = context; - this.ingestor = ingestor; - this.timestampExtractor = timestampExtractor; - this.desiredUnprocessed = desiredUnprocessedPerPartition; - this.consumedOffsets = new HashMap<>(); - } - - public StampedRecord record() { - return currRecord; - } - - public ProcessorNode node() { - return currNode; - } - - public void setNode(ProcessorNode node) { - currNode = node; - } - - public Set partitions() { - return queuesPerPartition.keySet(); - } - - /** - * Adds a partition and its receiver to this stream synchronizer - * - * @param partition the partition - * @param source the instance of KStreamImpl - */ - @SuppressWarnings("unchecked") - public void addPartition(TopicPartition partition, SourceNode source) { - synchronized (this) { - RecordQueue recordQueue = queuesPerPartition.get(partition); - - if (recordQueue == null) { - queuesPerPartition.put(partition, createRecordQueue(partition, source)); - } else { - throw new IllegalStateException("duplicate partition"); - } - } - } - - /** - * Adds records - * - * @param partition the partition - * @param iterator the iterator of records - */ - @SuppressWarnings("unchecked") - public void addRecords(TopicPartition partition, Iterator> iterator) { - synchronized (this) { - newRecordBuffer.addLast(new NewRecords(partition, iterator)); - } - } - - @SuppressWarnings("unchecked") - private void ingestNewRecords() { - for (NewRecords newRecords : newRecordBuffer) { - TopicPartition partition = newRecords.partition; - Iterator> iterator = newRecords.iterator; - - RecordQueue recordQueue = queuesPerPartition.get(partition); - if (recordQueue != null) { - boolean wasEmpty = recordQueue.isEmpty(); - - while (iterator.hasNext()) { - ConsumerRecord record = iterator.next(); - - // deserialize the raw record, extract the timestamp and put into the queue - Deserializer keyDeserializer = ((SourceNode) recordQueue.source()).keyDeserializer; - Deserializer valDeserializer = ((SourceNode) recordQueue.source()).valDeserializer; - - Object key = keyDeserializer.deserialize(record.topic(), record.key()); - Object value = valDeserializer.deserialize(record.topic(), record.value()); - ConsumerRecord deserializedRecord = new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), key, value); - - long timestamp = timestampExtractor.extract(record.topic(), key, value); - recordQueue.add(new StampedRecord(deserializedRecord, timestamp)); - buffered++; - } - - int queueSize = recordQueue.size(); - - // add this record queue to be considered for processing - if (wasEmpty && queueSize > 0) - queuesByTime.offer(recordQueue); - - // if we have buffered enough for this partition, pause - if (queueSize >= this.desiredUnprocessed) { - ingestor.pause(partition); - } - } - } - newRecordBuffer.clear(); - } - - /** - * Schedules a punctuation for the processor - * - * @param punctuator the punctuator requesting scheduler - * @param interval the interval in milliseconds - */ - public void schedule(Punctuator punctuator, long interval) { - punctuationQueue.schedule(new PunctuationSchedule(punctuator, interval)); - } - - /** - * Processes one record - */ - @SuppressWarnings("unchecked") - public boolean process() { - synchronized (this) { - boolean readyForNextExecution = false; - ingestNewRecords(); - - // take the next record queue with the smallest estimated timestamp to process - RecordQueue recordQueue = queuesByTime.poll(); - - if (recordQueue == null) { - return false; - } - - if (recordQueue.size() == 0) throw new IllegalStateException("empty record queue"); - - if (recordQueue.size() == this.desiredUnprocessed) { - ingestor.unpause(recordQueue.partition(), recordQueue.offset()); - } - - long trackedTimestamp = recordQueue.timestamp(); - currRecord = recordQueue.next(); - currNode = recordQueue.source(); - - if (streamTime < trackedTimestamp) streamTime = trackedTimestamp; - - currNode.process(currRecord.key(), currRecord.value()); - consumedOffsets.put(recordQueue.partition(), currRecord.offset()); - - // TODO: local state flush and downstream producer flush - // need to be done altogether with offset commit atomically - if (commitRequested) { - // flush local state - context.flush(); - - // flush produced records in the downstream - context.recordCollector().flush(); - - // commit consumed offsets - ingestor.commit(consumedOffsets()); - } - - if (commitRequested) ingestor.commit(Collections.singletonMap( - new TopicPartition(currRecord.topic(), currRecord.partition()), - currRecord.offset())); - - // update this record queue's estimated timestamp - if (recordQueue.size() > 0) { - readyForNextExecution = true; - queuesByTime.offer(recordQueue); - } - - buffered--; - currRecord = null; - - punctuationQueue.mayPunctuate(streamTime); - - return readyForNextExecution; - } - } - - /** - * Returns consumed offsets - * - * @return the map of partition to consumed offset - */ - public Map consumedOffsets() { - return this.consumedOffsets; - } - - /** - * Request committing the current record's offset - */ - public void commitOffset() { - this.commitRequested = true; - } - - public void close() { - queuesByTime.clear(); - queuesPerPartition.clear(); - } - - protected RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { - return new RecordQueue(partition, source); - } - - private static class NewRecords { - final TopicPartition partition; - final Iterator> iterator; - - NewRecords(TopicPartition partition, Iterator> iterator) { - this.partition = partition; - this.iterator = iterator; - } - } -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java deleted file mode 100644 index 0becd1722e572..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/KStreamBranchTest.java +++ /dev/null @@ -1,182 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming; - -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.Predicate; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; -import org.junit.Test; - -import java.lang.reflect.Array; - -import static org.junit.Assert.assertEquals; - -public class KStreamBranchTest { - -<<<<<<< HEAD -<<<<<<< HEAD - private String topicName = "topic"; - - private KStreamMetadata streamMetadata = new KStreamMetadata(Collections.singletonMap(topicName, new PartitioningInfo(1))); - - @SuppressWarnings("unchecked") - @Test - public void testKStreamBranch() { - - Predicate isEven = new Predicate() { - @Override - public boolean apply(Integer key, String value) { - return (key % 2) == 0; - } - }; - Predicate isMultipleOfThree = new Predicate() { - @Override - public boolean apply(Integer key, String value) { - return (key % 3) == 0; - } - }; - Predicate isOdd = new Predicate() { - @Override - public boolean apply(Integer key, String value) { - return (key % 2) != 0; - } - }; - - final int[] expectedKeys = new int[] { 1, 2, 3, 4, 5, 6, 7 }; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology initializer = new MockKStreamTopology(); ->>>>>>> wip - KStreamSource stream; - KStream[] branches; - MockProcessor[] processors; - - stream = new KStreamSource<>(null, initializer); - branches = stream.branch(isEven, isMultipleOfThree, isOdd); - - assertEquals(3, branches.length); - - processors = (MockProcessor[]) Array.newInstance(MockProcessor.class, branches.length); - for (int i = 0; i < branches.length; i++) { - processors[i] = new MockProcessor<>(); - branches[i].process(processors[i]); - } - - for (int i = 0; i < expectedKeys.length; i++) { - stream.receive(expectedKeys[i], "V" + expectedKeys[i], 0L); -======= - private String topicName = "topic"; -======= - private String topic1 = "topic"; ->>>>>>> adding files - - private KStreamBuilder topology = new MockKStreamBuilder(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - @SuppressWarnings("unchecked") - @Test - public void testKStreamBranch() { - - Predicate isEven = new Predicate() { - @Override - public boolean apply(Integer key, String value) { - return (key % 2) == 0; - } - }; - Predicate isMultipleOfThree = new Predicate() { - @Override - public boolean apply(Integer key, String value) { - return (key % 3) == 0; - } - }; - Predicate isOdd = new Predicate() { - @Override - public boolean apply(Integer key, String value) { - return (key % 2) != 0; - } - }; - - final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - - KStream stream; - KStream[] branches; - MockProcessor[] processors; - - stream = topology.from(keyDeserializer, valDeserializer, topic1); - branches = stream.branch(isEven, isMultipleOfThree, isOdd); - - assertEquals(3, branches.length); - - processors = (MockProcessor[]) Array.newInstance(MockProcessor.class, branches.length); - for (int i = 0; i < branches.length; i++) { - processors[i] = new MockProcessor<>(); - branches[i].process(processors[i]); - } - - for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "V" + expectedKeys[i]); - } - - assertEquals(3, processors[0].processed.size()); -<<<<<<< HEAD - assertEquals(1, processors[1].processed.size()); - assertEquals(3, processors[2].processed.size()); - - // ----------------------------------------------- - - stream = topology.from(keyDeserializer, valDeserializer, topicName); - branches = stream.branch(isEven, isOdd, isMultipleOfThree); - - assertEquals(3, branches.length); - - processors = (MockProcessor[]) Array.newInstance(MockProcessor.class, branches.length); - for (int i = 0; i < branches.length; i++) { - processors[i] = new MockProcessor<>(); - branches[i].process(processors[i]); - } - - for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().receive(expectedKeys[i], "V" + expectedKeys[i]); - } - - assertEquals(3, processors[0].processed.size()); - assertEquals(4, processors[1].processed.size()); - assertEquals(0, processors[2].processed.size()); ->>>>>>> compile and test passed -======= - assertEquals(2, processors[1].processed.size()); - assertEquals(4, processors[2].processed.size()); ->>>>>>> adding files - } -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java deleted file mode 100644 index e9d3beda17d75..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/KStreamJoinTest.java +++ /dev/null @@ -1,744 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming; - -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.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.KStreamWindowed; -import org.apache.kafka.streaming.kstream.KeyValue; -import org.apache.kafka.streaming.kstream.KeyValueMapper; -import org.apache.kafka.streaming.kstream.ValueJoiner; -import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; -import org.apache.kafka.test.MockProcessorContext; -import org.apache.kafka.test.UnlimitedWindow; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class KStreamJoinTest { - - private String topic1 = "topic1"; - private String topic2 = "topic2"; - - private KStreamBuilder topology = new MockKStreamBuilder(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - private ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; - } - }; - - private ValueMapper valueMapper = new ValueMapper() { - @Override - public String apply(String value) { - return "#" + value; - } - }; - - private ValueMapper> valueMapper2 = new ValueMapper>() { - @Override - public Iterable apply(String value) { - return (Iterable) Utils.mkSet(value); - } - }; - - private KeyValueMapper keyValueMapper = - new KeyValueMapper() { - @Override - public KeyValue apply(Integer key, String value) { - return KeyValue.pair(key, value); - } - }; - - KeyValueMapper> keyValueMapper2 = - new KeyValueMapper>() { - @Override - public KeyValue> apply(Integer key, String value) { - return KeyValue.pair(key, (Iterable) Utils.mkSet(value)); - } - }; - - - @Test - public void testJoin() { - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -<<<<<<< HEAD -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology initializer = new MockKStreamTopology(); ->>>>>>> wip - processor = new TestProcessor<>(); -======= - processor = new MockProcessor<>(); ->>>>>>> removing io.confluent imports: wip - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); -======= - final int[] expectedKeys = new int[]{0, 1, 2, 3}; ->>>>>>> compile and test passed - - KStream stream1; - KStream stream2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; - String[] expected; - - processor = new MockProcessor<>(); - stream1 = topology.from(keyDeserializer, valDeserializer, topic1); - stream2 = topology.from(keyDeserializer, valDeserializer, topic2); - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); - - windowed1.join(windowed2, joiner).process(processor); - - MockProcessorContext context = new MockProcessorContext(null, null); - topology.init(context); - context.setTime(0L); - - // push two items to the main stream. the other stream's window is empty - - for (int i = 0; i < 2; i++) { - ((KStreamSource) stream1).source().process(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++) { - ((KStreamSource) stream2).source().process(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++) { - ((KStreamSource) stream1).source().process(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++) { - ((KStreamSource) stream2).source().process(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)); - } - } - - @Test - public void testJoinPrior() { - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - - KStream stream1; - KStream stream2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; - String[] expected; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology initializer = new MockKStreamTopology(); -<<<<<<< HEAD ->>>>>>> wip - processor = new TestProcessor<>(); -======= - processor = new MockProcessor<>(); ->>>>>>> removing io.confluent imports: wip - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); -======= - KStreamTopology initializer = new MockKStreamTopology(); - processor = new MockProcessor<>(); - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); ->>>>>>> compile and test passed - - boolean exceptionRaised = false; - - try { - windowed1.joinPrior(windowed2, joiner).process(processor); - - KStreamContext context = new MockKStreamContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } -======= - processor = new MockProcessor<>(); -<<<<<<< HEAD - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - windowed1 = stream1.with(new UnlimitedWindow<>()); - windowed2 = stream2.with(new UnlimitedWindow<>()); ->>>>>>> wip -======= - stream1 = topology.from(keyDeserializer, valDeserializer, topic1); - stream2 = topology.from(keyDeserializer, valDeserializer, topic2); - windowed1 = stream1.with(new UnlimitedWindow()); - windowed2 = stream2.with(new UnlimitedWindow()); ->>>>>>> adding files - - windowed1.joinPrior(windowed2, joiner).process(processor); - - MockProcessorContext context = new MockProcessorContext(null, null); - topology.init(context); - - // push two items to the main stream. the other stream's window is empty - - for (int i = 0; i < 2; i++) { - context.setTime(i); - - ((KStreamSource) stream1).source().process(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 - // no corresponding item in the main window has a newer timestamp - - for (int i = 0; i < 2; i++) { - context.setTime(i + 1); - - ((KStreamSource) stream2).source().process(expectedKeys[i], "Y" + expectedKeys[i]); - } - - assertEquals(0, processor.processed.size()); - - processor.processed.clear(); - - // push all items with newer timestamps to the main stream. this should produce two items. - - for (int i = 0; i < expectedKeys.length; i++) { - context.setTime(i + 2); - - ((KStreamSource) stream1).source().process(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 with older timestamps to the other stream. this should produce six items - for (int i = 0; i < expectedKeys.length; i++) { - context.setTime(i); - - ((KStreamSource) stream2).source().process(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)); - } -<<<<<<< HEAD - - } - - @Test - public void testMap() { - KStream stream1; - KStream stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; - - processor = new MockProcessor<>(); - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -<<<<<<< HEAD - processor = new TestProcessor<>(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip - processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.map(keyValueMapper); - mapped2 = stream2.map(keyValueMapper); ->>>>>>> new api model -======= - KStreamTopology initializer = new MockKStreamTopology(); - processor = new TestProcessor<>(); ->>>>>>> wip -======= - processor = new MockProcessor<>(); ->>>>>>> removing io.confluent imports: wip -======= - boolean exceptionRaised; ->>>>>>> compile and test passed - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.map(keyValueMapper); - mapped2 = stream2.map(keyValueMapper); - - exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow<>()); - windowed2 = mapped2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - stream1.bind(context, streamMetadata); - stream2.bind(context, streamMetadata); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.map(keyValueMapper); - mapped2 = stream2.map(keyValueMapper); - - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow<>()); - windowed2 = stream2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.map(keyValueMapper); - mapped2 = stream2.map(keyValueMapper); - - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow<>()); - windowed2 = mapped2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - } - - @Test - public void testFlatMap() { - KStream stream1; - KStream stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; - - processor = new MockProcessor<>(); - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); - processor = new TestProcessor<>(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip - processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.flatMap(keyValueMapper2); - mapped2 = stream2.flatMap(keyValueMapper2); ->>>>>>> new api model -======= - KStreamTopology initializer = new MockKStreamTopology(); -<<<<<<< HEAD - processor = new TestProcessor<>(); ->>>>>>> wip -======= - processor = new MockProcessor<>(); ->>>>>>> removing io.confluent imports: wip -======= - boolean exceptionRaised; ->>>>>>> compile and test passed - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.flatMap(keyValueMapper2); - mapped2 = stream2.flatMap(keyValueMapper2); - - exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow<>()); - windowed2 = mapped2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.flatMap(keyValueMapper2); - mapped2 = stream2.flatMap(keyValueMapper2); - - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow<>()); - windowed2 = stream2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.flatMap(keyValueMapper2); - mapped2 = stream2.flatMap(keyValueMapper2); - - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow<>()); - windowed2 = mapped2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertTrue(exceptionRaised); - } - - @Test - public void testMapValues() { - KStream stream1; - KStream stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; - - KStreamTopology initializer = new MockKStreamTopology(); - processor = new MockProcessor<>(); - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); - processor = new TestProcessor<>(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip - processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.mapValues(valueMapper); - mapped2 = stream2.mapValues(valueMapper); ->>>>>>> new api model -======= - KStreamTopology initializer = new MockKStreamTopology(); -<<<<<<< HEAD - processor = new TestProcessor<>(); ->>>>>>> wip -======= - processor = new MockProcessor<>(); ->>>>>>> removing io.confluent imports: wip -======= - boolean exceptionRaised; ->>>>>>> compile and test passed - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.mapValues(valueMapper); - mapped2 = stream2.mapValues(valueMapper); - - exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow<>()); - windowed2 = mapped2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.mapValues(valueMapper); - mapped2 = stream2.mapValues(valueMapper); - - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow<>()); - windowed2 = stream2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.mapValues(valueMapper); - mapped2 = stream2.mapValues(valueMapper); - - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow<>()); - windowed2 = mapped2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - } - - @Test - public void testFlatMapValues() { - KStream stream1; - KStream stream2; - KStream mapped1; - KStream mapped2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessor processor; - - KStreamTopology initializer = new MockKStreamTopology(); - processor = new MockProcessor<>(); - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); - processor = new TestProcessor<>(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip - processor = new TestProcessor<>(); - stream1 = new KStreamSource<>(null, initializer); - stream2 = new KStreamSource<>(null, initializer); - mapped1 = stream1.flatMapValues(valueMapper2); - mapped2 = stream2.flatMapValues(valueMapper2); ->>>>>>> new api model -======= - KStreamTopology initializer = new MockKStreamTopology(); -<<<<<<< HEAD - processor = new TestProcessor<>(); ->>>>>>> wip -======= - processor = new MockProcessor<>(); ->>>>>>> removing io.confluent imports: wip -======= - boolean exceptionRaised; ->>>>>>> compile and test passed - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.flatMapValues(valueMapper2); - mapped2 = stream2.flatMapValues(valueMapper2); - - exceptionRaised = false; - windowed1 = stream1.with(new UnlimitedWindow<>()); - windowed2 = mapped2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.flatMapValues(valueMapper2); - mapped2 = stream2.flatMapValues(valueMapper2); - - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow<>()); - windowed2 = stream2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); - - try { - stream1 = topology.from(keyDeserializer, valDeserializer, topicName); - stream2 = topology.from(keyDeserializer, valDeserializer, topicName); - mapped1 = stream1.flatMapValues(valueMapper2); - mapped2 = stream2.flatMapValues(valueMapper2); - - exceptionRaised = false; - windowed1 = mapped1.with(new UnlimitedWindow<>()); - windowed2 = mapped2.with(new UnlimitedWindow<>()); - - windowed1.join(windowed2, joiner).process(processor); - - KStreamContext context = new MockProcessorContext(null, null); - - } catch (NotCopartitionedException e) { - exceptionRaised = true; - } - - assertFalse(exceptionRaised); -======= ->>>>>>> wip - } - - // TODO: test for joinability -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java deleted file mode 100644 index be11bb77611c8..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java +++ /dev/null @@ -1,150 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamContext; -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamTopology; -======= -import io.confluent.streaming.KStreamInitializer; ->>>>>>> new api model -======= -import io.confluent.streaming.KStreamTopology; ->>>>>>> wip -import io.confluent.streaming.ValueMapper; -import io.confluent.streaming.testutil.MockKStreamContext; -import io.confluent.streaming.testutil.MockKStreamTopology; -import io.confluent.streaming.testutil.TestProcessor; -import org.apache.kafka.clients.processor.KStreamContext; -======= -import org.apache.kafka.stream.internal.PartitioningInfo; -======= -import org.apache.kafka.stream.internals.PartitioningInfo; ->>>>>>> compile and test passed -======= -import org.apache.kafka.clients.processor.internals.PartitioningInfo; -<<<<<<< HEAD ->>>>>>> wip -import org.apache.kafka.stream.topology.KStreamTopology; -import org.apache.kafka.stream.topology.ValueMapper; -import org.apache.kafka.stream.topology.internals.KStreamMetadata; -import org.apache.kafka.stream.topology.internals.KStreamSource; -======= -import org.apache.kafka.stream.internals.KStreamMetadata; -======= -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -<<<<<<< HEAD:stream/src/test/java/org/apache/kafka/stream/KStreamMapValuesTest.java -<<<<<<< HEAD ->>>>>>> wip -import org.apache.kafka.stream.internals.KStreamSource; ->>>>>>> wip -import org.apache.kafka.test.MockKStreamTopology; -======= -import org.apache.kafka.stream.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamTopologyBuilder; ->>>>>>> Refactor Processor and KStream APIs -======= -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.ValueMapper; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; ->>>>>>> Some package and class renaming, fix KafkaSource constrcution at builder:stream/src/test/java/org/apache/kafka/streaming/KStreamMapValuesTest.java -import org.apache.kafka.test.MockProcessor; -<<<<<<< HEAD -import org.apache.kafka.test.MockKStreamContext; ->>>>>>> removing io.confluent imports: wip -======= ->>>>>>> wip -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class KStreamMapValuesTest { - - private String topicName = "topic"; - - private KStreamBuilder topology = new MockKStreamBuilder(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - @Test - public void testFlatMapValues() { - - ValueMapper mapper = - new ValueMapper() { - @Override - public Integer apply(String value) { - return value.length(); - } - }; - - final int[] expectedKeys = new int[]{1, 10, 100, 1000}; - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology initializer = new MockKStreamTopology(); ->>>>>>> wip - KStreamSource stream; - MockProcessor processor; -======= - KStreamTopology initializer = new MockKStreamTopology(); - KStreamSource stream; - MockProcessor processor; ->>>>>>> compile and test passed - - processor = new MockProcessor<>(); - stream = new KStreamSource<>(null, initializer); -======= - KStream stream; - MockProcessor processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); ->>>>>>> wip - stream.mapValues(mapper).process(processor); - - for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], Integer.toString(expectedKeys[i])); - } - - assertEquals(4, processor.processed.size()); - - String[] expected = new String[]{"1:1", "10:2", "100:3", "1000:4"}; - - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); - } - } - -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java b/stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java deleted file mode 100644 index 99689d333c5e4..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/KStreamWindowedTest.java +++ /dev/null @@ -1,120 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming; - -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.WindowDef; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessorContext; -import org.apache.kafka.test.UnlimitedWindow; -import org.junit.Test; - -import java.util.Iterator; - -import static org.junit.Assert.assertEquals; - -public class KStreamWindowedTest { - - private String topicName = "topic"; - - private KStreamBuilder topology = new MockKStreamBuilder(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - @Test - public void testWindowedStream() { - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - -<<<<<<< HEAD -<<<<<<< HEAD - KStreamSource stream; - Window window; -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null) { - }; ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology initializer = new MockKStreamTopology(); ->>>>>>> wip -======= - KStreamSource stream; - Window window; - KStreamTopology initializer = new MockKStreamTopology(); ->>>>>>> compile and test passed -======= - KStream stream; -<<<<<<< HEAD - Window window; ->>>>>>> wip -======= - WindowDef window; ->>>>>>> adress comments - - window = new UnlimitedWindow<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); - stream.with(window); - - MockProcessorContext context = new MockProcessorContext(null, null); - topology.init(context); - context.setTime(0L); - - // two items in the window - - for (int i = 0; i < 2; i++) { - ((KStreamSource) stream).source().process(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++) { - ((KStreamSource) stream).source().process(expectedKeys[i], "Y" + expectedKeys[i]); - } - - assertEquals(2, countItem(window.find(0, 0L))); - assertEquals(2, countItem(window.find(1, 0L))); - assertEquals(1, countItem(window.find(2, 0L))); - assertEquals(1, countItem(window.find(3, 0L))); - } - - - private int countItem(Iterator iter) { - int i = 0; - while (iter.hasNext()) { - i++; - iter.next(); - } - return i; - } -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java b/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java deleted file mode 100644 index 6ce4825600711..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/kstream/internals/KStreamSourceTest.java +++ /dev/null @@ -1,84 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.kstream.internals; - -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streaming.kstream.KStream; -import org.apache.kafka.streaming.kstream.KStreamBuilder; -import org.apache.kafka.streaming.kstream.internals.KStreamSource; -import org.apache.kafka.test.MockKStreamBuilder; -import org.apache.kafka.test.MockProcessor; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class KStreamSourceTest { - - private String topicName = "topic"; - - private KStreamBuilder topology = new MockKStreamBuilder(); - private StringDeserializer keyDeserializer = new StringDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - @Test - public void testKStreamSource() { - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -<<<<<<< HEAD -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology initializer = new MockKStreamTopology(); ->>>>>>> wip - TestProcessor processor = new TestProcessor<>(); -======= - MockProcessor processor = new MockProcessor<>(); ->>>>>>> removing io.confluent imports: wip -======= - KStreamTopology initializer = new MockKStreamTopology(); -======= ->>>>>>> wip - MockProcessor processor = new MockProcessor<>(); ->>>>>>> compile and test passed - - KStream stream = topology.from(keyDeserializer, valDeserializer, topicName); - stream.process(processor); - - final String[] expectedKeys = new String[]{"k1", "k2", "k3"}; - final String[] expectedValues = new String[]{"v1", "v2", "v3"}; - - for (int i = 0; i < expectedKeys.length; i++) { - ((KStreamSource) stream).source().process(expectedKeys[i], expectedValues[i]); - } - - assertEquals(3, processor.processed.size()); - - for (int i = 0; i < expectedKeys.length; i++) { - assertEquals(expectedKeys[i] + ":" + expectedValues[i], processor.processed.get(i)); - } - } -} diff --git a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamGroupTest.java b/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamGroupTest.java deleted file mode 100644 index 659de88f03f30..0000000000000 --- a/stream/src/test/java/org/apache/kafka/streaming/processor/internals/StreamGroupTest.java +++ /dev/null @@ -1,204 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streaming.processor.internals; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streaming.processor.TimestampExtractor; -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.streaming.processor.internals.StreamGroup; -import org.apache.kafka.test.MockProcessorContext; -import org.apache.kafka.test.MockSourceNode; -import org.junit.Test; - -import java.util.Arrays; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -public class StreamGroupTest { - - private static Serializer serializer = new IntegerSerializer(); - private static Deserializer deserializer = new IntegerDeserializer(); - -<<<<<<< HEAD -<<<<<<< HEAD - public MockKStreamSource() { -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - super(null, deserializer, deserializer, new MockKStreamTopology()); -======= - super(null, deserializer, deserializer, new KStreamInitializerImpl(serializer, serializer, deserializer, deserializer)); ->>>>>>> new api model -======= - super(null, deserializer, deserializer, new KStreamInitializerImpl()); ->>>>>>> wip -======= - super(null, deserializer, deserializer, new MockKStreamTopology()); ->>>>>>> wip - } -======= - private static class MockKStreamSource extends KStreamSource { ->>>>>>> compile and test passed - - public int numReceived = 0; - public ArrayList keys = new ArrayList<>(); - public ArrayList values = new ArrayList<>(); - public ArrayList timestamps = new ArrayList<>(); - - public MockKStreamSource() { - super(null, deserializer, deserializer, new MockKStreamTopology()); - } - - @Override - public void receive(Object key, Object value, long timestamp) { - this.numReceived++; - this.keys.add(key); - this.values.add(value); - this.timestamps.add(timestamp); - } - - } - -======= ->>>>>>> wip - @SuppressWarnings("unchecked") - @Test - public void testAddPartition() { - - MockIngestor mockIngestor = new MockIngestor(); - - StreamGroup streamGroup = new StreamGroup( - new MockProcessorContext(serializer, deserializer), - mockIngestor, - new TimestampExtractor() { - public long extract(String topic, Object key, Object value) { - if (topic.equals("topic1")) - return ((Integer) key).longValue(); - else - return ((Integer) key).longValue() / 10L + 5L; - } - }, - 3 - ); - - TopicPartition partition1 = new TopicPartition("topic1", 1); - TopicPartition partition2 = new TopicPartition("topic2", 1); - MockSourceNode source1 = new MockSourceNode(deserializer, deserializer); - MockSourceNode source2 = new MockSourceNode(deserializer, deserializer); - MockSourceNode source3 = new MockSourceNode(deserializer, deserializer); - - streamGroup.addPartition(partition1, source1); - mockIngestor.addPartitionStreamToGroup(streamGroup, partition1); - - streamGroup.addPartition(partition2, source2); - mockIngestor.addPartitionStreamToGroup(streamGroup, partition2); - - Exception exception = null; - try { - streamGroup.addPartition(partition1, source3); - } catch (Exception ex) { - exception = ex; - } - assertTrue(exception != null); - - byte[] recordValue = serializer.serialize(null, new Integer(10)); - - mockIngestor.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 1, serializer.serialize(partition1.topic(), new Integer(10)), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 2, serializer.serialize(partition1.topic(), new Integer(20)), recordValue) - )); - - mockIngestor.addRecords(partition2, records( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 1, serializer.serialize(partition1.topic(), new Integer(300)), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 2, serializer.serialize(partition1.topic(), new Integer(400)), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 3, serializer.serialize(partition1.topic(), new Integer(500)), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 4, serializer.serialize(partition1.topic(), new Integer(600)), recordValue) - )); - - streamGroup.process(); - assertEquals(source1.numReceived, 1); - assertEquals(source2.numReceived, 0); - - assertEquals(mockIngestor.paused.size(), 1); - assertTrue(mockIngestor.paused.contains(partition2)); - - mockIngestor.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 3, serializer.serialize(partition1.topic(), new Integer(30)), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 4, serializer.serialize(partition1.topic(), new Integer(40)), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 5, serializer.serialize(partition1.topic(), new Integer(50)), recordValue) - )); - - streamGroup.process(); - assertEquals(source1.numReceived, 2); - assertEquals(source2.numReceived, 0); - - assertEquals(mockIngestor.paused.size(), 2); - assertTrue(mockIngestor.paused.contains(partition1)); - assertTrue(mockIngestor.paused.contains(partition2)); - - streamGroup.process(); - assertEquals(source1.numReceived, 3); - assertEquals(source2.numReceived, 0); - - streamGroup.process(); - assertEquals(source1.numReceived, 3); - assertEquals(source2.numReceived, 1); - - assertEquals(mockIngestor.paused.size(), 1); - assertTrue(mockIngestor.paused.contains(partition2)); - - streamGroup.process(); - assertEquals(source1.numReceived, 4); - assertEquals(source2.numReceived, 1); - - assertEquals(mockIngestor.paused.size(), 1); - - streamGroup.process(); - assertEquals(source1.numReceived, 4); - assertEquals(source2.numReceived, 2); - - assertEquals(mockIngestor.paused.size(), 0); - - streamGroup.process(); - assertEquals(source1.numReceived, 5); - assertEquals(source2.numReceived, 2); - - streamGroup.process(); - assertEquals(source1.numReceived, 5); - assertEquals(source2.numReceived, 3); - - streamGroup.process(); - assertEquals(source1.numReceived, 5); - assertEquals(source2.numReceived, 4); - - assertEquals(mockIngestor.paused.size(), 0); - - streamGroup.process(); - assertEquals(source1.numReceived, 5); - assertEquals(source2.numReceived, 4); - } - - private Iterable> records(ConsumerRecord... recs) { - return Arrays.asList(recs); - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java index 40a6ce4f396eb..c09f38381863f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java @@ -1,53 +1,3 @@ -<<<<<<< HEAD -package org.apache.kafka.clients.processor; - -/** - * Created by yasuhiro on 6/17/15. - */ -public interface Processor { - -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - void init(KStreamContext context); -======= - public interface ProcessorContext { - - void send(String topic, Object key, Object value); - - void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer); - - void schedule(long timestamp); - - void commit(); - - String topic(); - - int partition(); - - long offset(); - - long timestamp(); - - KStreamContext kstreamContext(); - - } - - void init(ProcessorContext context); ->>>>>>> new api model -======= - void init(KStreamContext context); ->>>>>>> removed ProcessorContext -======= - void init(ProcessorContext context); ->>>>>>> removing io.confluent imports: wip - - void process(K key, V value); - - void punctuate(long streamTime); - - void close(); -======= /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -76,5 +26,4 @@ public interface Processor { void punctuate(long streamTime); void close(); ->>>>>>> wip } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 108b9ec031b47..63694dbdd1922 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -84,7 +84,6 @@ public ProcessorContextImpl(int id, Consumer restoreConsumer = new KafkaConsumer<>( config.getConsumerConfigs(), - null /* no callback for restore consumer */, new ByteArrayDeserializer(), new ByteArrayDeserializer()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 42446d280858e..7c9bad284f59b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -29,6 +29,7 @@ import java.io.File; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -95,10 +96,10 @@ public void register(StateStore store, RestoreFunc restoreFunc) { // subscribe to the store's partition TopicPartition storePartition = new TopicPartition(store.name(), id); - if (!restoreConsumer.subscriptions().isEmpty()) { + if (!restoreConsumer.subscription().isEmpty()) { throw new IllegalStateException("Restore consumer should have not subscribed to any partitions beforehand"); } - restoreConsumer.subscribe(storePartition); + 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 @@ -132,7 +133,7 @@ public void register(StateStore store, RestoreFunc restoreFunc) { restoredOffsets.put(storePartition, newOffset); // un-subscribe the change log partition - restoreConsumer.unsubscribe(storePartition); + restoreConsumer.subscribe(Collections.emptyList()); } public StateStore getStore(String name) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index a673db7e1620f..990f21b155788 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -18,7 +18,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; +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.streams.StreamingConfig; @@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -75,7 +76,7 @@ public class StreamThread extends Thread { private long lastCommit; private long recordsProcessed; - protected final ConsumerRebalanceCallback rebalanceCallback = new ConsumerRebalanceCallback() { + protected final ConsumerRebalanceListener rebalanceCallback = new ConsumerRebalanceListener() { @Override public void onPartitionsAssigned(Consumer consumer, Collection assignment) { addPartitions(assignment); @@ -105,7 +106,6 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce log.info("Creating consumer client for stream thread [" + this.getName() + "]"); this.consumer = new KafkaConsumer<>(config.getConsumerConfigs(), - rebalanceCallback, new ByteArrayDeserializer(), new ByteArrayDeserializer()); @@ -186,9 +186,7 @@ private void runLoop() { try { int totalNumBuffered = 0; - for (String topic : builder.sourceTopics()) { - consumer.subscribe(topic); - } + consumer.subscribe(new ArrayList<>(builder.sourceTopics()), rebalanceCallback); while (stillRunning()) { // try to fetch some records if necessary diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index d381f3d7b009d..eac2231585f03 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -48,35 +48,10 @@ public void testFilter() { KStreamBuilder builder = new KStreamBuilder(); final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology initializer = new MockKStreamTopology(); ->>>>>>> wip - KStreamSource stream; - MockProcessor processor; -======= KStreamTopology initializer = new MockKStreamTopology(); KStreamSource stream; -======= - KStream stream; -<<<<<<< HEAD ->>>>>>> wip MockProcessor processor; ->>>>>>> compile and test passed -======= MockProcessorDef processor; ->>>>>>> kstream test fix processor = new MockProcessorDef<>(); stream = builder.from(keyDeserializer, valDeserializer, topicName); @@ -98,32 +73,8 @@ public void testFilterOut() { KStream stream; MockProcessorDef processor; -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology initializer = new MockKStreamTopology(); ->>>>>>> wip - KStreamSource stream; - MockProcessor processor; -======= - processor = new MockProcessor<>(); - stream = topology.from(keyDeserializer, valDeserializer, topicName); -======= - processor = new MockProcessorDef<>(); - stream = builder.from(keyDeserializer, valDeserializer, topicName); ->>>>>>> kstream test fix + KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); stream.filterOut(isMultipleOfThree).process(processor); ->>>>>>> compile and test passed KStreamTestDriver driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { From b05f4aca7dac9a78493bfe45e4429b4a0c886a6d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 9 Sep 2015 17:38:00 -0700 Subject: [PATCH 221/275] fix KStreamFlatMapTest.java --- kafka-work.iws | 1266 ----------------- .../kstream/internals/KStreamFilterTest.java | 10 +- 2 files changed, 4 insertions(+), 1272 deletions(-) delete mode 100644 kafka-work.iws diff --git a/kafka-work.iws b/kafka-work.iws deleted file mode 100644 index 8f4e45d4be93f..0000000000000 --- a/kafka-work.iws +++ /dev/null @@ -1,1266 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - org.apache.kafka.streaming.examples.* - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - localhost - 5050 - - - - - - - - - - - - - 1438819547964 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index eac2231585f03..b80e1e2028ef7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -48,9 +48,7 @@ public void testFilter() { KStreamBuilder builder = new KStreamBuilder(); final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; - KStreamTopology initializer = new MockKStreamTopology(); - KStreamSource stream; - MockProcessor processor; + KStream stream; MockProcessorDef processor; processor = new MockProcessorDef<>(); @@ -73,7 +71,8 @@ public void testFilterOut() { KStream stream; MockProcessorDef processor; - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); + processor = new MockProcessorDef<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); stream.filterOut(isMultipleOfThree).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); @@ -83,5 +82,4 @@ public void testFilterOut() { assertEquals(5, processor.processed.size()); } - -} +} \ No newline at end of file From d2f1d55103512498ad1f2adb396b68cac9542595 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 9 Sep 2015 17:39:31 -0700 Subject: [PATCH 222/275] fix KStreamFlatMapTest --- .../kstream/internals/KStreamFlatMapTest.java | 48 +------------------ 1 file changed, 1 insertion(+), 47 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index a92766112152f..e87223e73d886 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -56,58 +56,12 @@ public Iterable> apply(Integer key, String value) { final int[] expectedKeys = {0, 1, 2, 3}; -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology topology = new MockKStreamTopology(); - -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology topology = new MockKStreamTopology(); - ->>>>>>> wip - KStreamSource stream; - MockProcessor processor; - -<<<<<<< HEAD - processor = new TestProcessor<>(); -<<<<<<< HEAD -<<<<<<< HEAD - stream = new KStreamSource<>(null, topology); -======= - stream = new KStreamSource<>(null, initializer); ->>>>>>> new api model -======= -======= - processor = new MockProcessor<>(); ->>>>>>> removing io.confluent imports: wip - stream = new KStreamSource<>(null, topology); ->>>>>>> wip - stream.flatMap(mapper).process(processor); -======= - KStreamTopology topology = new MockKStreamTopology(); - - KStreamSource stream; -======= KStream stream; -<<<<<<< HEAD ->>>>>>> wip - MockProcessor processor; -======= MockProcessorDef processor; ->>>>>>> kstream test fix processor = new MockProcessorDef<>(); stream = builder.from(keyDeserializer, valDeserializer, topicName); stream.flatMap(mapper).process(processor); ->>>>>>> compile and test passed KStreamTestDriver driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { @@ -123,4 +77,4 @@ public Iterable> apply(Integer key, String value) { } } -} +} \ No newline at end of file From 050aade23f3579fcee8a375d8691fbcab0705f9b Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 10 Sep 2015 10:39:08 -0700 Subject: [PATCH 223/275] misc fixes after rebase --- .gitignore | 2 +- README.md | 2 +- .../kafka/clients/consumer/KafkaConsumer.java | 1 - kafka-work.ipr | 209 ------------------ .../apache/kafka/clients/ClientRequest.class | Bin 2117 -> 0 bytes .../apache/kafka/clients/ClientResponse.class | Bin 1880 -> 0 bytes .../apache/kafka/clients/ClientUtils.class | Bin 3513 -> 0 bytes ...ConnectionStates$NodeConnectionState.class | Bin 1090 -> 0 bytes .../clients/ClusterConnectionStates.class | Bin 3360 -> 0 bytes .../kafka/clients/CommonClientConfigs.class | Bin 3528 -> 0 bytes .../kafka/clients/ConnectionState.class | Bin 1123 -> 0 bytes .../kafka/clients/InFlightRequests.class | Bin 3688 -> 0 bytes .../apache/kafka/clients/KafkaClient.class | Bin 1068 -> 0 bytes .../org/apache/kafka/clients/Metadata.class | Bin 4076 -> 0 bytes .../apache/kafka/clients/NetworkClient.class | Bin 15727 -> 0 bytes .../clients/RequestCompletionHandler.class | Bin 222 -> 0 bytes .../kafka/clients/consumer/CommitType.class | Bin 1077 -> 0 bytes .../kafka/clients/consumer/Consumer.class | Bin 2045 -> 0 bytes .../consumer/ConsumerCommitCallback.class | Bin 342 -> 0 bytes .../clients/consumer/ConsumerConfig.class | Bin 12655 -> 0 bytes .../consumer/ConsumerRebalanceCallback.class | Bin 444 -> 0 bytes .../clients/consumer/ConsumerRecord.class | Bin 2181 -> 0 bytes ...nsumerRecords$ConcatenatedIterable$1.class | Bin 2079 -> 0 bytes ...ConsumerRecords$ConcatenatedIterable.class | Bin 1709 -> 0 bytes .../clients/consumer/ConsumerRecords.class | Bin 4221 -> 0 bytes .../consumer/ConsumerWakeupException.class | Bin 448 -> 0 bytes .../clients/consumer/KafkaConsumer$1.class | Bin 1403 -> 0 bytes .../clients/consumer/KafkaConsumer$2.class | Bin 1802 -> 0 bytes .../clients/consumer/KafkaConsumer.class | Bin 25520 -> 0 bytes .../kafka/clients/consumer/MockConsumer.class | Bin 10225 -> 0 bytes .../NoOffsetForPartitionException.class | Bin 530 -> 0 bytes .../consumer/OffsetResetStrategy.class | Bin 1192 -> 0 bytes ...lient$RequestFutureCompletionHandler.class | Bin 1083 -> 0 bytes .../internals/ConsumerNetworkClient.class | Bin 9748 -> 0 bytes .../consumer/internals/Coordinator$1.class | Bin 1724 -> 0 bytes .../consumer/internals/Coordinator$2.class | Bin 1910 -> 0 bytes .../Coordinator$CoordinatorMetrics$1.class | Bin 1719 -> 0 bytes .../Coordinator$CoordinatorMetrics$2.class | Bin 1824 -> 0 bytes .../Coordinator$CoordinatorMetrics.class | Bin 4037 -> 0 bytes ...ordinator$CoordinatorResponseHandler.class | Bin 4181 -> 0 bytes ...ordinator$HeartbeatCompletionHandler.class | Bin 5438 -> 0 bytes .../Coordinator$HeartbeatTask$1.class | Bin 2514 -> 0 bytes .../internals/Coordinator$HeartbeatTask.class | Bin 3425 -> 0 bytes ...Coordinator$JoinGroupResponseHandler.class | Bin 6376 -> 0 bytes ...rdinator$OffsetCommitResponseHandler.class | Bin 6253 -> 0 bytes ...ordinator$OffsetFetchResponseHandler.class | Bin 6119 -> 0 bytes .../Coordinator$RebalanceCallback.class | Bin 464 -> 0 bytes .../consumer/internals/Coordinator.class | Bin 19663 -> 0 bytes .../consumer/internals/DelayedTask.class | Bin 168 -> 0 bytes .../internals/DelayedTaskQueue$Entry.class | Bin 1181 -> 0 bytes .../consumer/internals/DelayedTaskQueue.class | Bin 2236 -> 0 bytes .../consumer/internals/Fetcher$1.class | Bin 1945 -> 0 bytes .../consumer/internals/Fetcher$2.class | Bin 2133 -> 0 bytes .../Fetcher$FetchManagerMetrics.class | Bin 4008 -> 0 bytes .../internals/Fetcher$PartitionRecords.class | Bin 1191 -> 0 bytes .../clients/consumer/internals/Fetcher.class | Bin 23886 -> 0 bytes .../consumer/internals/Heartbeat.class | Bin 1599 -> 0 bytes .../NoAvailableBrokersException.class | Bin 497 -> 0 bytes .../NoOpConsumerRebalanceCallback.class | Bin 1179 -> 0 bytes .../consumer/internals/RequestFuture$1.class | Bin 1936 -> 0 bytes .../consumer/internals/RequestFuture.class | Bin 5732 -> 0 bytes .../internals/RequestFutureAdapter.class | Bin 1373 -> 0 bytes .../internals/RequestFutureListener.class | Bin 345 -> 0 bytes .../internals/SendFailedException.class | Bin 561 -> 0 bytes .../internals/StaleMetadataException.class | Bin 482 -> 0 bytes ...ubscriptionState$TopicPartitionState.class | Bin 4139 -> 0 bytes .../internals/SubscriptionState.class | Bin 10200 -> 0 bytes .../producer/BufferExhaustedException.class | Bin 515 -> 0 bytes .../kafka/clients/producer/Callback.class | Bin 231 -> 0 bytes .../KafkaProducer$FutureFailure.class | Bin 1804 -> 0 bytes .../clients/producer/KafkaProducer.class | Bin 21339 -> 0 bytes .../producer/MockProducer$Completion.class | Bin 1819 -> 0 bytes .../kafka/clients/producer/MockProducer.class | Bin 9644 -> 0 bytes .../kafka/clients/producer/Partitioner.class | Bin 320 -> 0 bytes .../kafka/clients/producer/Producer.class | Bin 1213 -> 0 bytes .../clients/producer/ProducerConfig.class | Bin 16285 -> 0 bytes .../clients/producer/ProducerRecord.class | Bin 3300 -> 0 bytes .../clients/producer/RecordMetadata.class | Bin 1112 -> 0 bytes .../producer/internals/BufferPool.class | Bin 6445 -> 0 bytes .../internals/DefaultPartitioner.class | Bin 2282 -> 0 bytes .../internals/ErrorLoggingCallback.class | Bin 1986 -> 0 bytes .../internals/FutureRecordMetadata.class | Bin 3048 -> 0 bytes .../internals/ProduceRequestResult.class | Bin 1684 -> 0 bytes .../internals/RecordAccumulator$1.class | Bin 1275 -> 0 bytes .../internals/RecordAccumulator$2.class | Bin 1280 -> 0 bytes .../internals/RecordAccumulator$3.class | Bin 1284 -> 0 bytes ...dAccumulator$IncompleteRecordBatches.class | Bin 1849 -> 0 bytes .../RecordAccumulator$ReadyCheckResult.class | Bin 915 -> 0 bytes ...RecordAccumulator$RecordAppendResult.class | Bin 824 -> 0 bytes .../internals/RecordAccumulator.class | Bin 16641 -> 0 bytes .../internals/RecordBatch$Thunk.class | Bin 788 -> 0 bytes .../producer/internals/RecordBatch.class | Bin 4607 -> 0 bytes .../clients/producer/internals/Sender$1.class | Bin 1365 -> 0 bytes .../internals/Sender$SenderMetrics$1.class | Bin 1494 -> 0 bytes .../internals/Sender$SenderMetrics$2.class | Bin 1501 -> 0 bytes .../internals/Sender$SenderMetrics.class | Bin 9145 -> 0 bytes .../clients/producer/internals/Sender.class | Bin 14836 -> 0 bytes .../org/apache/kafka/common/Cluster.class | Bin 7232 -> 0 bytes .../apache/kafka/common/Configurable.class | Bin 232 -> 0 bytes .../apache/kafka/common/KafkaException.class | Bin 821 -> 0 bytes .../org/apache/kafka/common/Metric.class | Bin 201 -> 0 bytes .../org/apache/kafka/common/MetricName.class | Bin 3956 -> 0 bytes .../org/apache/kafka/common/Node.class | Bin 1881 -> 0 bytes .../apache/kafka/common/PartitionInfo.class | Bin 2326 -> 0 bytes .../apache/kafka/common/TopicPartition.class | Bin 1531 -> 0 bytes .../kafka/common/config/AbstractConfig.class | Bin 8093 -> 0 bytes .../kafka/common/config/ConfigDef$1.class | Bin 1667 -> 0 bytes .../kafka/common/config/ConfigDef$2.class | Bin 1082 -> 0 bytes .../common/config/ConfigDef$ConfigKey.class | Bin 1741 -> 0 bytes .../common/config/ConfigDef$Importance.class | Bin 1255 -> 0 bytes .../kafka/common/config/ConfigDef$Range.class | Bin 2019 -> 0 bytes .../kafka/common/config/ConfigDef$Type.class | Bin 1460 -> 0 bytes .../common/config/ConfigDef$ValidString.class | Bin 1935 -> 0 bytes .../common/config/ConfigDef$Validator.class | Bin 293 -> 0 bytes .../kafka/common/config/ConfigDef.class | Bin 10278 -> 0 bytes .../kafka/common/config/ConfigException.class | Bin 1261 -> 0 bytes .../kafka/common/errors/ApiException.class | Bin 944 -> 0 bytes ...umerCoordinatorNotAvailableException.class | Bin 936 -> 0 bytes .../errors/CorruptRecordException.class | Bin 946 -> 0 bytes .../common/errors/DisconnectException.class | Bin 873 -> 0 bytes .../errors/IllegalGenerationException.class | Bin 894 -> 0 bytes .../common/errors/InterruptException.class | Bin 1055 -> 0 bytes .../errors/InvalidMetadataException.class | Bin 888 -> 0 bytes .../errors/InvalidRequiredAcksException.class | Bin 526 -> 0 bytes .../common/errors/InvalidTopicException.class | Bin 873 -> 0 bytes .../errors/LeaderNotAvailableException.class | Bin 535 -> 0 bytes .../common/errors/NetworkException.class | Bin 870 -> 0 bytes .../NotCoordinatorForConsumerException.class | Bin 918 -> 0 bytes ...otEnoughReplicasAfterAppendException.class | Bin 559 -> 0 bytes .../errors/NotEnoughReplicasException.class | Bin 894 -> 0 bytes .../NotLeaderForPartitionException.class | Bin 912 -> 0 bytes .../OffsetLoadInProgressException.class | Bin 903 -> 0 bytes .../errors/OffsetMetadataTooLarge.class | Bin 876 -> 0 bytes .../errors/OffsetOutOfRangeException.class | Bin 891 -> 0 bytes .../errors/RecordBatchTooLargeException.class | Bin 894 -> 0 bytes .../errors/RecordTooLargeException.class | Bin 879 -> 0 bytes .../common/errors/RetriableException.class | Bin 864 -> 0 bytes .../errors/SerializationException.class | Bin 974 -> 0 bytes .../common/errors/TimeoutException.class | Bin 864 -> 0 bytes .../errors/UnknownConsumerIdException.class | Bin 894 -> 0 bytes .../errors/UnknownServerException.class | Bin 876 -> 0 bytes .../UnknownTopicOrPartitionException.class | Bin 922 -> 0 bytes .../CompoundStat$NamedMeasurable.class | Bin 932 -> 0 bytes .../kafka/common/metrics/CompoundStat.class | Bin 438 -> 0 bytes .../metrics/JmxReporter$KafkaMbean.class | Bin 5120 -> 0 bytes .../kafka/common/metrics/JmxReporter.class | Bin 5932 -> 0 bytes .../kafka/common/metrics/KafkaMetric.class | Bin 1940 -> 0 bytes .../kafka/common/metrics/Measurable.class | Bin 204 -> 0 bytes .../kafka/common/metrics/MeasurableStat.class | Bin 235 -> 0 bytes .../kafka/common/metrics/MetricConfig.class | Bin 2153 -> 0 bytes .../apache/kafka/common/metrics/Metrics.class | Bin 6409 -> 0 bytes .../common/metrics/MetricsReporter.class | Bin 409 -> 0 bytes .../apache/kafka/common/metrics/Quota.class | Bin 959 -> 0 bytes .../metrics/QuotaViolationException.class | Bin 502 -> 0 bytes .../apache/kafka/common/metrics/Sensor.class | Bin 6638 -> 0 bytes .../apache/kafka/common/metrics/Stat.class | Bin 192 -> 0 bytes .../kafka/common/metrics/stats/Avg.class | Bin 1615 -> 0 bytes .../kafka/common/metrics/stats/Count.class | Bin 1532 -> 0 bytes .../metrics/stats/Histogram$BinScheme.class | Bin 312 -> 0 bytes .../stats/Histogram$ConstantBinScheme.class | Bin 1333 -> 0 bytes .../stats/Histogram$LinearBinScheme.class | Bin 1359 -> 0 bytes .../common/metrics/stats/Histogram.class | Bin 2442 -> 0 bytes .../kafka/common/metrics/stats/Max.class | Bin 1601 -> 0 bytes .../kafka/common/metrics/stats/Min.class | Bin 1616 -> 0 bytes .../common/metrics/stats/Percentile.class | Bin 684 -> 0 bytes .../common/metrics/stats/Percentiles$1.class | Bin 1033 -> 0 bytes .../stats/Percentiles$BucketSizing.class | Bin 1292 -> 0 bytes .../stats/Percentiles$HistogramSample.class | Bin 1619 -> 0 bytes .../common/metrics/stats/Percentiles.class | Bin 6038 -> 0 bytes .../kafka/common/metrics/stats/Rate$1.class | Bin 978 -> 0 bytes .../metrics/stats/Rate$SampledTotal.class | Bin 1635 -> 0 bytes .../kafka/common/metrics/stats/Rate.class | Bin 2865 -> 0 bytes .../metrics/stats/SampledStat$Sample.class | Bin 1134 -> 0 bytes .../common/metrics/stats/SampledStat.class | Bin 3349 -> 0 bytes .../kafka/common/metrics/stats/Total.class | Bin 888 -> 0 bytes .../common/network/ByteBufferReceive.class | Bin 1322 -> 0 bytes .../kafka/common/network/ByteBufferSend.class | Bin 1585 -> 0 bytes .../network/InvalidReceiveException.class | Bin 618 -> 0 bytes .../kafka/common/network/MultiSend.class | Bin 3217 -> 0 bytes .../kafka/common/network/NetworkReceive.class | Bin 3055 -> 0 bytes .../kafka/common/network/NetworkSend.class | Bin 1061 -> 0 bytes .../apache/kafka/common/network/Receive.class | Bin 306 -> 0 bytes .../kafka/common/network/Selectable.class | Bin 832 -> 0 bytes .../network/Selector$SelectorMetrics$1.class | Bin 1346 -> 0 bytes .../network/Selector$SelectorMetrics.class | Bin 8067 -> 0 bytes .../network/Selector$Transmissions.class | Bin 1034 -> 0 bytes .../kafka/common/network/Selector.class | Bin 15681 -> 0 bytes .../apache/kafka/common/network/Send.class | Bin 325 -> 0 bytes .../kafka/common/protocol/ApiKeys.class | Bin 2633 -> 0 bytes .../apache/kafka/common/protocol/Errors.class | Bin 8391 -> 0 bytes .../kafka/common/protocol/ProtoUtils.class | Bin 2456 -> 0 bytes .../kafka/common/protocol/Protocol.class | Bin 12722 -> 0 bytes .../common/protocol/SecurityProtocol.class | Bin 2453 -> 0 bytes .../kafka/common/protocol/types/ArrayOf.class | Bin 2464 -> 0 bytes .../kafka/common/protocol/types/Field.class | Bin 2201 -> 0 bytes .../kafka/common/protocol/types/Schema.class | Bin 5250 -> 0 bytes .../protocol/types/SchemaException.class | Bin 498 -> 0 bytes .../kafka/common/protocol/types/Struct.class | Bin 8367 -> 0 bytes .../kafka/common/protocol/types/Type$1.class | Bin 1704 -> 0 bytes .../kafka/common/protocol/types/Type$2.class | Bin 1720 -> 0 bytes .../kafka/common/protocol/types/Type$3.class | Bin 1723 -> 0 bytes .../kafka/common/protocol/types/Type$4.class | Bin 1714 -> 0 bytes .../kafka/common/protocol/types/Type$5.class | Bin 2113 -> 0 bytes .../kafka/common/protocol/types/Type$6.class | Bin 2015 -> 0 bytes .../kafka/common/protocol/types/Type.class | Bin 1229 -> 0 bytes .../common/record/ByteBufferInputStream.class | Bin 951 -> 0 bytes .../record/ByteBufferOutputStream.class | Bin 1474 -> 0 bytes .../kafka/common/record/CompressionType.class | Bin 2310 -> 0 bytes .../kafka/common/record/Compressor$1.class | Bin 884 -> 0 bytes .../kafka/common/record/Compressor.class | Bin 8487 -> 0 bytes .../record/InvalidRecordException.class | Bin 485 -> 0 bytes .../record/KafkaLZ4BlockInputStream.class | Bin 5162 -> 0 bytes .../record/KafkaLZ4BlockOutputStream$BD.class | Bin 1605 -> 0 bytes .../KafkaLZ4BlockOutputStream$FLG.class | Bin 2914 -> 0 bytes .../record/KafkaLZ4BlockOutputStream.class | Bin 4572 -> 0 bytes .../apache/kafka/common/record/LogEntry.class | Bin 1218 -> 0 bytes .../MemoryRecords$RecordsIterator.class | Bin 3595 -> 0 bytes .../kafka/common/record/MemoryRecords.class | Bin 5566 -> 0 bytes .../apache/kafka/common/record/Record.class | Bin 7678 -> 0 bytes .../apache/kafka/common/record/Records.class | Bin 524 -> 0 bytes .../common/requests/AbstractRequest$1.class | Bin 1164 -> 0 bytes .../common/requests/AbstractRequest.class | Bin 2628 -> 0 bytes .../requests/AbstractRequestResponse.class | Bin 1424 -> 0 bytes .../requests/ConsumerMetadataRequest.class | Bin 3355 -> 0 bytes .../requests/ConsumerMetadataResponse.class | Bin 3068 -> 0 bytes .../requests/FetchRequest$PartitionData.class | Bin 567 -> 0 bytes .../kafka/common/requests/FetchRequest.class | Bin 8557 -> 0 bytes .../FetchResponse$PartitionData.class | Bin 674 -> 0 bytes .../kafka/common/requests/FetchResponse.class | Bin 6236 -> 0 bytes .../common/requests/HeartbeatRequest.class | Bin 3738 -> 0 bytes .../common/requests/HeartbeatResponse.class | Bin 1934 -> 0 bytes .../common/requests/JoinGroupRequest.class | Bin 4986 -> 0 bytes .../common/requests/JoinGroupResponse.class | Bin 5149 -> 0 bytes .../ListOffsetRequest$PartitionData.class | Bin 595 -> 0 bytes .../common/requests/ListOffsetRequest.class | Bin 8121 -> 0 bytes .../ListOffsetResponse$PartitionData.class | Bin 772 -> 0 bytes .../common/requests/ListOffsetResponse.class | Bin 6013 -> 0 bytes .../common/requests/MetadataRequest.class | Bin 4481 -> 0 bytes .../common/requests/MetadataResponse.class | Bin 7925 -> 0 bytes .../OffsetCommitRequest$PartitionData.class | Bin 916 -> 0 bytes .../common/requests/OffsetCommitRequest.class | Bin 10053 -> 0 bytes .../requests/OffsetCommitResponse.class | Bin 5093 -> 0 bytes .../common/requests/OffsetFetchRequest.class | Bin 6668 -> 0 bytes .../OffsetFetchResponse$PartitionData.class | Bin 932 -> 0 bytes .../common/requests/OffsetFetchResponse.class | Bin 6111 -> 0 bytes .../common/requests/ProduceRequest.class | Bin 7363 -> 0 bytes .../ProduceResponse$PartitionResponse.class | Bin 1077 -> 0 bytes .../common/requests/ProduceResponse.class | Bin 6010 -> 0 bytes .../kafka/common/requests/RequestHeader.class | Bin 3097 -> 0 bytes .../kafka/common/requests/RequestSend.class | Bin 1900 -> 0 bytes .../common/requests/ResponseHeader.class | Bin 1878 -> 0 bytes .../kafka/common/requests/ResponseSend.class | Bin 1762 -> 0 bytes .../serialization/ByteArrayDeserializer.class | Bin 1144 -> 0 bytes .../serialization/ByteArraySerializer.class | Bin 1138 -> 0 bytes .../common/serialization/Deserializer.class | Bin 466 -> 0 bytes .../serialization/IntegerDeserializer.class | Bin 1619 -> 0 bytes .../serialization/IntegerSerializer.class | Bin 1355 -> 0 bytes .../common/serialization/Serializer.class | Bin 460 -> 0 bytes .../serialization/StringDeserializer.class | Bin 2147 -> 0 bytes .../serialization/StringSerializer.class | Bin 2131 -> 0 bytes .../common/utils/AbstractIterator$1.class | Bin 897 -> 0 bytes .../common/utils/AbstractIterator$State.class | Bin 1323 -> 0 bytes .../kafka/common/utils/AbstractIterator.class | Bin 2342 -> 0 bytes .../kafka/common/utils/CollectionUtils.class | Bin 2901 -> 0 bytes .../kafka/common/utils/CopyOnWriteMap.class | Bin 4034 -> 0 bytes .../org/apache/kafka/common/utils/Crc32.class | Bin 28501 -> 0 bytes .../kafka/common/utils/KafkaThread$1.class | Bin 1388 -> 0 bytes .../kafka/common/utils/KafkaThread.class | Bin 1256 -> 0 bytes .../kafka/common/utils/SystemTime.class | Bin 804 -> 0 bytes .../org/apache/kafka/common/utils/Time.class | Bin 193 -> 0 bytes .../apache/kafka/common/utils/Utils$1.class | Bin 1182 -> 0 bytes .../org/apache/kafka/common/utils/Utils.class | Bin 13100 -> 0 bytes .../kafka/streaming/KafkaStreaming.class | Bin 2626 -> 0 bytes .../kafka/streaming/StreamingConfig.class | Bin 6417 -> 0 bytes .../streaming/examples/KStreamJob$1.class | Bin 1009 -> 0 bytes .../streaming/examples/KStreamJob$2.class | Bin 1423 -> 0 bytes .../streaming/examples/KStreamJob$3.class | Bin 1076 -> 0 bytes .../streaming/examples/KStreamJob$4.class | Bin 1009 -> 0 bytes .../kafka/streaming/examples/KStreamJob.class | Bin 3323 -> 0 bytes .../streaming/examples/ProcessorJob$1.class | Bin 256 -> 0 bytes .../ProcessorJob$MyProcessorDef$1.class | Bin 3821 -> 0 bytes .../ProcessorJob$MyProcessorDef.class | Bin 1180 -> 0 bytes .../streaming/examples/ProcessorJob.class | Bin 2995 -> 0 bytes .../WallclockTimestampExtractor.class | Bin 723 -> 0 bytes .../kafka/streaming/kstream/KStream.class | Bin 3250 -> 0 bytes .../streaming/kstream/KStreamBuilder.class | Bin 2434 -> 0 bytes .../streaming/kstream/KStreamWindowed.class | Bin 727 -> 0 bytes .../kafka/streaming/kstream/KeyValue.class | Bin 1045 -> 0 bytes .../streaming/kstream/KeyValueMapper.class | Bin 344 -> 0 bytes .../kafka/streaming/kstream/Predicate.class | Bin 295 -> 0 bytes .../SlidingWindowDef$SlidingWindow$1.class | Bin 2184 -> 0 bytes ...dowDef$SlidingWindow$RestoreFuncImpl.class | Bin 2396 -> 0 bytes .../SlidingWindowDef$SlidingWindow.class | Bin 9359 -> 0 bytes .../streaming/kstream/SlidingWindowDef.class | Bin 3388 -> 0 bytes .../kafka/streaming/kstream/ValueJoiner.class | Bin 342 -> 0 bytes .../kafka/streaming/kstream/ValueMapper.class | Bin 301 -> 0 bytes .../kafka/streaming/kstream/Window.class | Bin 627 -> 0 bytes .../kafka/streaming/kstream/WindowDef.class | Bin 386 -> 0 bytes .../kstream/internals/FilteredIterator.class | Bin 1639 -> 0 bytes .../kstream/internals/KStreamBranch$1.class | Bin 277 -> 0 bytes ...KStreamBranch$KStreamBranchProcessor.class | Bin 2147 -> 0 bytes .../kstream/internals/KStreamBranch.class | Bin 1576 -> 0 bytes .../kstream/internals/KStreamFilter$1.class | Bin 277 -> 0 bytes ...KStreamFilter$KStreamFilterProcessor.class | Bin 2137 -> 0 bytes .../kstream/internals/KStreamFilter.class | Bin 1844 -> 0 bytes .../kstream/internals/KStreamFlatMap$1.class | Bin 280 -> 0 bytes ...treamFlatMap$KStreamFlatMapProcessor.class | Bin 2516 -> 0 bytes .../kstream/internals/KStreamFlatMap.class | Bin 1894 -> 0 bytes .../internals/KStreamFlatMapValues$1.class | Bin 298 -> 0 bytes ...Values$KStreamFlatMapValuesProcessor.class | Bin 2498 -> 0 bytes .../internals/KStreamFlatMapValues.class | Bin 1812 -> 0 bytes .../kstream/internals/KStreamImpl.class | Bin 11028 -> 0 bytes .../kstream/internals/KStreamJoin$1.class | Bin 1187 -> 0 bytes .../internals/KStreamJoin$Finder.class | Bin 1067 -> 0 bytes .../KStreamJoin$KStreamJoinProcessor$1.class | Bin 1689 -> 0 bytes .../KStreamJoin$KStreamJoinProcessor.class | Bin 3227 -> 0 bytes .../kstream/internals/KStreamJoin.class | Bin 2375 -> 0 bytes .../kstream/internals/KStreamMap$1.class | Bin 268 -> 0 bytes .../KStreamMap$KStreamMapProcessor.class | Bin 2222 -> 0 bytes .../kstream/internals/KStreamMap.class | Bin 1806 -> 0 bytes .../internals/KStreamMapValues$1.class | Bin 286 -> 0 bytes ...KStreamMapValues$KStreamMapProcessor.class | Bin 2089 -> 0 bytes .../kstream/internals/KStreamMapValues.class | Bin 1710 -> 0 bytes ...sThrough$KStreamPassThroughProcessor.class | Bin 1842 -> 0 bytes .../internals/KStreamPassThrough.class | Bin 1069 -> 0 bytes .../kstream/internals/KStreamProcessor.class | Bin 1205 -> 0 bytes .../kstream/internals/KStreamWindow$1.class | Bin 277 -> 0 bytes ...KStreamWindow$KStreamWindowProcessor.class | Bin 2752 -> 0 bytes .../kstream/internals/KStreamWindow.class | Bin 1855 -> 0 bytes .../internals/KStreamWindowedImpl.class | Bin 4116 -> 0 bytes .../internals/WindowSupport$Value.class | Bin 1484 -> 0 bytes .../internals/WindowSupport$ValueList.class | Bin 3042 -> 0 bytes .../WindowSupport$ValueListIterator.class | Bin 2126 -> 0 bytes .../kstream/internals/WindowSupport.class | Bin 2341 -> 0 bytes .../kafka/streaming/processor/Processor.class | Bin 424 -> 0 bytes .../processor/ProcessorContext.class | Bin 1281 -> 0 bytes .../streaming/processor/ProcessorDef.class | Bin 214 -> 0 bytes .../streaming/processor/RestoreFunc.class | Bin 166 -> 0 bytes .../streaming/processor/StateStore.class | Bin 241 -> 0 bytes .../processor/TimestampExtractor.class | Bin 232 -> 0 bytes .../processor/TopologyBuilder$1.class | Bin 267 -> 0 bytes .../TopologyBuilder$NodeFactory.class | Bin 346 -> 0 bytes ...TopologyBuilder$ProcessorNodeFactory.class | Bin 1633 -> 0 bytes .../TopologyBuilder$SinkNodeFactory.class | Bin 2230 -> 0 bytes .../TopologyBuilder$SourceNodeFactory.class | Bin 2134 -> 0 bytes .../streaming/processor/TopologyBuilder.class | Bin 8232 -> 0 bytes .../internals/MinTimestampTracker.class | Bin 2266 -> 0 bytes .../internals/PartitionGroup$1.class | Bin 1527 -> 0 bytes .../processor/internals/PartitionGroup.class | Bin 4427 -> 0 bytes .../internals/ProcessorContextImpl.class | Bin 10324 -> 0 bytes .../processor/internals/ProcessorNode.class | Bin 2779 -> 0 bytes .../internals/ProcessorStateManager.class | Bin 7712 -> 0 bytes .../internals/ProcessorTopology.class | Bin 3668 -> 0 bytes .../internals/PunctuationQueue.class | Bin 2576 -> 0 bytes .../internals/PunctuationSchedule.class | Bin 1275 -> 0 bytes .../internals/RecordCollector$1.class | Bin 1801 -> 0 bytes .../processor/internals/RecordCollector.class | Bin 3402 -> 0 bytes .../processor/internals/RecordQueue.class | Bin 4189 -> 0 bytes .../processor/internals/SinkNode.class | Bin 3306 -> 0 bytes .../processor/internals/SourceNode.class | Bin 2590 -> 0 bytes .../processor/internals/Stamped.class | Bin 978 -> 0 bytes .../processor/internals/StampedRecord.class | Bin 1854 -> 0 bytes .../processor/internals/StreamTask.class | Bin 10527 -> 0 bytes .../processor/internals/StreamThread$1.class | Bin 1903 -> 0 bytes .../StreamThread$KafkaStreamingMetrics.class | Bin 2462 -> 0 bytes .../processor/internals/StreamThread.class | Bin 12434 -> 0 bytes .../internals/TimestampTracker.class | Bin 511 -> 0 bytes .../apache/kafka/streaming/state/Entry.class | Bin 1283 -> 0 bytes ...tore$MemoryStore$MemoryStoreIterator.class | Bin 2150 -> 0 bytes .../InMemoryKeyValueStore$MemoryStore.class | Bin 4032 -> 0 bytes .../state/InMemoryKeyValueStore.class | Bin 1518 -> 0 bytes .../streaming/state/KeyValueIterator.class | Bin 391 -> 0 bytes .../kafka/streaming/state/KeyValueStore.class | Bin 982 -> 0 bytes .../state/MeteredKeyValueStore$1.class | Bin 1824 -> 0 bytes ...eyValueStore$MeteredKeyValueIterator.class | Bin 2680 -> 0 bytes .../state/MeteredKeyValueStore.class | Bin 11596 -> 0 bytes .../streaming/state/OffsetCheckpoint.class | Bin 5907 -> 0 bytes .../state/RocksDBKeyValueStore$1.class | Bin 274 -> 0 bytes ...RocksDBStore$LexicographicComparator.class | Bin 1441 -> 0 bytes ...re$RocksDBStore$RocksDBRangeIterator.class | Bin 1614 -> 0 bytes ...ueStore$RocksDBStore$RocksDbIterator.class | Bin 2160 -> 0 bytes .../RocksDBKeyValueStore$RocksDBStore.class | Bin 7803 -> 0 bytes .../state/RocksDBKeyValueStore.class | Bin 1413 -> 0 bytes out/test/clients/log4j.properties | 21 -- .../kafka/clients/ClientUtilsTest.class | Bin 1224 -> 0 bytes .../apache/kafka/clients/MetadataTest$1.class | Bin 1619 -> 0 bytes .../apache/kafka/clients/MetadataTest.class | Bin 4420 -> 0 bytes .../clients/MockClient$FutureResponse.class | Bin 749 -> 0 bytes .../org/apache/kafka/clients/MockClient.class | Bin 6787 -> 0 bytes .../kafka/clients/NetworkClientTest$1.class | Bin 249 -> 0 bytes ...etworkClientTest$TestCallbackHandler.class | Bin 1024 -> 0 bytes .../kafka/clients/NetworkClientTest.class | Bin 7304 -> 0 bytes .../clients/consumer/KafkaConsumerTest.class | Bin 2103 -> 0 bytes .../clients/consumer/MockConsumerTest.class | Bin 2864 -> 0 bytes .../ConsumerNetworkClientTest$1.class | Bin 311 -> 0 bytes ...merNetworkClientTest$TestDelayedTask.class | Bin 1076 -> 0 bytes .../internals/ConsumerNetworkClientTest.class | Bin 5664 -> 0 bytes .../internals/CoordinatorTest$1.class | Bin 1521 -> 0 bytes .../CoordinatorTest$MockCommitCallback.class | Bin 1394 -> 0 bytes ...oordinatorTest$MockRebalanceCallback.class | Bin 1662 -> 0 bytes .../consumer/internals/CoordinatorTest.class | Bin 17373 -> 0 bytes .../internals/DelayedTaskQueueTest$1.class | Bin 296 -> 0 bytes .../DelayedTaskQueueTest$TestTask.class | Bin 1418 -> 0 bytes .../internals/DelayedTaskQueueTest.class | Bin 2833 -> 0 bytes .../consumer/internals/FetcherTest.class | Bin 9942 -> 0 bytes .../consumer/internals/HeartbeatTest.class | Bin 1929 -> 0 bytes .../internals/RequestFutureTest$1.class | Bin 1784 -> 0 bytes .../internals/RequestFutureTest$2.class | Bin 1784 -> 0 bytes .../internals/RequestFutureTest.class | Bin 2160 -> 0 bytes .../internals/SubscriptionStateTest.class | Bin 4644 -> 0 bytes .../clients/producer/KafkaProducerTest.class | Bin 2598 -> 0 bytes .../clients/producer/MockProducerTest.class | Bin 5963 -> 0 bytes .../clients/producer/ProducerRecordTest.class | Bin 1691 -> 0 bytes .../clients/producer/RecordSendTest$1.class | Bin 1654 -> 0 bytes .../clients/producer/RecordSendTest.class | Bin 3393 -> 0 bytes .../producer/internals/BufferPoolTest$1.class | Bin 1569 -> 0 bytes .../producer/internals/BufferPoolTest$2.class | Bin 1589 -> 0 bytes .../BufferPoolTest$StressTestThread.class | Bin 1690 -> 0 bytes .../producer/internals/BufferPoolTest.class | Bin 6434 -> 0 bytes .../internals/DefaultPartitionerTest.class | Bin 2835 -> 0 bytes .../internals/RecordAccumulatorTest$1.class | Bin 1883 -> 0 bytes .../RecordAccumulatorTest$1TestCallback.class | Bin 1589 -> 0 bytes .../internals/RecordAccumulatorTest.class | Bin 13392 -> 0 bytes .../producer/internals/SenderTest.class | Bin 7552 -> 0 bytes .../AbstractConfigTest$TestConfig.class | Bin 1651 -> 0 bytes .../common/config/AbstractConfigTest.class | Bin 2464 -> 0 bytes .../kafka/common/config/ConfigDefTest.class | Bin 6953 -> 0 bytes .../common/metrics/FakeMetricsReporter.class | Bin 1214 -> 0 bytes .../common/metrics/JmxReporterTest.class | Bin 1563 -> 0 bytes .../MetricsTest$ConstantMeasurable.class | Bin 762 -> 0 bytes .../kafka/common/metrics/MetricsTest.class | Bin 10894 -> 0 bytes .../common/metrics/stats/HistogramTest.class | Bin 4469 -> 0 bytes .../network/SelectorTest$EchoServer$1.class | Bin 1857 -> 0 bytes .../network/SelectorTest$EchoServer.class | Bin 2168 -> 0 bytes .../kafka/common/network/SelectorTest.class | Bin 9432 -> 0 bytes .../types/ProtocolSerializationTest.class | Bin 5359 -> 0 bytes .../common/record/MemoryRecordsTest.class | Bin 3851 -> 0 bytes .../kafka/common/record/RecordTest.class | Bin 4539 -> 0 bytes .../common/requests/RequestResponseTest.class | Bin 11233 -> 0 bytes .../SerializationTest$SerDeser.class | Bin 1255 -> 0 bytes .../serialization/SerializationTest.class | Bin 4681 -> 0 bytes .../AbstractIteratorTest$ListIterator.class | Bin 1380 -> 0 bytes .../common/utils/AbstractIteratorTest.class | Bin 2011 -> 0 bytes .../apache/kafka/common/utils/CrcTest.class | Bin 1544 -> 0 bytes .../apache/kafka/common/utils/MockTime.class | Bin 900 -> 0 bytes .../apache/kafka/common/utils/UtilsTest.class | Bin 3389 -> 0 bytes .../org/apache/kafka/test/MetricsBench.class | Bin 3354 -> 0 bytes .../apache/kafka/test/Microbenchmarks$1.class | Bin 1745 -> 0 bytes .../apache/kafka/test/Microbenchmarks$2.class | Bin 1550 -> 0 bytes .../apache/kafka/test/Microbenchmarks$3.class | Bin 1772 -> 0 bytes .../apache/kafka/test/Microbenchmarks$4.class | Bin 1592 -> 0 bytes .../apache/kafka/test/Microbenchmarks$5.class | Bin 1451 -> 0 bytes .../apache/kafka/test/Microbenchmarks.class | Bin 5881 -> 0 bytes .../kafka/test/MockMetricsReporter.class | Bin 1503 -> 0 bytes .../org/apache/kafka/test/MockSelector.class | Bin 3363 -> 0 bytes .../apache/kafka/test/MockSerializer.class | Bin 1400 -> 0 bytes .../org/apache/kafka/test/TestUtils.class | Bin 2885 -> 0 bytes .../internals/FilteredIteratorTest$1.class | Bin 1455 -> 0 bytes .../internals/FilteredIteratorTest$2.class | Bin 1457 -> 0 bytes .../internals/FilteredIteratorTest$3.class | Bin 1454 -> 0 bytes .../internals/FilteredIteratorTest.class | Bin 2662 -> 0 bytes .../internals/KStreamBranchTest$1.class | Bin 1298 -> 0 bytes .../internals/KStreamBranchTest$2.class | Bin 1298 -> 0 bytes .../internals/KStreamBranchTest$3.class | Bin 1298 -> 0 bytes .../kstream/internals/KStreamBranchTest.class | Bin 4070 -> 0 bytes .../internals/KStreamFilterTest$1.class | Bin 1273 -> 0 bytes .../kstream/internals/KStreamFilterTest.class | Bin 3799 -> 0 bytes .../internals/KStreamFlatMapTest$1.class | Bin 2145 -> 0 bytes .../internals/KStreamFlatMapTest.class | Bin 3791 -> 0 bytes .../KStreamFlatMapValuesTest$1.class | Bin 1585 -> 0 bytes .../internals/KStreamFlatMapValuesTest.class | Bin 3764 -> 0 bytes .../kstream/internals/KStreamJoinTest$1.class | Bin 1381 -> 0 bytes .../kstream/internals/KStreamJoinTest$2.class | Bin 1299 -> 0 bytes .../kstream/internals/KStreamJoinTest$3.class | Bin 1336 -> 0 bytes .../kstream/internals/KStreamJoinTest$4.class | Bin 1639 -> 0 bytes .../kstream/internals/KStreamJoinTest$5.class | Bin 1795 -> 0 bytes .../kstream/internals/KStreamJoinTest.class | Bin 6152 -> 0 bytes .../kstream/internals/KStreamMapTest$1.class | Bin 1648 -> 0 bytes .../kstream/internals/KStreamMapTest.class | Bin 3706 -> 0 bytes .../internals/KStreamMapValuesTest$1.class | Bin 1285 -> 0 bytes .../internals/KStreamMapValuesTest.class | Bin 3500 -> 0 bytes .../internals/KStreamWindowedTest.class | Bin 4011 -> 0 bytes .../internals/MinTimestampTrackerTest.class | Bin 2134 -> 0 bytes .../internals/StreamTaskTest$1.class | Bin 1081 -> 0 bytes .../internals/StreamTaskTest$2.class | Bin 1275 -> 0 bytes .../processor/internals/StreamTaskTest.class | Bin 7533 -> 0 bytes .../apache/kafka/test/KStreamTestDriver.class | Bin 2796 -> 0 bytes .../kafka/test/MockProcessorContext.class | Bin 5842 -> 0 bytes .../test/MockProcessorDef$MockProcessor.class | Bin 1948 -> 0 bytes .../apache/kafka/test/MockProcessorDef.class | Bin 1161 -> 0 bytes .../apache/kafka/test/MockSourceNode.class | Bin 2292 -> 0 bytes .../kafka/test/MockTimestampExtractor.class | Bin 677 -> 0 bytes ...UnlimitedWindowDef$UnlimitedWindow$1.class | Bin 2395 -> 0 bytes .../UnlimitedWindowDef$UnlimitedWindow.class | Bin 3531 -> 0 bytes .../kafka/test/UnlimitedWindowDef.class | Bin 1322 -> 0 bytes projectFilesBackup/kafka-work.ipr | 121 ---------- .../streaming/ProcessorKStreamJob.java | 59 ----- .../internals/KStreamFlatMapValuesTest.java | 29 +-- .../kstream/internals/KStreamMapTest.java | 27 --- .../processor/internals/StreamTaskTest.java | 2 +- .../kafka/test/MockProcessorContext.java | 179 --------------- temp | 6 - temp.sh | 4 - temp2 | 6 - 504 files changed, 4 insertions(+), 664 deletions(-) delete mode 100644 kafka-work.ipr delete mode 100644 out/production/clients/org/apache/kafka/clients/ClientRequest.class delete mode 100644 out/production/clients/org/apache/kafka/clients/ClientResponse.class delete mode 100644 out/production/clients/org/apache/kafka/clients/ClientUtils.class delete mode 100644 out/production/clients/org/apache/kafka/clients/ClusterConnectionStates$NodeConnectionState.class delete mode 100644 out/production/clients/org/apache/kafka/clients/ClusterConnectionStates.class delete mode 100644 out/production/clients/org/apache/kafka/clients/CommonClientConfigs.class delete mode 100644 out/production/clients/org/apache/kafka/clients/ConnectionState.class delete mode 100644 out/production/clients/org/apache/kafka/clients/InFlightRequests.class delete mode 100644 out/production/clients/org/apache/kafka/clients/KafkaClient.class delete mode 100644 out/production/clients/org/apache/kafka/clients/Metadata.class delete mode 100644 out/production/clients/org/apache/kafka/clients/NetworkClient.class delete mode 100644 out/production/clients/org/apache/kafka/clients/RequestCompletionHandler.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/CommitType.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/Consumer.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerCommitCallback.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerConfig.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecord.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable$1.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/ConsumerWakeupException.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$1.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$2.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/MockConsumer.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/OffsetResetStrategy.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient$RequestFutureCompletionHandler.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$1.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$2.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics$1.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics$2.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorResponseHandler.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$HeartbeatCompletionHandler.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$HeartbeatTask$1.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$HeartbeatTask.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$JoinGroupResponseHandler.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetCommitResponseHandler.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetFetchResponseHandler.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$RebalanceCallback.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTask.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue$Entry.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$1.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$2.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$FetchManagerMetrics.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$PartitionRecords.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/Heartbeat.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFuture$1.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFuture.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFutureAdapter.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFutureListener.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/SendFailedException.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/StaleMetadataException.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState$TopicPartitionState.class delete mode 100644 out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/BufferExhaustedException.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/Callback.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/KafkaProducer$FutureFailure.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/KafkaProducer.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/MockProducer$Completion.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/MockProducer.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/Partitioner.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/Producer.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/ProducerConfig.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/ProducerRecord.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/RecordMetadata.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/BufferPool.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/DefaultPartitioner.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/ProduceRequestResult.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$1.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$2.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$3.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$IncompleteRecordBatches.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$ReadyCheckResult.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$RecordAppendResult.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch$Thunk.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/Sender$1.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics$1.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics$2.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics.class delete mode 100644 out/production/clients/org/apache/kafka/clients/producer/internals/Sender.class delete mode 100644 out/production/clients/org/apache/kafka/common/Cluster.class delete mode 100644 out/production/clients/org/apache/kafka/common/Configurable.class delete mode 100644 out/production/clients/org/apache/kafka/common/KafkaException.class delete mode 100644 out/production/clients/org/apache/kafka/common/Metric.class delete mode 100644 out/production/clients/org/apache/kafka/common/MetricName.class delete mode 100644 out/production/clients/org/apache/kafka/common/Node.class delete mode 100644 out/production/clients/org/apache/kafka/common/PartitionInfo.class delete mode 100644 out/production/clients/org/apache/kafka/common/TopicPartition.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/AbstractConfig.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$1.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$2.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$ConfigKey.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$Importance.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$Range.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$Type.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$ValidString.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef$Validator.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigDef.class delete mode 100644 out/production/clients/org/apache/kafka/common/config/ConfigException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/ApiException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/CorruptRecordException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/DisconnectException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/IllegalGenerationException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/InterruptException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/InvalidMetadataException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/InvalidRequiredAcksException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/InvalidTopicException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/LeaderNotAvailableException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/NetworkException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/NotEnoughReplicasException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/NotLeaderForPartitionException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/OffsetLoadInProgressException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/OffsetMetadataTooLarge.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/OffsetOutOfRangeException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/RecordBatchTooLargeException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/RecordTooLargeException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/RetriableException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/SerializationException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/TimeoutException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/UnknownConsumerIdException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/UnknownServerException.class delete mode 100644 out/production/clients/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/CompoundStat$NamedMeasurable.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/CompoundStat.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/JmxReporter$KafkaMbean.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/JmxReporter.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/KafkaMetric.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/Measurable.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/MeasurableStat.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/MetricConfig.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/Metrics.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/MetricsReporter.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/Quota.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/QuotaViolationException.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/Sensor.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/Stat.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Avg.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Count.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Histogram$BinScheme.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Histogram$ConstantBinScheme.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Histogram$LinearBinScheme.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Histogram.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Max.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Min.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Percentile.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles$1.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles$BucketSizing.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles$HistogramSample.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Rate$1.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Rate$SampledTotal.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Rate.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/SampledStat$Sample.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/SampledStat.class delete mode 100644 out/production/clients/org/apache/kafka/common/metrics/stats/Total.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/ByteBufferReceive.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/ByteBufferSend.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/InvalidReceiveException.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/MultiSend.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/NetworkReceive.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/NetworkSend.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/Receive.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/Selectable.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/Selector$SelectorMetrics$1.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/Selector$SelectorMetrics.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/Selector$Transmissions.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/Selector.class delete mode 100644 out/production/clients/org/apache/kafka/common/network/Send.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/ApiKeys.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/Errors.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/ProtoUtils.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/Protocol.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/SecurityProtocol.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/ArrayOf.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Field.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Schema.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/SchemaException.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Struct.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$1.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$2.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$3.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$4.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$5.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type$6.class delete mode 100644 out/production/clients/org/apache/kafka/common/protocol/types/Type.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/ByteBufferInputStream.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/ByteBufferOutputStream.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/CompressionType.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/Compressor$1.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/Compressor.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/InvalidRecordException.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream$BD.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream$FLG.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/LogEntry.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/MemoryRecords$RecordsIterator.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/MemoryRecords.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/Record.class delete mode 100644 out/production/clients/org/apache/kafka/common/record/Records.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/AbstractRequest$1.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/AbstractRequest.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/AbstractRequestResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataRequest.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/FetchRequest$PartitionData.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/FetchRequest.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/FetchResponse$PartitionData.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/FetchResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/HeartbeatRequest.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/HeartbeatResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/JoinGroupRequest.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/JoinGroupResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ListOffsetRequest$PartitionData.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ListOffsetRequest.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse$PartitionData.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/MetadataRequest.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/MetadataResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest$PartitionData.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetCommitResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetFetchRequest.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse$PartitionData.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ProduceRequest.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ProduceResponse$PartitionResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ProduceResponse.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/RequestHeader.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/RequestSend.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ResponseHeader.class delete mode 100644 out/production/clients/org/apache/kafka/common/requests/ResponseSend.class delete mode 100644 out/production/clients/org/apache/kafka/common/serialization/ByteArrayDeserializer.class delete mode 100644 out/production/clients/org/apache/kafka/common/serialization/ByteArraySerializer.class delete mode 100644 out/production/clients/org/apache/kafka/common/serialization/Deserializer.class delete mode 100644 out/production/clients/org/apache/kafka/common/serialization/IntegerDeserializer.class delete mode 100644 out/production/clients/org/apache/kafka/common/serialization/IntegerSerializer.class delete mode 100644 out/production/clients/org/apache/kafka/common/serialization/Serializer.class delete mode 100644 out/production/clients/org/apache/kafka/common/serialization/StringDeserializer.class delete mode 100644 out/production/clients/org/apache/kafka/common/serialization/StringSerializer.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/AbstractIterator$1.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/AbstractIterator$State.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/AbstractIterator.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/CollectionUtils.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/CopyOnWriteMap.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/Crc32.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/KafkaThread$1.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/KafkaThread.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/SystemTime.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/Time.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/Utils$1.class delete mode 100644 out/production/clients/org/apache/kafka/common/utils/Utils.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/KafkaStreaming.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/StreamingConfig.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$2.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$3.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$4.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/examples/KStreamJob.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/examples/WallclockTimestampExtractor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/KStream.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/KStreamBuilder.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/KStreamWindowed.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/KeyValue.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/KeyValueMapper.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/Predicate.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$RestoreFuncImpl.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/ValueJoiner.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/ValueMapper.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/Window.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/WindowDef.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/FilteredIterator.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranch$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranch$KStreamBranchProcessor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranch.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter$KStreamFilterProcessor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap$KStreamFlatMapProcessor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues$KStreamFlatMapValuesProcessor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamImpl.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$Finder.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$KStreamJoinProcessor$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$KStreamJoinProcessor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMap$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMap$KStreamMapProcessor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMap.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$KStreamMapProcessor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough$KStreamPassThroughProcessor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$KStreamWindowProcessor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindowedImpl.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$Value.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueList.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueListIterator.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/Processor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/ProcessorContext.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/ProcessorDef.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/RestoreFunc.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/StateStore.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TimestampExtractor.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$NodeFactory.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$ProcessorNodeFactory.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$SinkNodeFactory.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$SourceNodeFactory.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorNode.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorTopology.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/PunctuationQueue.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/PunctuationSchedule.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/RecordCollector$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/RecordCollector.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/RecordQueue.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/SinkNode.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/SourceNode.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/Stamped.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/StampedRecord.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/StreamTask.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$KafkaStreamingMetrics.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/processor/internals/TimestampTracker.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/Entry.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore$MemoryStoreIterator.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/KeyValueIterator.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/KeyValueStore.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$MeteredKeyValueIterator.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/OffsetCheckpoint.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$1.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$LexicographicComparator.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$RocksDBRangeIterator.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$RocksDbIterator.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore.class delete mode 100644 out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore.class delete mode 100644 out/test/clients/log4j.properties delete mode 100644 out/test/clients/org/apache/kafka/clients/ClientUtilsTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/MetadataTest$1.class delete mode 100644 out/test/clients/org/apache/kafka/clients/MetadataTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/MockClient$FutureResponse.class delete mode 100644 out/test/clients/org/apache/kafka/clients/MockClient.class delete mode 100644 out/test/clients/org/apache/kafka/clients/NetworkClientTest$1.class delete mode 100644 out/test/clients/org/apache/kafka/clients/NetworkClientTest$TestCallbackHandler.class delete mode 100644 out/test/clients/org/apache/kafka/clients/NetworkClientTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/KafkaConsumerTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/MockConsumerTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$1.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$TestDelayedTask.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$1.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockCommitCallback.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockRebalanceCallback.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest$1.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest$TestTask.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/FetcherTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/HeartbeatTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/RequestFutureTest$1.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/RequestFutureTest$2.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/RequestFutureTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/KafkaProducerTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/MockProducerTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/ProducerRecordTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/RecordSendTest$1.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/RecordSendTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest$1.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest$2.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest$StressTestThread.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1TestCallback.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.class delete mode 100644 out/test/clients/org/apache/kafka/clients/producer/internals/SenderTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/config/AbstractConfigTest$TestConfig.class delete mode 100644 out/test/clients/org/apache/kafka/common/config/AbstractConfigTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/config/ConfigDefTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/metrics/FakeMetricsReporter.class delete mode 100644 out/test/clients/org/apache/kafka/common/metrics/JmxReporterTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/metrics/MetricsTest$ConstantMeasurable.class delete mode 100644 out/test/clients/org/apache/kafka/common/metrics/MetricsTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/metrics/stats/HistogramTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer$1.class delete mode 100644 out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer.class delete mode 100644 out/test/clients/org/apache/kafka/common/network/SelectorTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/record/MemoryRecordsTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/record/RecordTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/requests/RequestResponseTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/serialization/SerializationTest$SerDeser.class delete mode 100644 out/test/clients/org/apache/kafka/common/serialization/SerializationTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/utils/AbstractIteratorTest$ListIterator.class delete mode 100644 out/test/clients/org/apache/kafka/common/utils/AbstractIteratorTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/utils/CrcTest.class delete mode 100644 out/test/clients/org/apache/kafka/common/utils/MockTime.class delete mode 100644 out/test/clients/org/apache/kafka/common/utils/UtilsTest.class delete mode 100644 out/test/clients/org/apache/kafka/test/MetricsBench.class delete mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks$1.class delete mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks$2.class delete mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks$3.class delete mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks$4.class delete mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks$5.class delete mode 100644 out/test/clients/org/apache/kafka/test/Microbenchmarks.class delete mode 100644 out/test/clients/org/apache/kafka/test/MockMetricsReporter.class delete mode 100644 out/test/clients/org/apache/kafka/test/MockSelector.class delete mode 100644 out/test/clients/org/apache/kafka/test/MockSerializer.class delete mode 100644 out/test/clients/org/apache/kafka/test/TestUtils.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$1.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$2.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$3.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$1.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$2.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$3.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest$1.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilterTest.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest$1.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapTest.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest$1.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$1.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$2.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$3.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$4.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$5.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest$1.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest$1.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$1.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$2.class delete mode 100644 out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest.class delete mode 100644 out/test/stream/org/apache/kafka/test/KStreamTestDriver.class delete mode 100644 out/test/stream/org/apache/kafka/test/MockProcessorContext.class delete mode 100644 out/test/stream/org/apache/kafka/test/MockProcessorDef$MockProcessor.class delete mode 100644 out/test/stream/org/apache/kafka/test/MockProcessorDef.class delete mode 100644 out/test/stream/org/apache/kafka/test/MockSourceNode.class delete mode 100644 out/test/stream/org/apache/kafka/test/MockTimestampExtractor.class delete mode 100644 out/test/stream/org/apache/kafka/test/UnlimitedWindowDef$UnlimitedWindow$1.class delete mode 100644 out/test/stream/org/apache/kafka/test/UnlimitedWindowDef$UnlimitedWindow.class delete mode 100644 out/test/stream/org/apache/kafka/test/UnlimitedWindowDef.class delete mode 100644 projectFilesBackup/kafka-work.ipr delete mode 100644 src/main/java/io/confluent/streaming/ProcessorKStreamJob.java delete mode 100644 temp delete mode 100755 temp.sh delete mode 100644 temp2 diff --git a/.gitignore b/.gitignore index 37f242bc2e8de..dbc05075b8d27 100644 --- a/.gitignore +++ b/.gitignore @@ -34,4 +34,4 @@ gradle/wrapper/* results tests/results .ducktape -tests/.ducktape \ No newline at end of file +tests/.ducktape diff --git a/README.md b/README.md index eb1538c27c455..a4a3f7876472c 100644 --- a/README.md +++ b/README.md @@ -117,4 +117,4 @@ To contribute follow the instructions here: * http://kafka.apache.org/contributing.html We also welcome patches for the website and documentation which can be found here: - * https://svn.apache.org/repos/asf/kafka/site \ No newline at end of file + * https://svn.apache.org/repos/asf/kafka/site diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index f99cfb9ee41ae..d1ecbe9b7f194 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -21,7 +21,6 @@ import org.apache.kafka.clients.consumer.internals.Fetcher; import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.SubscriptionState; -import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; diff --git a/kafka-work.ipr b/kafka-work.ipr deleted file mode 100644 index e253c951c1028..0000000000000 --- a/kafka-work.ipr +++ /dev/null @@ -1,209 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/out/production/clients/org/apache/kafka/clients/ClientRequest.class b/out/production/clients/org/apache/kafka/clients/ClientRequest.class deleted file mode 100644 index 7d0ae68b2ef3e094f9503f2947ecb3a16e8de7c8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2117 zcmcgsU31%15IxtHEX7uuBAT`?`EsGb7HO1_51?s6vD?rFC!I+;nIUhoeVeGUB_k^o zegn@u@C*6^Gcbdb8JOXb2YwW=_eyc>L`{bmUUpY|@9v(nyH|hx{o5Y^7V$KTYgo!+ z6w5dHvf_4w9NtHy(8 zPOV8Ev+5JoT_*%M6k(BBCI*V|{fRxDLtIA1$< z)A9E0$V*s;cS0K6rqh;eXussrJYh%|OSBbp6!jmsA$Q<(D!qkRN=)#kDKO;XoqguE zsPP1b_BOlY9|d*3?#fvVbahrxGcbv1AvO$L#|;Bx7&mYW?-;m=ynzYi8E%|!LixyT zHh91=K9I3E_()a_e1WPEn*?rV>U8C$e~YlqOvSHbm0@gV>avXny||Kr_fce+zS!Nk zIRj`IuAbq3T6;mycffTzWoi_SJ>#_7^fVdn7B9~H;u^#qplui{?p<~~kS;>8^7{Lj zr(f16{o7QSVjg6fVh~c(RHUX6NlmjMJw&aU#1zQ_ebt~LL)x$=$MZ^FO`m}MLf`Vr#l**6|57atjR9Y zR$lEwvGPh6sx{t4#v09Zp`C%4kO6a)a-P6{h!HFh>CJWnxoeh~E)uLFv)&(7!yvkeNM3&iWOZV;JUeWGlzGVva<% zN;Vy9KR`FHzJhALMSjyHag4WrrbKDkOHl;4C*c}Vff#Oy{MV6sslob(4w!}ykfIL_ zGxT{AcS#oLjx7#Js{+iAWJE)X5|{fDt8rrb>ti+{SQV)HgVz&-?Ox?_TzQx(D}O>$ zUO`C<1pU-D`jQOyQB#r1JrKypFqZ&EIS*#5FV7j1d7Aub8>0D$Hl%+#$7DM(c%sLo RzlYlS9_}UvFUS&${{pcixDfyV diff --git a/out/production/clients/org/apache/kafka/clients/ClientResponse.class b/out/production/clients/org/apache/kafka/clients/ClientResponse.class deleted file mode 100644 index e7802d3d34b21d1129c59d081a34aca60ff3aa5f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1880 zcmb7ET~pIg5Iwg|+fY)$C@O+l6|oIy{6J9wseq`pzzjIf$oLY{D+EeXlN5(Xe~m9X zqcc)xbjC*?{85g3lYmen=*#Zr-rYUBXZPmkudm+$3}G&dF5GCtaoiMQNQhfP+|H61 z&f*U4W^gYH88>8%2t6vqn2h^EkIN_t{XoWqj7ecmW$-YAX$INlCbu?tm4THRl&a;K zj%{<(4~`ZYGA@7J;GWM=s5tJjZqyBPh3jj^t2IM6YZkYCPoE6*eCRnsd0g(*9oyp* zPIZ$(kBM~F*Bx80yN>UePEGeW>)g{9e79lxlpsB7*_MBw;pIQ0mKXmMrTCmdnsln1 zp`&8ie73Q^#NB7cQjJd3iennJ=Z0&E@90qSS1gaA7z5NoMA&`EL`d*6Yn{?23q_*2 z?O~eEiEFL!52=G`Q0D8g&Ii-lzG8Uu(P$WQ3%+5l%^3Blz0PRV6~pJYxj93v z%DzK0vFv4rNx z15JY^ErZe6&ymd2*IOi!Cf%cb0n_qZNN8Qs7LwY)79_1ZwT09tx=s)PXg!4MMFQzV z8?8tO`U&$}s7=Ak0>})m@h(Uru#N~#BuoCb}5RAcSG(<53#RHwpb~ZyMGwaN31n>LC z`v4RVJkW0*Rl!Jke8MXIWcerj36|yc^vq^27AdQ&qz&*we5_s74!{uaPye44~6 z+@fJm5_Pzx3Af_5CftrY)Ua2>ok?i8D+vQ*YA`iqG{_`$#WS8jR!t|=U?tIneHtc{ zurZay0yt`L6UZs4cWdwx@YOU(Am4;(+@oQC5(jW^BUa$P1nzIb19(uwLrFZ0M>ISt zP;a|gfz~0{%cgyMeDl8akekg)uU8Xoy&<3(5re>z=+KctdXU42Yfj3b zAdZ>@OmWZTexS;@eWDs$j-e8nU4_S#94HL-R;r zn3FqMHpEK!GuqC#z zM5D@2j<|WxlmnKk^OmxT-Jps>$6Bl-jHTl2^E~5#s!%4DjwkUHvrEU*ct*#wc#a~f zYe~oRIH=(T9WUY~9WUb*=CtZXl`X@ansS{qqdsnB)BWM6SgHbxs-<3)x9p5$F0Vgb z)Li!oMMfS)0#&0Kbf#7NNwIcQmXjKD-M|k#Be%hq-n8`mo|KMP@tTg;@rLsLO@YOs z@|wBMqSA2)2NmUE9Y^qtz~=44BPljE*Jev_oz%ENY%{5G*ZWtcZ{cl$3rl|1Y+3>< z<{VJ{>5Lv6#X&+d?3Vmfy8TqvnwCyYE)DPKc()Ynup1WhRKt5Z-p4T=tFeYb9*BGj z+*DZSDGeX!_z)intUPl(O1FWAk98c!Cl%>(!`MD)vYzLK7!=zn`*UeyFpwVG0pos7 z38_L~W?EatWsEo+XA-BFt`{9(DDdU)sEs(L}VICqXd^gA+z)Nxxn**j2t2vD_G5IBU+rQo4lVMlL6v4BOUOSfb0M zP%(5W5xCKoK&9J-TZb*RmPFMzEGOXYooB0Pt3fiTdW&xPqfVIq(~`#3^D3+oSU@*K z-?#3iuqc@2U0R?a%q)S34eSkJj05ughjxN0QJMWiCL_?5ym?)`ES zb$d>reyC#>F@YWHXVD;V7@u$w7x*69aMvlk*)@a2J9w$U}RI3{}<+GD4XXm9ud zOHZO{&q?SM)7(0L1}!9RnZY?H(E8m;EZEZ?Yj2prxhJr&C*B^PLEGU1WBYr?PekPZK^;$ybaJG?q zaZ;Vn{mq^cQup_S-UN0EB3vd1cskv}2BjlY?IJ;)CydIy^uo#kP z#?k_~OT#?6G;}&ihnnf+JUX|KE^frO5U>uO&Lim8V<-fo8#^#e{j>l#5FC}p-MA4u zX`g~5%3HN_6L+rUj)uA&8b($W+}TA8M~juPEEK>5Z(Z>fR(^?;ss?rsj!`vSz}0T9 Hm{tD(iNU^D diff --git a/out/production/clients/org/apache/kafka/clients/ClusterConnectionStates$NodeConnectionState.class b/out/production/clients/org/apache/kafka/clients/ClusterConnectionStates$NodeConnectionState.class deleted file mode 100644 index e8dc3026f4c3b3846d63ed463c7e471d6bf17987..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1090 zcmb7D+iuf95IyUpabnyHZlPQPxTS#OwsC>-RD=XUm54%-NL8tin`BAdx^`r5_!ce_ z-vAO#C6M?4J_<4GSVC<=1P`7W&z_kxm))PgzWo5OiAM%zuvEe{mX)}zW5qxoOGVsK z`mPf9ig1*!6yYkps$*S8m0>E8TnYxa5yf4P_jvn2ct`xr5%=1GFG87kwI~clTl!Jh zRM;(sg@7kAb3c_*9QWkQgn`u=3XlEJmrod~&cA1@SN0h4wWvdd%r^W`?DUUYBHrb# zfILLROMP3T zeKQng)ReIwc4;n7rE$pLaxdUv*Bg2%%xFgaxGlDQ)lIoJp0cU}P0YhHF^f497hy3h zo=^9z?*|JpcQ0$5%6M01R_~s(IBq zq&vZobJXq%yw?+9hhg0r9l+>JvLf$jh!Ua zxBi9m*{UFIeIpPaZkbA)e~s0eS+MV6nL4Wl^(w%`5q;FprjwMNPe3FRq89a zPFh)thNV`Sq7)6SDWiE!Xm#d%M}GelQ&wRB-Tew}fTC3z+PMJ?>jK#Wn4bx&LGc&T bxKAmkIn0r9YPgl6Y+NI4(AhNBa23XH84?qG diff --git a/out/production/clients/org/apache/kafka/clients/ClusterConnectionStates.class b/out/production/clients/org/apache/kafka/clients/ClusterConnectionStates.class deleted file mode 100644 index 19069c4afa541564af5b29d0f48b1424e6739a26..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3360 zcmb_d-BS}+96g&4mMlxq2%&xwsUIPrQHy9nQ3;?Rgtj0STWy!Hgq3ACO*XCer89l$ zOh4ZGW}oeg&a{k5Ei-MO>P-J8rtP_#H3=ngpkrpTA9wHXp7T4ubAR`*zkmG=Krf04 z4q!Bd{kSUQngW06WvmI~xGrNN1R0Ydq%kF)-cj%_-jgvM{NkU|PkfqD`00j-m=&)# z6|VNaj5!&%6trQq5g&;AhvITuK`ZXa_(;ab685C+1wCb3mY#78+nRPXN0*?EaUh>E zwL(EJNQmV1OliuHmRYtJ7biI=jY|kwjP_z$ChlsVYRRHwn8`^kH^8scM%L1tVxB7+ zo4;Nty_%YpO-?&`!^#d!*!gTy%W0Vt`GMqsPf%tgWEN>~Yl+ZF^SbDm+GOy=nw0V;wkqJAanX_8n5bxe#z*#bw-kBZq zA+nOuERUvyXnbpu<6XBT_>2XC?aEcHfCRr~ef`c=e?PhILcBUynU&t~kbuep53XomnU^JrFL%6WIU=*sRRb}CsaTUiyRm~s>=DlFJ4a^m$cQW82UGsjF*&uXSyn2~!KJtz8&g!ameQX?2D z8s>tYmvAU;$GHFV_hXB8(OTd&AM=XM$e4;xkXKQFBcLxyIJMKXCG4tzPtD(D3*;7z ziNZnfDi1^KNn@dUx{=VZAP0U#!2gJmepEOdM+@Iti5?}U;E-^+^_4mF;Utp8CL-mCkzor{ zJ&iM*_ZCN+;NL)0M(@GPg6P@p5%t$2`a>O}y@WkWqJ1QK{&f)T!#PjURc0bHlm1$Y zh8}w|o(oo_Z4sJAgL>!}G(B!b^J7MrRNaRLKBq`_m_$cdzfmfAm9($Bqcm;XBCxmC2p*0Ae2cKdO& zVmrmP*9{Vzb;-nBueO)Tw3o@Wdt@XG;DQI^Fj{!KbCXPyJ%3_%kL$U}!!5NdA>Kw9 zcd8IO%ZQyGVwjZ`mh!o&GHOE{6up;)qb_crK}$Ubi)4`f-wXzE$fGiqOqcI*j31;336!W$s#sXN#$@k=dhP^0~*;&M9vZ}zXBmt1K}PK?i1njZ6jR4 zke9a&utie&UPI?FkK>0(|K%TIxliGVPE@+b51>Z>g{7J7J0Be=P9=ht|0Hxd&hTG870f$$+AxlSNWt7(gse@ VC!{~@CNY9Tyep(9bMqX8@-K5t;N<`S diff --git a/out/production/clients/org/apache/kafka/clients/CommonClientConfigs.class b/out/production/clients/org/apache/kafka/clients/CommonClientConfigs.class deleted file mode 100644 index 5094e059ab5081e74f0457f9e174c243418700a4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3528 zcmb7H+mhQv5FO=0HX$K5E&)O^;kpYOnS^jlLcsBA6DMAKvCRgaTw`f$O;*wfX}re# z4qw0%RaC(T@We;($}doy9?4$qB@e0Ea_MM#db&@a?$Q4B-|v4i#%{6hDRz|IKEdvs zV0RsM>3*lv@AbR&%|XxWZh76_pwVfsHy=3cLhGs65w%Dp<66&TDjDB#*qMe-aw8J6 zC8CL3VqZDzWw_YuVXf0xV)q>O+J@J!uhskY!AAZ2LH&VO`d(sRW7~m6yX_ks-1FLN zgZtmDuY29aj=ICnR^MCW`wn}h>ovUQmbc%&0YCee?3Oi$oolq3Ub{bNt}VQKl-c5~ zHC%Vtn{d!+x4lMxaKGMo+*w~AY|QytVxZ{Kp$2yQL!{sJ`rRK6+uB6>2NvzNM-ICR zV!F*nZ_ul6Y_`0?lV*FZ^Q3z6V~1USrjd2ia@ZSli}rULrK{c8H;UreW2)=e?651v z&Oz7P>~#B7gI2xQ^LBT)LCv#l?C~8qDVb|_+C5wT<{EOT(6x*H8Hy2vk7txl;6yfsaF%zoXVQx!jy=l%Y`$bC-FjCsDwML_lG$vzXuYA(ngO^~P=D@5U-U-?Yy+ zS6$b=TeB7}@2SL>v=O|{GpF31f~_JTN~;>{UV;zCS`y7;Ng_poQodYp7mFG9LyT0( zgIOYC1>Huol~uT(r;|ufkZ40?G-968K??ZtOixg{8J}Vv^H?Kf1J_(9lKkn(I9w%4 ze1XJ7nHtA2fu1HSuEQ?I(uhD9;l|=8SB#|_qhbCab)+&=I-r^x6|F9-o_S=+ARF!$m)D9lfWHLMk7dlI5QGCd6ZMGezq#6O7RUqdl(DpGLp1@ zRbynFnv$J`8%j>_G3u1q`B@GNv7;5Vo5@rPBLiq^-p4dQX$wsVidR7O z!XSqvm7QM2IQMm&PK>0fVBHk6$39|f=@or?t}t*>BaCdi3)#QBccC9a{{{DS2MU~0 z_@kN@_AYNyfHA6JRe{)wi(tcAc1yc*_9IgqG7>brNZ2_~qluLzaj-Zp6sEW*A!`ig z;=V(!G=+)?xQ{lb4k%jhQT)^VBx*YmO~niin3Np>S!8mI3my`AwDCO}9D8#^{NlT8 zE=q{V&Se#Va2m|tHX`b0!VBfXs2s-B$Gc_NGp%u&j>{a$U zJI0PPd~yLDVJCo>D)=PusS17(_;dxI0e-21&jO#T;Pb#2D)=Js%N6_z@T(R48t|nG zejWIY3Vsv#as|Hye5Hb~0`m&K2K;sfzXSYk1-}RUeg%I3yj;PwzEZ&-0)JG&tH9SQ z*afat@W)`*p)W@KegcMT7BhDI!{3HGsXk1hR_ADwdDbZpD@hkDg> zUCZ=s&ozARTO|f%da>R-V2IDK$Q$jw#+#bKkbgT5Vf2hUBWuX8oypF1QL1|HErvwH zcCF^9-?avZyz9`395A+y7%G{@2|wey!`+^4_ygPRm9q1Q(;A-1oi-oXf_lm?xx7zl zw12r7qGj8){R+d%e^S@8Z3f9dwyD~oOh*>3AGq))B*#K(L^BM}>&B z_YG1g-~ diff --git a/out/production/clients/org/apache/kafka/clients/InFlightRequests.class b/out/production/clients/org/apache/kafka/clients/InFlightRequests.class deleted file mode 100644 index 1d3736a23d2a67116d3f94a9f07ceac6703afb97..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3688 zcmbtXS#uOs7(G2Rq%)Hyfia0d2n+~hg@LFDNrXVyq7#;cT}3*RPNqpu_t4W55?n!X z7ZvpZpDbS}t9ZaF79&;alT`UT{1p~1=XUpO$wWd~sod`V?zfz8y?1{9`=_4)JcF+z zScAhdjzkc^;V@zt63cJ|^*9>AF&wYK37nL1O2+93A~-C*U#dYEFN@_B8E2~TY6NvS z8=;^oh$AwR;yEf7O=uYtZ{tEVRRbO8YA^vqEa?bLSYjh15&i~*2hiN$=RBO zGv&Lz-lw6OY=^$Kz^7O{TQ?ggQ2%#*>DwnNBv|?>>E4Jv;a( zp{iFmb!Uf!Q0qY3aX$L3QQECb=%#ismmbmVqw0u3N;F|5RpYp7>*CudhMbg6uRG?q zIXf6M8B|9{DbqT?P&rTmOZEdMVZmbbg+fHPbase{p|RYX=}g|E{}WFMi-#RGIl(l# z@nP*rP<-S^a$1gIT;Do3g>4l8hDZccHswg@t!Q9@ zb;as7)T}eil9R9`uQ*X4)g)Duo1=r4DDZ|#rcO&(#8MXVs@KcAso%;8y%ojSSi#md zu8T%RC-CW5j%vuDKy}j4RC||Uu!1U?EQ#}gquHXyStdgZWPPk)M~1DOoz(W}qOBTc zH^?S24ivN^$I2~^h68Fg#kx|k4(k=H#ZwBlU{b~f1yh)oaZ$l*c%6+~6miD^BUq&D zvTb$RB`bIXZz^~TZ!5Tj%Mw<*T<%~QFbr*6HCW1yws$J2WxTnyb4bf(EHkUc#%wDc z%WC$dX2-@XJ9Z(ZCsQ#erP*3c<-cjgbaUgFI|O5X`DZr?pWach9o-7v#T5w~7FJLS z-oq6I?~COFTvqTQ_AB@ZAItbe!Kb*&{3`fN?0k+ZGQLo74PPqwN`xa?;+CIQ32pNl zo}CO9a8bdbk#n5kd7bXSc1y*vY`5?@rZ%^>RhVj4I&GQpd?4aO-s~4Wz!K}63ub

=u%lLdp#Ne;aScV~n2Mf8(*#g>dM$;yIYhl5i zf6tE|9yXz@ev!>qHK`grk|iw5t8!!1Pa*GgWc%qRd=IvLfc+pb%iKgzH#*A9bgqOYULmFP*n2 z;=%O7wFR{_-bBOeX}Sonr2wx5J?JHvS}dOh&!gnwg%Nm&Cc+E*X?&hSJFt_hz`2`v zMG|Yh_VFoKC z3=&C`DDLHaSt)WAKG?L;jo#Vb8~^;#RK(~Nx=u~!IA$i;^F{&dI{pqiz&MFab@`|R z4mbrLb>)XWmTzu;cZb-)oKOX6`-!f1LDO^rm4HGoyHE(U*mxf z;G+-|C$tqCpnZsr#+fr`Jahcx=eO?w@B%giSPNi1fDHzLv_~l7O9ty<&twS~sfbRI z55&np@W`B>8_nOE7~9>}M+W0rncR$w))+Zy^Z_byD|(l~>@}?;q{5iN+J9sV)iH~j zQn%|;l|=0<-}iWg7dcwjZhat9YLsHI7h08N!@Tw-| zWz8`Dkg}=K7Tdd2ZJGv^Ds;G4NN$Cto)S9YeV0j{(Da6Tcz>I`yEU>9)(PznpWdx( zQ=zb~XGDx~VTNLDB=k?e(CDS^D4CqPYAyRF*N(wx*m}cYspj&<`JBeDzX`9&Y?v|l zRJZujnrUX7iHzcVsXW9>+x~l3S!oW-t9Hujw7td`F~B_aosn^-`XG2og#_4T7MpPIhNE3$r`R%q)Zg zYFllqR@+(>5!$NQ(pqgr$%o~6^!NTB{k1>$>Cwj~9|iqmo2F&Lp^UWN>BhWT?t;Hi>hXkkxq^zOUj!5{m-) zi*2}s=aP6HFR1uI96wB=8$U|oMZ6?Bf2`ssDt?;8T{siR%QC+r!>e)pOy<{;SRvwR zye=w#F4}$}7k??1-iYH@NnFO8ZFmb;RJ@(URlK9(nu1o#E-2_aWIKh7Yvs3|${ex_ z1=AT+(CL_Y$8^W{7}?WyK0o3rP!4jtWO_!<@Qi&Em7-(HwtCid+@f8kz_5aL%W%EN zgS&FNOrA%p*{tch`Kr})qGqkC>zR&%Uf%(uVr0k7%xNQk+Q?+>Qpql7_68#KCs{Vl z+!Ga=HfiXjf|zGlidkA%bm)|E*2q-7qLmpnJ<29Vi-oe`RUL}1n|-<~c(jaiAv5YZ z#d2YAh&N)pishm=L_|`3#}q{N+Bx1`bf{Q14_8ZLrgPL7v$#X3vW9icaEkIAZbrQE zBK=smKonNdEPHNdBm|9?qN#%i#Y8l3df9Q_O7$&pZ*KIhMlPpdX==XrzAhgM@y##o;zD%HEEHKmiyX6L zFcsxO?IgKxB5PJW)|ac|I&(`Tj)s}(w?=; z9-VW2^WTsSDa>kErR!7wH?uxKt;_?nkT;4}qr@dgjHG*Z`_%G5yqUm`**&TtzAI~m zg`6C+_4veEOlqu1C)4={9fcigU8IlXLq`k1Nj(l_g+qGc6z*JL(wu0%1c;sXu8!)Xn_ zM^QmW*lZy(F&)Qt+{{rLwX4m(ENeI%aKfHy(aH%k>HGbph>w9s4G)bxxl7k@6MvB5 zk7NzKRVn?o=vCN)f9nr+{R>B!hAGwUf>-Pjr~(|M;_@yuLPPS(uVKZ}Ui zr)H_*oo7M$s%n%wrxyZygMI9Uj)ut*!z10bvgxTtNKoD;jpNbtd>52hW!b9Zuv8>V zBa&*eLqzI>jb&6SW|=&4-$Fb%j|svgl5_87Fa3|>xMv4xXS>z7U^a)Y2o3rH@3mRf zSz)9>>CkwM7RXI3(_;QSnX}d{z8V3WFI9O{8dlXjl2_1^8gBL|l%T7fb6>>-D`$>X z`I_FjK-5i9g{aTp5hs_;RlDEFdbYz%CJLq(yz~{2uo?;3OXhJ0gO}5+Zv)arn%@>c zf}b9&X|B42NBo&o!ns8_#GlD3{!C)=XVQs3lTkQ#34uVtE`RRL~r?myE!Il1XP-&JA1g&0#WS6!+a7o`$9EW&`L$Xw)8Y&={1v3)4h|3 zrxTM%$dF98PSVBm_&q%V}yAeZGf2 z_hC1AbdT?lPO~%`IwUK>NaEQhgv--j?8hUalNru(7ERwq`zKtqP!$mKHxD#L7!4Kn zl5>Z{%MVjp(9n2!@y5^5)q>*<#{;B}pbd}udU^ucx?>s+;ycvTi$fR*HNL_yRcg#o zW5-pfiJR!+GRgnAvBpfyV96e?6Z}{CJP_^eiE{5_UTO6m@9?945-V{6DV+2Tr>T@G z>xMHp>?5+0AG=3RC|6NDM#(5z+73tP7s)jAYX^N7n3Cy;7a zYhz7#CF^({#S@f#h{X}7j3hI!CoW${`yF(j=vq3Bo-b#xY#^c!#6`V>}ot>Iz{`ZRjK)O+IQK`M^MTO0DhQ;Bquv|P1k8xCC;;bLd5N~ax#4*AX z7vJkJ=5d_OCaC1`Bu`tY_c6l0nQ$*B#3#9m`G9sJ+W8--kx2Kyu_^K|>UoN{BE0)F z-otnCy%6Qss3XRc{*5F3UktTwk1UO>dLJwMmqxbSF9o`IMKma~7bD8eum7H&!dk!H zf+%~4E^v`ay~M~aVgsJTM*g&sE|sR}X0J-ciBx|*ihj1$Gqf*>Z%4~lh^u&3QU1x^ z>gACUq>i5>trTLZON)vt(^xmRU0!NRzZMz@7-o#RvdD#DK1iL?Hf^GZZF5fx8r2kZ z-s`@qpkvzVzP8mn#w6weS-n9VrM!0p45ACUa7d<4u$Cgv3d3v^ewo7hPqCrvUMaB| zq&T&Ij=l&Q{kqoA@0-NuEmr9j{-ArCKku$$HQvEmTqA*AuVYDjb+2~Fx0(;m4ix-4 z=*2jzE%0xMzzHGw447zG3%ce8t$olGYpkv{=Iau6*i!GpX0mE(zyPujo6v8N)hV0~ z9mx=L$)FTcQV64&>257>Sz)IvbXlEyd1isjakg0PebVdO<$J4(WWVtz>% diff --git a/out/production/clients/org/apache/kafka/clients/NetworkClient.class b/out/production/clients/org/apache/kafka/clients/NetworkClient.class deleted file mode 100644 index 3a959fc5e2946b7b31f830498b1f5ee169d7c942..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 15727 zcmb_j33yc1^*`s%@+OlPmOz372*Vfk{QJB`}2L=_uhT?o_p>& zzq8!)Cf|7Pi%%2L8R|AaJ;aSA^aQVzrb(Jr2Cw#$htHFTH3m2P$;WHuVVyMROS4{@ z3k+`Y(-3Ync!O(KUtga*Y?L>fq`6Rkq;|3=T?()+q-00&=<(l+yO0H=?P3CSHzO9tE^A4f2)6bXj zBn$e^(y9ClB8j4t`+p zOTzRIef%SN^JC%UC#C#TzSZEL`S@k&{J9@%{zB&bQs(^1$G?_2zY$))>Eqvuj(_Ll zSA6_?gI^Uz{lQOn^B)cVlkoUwdH9R$>94}~-=y#FKK_T0{-;oVO?v+2<9`bV|0CV6 z`}n^;e#7AZN!Odw9FzXHWZkz7e#cLb@w>vzdp_G*&@pP(sr85W2BwNwIF)WnG=avVZQ*E4WV=${tzh!bkH(|v z1xy`P1-4gIGX=&D)WzU^G^}W7n0h{wyB?fiQ;pHM)zsa&#Y(OfmNa-;!?E+j$*4R# zo$ho;6wDT4&snm`MoztcgtUU4;T;1tvN~VOSSe{aKv^7a-I{1?6M1-&RyeYgX=0(Z zY*B~|>sCyKP~D;urf~%%kv9}(5l?JqDgzi>x2_C#Ihz`qgXt1z@q`wIw(`)u42raX zmkIB~tng|?Q)}$4c{PwCa6U}_XiCs&i@u_2u)bt!9;4cxm=+WoSwISoat7EI)Z_H^qt)sXd%`=XUt3{*zYkW`m58S>b?R~U?67YS1vw)f?X(izu-lX5Zf&B; z+L6w9t#w^+a3DB3_X_U8c1erUX{)mf^G0qDN7Ks^$;>2e=ul6_@YiM*D9LaLKo1+d z>B`1LzKQMZ%9vT*fU)yWczX-#kTn>}Z72@{0@474e5R7-XnQ=I?oMK7XBSYw2cfUs zU1wL!N?VIWp{Epp;9w0K%FDF+M23DaUPe?6UQ5hkQ11sLJ90q8sZVsrp?Fu-RD_L! z=868{aRA8y{BoGMgqcl+49oUOUCBf`(VB==FY1b(Ywb+c6#}t6)UmfzbwBFBvTLga zcNHoCk0NUz$ZQX9wYt0XW?;*n)>uL;XlTqrU}{W&?T9#5ab^&rBoKuX1nql4BO2Mk zWVUuElMV4U9iw5(QIHki2Gska*=|o8ga}_V>jc1^=d}gVkHF?S!tqGVs!!O^i-@-b z{gDo&a=v7Rc&I5w8P8=l@dyNX_Q3fc1Sfi_Q3DurvI+3!O(nXMt&pS(`om*~gNf6M zGMA@k+g;f(0-xTn1_^}Q3|G1(A)ygDT@meMp_PVDNlEI`-6~!dj$5ZXqJ&~|GXin~ zxw9Q8;VI<8hBABvU4HJPB_Shd17;rzTf�TNbU|NMFrNUl(}}KV zD|SD00FA<@g#jKhr!);J=xktc+^}1C&!baa$JZaUvaumBwGe0-+Y{K?1P5CS-YrZNMDhM z_siX9%F)N97rD%&9y(;wVd^#MS$YoH)?nY^>}X)7kNL`P(wFGVa5+V16*>9I;!92X zA{{a5X*z1sGtzvOjzSpvT&A7=8pMS}C+sPARBMQ*EgW`DRjN#rz9O@}MqfASd3wR5 zU(idYDpNyDHB=2V`5azu(ofZJlYUFTGgP^$MyQdds!)}N8fB`JRKTQP({D^QTADFv z=v6h=R41z;Oyl#`jK!?>a7^dr*(4Q*!Q2S@0J*q38q=AsL7@}MNKi(sivw@ zQ%zn10jd!0=#RV<2VTuh{{-8lr|2M4%{pW+T{x^;k)V}KCX&HOw;Xtb4b`iH(RdKv zIMUr}C4)|U2&OxP5kpNg)pRu@LtCq4Q8F3cDT%$w0UmAg$@9|jO!BdpJA#qWnzt*W2m{Nnx|?r>-Rq-AUjB;_*I3V>FftClcL zE;cX|!Nt9qs!^>pRg;Ko74E>yY5+!6n(90uQgl-pB92dR(g2gkiRG*jn$2o0(g9Pg zQ|Fr;RO?Nis4g&6i>WrKjfUD}steUcOf!nHSunnvJeTL0>S8W6)n+iw(;3)0tr(50 zN6)y+986wRVUuU6dQ)vtt%icToVQft8paf}f^AZq5qx(#^{U|ZXu2bq3BW)_Fd7M7 zer2#0jj1g5nyO8;o2o-aO?8Rd$~3p3J)Y1uBFDm=IZBxjT7)EQH>NsG6&K@7h!$*j zmKRQ?e*#V#qH|#)l1(P82&Q+*^h;IJPzak-1`DT9e1VDAhvTgl{8S`s146pn$~qi8 zZm~ot`RfU{*m7ERn`)bgVY{$AKAVX{W|&PxR-zm*ysB#XL&w$B}-D-PbAH!}hS69gF zE2X*0hW%7JJ3rds4 z3KD@1GX7tHw@@VBrEsJ=t3gNmSoAWhu*kJ=C6pARi&LUKKP_(ysw@~z18cRHUpccM zB3Es>1YyBcSBvFeW3zLusjgGkLvuQyz*kf^plQ}s)vFgq z9kms%?MNoJ+r_x*6M#?tVGX8HY=)EPplnubMAj!dH37-~beYzW+5++#y8~gu7me5n zg`9z%f?A#fLBi^94ac*IVP(P63u=mmU!c5MfX9qm$=bV9g*2BmG}o_cYFb*qb}3v# zp>jmwso>I!Q$;uiH`cGJVq>mOik=e&%gP~^v3a01SFOtw@FZ)-E=r>*M+Q=#Erm$L z@Kz?I5;hBR8%(^oZveNYC~ClfoJv0q|h||9kj^ug*2MMp1wN1UUUi4_2F1-3qA%wC8)$Yw`U%YiZkDden=CR>$;SDSu2pG zI7=6s1S3trM(vUoB2^&;4&dG?V~Y-r&Tgj$>+O!qy_AU}5nYuHt;?8?N74Mn|8*6sJJ*NKjYXI6~l>WXHj&o&IBE4(v?x~=HZ z5m+2Yyu~}y&L>Sk)6*4BN|krHBai_D3dv-UAIuMOAUXRukZ=;B0`=;usm=J>R16DQ zIjol2xPyrbsV?W80Gsi(x)a}^XA3^wmy#nDon07Oh=`^eCEgk}fN&^V*3cwz*wshm?5BQL8g_GLNRTUteuRI-b>|Z(9si6*m znO=~@xkV&6ahBXVy?cTB&)Be%IYgh!}^MPE3Qfkwfc(EtG=T8it8Ax^CEo%?Kkmsj676=YfI=5 zacJ5>QlaSw$rYM$kldjo%L_xZcX3#}+I$ccj zXfrLKFg4K@YNl3$~q`)AIYi z7O7y9q3=XsKFt0Bz2u1DGK`k-6GPL2Jyf}yJnjRbr^r98hen~}q}^14H-{+TI)G6w zlKoU_x(V#1i%Ka8#ig;AZJOVp=3`=(--+}?`jO^$BK;T~Df$U0xJi|cMDI`OXAU3d zI((GnEk61&3{`qbuO@MQ-V$ZmCCai(lsQWUV1DmGT+gOZncRc>|ag(M}R6)w)4>YuQ16@6cF-eg#JHdB9<5Bc5cN zW1VdVcWZ_Wa5!N%8See$&eNbR&d2gLJT|-jSgijXsEVFEq((19AxyqdI18QSEL7qw zH0JREVmy>bwIoZm#G%Uc3jN+8^&oa6s9gpoC-%^!m7(d7*dCg6geJEfp;KU2Q_4^6 zp{nWlUp^I055Zce9mUeJvFW%fncGV<6dXwgpQ9mT!uKK{_Av|}VP@eQ%v`vFrT7dZ zHf3XUS(dkD^eX)UJo#XOB4-<;TF$dX&MZZF^uleZDUiu<(0CbZyCy)}b3C}Kkf$D+ z8J=L39H#0y2A)U4bVkxF2)<-aNu{UKz>7-#wzRTTFyaeMI7G8CQjM=HJxr(f(itw= zPs3}=o}`-cIf1fXn(Kn+n+G}6)|Q0=Wf)&)U*_AFvmlNIGVpAs+981NH=QMS zNCKas#kE5NLwl&chnDQ7NzU!kz|dY=ru16FWUb|aVS26oeMbXjk7K9HX+8YZnQ+Ko zf!xl4p2Y4iqETpHB0qFF4pu!A7?}?Xu7|lThve1+jT<1ZO~7Rb^n58ay#un{1?k=h z8Qw~Np@-?Op!#>veGQcV0gv!cQ2RG1y{>g59%U1P!Jjd<3~%M`uh8PdG>!g-c|%3^ zSjkWFDMjf&ShI}&i`gc0 zZ1=o@9zp&BDtVWtx!nuh@6s5z`#gjGSN1N240_X`V|_H6{06<{*8l82gWe`K=$*Hy z>@7&}UC>bU9_aRgJ&hguA8;vtc%A6HmrzQRn`ynYLS@zL4gt{T?xo6rI|TH=pEUN; zN=57U_x&m0t`)FhY76_&vCX z=CGIQ*r&xVPG~GdEFzI`ur4d&b&lampY6ai0<*`zf$<`+(d5!@>CMr7P?8JMf$6v* z60EF+fz?GAE=L%5Kp3(Tj9mk~Dbz!&_EN3LbNW*0fA}ikiWa zJJj+Nm2GfuMjmpA)(&vQT%AXJOje&`*w3XP%JN;%AwE~0LELZSX&q~4YR5*uEX#9% zp1FCu^=EgB4~vDVAP3E*KsJ@LL1kezZY?{N^Zdw{qL5yLA=D%|%`X+=qmJK(GJT<_)meRsgrz>tl=T4Zm;=Ta#*0)Z>3 zoSP`ftLSuI4c~hnEOQNQ;k9J(I=X__)75+dU5_!h@w74bU4Z8RcMS987PVI5QNWPzH zqbb}DVRX;}j?zlL1Y~2>%AHz#XMoch3h99Dr72WPukaZ3f!D04BHH8y&yfKhOM3o8 zZvbgx{6oZJ4TC#>n^t9AHf&nSuLf*M@P0_56k{bPhTlc~VxT1M%1WZW))VmbP=|KQ zh6o{Ai~aY|C5LHituFu%f-EQ2L!AQ0!xY!=dMNP>b=CSqnfC$TA-c4pf-ST;RY>+0D?RLGFb&9oXC?fE==d} zWIhEpI|a0+@TpoG2eR5Ykk!V4LE4aa8Eu?JZd)6Dbe#^aKq}CB%-|{n5`(8290Fda z5%uBlgg1yUdH_*><>9lv<4Ocr0^?47C_6&kEyypo*;?8Tm_ju>dI<;Dy%dl%=rXwI z%Ma5P&WkJEBuczMJDj~V3=nVyyvX^k+D~I;{4QsFFJ0}DP%io4H6nfcm=qE}#y63N zZ-%S5g(~?rn#3Q%pXYppmhqj`%pao&-$gOLTVs2HHu=u1Q-)QG;Rmu^PJ2}^hUL9^Uxe{f5!Av10bXw$z9GR)!rFntJi z-~+TD#39;+lk$G5%%Iqv?-m~3-?uf7Pk8*jn4u%W?dx*@x4rQ_0Qf$t=6jLO+y`KP z7QnurcJTvr6F*2F=7;D$ewZHMM*#8z)XR_2v;4WN$ag_0XCfr}0Odv6u^52zE}iy> zV|gyCkms^cer^zytt@v?7`pEvimviXThjf05Ch|epeh-}q z@y_MLdfSq4OoSE5K~GRM&)K3Ns8mv-q2T!~sseBLO-zTbjcw%jcEGARLU-fjIIl&{-k&%`pRDy9q1`PrLV#Qk?GZeG>JWXp)_a6* zX&LVhcp2k9BhlS9S%d_b?DL4~FMzEVvlx*YPNo7=qjfK4S)&Cg`l{gte726nlj$&0 z91mnS1fgjW%rcWM+oKj^l${ROV+Z(=oGo2WVUGB!M?=U&_BPFUo<@j<@9Cv|ibCV1 zo-`vb^I z4smVv!C6zm)AapRjeAfB3n=`YEHfX>&J**WF&>p5#6+7Ruxm%|6-Ysde?VTw;gEkw zBl*V^;GfWF{uz>^UjTQ%1Z;l=ulH-DKEHw2`z(CzDGsmG1uFV%u*Wv}-H z`1PWIcoB~0HfZfA{|2Q0CY<0g#Fw{d3BOI}hre#jc(Vy$ eRsNNfKXI6z`~ua8{c~PDuNtS273foZ?*9QO$Hj60 diff --git a/out/production/clients/org/apache/kafka/clients/RequestCompletionHandler.class b/out/production/clients/org/apache/kafka/clients/RequestCompletionHandler.class deleted file mode 100644 index ce0fd8af36d252cf2c77ca3d364fd0f22a196103..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 222 zcmZ{fu?oU47=$miw(8^yxGRW3aC0n(gCJCJkMXy*jY(}%AI`xC@S((1T)W)e@_lgc z&+`Rfg<*_1#(-gDbYgCmB1a7K>CV&{7nP7#;-$Ehf=iW?c9tifgc5PcgvapJmh8z9h9C?5slw8TUzr$kjsR3uQ;9$X?toHoWy-TI?)>`MGA zNF|Cupq}|rh*>8hso;{ewCmk@GjHb2{QCXvCxC6dC_u)BiYIugU|GR46&fmHGxAte z$!G{fH4j~!n+moVWUIH|W~g<-XkhX)ZXY}5DL*>prtP^-5D!f|42FsCL}okmeK$V* zc;+-1vb$qPWuv>>dADaVnD6I}kvr#J;tUz;6=QB^xgCCR81#-CIQxm;ccMey_vl4Q zqaGYFysC6g_&GN{9t=z?j@)3-Fy>9N)Cq0wb$R59@DmO>;6726{*BL&Yr26O?=Y1A zL;uF;lIHQTOH~&vcM$M6i71j(5r%w|L?1i1WM18Yj%oKCbKXL4s z(t9~8Put_eVPgjNw_VqxHd=KWU5(~lYOM%zfKK8EmR3bcWRX!o2{%B@bbCanSn3Gu z3o=)bKGBWg7VWvzrW3x5+Y|gYGU=;YlB-{ly@D+8nQQnv^rDSXSix#)Cp8Eq80(Pe z1C;9K+9h(IQ}Zm{)Lp?nkvi_Dn04gw0PB-P-cliA)xJaSl@#d``3n@%iz=s=h3pS9 jNh6x1o;<>Wbpa;W#>J_W4yi)v>GU^dct~=HVjlkiG$!`S diff --git a/out/production/clients/org/apache/kafka/clients/consumer/Consumer.class b/out/production/clients/org/apache/kafka/clients/consumer/Consumer.class deleted file mode 100644 index b292fcd4307e32b88754b2eff60b558cac9cc2af..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2045 zcmc&#+iuf95S?w)q)iBExrF<@fYX#3B*X)bB1A+b(u5)sMMy|U8+RMFcD$A^Ab86^ z@x%ikz(*lw*KTlbk_$*YM4OqNIWuQ>=lI9ZZ{GpnF+9w{gBp5LoN*fkT7PV+4%p{aQbS!J~ zzO$dG#X^_Ber@k*VpRA0N6ZY2-W*?jHZ1aRp*JKp?b{41=Q|?wOt#BytX4c0;2lYi zz}BqpFOqBm>%WgA>I=VK%9X5`B|u;af6MskD`yB~U oVBSr*HMVb~eJ8QMnHj;|81G(;cONu`SCK1nix_S}Rax7=0A`7DWB>pF diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerCommitCallback.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerCommitCallback.class deleted file mode 100644 index 3f32f950207f5c61f0fb8d469d68392924d4fe13..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 342 zcmZ{gy-ou$5QN7eMR_&~9)O2Jm`j8B z>1K*=W_I5{Uf%#*zii`^qKvQR#)T zx$wMZzwMoldAT#GCTmxuy4lxuft%m$)@Yx$dBp*ffAa!o!}Y_t537wVUkrbsFP0&2WSHT diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerConfig.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerConfig.class deleted file mode 100644 index 375d64ce9fa9c8e98524ca4213684dccae0183ff..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12655 zcmc&*37i|%dH>!X*6dowwiesifIY+p*2ijna@dPO*2=c*JiWAEZYaeoUhe z#pus_sfW(+Uk~-tZ8BvrN5?+#ONso&S>;EJ@hm^ z(?fqte;1=?HTruV|ENYEyCuU)1PJ8hx2(zS2uyrT@_AYZ|?z(SK_6 zUmCrv(SLW+G9r!sN29NIQU^c&uSVbKBzg-TU(x8B8ojE~|7r9sjlSIt?eKrwdj$!h zi4IM4hGQj7bZMen6N@wv(}bpp9!>OOYq2;c;+j~ZiHkI`R1?cIv0M`gP4sEvVoj{j z#7a%9>Ll%kJ2Y`gOsp2PC_O&5KYKvXrV+=j3>tN#IAso=G|DH9!J<>EIW~UmvQ-&O z%b#JhyhqT|0~6y%k4$EVC&T55kvol3#$eU3D}y=Twd@MemSBzN8@7MUs5VSN@#eDO z@wA}TxlAsX9Uq&_XGb&RNAr`TxhB1p^UTRbm+TuK&*$}>OQjD(by1O4O=R*D zC+3@32cagi>D**4HF^XNb38jXJbpa7c1?&ec62nrYAJWEnuLvxj8DK}CetIZeMI~H z;3{`?6f+Zw`7N*+lY^?GD@Xk(=%Pm zs9J9_-Mpi6ilBS?Rr+Z7R&{XHsP8#%G7484zJ=5jrFKDUHlDMupsuu2LaJIiV%g?c zqgDW$Mxlz4#E4Tgs>cj)4>%Z%spRPK0+|4t5@6({ITK}2(9R1HOVILDoIcx0_1fmN z(jN9++0NQbJ!EfJJ5xgnj3(*G$avvSv*_>Hi^P5H1u#0aiB~Sk`9|?1Oehr;6Ke$Z z}Bx3it0HJu;MV~Avb%0h5# zpJ&&qW!k>SosHM1nJ)J>@l+PCtF^JSN$QNDVDp9yGiuuo$Oz(*uaR zxVV(wBWUZ~uI{yk7qkp{D_M%<>6o}IE-n{q<6@oAp=_Cs+AsuN++rl+L*k0Kc!O9U zr!rN*nAi{#SH{Iv;_A5Q7wcnUV_a+!n+5gH<#PVs6A~93$M<~Ks3%biPMNM36I+f>=d+RF1bMHv?QCBkTSL>I!Y+5yHay{L5^oD2Z zWyjS`bg~8DO`*5d{V7vtTcMKaMaQ<0I8l!noo5bn`>YqM4bL~-q@KqRObeIOr>$yL zuNfyzNfsL#fCD*n%XLk!&ZJJ6%*^zZRh(+h8m_7HtW~zml0IbuWVV{>sDrPoY9)BV z9(MddB)dAJTQzj`K7@%d&4PN&M2VR$SzghBE+tC2A$gMe2v}noLXzH;)2Np80xLk_ zJ8n%MDxz%eWvbhDv;MYi;pgr0dHa@RGC72314dHMS$5Inh0tp#Im>n@w51qgv7)@T zD|jxifui-A$>kabs_XqqcQs>1FHYf1V(O(C+o)Nn;MJLpTR=V($2Oq?+ws|qKB8BN zG*<1On49BA~K1D1V|)*ahqdQPJsAT=!nRg+19vh*>2f_#K)5+pFsqJT zVQ&o##n}tR2d4LQ*SxD?xe(GSV;MX#qy*!8x@lodsasL58MaX|YwWD$hArJ1-5N+0 zW?&umePwE|QSgecRWSQ~r*0KD2KLnh+sP0ZC_on_xN_Mup|Tcbj$hxb%t)ym& z+$e5}ixg*{_c5zw7x7*~@WJb3Fth5LyXmdn*yvIh%Iu7$qC({g&ORzY;Twg9A0-06 z3f$emIlY4LY1-D3=k8*tlR2h^UvTWRQx2oH)J)$f8NQLM8FwdPeyRTq>wkY-92D8O zIK+}Y%96pcEQgIM&A}dDLsGLm6PuSj-6$jH1*vOVrYJ$J&7Cl9 z?%k6{1CheF8{gC#9x6+2}B4NCJ2>@F!Ijth%bnNI}To&NT03 z=vAk}7SN4)y^04}fMv9)T~KDKjgqp8;&s47QVnloxH63BrKwbo!94zbw1I=Yh4%v~Y$aPH|d2FSMkbv*_U>J4EaXFiUIj(ga z9dH7~Z|GmjFzcJdJ$(~&d>$r4+o>Au;bg7XxR$b}eI4P0X9N8JdQXhN2rypAPz`w( z@vh5N5vxiy%K$Q~FrkHd?0Sm^`$r=|LDEYaG~UJ1#hYiwq4GA zB@2ym8J(E2DP{8PzN{zdnx>vlA5r`(JiR|RKA9UE#>Lb|-jZ{$EGA5{X1v5K-al`ah{<6L1?RfvLKn`cfNa0F$1*saz;cPdvwM@F&Os5PpnWbiwn zz|(AR8l{4*)}o`rp=6egMwPR@?=u3-unqkM*49icQ*P15ZjkgONkk32*mKzC7>6b8 z3Y=1|4@0v}j-VxJBPfd(9K3;`gH+4~3)U3Jb=k0bF5plmF!pi>)k~fLDlipSI_cuu zROA*7S*}}Vz>22Mq{AZEWe(wr(%P)0JJq(b@1tpV(UOM-^`}r!DpPgvYv=`-0R~{b z7SUxhEoMx|{hDlMkYm^qs45%*fe#^lFfX$7JRrSjPFCaoN>UATJD*kxRR;k$uTk4I ziqZp)V0p`vrTiS%f-bCD%`92q3`N^R!uHBDayxc1w5YP_tU&KJH=6<5q3c5ujyHAI zN^*%sO@=TrF~+rfT+p*DR4YX*VFCk$?4wNA>eAX^r%8^3+|bp|d~9Jal^*0+=xAKC z&Zg#OF4$f=H7{XgF5Cv(sWOFR>p1mbaloOfPTa%;AKDD!iAyjl3B*)J44Qi&F|7I* zNW2UI?9N1_74(kpUP_PvhukG|!Qtu_vJn+rw(*hNIj;`Xa>zem*rfp`$+|dK18_Ro z^{B2XiN??L@v4b^(!+yh8ZM5*{McpV_$0*B$GH$d`V#EL4F$9y9SJ8hP=`~)rGp*} zo2ga@;i_CtFE(&;mW5d!G)fNi%||-(Za&s?S16Bbd_Zr-X1|%NB;nU#(1HOO{ZmLP z1#Ux2W)t;+h2mr=_BqWCd|3B#Tw_j&X5>553v*-p_>G56Vv2Hmcs> zm@{5a2WQrakZw94D`-DX&q!AToGsDbmC!h-9oe1CyUik>nWRrO+X9r0@T`YJw7ecb zto7o>C-dnR(VKrdh5ou_U^yk72QOfDY==Q4zzM`fh6uPW1)+~-Mjl6@e#ahw%MDB; zyYXoRAPy@hvA}g3ZDwnBP7aCzyg8($c1B|7ipM8O+|o>;N!S9A+cLh8K%)n$T(ad6 zgfAdCznHQoH}Cd2vvRYF0*8Yzi~vd!ll)!dGgP1tH9PpBI; z$jWC&#erVr^3L}HNQ0(+sCIX_6F0&}b=sK0fhIg*Kq0PKL9RhRaA@=v^sHRmkRABY znTx)-Q9Q`t!!0YE2%MnOD*;t&uEdKjnj;*j9*rVIsLH$9{|DVw@a5n()04LWRtfIG z$JLwDRX)uz+^mBL%izGziM8c4Z8rj~OG{bqpg-(^FA5;Vz}!U8cJG@i@a;&^NBm;} z-Y&InnQ_&Pk^(18RjX|mH0m`L-?W*2nv)KEvp;x+Aa8KwAny1(u~^V`!Ck(b3SNOl zF7Jbp==xv?FZf!wLcvBc>^X)DAM`41Tju1&9`+pSpXY5vo5)I3{>-BN)q{Mc5xs|+ zfxFcp1zK1^D-^Qj3?4zt?PEXFsA0dwzH`v8RfD&ceUjmJBtfw{FR9v~64gb(Grx#< zJu@|V-jK(;M^tt76VJkxT&WryrKxKqo68FtT(AZtH3+feb-bjtJ#q!o2In~B^C-}D zFni#jpj`{!yR{k|EW(KesZY>@^Jr+{G+8a@e+gDR=R2DX5f&mI8QNvR+cN+7ASFA7 zuY8{T9`N7t`Y>EBz&?R@N3K-yns1?WAW9gT^baStaqGkgakz4HCF zT9cFSi|=-gjI-w^7i_Z$b!Xol{KgD_gzvxv4M)|ezU}xfoNj}p$F4UrejILw@6bjv zsj)UE+~+vBZ-ZZ~?7ubYdcimrU)1v05!~5xZ8{Xi$@OxK0t)?lm@TTcbJa^Eg7(W?-*J8dJfpyH+BJg_5+YxvLU?&1^2Yx*Q@520D z5%?O+yAk+W%zF`d1Lpk*yb1G-2z&(dry_73^V1PHjrqGH@G$0QB5(%tZ;HVCG5>=R zcnjbkiom;p|HBb@2=i}_z&8N?kqCSf;I~BJ>j3{~1U>-x$0G0u;2)2`w*mf%2z)!> zpNzn71pHGG_$c`M=?Hua=I@EXCoumr5%^Zj|7-;A2K;jo_zu87AAvQ%Z;ikP;9rQq z1?t8ZxW7oh1Yi1Pc+^J;pTgpI$0H<8oS}}fflUulX9sP5l9ml5x;LMuMGsM|la8wa zZA)`t-~(9JMZbc-*HR4cLiQ7M15j2>0%3Q5CgPLBR9r%uRGyNuJJL$JD+QNSuqf7kzoA~!P@!tmg-^P3ob-YS?jNZPCeg|L& z{jQYr_X0T|YsuLIIeQ`JX<7`SmIqRvrub=E!eU}XiW!W|k#i^H+(k>oHMCq@OY6jS zv|d~fId`|o`TO(^X8SyHz7w-b&Xr(rKIxh=t&7ngEDvOQ7f|0#?+Ikefdnt^*)&TR zZ9YRw{}e!{v`8;ch`r$SM#+`hrH9^2?*k5N>kpx6AvGPBTIG>H!jn?o`+@Pt^e2GX zN%nvmTWZNHEyGr)Y58eN@May6&Gu1Tq~&Ic;1c=(F^)#F^nfDN@d_YbmUx8vV1}MQMJN88rNLR+n%Kq@+h=J<*IC-hn7f$LHNafncTYR& zwTyaQz{S?Y^^CY1TqO2H@m3{<81Dw2*c+YLn7ENAZsLhl`@}w;NV7m)_%y!7-m1hf z&t!N;yd0g`n%K`X2f*jWbnq-4>cO8tq{BQjB4& diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.class deleted file mode 100644 index 18b4025af95f31fd61833327944c86b3fa459eb3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 444 zcmb7>F;BxV5QXnbX=q_Uf|(gQ;Ds$!B~-yw3DhF?i#dWD$F6LrKaPPPz>h*)B0!bs z0NcGg>(hJB?;o#k0B{9I8H_SGL^$>Co)Jc@ z;MYd~q{*k9gOlxm3*Pvu%{N;5CTPAjR*ao}2U&1ImYn7pSzC~LfcwOkIm%kUH3)`@5U diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecord.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecord.class deleted file mode 100644 index 5afdf7d2595c8645cb81913f7783c91126134957..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2181 zcmb7E?Q+{x5Ixr)vg|r-QPYOdw5Ul*Y{!Y4QfNusv}tI-iPJ)C4B=Z*6jzNZc@()r z-++gK8JMY^nRW(#GQ+#@3c#+UIR1i3W<2+v-MjbP-90P+`S+{e0Nlb(3YT#|ffX4K zQixzRiMQ|}FAsTHK2D&NLJId~tnA`(RwvN7I~Diqia@8b4|8GXUuzI9thklJ8r$8H8lN+QP|h^ z_O*g;nTG8h6m-WvXg)LC!fFV%4Bc^eDd9`KgfALLq>HC!D^fqIwWVcZ9jSPlzQ3t8 z{0x0XzVt=8GPEthMJ{|alkMtAN3LilOCt3ib~K4+vVjt`vj!AQBFdfOQ+S{jp>N@E&R0%OO_K}`>urnPIhbkdtn z#-DlhuA0U-_?8*p7dV?&JJaqTUBzua|4!gcUJVA{_p4Z9|M%3qyh`6d$fHpU@~S`P z6PeQB zbcL<-kJtb2xaYNShWLx|ZwF!CCaZ6$h$*zdX77UH>y7Np#@781GfcnflFTxuctE8U1; zmey$`X`LeLI^^)RG0aeI8OkP0D2F+U%v01dBwFXmo~NA=S0H@EHAH;GRYZNnb;Nzd z6-X!$R>QM7+#riNCh66qs2Pe{AnzE&KUBDsz|92S6|n?v(T@&-ro2aw(hSqeEm)zQ zlZfYDBK`}3s9%}nI~V-S(?LL(Ip=Zg;XDLsaGTbCI_alAx;rPNo5Y<^pziGC{_O6Z zh%PAl1IllDC~dNzcI_0ji$qIbc&GRXp`e$EpZkj6JP|GZv38{Sk+fjC@@u5F4{=6G zw=t4y<1B5;xd5MU;|)a#@P#%;m5YpDLT=-dG8W(@<2MPnG2UP7R##T#0fpJa3P-}x Lxl2>b*1dlLq!y{v diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable$1.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable$1.class deleted file mode 100644 index d2b830a226396cfb120fad0109dd97947626f304..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2079 zcmb_dTT|0O6#kZ80&S2ASQWfe5sH$cC@K|15Jd_)!tfxD<2GGNAWfJgb@Zp`s}H{F zjCgx+c=Ev?<#;wp0j1+ig);2r?4I*|XZJgspTE9+2QZ9#EojBX1P62xZ;JPkcdMt7|hGz#?vAgtFyBktqaASWkp{H<_yFv93r6*MBmS0K= zH_a*ABJB({d@MLaAAm#j`H0XzsRs;)Y1D*vYDq}j^UhRCiY?sPg>G6oL9-I=ZKA#_ zJFp3s{fwq;_9NIAM7|=*u!-0Q-{&N?34iA@eU_f$6wT9tCr%R!`Rno#7|zmIrTEek ztTCxN+AR^^QZLsEgcJXPHraiJ!te$fjbr}!HM%7anlv+Pp(P{}e}`oD6B1iUz7PET z<>Y&c7lq`QMvU6$lshNvB1R~~ajf`!15SmI(+8x96PQTyq C!ZG&% diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords$ConcatenatedIterable.class deleted file mode 100644 index 8f63aaeb171bc237d554c1a618bf80150c0e6001..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1709 zcmcIk>yFYu7(G)i1!Q*-k@fDXs9cO?6MqTe4{_rL;$;^x#>8YOBU`rHZrWn>9efx~ zG$xw(06vuQo3>n5>klGA+L`(0n{(!zeltIQe*F&M9`2-(z_kpLxUS+x1~HUW+){B{ zMVX<{wk@4|!lGxm>j{SsOu>-dwJhPZP42qFWk?(04D}1$F@M7~lUpNgx}ZTR`^Jbs zCJwD0MWi%qQNgb5j5L1C^#h?D^4Eu4(@jHIo~!A$4EuI2lnlWql+Cwo>h5-7fy(KSDw!-GB}mRIx1Am%O0|5gG$>Hud1TPu(KLShTQxz z2~1_k&qGtY7-D5va-G$EY(m>!S_HP}E=__#r6(^frZ!90tb#SVSQ+`VLH!&>2GlCl zW~yJHRBNBXKGGc+&z_ho&gRx;%$w9Zhw~xt z1AiYt`Ws>|h&%p)mISnNOwAsi?~F&Qj$ikL`YOWfk{}! kR};WKU4TvCvacc*!d$^sKYNSRg7Oq{@g`EFQ2Ic>U$GtCH~;_u diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerRecords.class deleted file mode 100644 index 174b0e512758573fe8319fa56150cf88a3fd32f5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4221 zcmcgwYjYE26n@?`?KbIBXep%#)QW(;LFA^=#3B}|wP`7|RtgGi(AsxT2;ADh8Peu7~TEQC%-V|u) z$(kw6)=hpU`fT0O#xuHr+Gm=&)sxY3IXx#(xpQD>WK3XFGHaz{+LV?$qQ{PDhmUEo zRL0OvI~Pl3&0Ky`w_-Ht-0aa)S!*H}7l;fSY1+?QG(2|i781#$+6gU|(adyg*Z5IA zWyj}Ay6H!;T$ch1U4^`DWMTu_6xYs_OA-NLWG5%HW^5!oWu%5Q%QkEyYkHm>LrEiN z2kpB%0Evy9b*qMk{dkjsSyF{~sWeuynTuWD@AR&Bu%a zn zI<8D*^QJ9O(c0E0Aox*?+|J1H)Ei4rlXLO7At~A9}D|lPQ zJ9t;1ww%-o-c#{DJ`h+{8n=C!jGopq+pRQ5yJ_z{nbN1^SQJ?5%*mF?e9mqe*IUf2 zxz5aIGV7(M4+R#NjBmFr?Ubxb6=(2~f{#_ah&~p9inG|I;uD-J(=Rp+73byB+PUNi z>H;dB#{~tSs`v~SReX+fD!#x)f!?|M?_NN;)j{t>GO;gJe1)$Se52x9BvlN`_}5^q zz=nHvqT)NWOQY*JU`mL)G-jJDi3%(!2+Qi1v!etxp<{KWb=KWle}D06!{5O@k-(DD z_MR+{J0;JTLoec#r#v_LuHtVU-UwXc_fVJ-ERJetAPmbO>dN*}Bu8DzpQEE3(Q>hoLLu7X5f$%P^O+PzzwN1y$4LsFumIY$~*g z39{OsQvrA9&8+H#)T6DYLU_(Glptej4W-mda^{sYPSZogd51nYuziR>1)BL`r6Gi6 zl0dwFg1-|&XhprO(&BRms(DY6aBV#`#(53#8f~A3Xz%zHp`Z9vff&DQc?BNlOaE&A zj#z^y`PArYV*`E2e^Jl+#X<=A2!A0w(0Luo zZ>YS|d4+3XCyIJr!FX#KZ#^ZA^erQ4arYM^X~I)@nm%e7(=*sg`=ywgs3l`^*SfeC z=M$0m2aO669#+pf5q7%~3WfG7sg>@yjs>%*8mm{PQT^vEB4g8t4#>D?P$Mt{RbX)4 z3>FAn!Y1C-3g)t~E4(Z`gGKVb?i%Vlu3_;d)OAW>gWR_;~aV4DL#yQ8s+Pum@c(nToJY+XG}rV+b+P)NYGk|Nu% zqo^-IakX}Ope#bT_LhT+g5E7RDRqhT0`0NQy5L}lwgmS!$nae#ngp1}H7t{vxCE+l zbapt=s;<5lqdc0DL`LCyd5K#6Nb*~f^9Op9hrh8RQS>o`M+62sJ-CQ zp^iN1y~*+;2e8YFyO&pqPSlNhg@BC5kIAGDljs~wq(pj&5a-<<;wv1B%<3vuN`!?| z94^9^hu$7CXBZ<~ljyB=A|~l)(L8n)k4~fIXRMUv?cdu>Ef@QjG0HWvk~o2-79um6 PS^iP}x4=F;R~7jWv;9(t diff --git a/out/production/clients/org/apache/kafka/clients/consumer/ConsumerWakeupException.class b/out/production/clients/org/apache/kafka/clients/consumer/ConsumerWakeupException.class deleted file mode 100644 index 97941d8f3a5bf1422ddeac0a6850bab50b006a8b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 448 zcmbVJyH3O~5S+~=7ZGwCcR+(^Xb@1~h6Wv?IY0s;8X(lZU^tV+j`BeK78Mc&AHYW; zHW8vhG`JY=jJ!Le^~dMiJAi$x1?XYyW5vfRVYyPJmRYRIN?UVtc1)mi!eDAmRZCOH zGHaB_jqM>q|4xwXm zZ22H$A(Ochsk~36NHVQVU5SKkn_QKm^>2$?nX0BZc}`SObHE89xUx-|s8ikAZ2a+V z?@>O<5DN$h2mf{>Z2j)Fd2WrkXk-7x!X_NvDsKquaN+UYXXe1?irN^Ai>!!_1K#c{ b=%srSGWJ;pMi?-LUF~5BLuUL#j1ER$D-dyd diff --git a/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$1.class b/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$1.class deleted file mode 100644 index d8bd6f8c7651502b701514de11c57da8d042d8f9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1403 zcmb7ET~8B16g|@|UAiu`prQy0YSmJZZAC;=H6c|K5%B>e@p0IW?b7Wm*^d%_mPXL{ z;1BRe8SgCHw0&y3$WZ z0TI@Zd56;m-b&l`0!KDJim>U`b*!0qjCm7Nm^N_*rimQZbyz0KsF-+y4U^7qQ^zwC zTX;_Tk12nl!jM<&mdj zkt?>N(5rdvwi8}LX+?FoYr_GPAX1-UVtfC~UX5XWg!AmhsVcN%YLal2D(nkYNgCX? zML;*8On0C&3c!E})s+#l?Frt!J9{EL_54;XzF@S4?cq`l0b!Wzo(99KQTEXT>??yR zl|7c}J;{Q!K`NhD#S2hRl}v9=ivAq(G&@7qXJ|+(PigfWth9WNROu(QZ{*T2OKpZ0 zLYu->vgeZEYhy$a6Wza1-+h#W7)MG`Cp#0rnl~3rr~M7c!+KGE)2k zdXH~ft<`ccv^*BHnLggd1e=g>E9UF*cIPxK5L1d)+@?NFPv{b^lVwmYlho7qAJ{r~ AX#fBK diff --git a/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$2.class b/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer$2.class deleted file mode 100644 index aa1f4153808f395d229942cab4068c4071df4c6b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1802 zcmbtU>rN9v6#k~IU0N0?7ZFsvRxJftZpI%)qG>Q8)=L2SXX11_l%d<1y1T{r3cigA z#OMznz=txPX$n}o&=Q+;=IqRz^L^hrXXe-MA3p)C;Bg8`+%rQq4GZ@N@c<9akh3sj zA#Y)h!9L(#*5#g8=5CGQX-WH4hc~#pC!8AJt#QZoq)=htxLO6xy6~N1+t?Ik?s4Uc zV%xpIkPP=^kX>L{>?6=i6aP4xabA^5hHDHTa~JmR!{HdIoA=BPL!ziFf?>EMm3ZB( zmxaI0%O0(amNeyOhx^iew-*yeCuMI#Dd87A9t46um-}GRAv(K6vL`wLJ~xg*#za%V}GmN6bHIy#HP1&(|5%S*(yax^5%^?s@S@6 zJsn6@eI>#@U9m84!@+`$LD)7@NZS~~qJ?cl-{f3|Bt+yb%%UOF!4@oLdsn zAOF4$eJZ5}iswwx3%$jiNr++nB7zE$P2ajTPkOV66o%XMxLRNsKrE9ntp?CceMb*) zoc=?&OuNHmy+f?EGMt}10?W@GA(sD!_}A9K42?-Th{Su0kUc?%L2jcMr`cF0Bo&ry zGoS9!CC61dZ6I6%X!+$2Dfl7S>~|#MIE6@??ZgLMPVh5eCNb59>83D=YqWQrfMU2o s*7Ui+m%4!k0q_1da2z*V6mC)SNe%AcF2=|jC0a?E&2Ip+c$7^20k*K}M*si- diff --git a/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer.class b/out/production/clients/org/apache/kafka/clients/consumer/KafkaConsumer.class deleted file mode 100644 index 2535a7a1d77573dd9788290b0b75332d98662ccd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 25520 zcmd6P34B!5+4pnKo#aj?7l8o+VI39`va*Tc%KAXe>a)$Uqtt*u?GwfpW;7vBGK&YhV%8w9?-Z$E!>mvhfK&w8Hi zoXOMgJ@Ob4EikqfktLq+;qnzzeAP#W_!?in&Mn{I%QyM;Ngur_zQtXiGR4!Tc&6a6 z^Pi83#kcvxcTDkJAB_~>F7>T-Vj0bhRT70(un=fsb^;(1fNz_b3?6hHCN zBt7wqUhz|Ie2F{%%qxD*FE9J(Y4Hng`K2j-<)f3tx4h!l{N)v&_>Fj#TYl>muN8^k ziQn_fAH3p^{P|B_@n;`hE&gJPzxrqa5BnS6|ITy$gYo<)zy6CauY1KCrg)Qwy=97j z`)Ij%yI8y<-ZjO0K3XLXV~ixP6nvMaG`!N|qtj#ozZZI?iILLllSR_!lf}~FZ^OLu z1XB+8(H4M~BZ}on={MylQ;zn^F+SQROT2QdSB~?_QXe(T@ur;Mqci11ubc$5Z4XUjp0l;oaG~1&gM^LURiF+3Vy6)RH}-pM^^LG9Iu?q z&GSq--zyh*z`RDN7$%H>|U!bca&mHfVnk*M{_ z)p8BLuJy{(ymFmS*2(n{sNBGxH!{)nUfIAen@qXcM~})aUb&Uu8@;lL@6D#%=9Sxh zbR8HhPnRt`-5JcA9X$MfjMkaVhWDFtr;l!j76Qlz_!7jcY-N($c&2uKviYflFP%Km zE*>5-M%-Q3*cm0_QZh}W5~y^QeLJUGhZV%#C_m3>e%c@DSi_Q^eRZ?QaA zo>wf-mlyEmLQ`Jkl@}L5B$sfX4|?UL{Bjw;e8?+5%=eEle=g^zkMi_a@a1Dn+LfmK zxGAsl%B!Jk@)JIKM!XHJlAq)O*RaBTiiy9L5x$N|xZW#o;CXKJ%A5FpGk>|oD{p1o zZ!_iXK6;6j>JHYpPkZH^UU`>S-tCq5;Dh{(SAN!%`+W4Oyq9O&Wy;Ta<$XT-9fP&x z=Y8@E@_xR25ldHoi7yYBa=%wT=%c@JpONw*zC6rd4)|oh95CgeDGv&=>cU|=TH76r z#q5|MqdU?m$iFcX?W~Sc3Agn{qjor69gIhMLT%M6)vYcZw>8Y-wn!w}9tz`gRM29DRb(QvRkR$ZGK3hak>*?U*pF*_Oxc8AWjG24P1vqgG(BH`*p zuV6fcH}@z^RMZ&i1U`LHkeUmjqfN22X~U9b%+7O|UGB+`cfM_EKC{D)gD)U#VGhFb zvuUj!28>QUo^ec0lg7Xk5A}d4GjdF$nZiPkRX3p*pcQJ-398M}KR>#xY%J&oJH9&- z?W$EhF`e0C$D^S(sPOC@^J(}_m+E@m6{Cj5`dVXc(NHf-8e^B|7@HrLM%UoRcrean zc>yHY9*hSCP0v3WFi@Q^;Dji!*}E#())ndKs0R+dU|&2^3yec?EU+^KjZQW2%7{X7 zxE~l8iP`NS!dXO3yP|dw@3V%m#5P94os4bN?r4kjg#l?q)IPhTC#k-oTsurw47 z#h1YzxTmZS_AZ&dO^~NH(hhMC-xvzp4ShYWcC;zj+Km=ikG5d&F8aZi%3wHf?bnq!h&f{AT)c&uDjRn{!Z-hg@4b+|M}+PducO3ijP;Fx@)BV-PD zhao^}>w?o(@c^m~s7#<_fE*V^0OXdYe8iN8R66}Aa#lgF96@SHre|;^2MhjJ7uA2j zc8r`g+a-cVoArE2W?O#QHbr_vZJUGe9vs0c2^BdpZ7V{tl36=k%IAoK9Z*4>2SNwc z9qVOOj*h0-DX0QdWG^4BSsBoIVj1tvz2I_=K(yPNKpsfU9x^CXfV$#7tO*srGp$9* znGl=yM!LHNojhxOu52K`PP8H0j6nhgNNA&2x@Oc?O-PLGFf14hR8J5=aTu%Co*uv!+cVtwPG3zoV`DMpNZgLXXYG&IeM3NlxgLSFcLr2?YTX$a! z=2_6xygE^aeH64y)v0wH3ef`P!!+EfXJH}xa=HObkgn4WeG8q*On%W^@arpGC{3W*= zKEh0b>Rc>6b{$o@kIEU@mYiMS!H*I!rHVK zuq(lmb7x<*FpWs`*~TV;s-#0EWi~3X zPNgZoMo5Pxw-Q+}3iPQ(p$;f@GEn(}c$^+SL%^eJI;yWy8rlml&ee>S(2 zgLIR0+~~7ih1&JSVB8Cl?}$sKw24;Lv@yd zyC$i_cef6sCBMF9gwL!3OWi)=*ajf*2&Y&*Lz#G4h~xT+6xB95@O z+YYj-`GRe2$f#$`nKMVwIYW^_TG8~F`L5-cnRUxOfbLz2N`bC7_b%N~qZLuDEvLk~ zEl1n3>~1-%Q$9U%g}%PVNME$gUK>&s>@*^)c$HgXxENuHfT$ESCp#a?2SKOnmi&tR zDi5B@my`LjfZM*tPxn}2f+!UI-$ZbIyinK=-V)RqzM`PFeo;`_L6f4`4C;?gWxAHYX zCmn5Qz23p&VZXD)NZ|+KY9FnN_dgKWvHZkXmfFIv?t3U=CR zctHN%5@SRO99C;25|5!A*jt6Y;d4-*wB#SeEvEdVCI2M>s2yc5Oa4Rt6BZyZs%ZxE2rB?3JKhzoZiIbrQ+wuP zflOG(tuaMis2xjzxf788f~VsGhw9+s90-n~+)>rq*MZ2Os&#Lij|Om8J9Lhn*38qq zE@)+5%oDsw+nCpCm_`&&^acQ{#i%25=wQr_Tk=i$iYXtpAZ^ z#23Vu7?F1@A%y|W#nFVS_O!DJ^RjW%OD_M>1WR~CfhFG+H(K&NdDt?DEwNvnGDw@< z5?)atXnr<`OedBKn4Cibh<4N|gQ}iTIH49Jofa&d-N8^?F`$%}QHlw=hoeS5RASn? zs@kG$tb+!;i%%3=hGajuTVNBQB;Bh7+OVswGMG^*%a)8bEpfN_Y|fRV`K9J^#lzFq z2I1k`1M!G*Tk&XLTRfobXwr=_R)!_|fso-5Ev8Xm8HI*vi9vCXpz)4QwdzS!`E*un zc1_^C3sC&52$+V~GKvfzJfTr+i5>Du3zNN1(3JFnN$$j)(dcUogEFLu)LD`x&Sbkh zxfzSbsdcsoI--#t(0~$da&#w)dwO=S9Dq6*(6khwI%Kw;!S0pOPGlWnclt2VISml# zLESG9j>H44b|8$-aTo&AIKeW8vwMt+m}QJGMp}m77-bowjWL!8u`K5vUCC;b-dDZu^6Ssc*~f;a+t_-PvWP^{4_=liF-UVPSKk3gBAfF;;Q}aie9dGHNYjwYY_Y z5UkoYjObdyo^Lbqa+YwKK*gQBjK*NAun*=i^W$B{FiU)fEye^+#d1f?;Ld5hY|9lB zRG0vNdE_J=CL-Ze15N;D8S9KX1RKeACkzPah!MvPl3(Cz0_m=1Lh_q(6&YVAH)VzS z0cNT`dj$NKv(8R-Syj$EGc4+Htk8M9LQ&e9l2Q-HN+AJ>+b(I(Y;x{4`qh) zj0HPvrCU9H-SJSR&b6U#I={Az%?PuNEqvK3_LxSaWi%PhY35#Mxq`}$z5sxu%8(Mv z#|bcFaqP3ER7u5|3y(gVUQ8KV-lJ3VqzasfCsNII*a*_+GN)w2(B9YC3xYk#?vX#( zaK&2L)fvv$+S}d7CYx|rPVb{%?k=8ig)>JfKqHnvj5@9{**j&|yHRl+-1%={p`y?=rV}SB6sJ*PYOhmgetD;GQI7tus*E~Xj5<+j zAv@(96rV8Tsz?N-sBj*zIQ?j%llBZpC?Tgx-^i@GQ6<@?&myC63L{d8DNxlwUYP?` zH)c-h19iBODRiAig#CzzQ}e>$nod>^`v$J&6|G~X)5-$?F)6i_+LAU$osMlr6$~e{ z>+3ddtZQ6@3Oiap8!Bw)s;g>4lv6icf>YZfc{Phs za1K=SqAr-VUhQnDC4-F%#E5zAu09*p%o89!KwuNI!DOg+h%4XDi62h;r#UQ1k?Xd0 z&bCcavOt2wGRJ^yG}0Ud>xnM8Bb?69ti;g*MWoToM=ugbZrycam%tK2$Zir8#R|!X zSGrhY`JIAG0xlSi_PgZB-iSrCD?fGVwHtLtEC+rWwJY~YWCv>9`Z{Ie&&~zP8LmSZ z^^%LWEFk&##b!lFMd5IWtY6)1p@F5yK$2_;M|V z{OBGrMpK$z6p4ngp^jhT=*lxrI;!8yjB=X?#!l^J7NSZP!7i1!bSSbtnt1B!>dwfq zcK%mlzZ`HJw$&$o;q_xKXJxzoW#Jy{fcawK6wrkLEcm5<*b9 zIPLtS;|*NMJ0>$bngF0&!J=zu!nH8B;Qdu~1~eVR$qkI0c4T5WxI-Y=6Y9n}1TNIK z1A-Zft?B8F<5)7HTSONHs=NxuAlAt@UBYwX%p`BIu2sMJ!dwWck2ZwW!9alnE_&Ab z9qT!sE{Jv8Huf+IY>mgloRBk(M{Rnc4;(JhyVWjRvLPJV4M-))LQ9Mf?5$Kqbk@aE zJtC=G`iqVVN24yWvb$T+=GwF$(-Xq*aA#)w35soY`_{w>PsD!NyvQUXE%#rde4aOP zwh)^WLzyDA)ZM2B)l?NnM?lvp^&vob+l`Uk32(dj$m=v)`CS@3OxMS>yM&(f7@wK7*mqrtrxRv3(NK*&MdjvPxxH>nX$+~^G@iwGr&P+fgb6c?Z+ zCgj>+@4>8|u)3uO^$rzC&(^HmiXWUdHo>rjZM)qSI|ypcB^hlu((9_Z+-{~4xMqd{ zQJXr-_!Vwk*`(n6La~(29)m@|YEEn(g6%oOoSZnj?HA|D;93#ue77BsU^z{9Yequq zE@5cHIt4$%z>eE4rxKoEV63jA0o>%(H%2X)%yC241c8P3?}`^)n27nM~dVzgASF`Ez=N8nX|s5S;s6C zOxHdXJiPi{Y5Ka;tq^Sic*BQ^*oJ_koJJ~NuCh^R^zcw3bH|s{SPh1^+fFW@Xfa~k zK9EST$C1+&kSArF`4iTRj@ zxnwDjMIeCDU@zj37Sxk&XZaOXz~G?QdaH{^wZP>G!0MF`)Ni$op^v|9-@=bVOF_+c0ZN% zQ+YpCl>007=Le~(pQ>vL%KdZtX>LuSe_oAQTG&tXZ=*?vXaUA8^q<^Miyov?`e|`L z)%4Snn!ODluTT0CVT5*t8O4<%|R<)EClzPh^rYQqddw^C0nKcPy z*5dnVE%>&!+`mpY4p7|zT3_zp&`%o&slKKNU)Prw4N^lt?l zzp=FNAT=4Z{SY;`9HMP4<^Jt@9t>N}Kudk4#RGJ@nx$|=d%3?we=ql+q3JL8@6Zne z^u8Jk2%XtdYH@cSjelzg==}#sl>2vrCH?e)K?>FkL$}tJQV6K+0JY=AW+Kq11AWT< zor#%u;T0d+@ozOFf;&%Hf`4nvDh4PtKxZAGuF5BAI@r?f?@?0hIY43b2tzax&q3-n zXdj(`&;4|EStY)edUXG2s(%dqW8B|x`a>zA+`rUwfMP%}?jYEgK<%8w!|uex9w-Zg z<*D$G>CZ$Qyx4ny&IO?J@O9cvR9uGp%mF%|HSz+bk^88mrl1UF=b}S&af| za-FBg*`06X^LfdVn1<*SdOngF-Kf4R={f%i_63+u~MwUUs?J*99FGZ zO~d3f_;X8Z#9B7I90r8A1(rq;vo09g|YQ*8G;UgvZ;g;Ao#aSCw|&7Nr@Z0)72;Qe&j zAbuE#*AJIJP|vrD$I-_Fp^QO4rK+&b7$bL*M+RxQY*S*>j%T7~smm2{3z(q@FC@lw zyMxIYbcxs@-skwgKLUmc2%8xbUOgWv_g_9pA3aD{7*t+g@z^rsWX~ASq`PQb#Td^# zEGb}HTF_4)yUZi@9exe3>#+{6WR&-jRa3|hrG;2%5;HHSwU~P?ty6bRMH#{mlDk30 zUK%aWrCIVknv44)c|I+d7gDXfh}Po1NnWfdWN$VLqCOL9#=*trg!D3}(b98bQ8d z#*cC{VLfdDm@No3`0i%Hm0-eEV8Yd4!Y9ClPts~Uub07#Pd+qU2yC2j(1Zod}TDV}`7s)|Tdl1wPg4##YsNLnr)=h1uY;#eoWIJ9{ zD{zqRENx^E-stKR56}&dPgym!=-OM9@tJava4K$5LD|#+ zx~ay5b~`QJQu83)Ea+w`N8`Dy_Wg8AKi!HafiUQ{Lv*`}vhJwyvFexl9++CIDt%Q6@ei!WhKKbMiz~bkqME-~-%IE1M`2tnRAJby_B5cXev5a1( z7WoTmmw%=%`D=>FS81>OEu7tJbd&r&-75b`cgjC2QLvjiiK3zlKB0iBs7rLirJ1yt zJ|KF~=A{z4MuhRi)_Z?aE%%EE+ENLV9@(qx3-?MW=u|2k{SFBjrX~-3hzDy@9)|Oh z@Hie~f8iMEvmrwqlXTq0lZqik^ zi%MLys!R#?(-%@{neu|hF!C9rXp%A7h1KLFR+GhEmIwK0v^ZCsmxdMh0#<;2zPP|) za0f78iF(V*D?~rtpMs`RWdk~3%yvQYCLwvng`6t$SaFfK81to)X70t1_}hiz52EU>ECAtHk5D*Z>rk z@Nc%d>*X<-70)=8JjQY=F;-B(SVc38S~>~$IXJV(IloSv=Ozi6nY<*W%FkAw}PZah-;bH-SEuZ-D7@)bpWo(0IQl$k)L4N)?)W zXe5n8Urok%Xa*VEK}HLWGA(;}MNYgQJI?6kIcT4zOu;zRkWn4fLjf)`Vi)lLU z6~-kl_EaVjtrYBGm_3!^3M>ydd$^a2J#9{)@G;@IPEiW>u=eg`{L3G^nTA0zIc(iW zX89lu)>j;)gMu0=57Hw7GM|lyLxP^5M{7Jx?w3nFDx~C5rD%DqpB~?L__+kdjtg*f zx`CPjs+qQ{yQa8`V|?Qyp!lOS+PDIW@G+WiTuIA~kJAR@Drzu3L0gSaQZw%FH?C0} zU>%W#!Ig1TII7kZSHDXpuBQ0JTU3>Z zFh2PoW788EZ2HRavFS#z>1MF$7O?47u<16i>2|Q`4zTG?u<0(a=}xfe?*BtJ32}}1 zl(QNRJKEI5+Qce>6ymGUDHWf5jpGxi@$2=dEKMqdhJ8bOH17FzRFGD&=1jdGW+d^L z2qSd#P(vG_LMv#Ex^JL*b=Rt-G7#g7;L!s#+}Ka0#)I&857A8HVX86?P>t~jtw5z~ zjgbn_rqOJ1g}7F!MN?9Fn)H%#t$qHMP#LV5(syX4DXu$A@FOa#yk2>Eu;GRTGej+P zlb6 zPzI)b2Q{(*`tIX7ImWr#S=0+8dMT>znp5*QfHb}hPJIuY`aZPj2QW21gqeAks*UGq ziSYuhHD09k#!tyMenw{*FDs4WdhR%sQf^QjTY{nhdr1RXV42efa+6%#>*69WLEZln z^w+%-{2Wi7>UXFF{G6PFp*WQ9c+}Hi?-fO#T)SeVVyvml0e}eI^lV!X?e&bDe^0#QR@ozfOc$?-L@8U0aA68Vb zm9wZ?g%uua+G$fQd)-{ zw}TybBzcX9Pfaq(VFzakCH?eVj#9=51kVZorGhw5F?l@0XoP3DOB$21q`@r-{2;w; zsVMo=Na*iO*j{=Gh;Tx$B$HOUn4uGR_Cr4^ElBwxWbeY1bE2Xs0F1`b7^U9q{N3`%%IO^rQ2{AfkwZ9DZ`64YfpVl=eb||Ka=syBwGrqfJwiJ{sqhilW8jkx2 z&vF`r=Tgrqnuz;!PpwM}XJlz1wh z+314dBoQvVlSmjO|4h-D^>y;71=hjZ`{F_RsiZ^n5_TSbR$qpqI%j~Z{Xef5q?Z}2 z{};|v*#ArCDdztbKk@Ki>%E$OdgYrmZUO=XRT}>dT*50)(1p4_p5U2^w>+E3=h;kt z&lVc%*-BGAja2SwqJ^GjTJG6Kb)M}=WKSpCb4F4N2o>_gK4OR1hrkg(1ED5yulSr3 zaq0k?dx4+MxBKvoGYK!D%x;_F^LWSaUL7861`k*Ze23`Oma_5z`fY|i_vnN`&+JRW z_o=F~dxws{F;vOF2gG$6o_9GqHUbRn(ms~A7G6uqPg|=vHxbPewn@MDOfpw9Hc+>)1O!HPSA*c`pdO9 z(FA_Ph99Z~e^t@g-&*{CAE1A5G&ZWA{<%GclFH3IS71qf3`^=tI??lSs`gw3!*I2W zJ*PPADMrdMkgSks)cIQiQJD32FdVqzSF;YTtqxt1Pp_vC(Y(11dU`#L z_T1p&&6q6Sa7zLtcr$v0R~!_NIPj{S<;xXr!TKcy=UXaXK2V3Ob}OM;JfSSqVHJxTMGtNwa zfV(Ivf&z7EQ4lxWP_`z7DBuPPxZ*Cjpd#)Iu881&?tSy-O)@Dn*3YkB-`scax##?6 zzxTBdKY#Ci08UaXLpT=`7Cs$989q~mb8w%9`$MR}guFZ;kI!28TnJTW=JPW11)=_8 zh@%H(^pHHhB#$r4#tj3Xe%)+;X=G*e}9Sh&B#zK5g z9^aS84=nsJgnB$KG*4LgQ3%K5$1?L1dHghlIy`CNXCXA;=koZ4Jf0HjU&`ZY5&bKf zn~>43h38WieiOnf{8k8lSB>A}4}$!U^7DOf(TsD;&|eH+)IBbvP4^wQde4+Z3vH zLI)ivd3v14DTKcw?+~} zLy36n)~wKu`0hm8dj3(Eksj_#M^e$G3+D6!yJJx&o=KBSJUu+*q*|l#jFXCoW9il% zzRf!_;fzC$R?3MaQvFoy4L)R4YW$wcVTAx_h*SqGM$; zJe+p=^DVu@$#Cg3A4JX+4NG0xO>CL-ZjPdF8ocXu+7*%PIiR}>*|b0RXB zwGq{xotO$h;glj66+*5sFCX4WG7It4B3Sydd7e|KX;_x;9O7=9JXey#s@z!bqe162 zTx7WWGf7-fk&$2&i|efPFVo9*Udoe{InTW*{RyS;1RsiauGc8T01eowN$t<6@qz-YV zf^{P`!ru+%*Y1eNom6`)oK8F3G1s?)Mq8|@xXsE$hn&Q4MnP>P`$!^59w+`+*gAb8 zrYnwz*3gScN%^aN>^#Ip$DnWXVPg?AZKa#U)5cO|3hVzXmwP-IuHpWEQ*sqfE)=ha zwymytm8M@PtSVU@+Mq0xWW9_Gr&5fyGt+4{PO0?%&^j}f3XixExu#?pK5b9;InZNT zwD;Gy@jS!x2oEv}>qsT3QExqGMjb5?@aJqB`MaC z&+)K$-95=whrz5UBY6fcin^dlCDhKC1iQGH@q`PD$wB&m(_DlP7ME%s;KQ@l$`Yjs z!a&+_2IZ(wKy<-|zfpEon&e{Zy_w9f}uNcmNO8I$9bzD zTQjpEPPW#N!@Gub`TSFv6dZ*M@=j%(o8G^hHJ`5Vkki&Qt+s~9Tu;37Q2LxiN@1}Z zjQ(Sp|CpFPc*r)Q^&ju2Yq`e4-tEJ^8!2oqLe%t%PURPh@ld2sZUpwysV19p-1L>6 z(y&Y6nl+s4=e@#%L768MP-v12UDdl@oUVAiB~mN&{fiNsa^wQZ34L=?C~we#d}(;l z8J0Hgun47`u0*EYy;&vKZd&6j%Uk;PI+09UPl37hQ^^X%*L24bueY+3?MMu#BF;Hc zxpvj~?V>oCdq5|@$u#uEeQdL;0(pDD2mI~UcN`-jfvsJaSZ8by9z%5PR()B z#?5NJt?F=-!pc%Nr)gicz=oravJoQ)zAnsD**m?-($Tg$MjdObdU-5T$Juxhrq4O3f`$eWXg1(Ah{mBPWmH%}%FpqD-H>CH)?ZdVuET zRpl&DR#S`i>~b#SZk8;5?Fq#7%bYB|_ODx=Smy?u%qE9-)?hTf)&2BT0(|lhl0#28 zHp~)aMbjIGk{NOfq@$N}f(SbJBn7WI4IOUx%SS*OY42`0uz}aQR`otIib?a~yI&LK z(}=H@y0aZZU*~7i&W{Ex#*AUzG(=V=9F^~xc^-6)B1S9cWsyuGd-&}#*O<(mZHWG1 zhWZ6aA zj$M2n1B=fVvRjT>Xr-7HSb~-OvkI$mA}-|WJR=Gh;bOi~C|-h>63ELqw)rbF7onHG zi>OqX)B^EruE<)j@oof0G2?cQjm%1SkMK78gD&qJN15s6x-nNDZb1b7+$WrGq=ZWT zHcF^VIu45(XQcv`d#Xl>`w5x`o$t;2k* zC*E~fiqo~0^+Y$H|Ni1&x29_F^liWZ_E2^XoviheQ>2XN$?fDYNfIOfZH-6$3uz!`OXKT6&guBoW27{}2H2hf;LS*KTyVT&m&^v3DoUs z8pR3xyJQSYH&38Jl$XjghGktX&7){MfExL2I$suAM$s(4&D=qQwTPvf$FW>&xNuFN zF7OcQgvl6MwR?ii;*S;OIKM8?G>(;JI6qfdR#RE9a0FN?=CD{CPbHeLm!=xTY0T3N z%*$>hXyPPI`EriZG+f3rDwXgIID#wj1{!bqq|Y=IUc;WNTex}06| z3Kou6P{&t!c3H{>B_Sqm-sxFrC$7^yQ|7M64TLSDee8-R*rb%3o$vH^_L4@ZuEV=% zA+d%nAy&f>dlsv!T|>NE8)Cm}2=Qk*DT9qo&1w{@b8S=py&4Qfi@_*uOeB)dLm2cB z23>@b1g4Q@Tbs}uG$yzYXxTRsoiw?3Y)Kz?9nOQ)#9rX(psli%K(bGXux3l293$?_(=A zro508qS7o+<|s~MaNfo_b0RVBd^e@;_vtpvlQ_$jnA1(xjGXr$QqGzJa&DrWn@f~) z3qIgA!5&v1TlASg8%xyM`zPu0LCW}$PloO3BHu{z4+*j+y4->fmL%h2l=1NrWhkQh zF!4)>pGW_g6C(Av2kUYtMB|U!$^Q<`zqnK4tqkOk;G-`0ef%w_l!dO>Ef#H4+&4Er z1nVBGzr8uv3C+oHloH37DEBZ`$9b5W(9%tEFZ3+9P@fTb_GkC}7>l@xKa&qaCAKtc zALp8snjqA3Sdgo?pP=5id3NY=8UOcDEC1h9wgk_? zT%D=XE>?KRS4@d`S?eW@_(q@KgBJ9$cnJaDRT|(kvw+W<7Vu*P{4E0hHUU3Iz~B8} z0$zl>agY%t8P!D#OGbsXs{?sAJyThKNJ~D>Th0@{a0uB-ja80Nas zk*&QSGbYf!tM=?MY`8xcLB57%Fp@_jVG^m;{pC3WyJD#Jv`Q>j zcEqDpl&CO!jG=rYm9$-k=AQq^pSTxs6?#K!qzXIxBZgv6KM01JFO?Xi@mQEUK8}cK z_&T6LoW>FUA&Yh@6NWQio6zHP9!!Nd}6 RHA0k+I@a=D4MGL1e*sl(iERJ? diff --git a/out/production/clients/org/apache/kafka/clients/consumer/OffsetResetStrategy.class b/out/production/clients/org/apache/kafka/clients/consumer/OffsetResetStrategy.class deleted file mode 100644 index 9ea44aafc01f39475ce74fcc11bac828612ee40f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1192 zcmbVL|4-9U5dXZbTf0&iOhkwH4NKkNc^vK zBw{oge)f+t-g_f5B0sH7?s9jZclWuw_kRER{tLhcHd0WqtRsiKidhu}9aC5lVpT(0 zr{JnUJk>BG^qPi@(58xK3`whWP__>kURvF-ZSs%YIdsiq-a6){<9lup^-QN5^!gn) zG;6I^&yDt75_S~w$ZdZrGHB(}zEzP)lxnS7W>A;vCF^b3W_a~(?6k=f?)Tju!|Pmr z{4C0D_k+ujw!FZt_B%~CJm5{AJgQ7zYcagZSx5YYn?4WPCiUe7?P7jx+8L|saKFw& zPt>_6rUbl0H8cMgq77_$ffsEv%w5BylCLu;(V<6kNZDRH;88!MO!1ruXj`T=NUY>nB#HZ4@QyZz8{_dGGD+t)08P0Y){4RPCM4Xooi!{%7m$3PY@4BW$g18Gbf zm=Pj_X@;rGrHF%Plz2XS$-(Ig~E$*z-Q<$Q6 zr0iq`1AsJJPz+D-q(GW^hTt}RKKrO5+oFVx|hDXa?9#`=|P{%`w zS%ij1lCVhgo*EHf=?9cXMvb3AJ4H$wx+0AUX&4D7#*LKI*vbVa>x`5BCVIJ#BQ> zC)B&!$Am_hoG{_~KpQ^DCquSx)G(IF(jbXce5|Z?ei>|Jr#dC9AM*R0(`-M_a?4>d znZ}&y#Jp7IBxXzKyfTJu7^^g8$sZ09I||fPMW-B$)w{6@qFA%Z(jZDqnormU+GK2v ziqjx0LkFCFOzb!;G9UiAWRKt{ro4&V{zO7LmL6gIuYrX2kscZ4 zG$5Q;Xh7DSzvRljbPNZN&T9HFj;TzPo4YgX*9KuY>zo_pm{D z`@g$Jc>XUXE{R5XT8XWaLK+>-KC9@k(jB-r2eC4v;Pvo~qpbJ~FDhy0^l`41cpx*^ z0HG;!SQ9I%%DNz4AeO{7cRqtIisQDdEf=H>__FWJ5^x3A#FNp)qO1p1&az4d$9@A#!coUfLD#U1mBOy#s#w1`xg&MkKLASuKZyVU diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.class deleted file mode 100644 index 24976a2274a3958cf5d18ad16676f846e7f3ce9c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 9748 zcmdT~d3+S-d4As2YG<_?BoIPIT;h`fcUNF? z)Jcz|ZPGZn?m_CKP3x+0+%|TC6oob+OZQ(N!l;e`V{F#M6k09*6 zJfkarp_jkZ%d>jbD zZz|BYBKS7`Q#bxgFaNEAd?$kc!FTobd3_~DA70SQi%R=@iuQeF^@B?M5HD5YW&B7l zujt1AD%@q=y;6x+@sfqfN^}Yo^Ce0yQXv5;qlywFl9UrvLJDG85+((y&`U%wm6q5M z?2$PNU8O74lw0OnGB1Mt#zazM$$U#{EmNu=VvBSXjInS;*pBv)#B()~{IkdsO1ySEt3 z4JIgcmk)VQP9M*AdBM;d&z(tUP8FgsoMoI2j3~1$SJB*uVo5BuyISNJl7d^WORiEX zf`G#eiZCJzLUsJ#g))|9Y!3*={iu}2>&%>WcZW0ST)IDEL&rzk(8=F%FW!MHP^)pwl_w`qRxi0v{M#9e#MD1D3> zB7E)tXV6?jvCR&%8pKVlEgK$T%5`7As-ADB)^ANYKT^KH%ybRzyS)B0cUse+2kNGl zzq7;n64*8k_8N3H-Q1`=r!U9a&lYFuDMubwUN+Zkt|!gl8qfwNSl;B9w^LGXzH`P& zg8B9=9&5vfVSG}va2(zM#_(;Btm0p@m+JyWplb&yrj4CP>x>0{+hyUo4U z_;){xK_Ye1GY9+b3O`mQfuUN?AD)8Ml^)U2@HH_RDwc=IDc2p$JY0qqcf%R4%LvMI z>70|aq>fX9hL@X0MKS01Go~}sSFOSP{2EShDCCUEVp?00P2Juq-EDi!cBD8ik)d~A zPy)>)YBb9P?<`r*Q@O6ZOqUQS5cZ=`V(b!>>mD?DoX}{>oYi{IB&GKmUU{vvT&2(C zD3206D6v@en^fCxQn|c8na(oqB0KKuj}Pm?AZy7DeD6z-WcuSf6FN{W^6heKb=tQv zh;xE9*F55wK+|iO4qT;x=LMQCzTPyX>%J z=MX~`w zfl3+8DaJ8t17qZ5oR(T$s0G1f+YN!i*P_+ZEU?cWXrdW08l@&AN zz*9Cqk5eiod$a$ANe zAA6nLAYq3RJO-GdlH0*-;>{#bn@H_UCQc6K6U>|PrkKPFdY$`1r=29fMhX_@)53f$ z?BXoK0>$ZrCaEntvbfGO)D(<-sUkc_!E7%TpiHXA^pq=jILtXnXCy+*n?j3ABvTSU z1}4n-Ii%7u^Z4y5d7Ni0!Dipgd#(87abZ?J7dk@GTbc$t9gbcDCB-+~-P?C?Yj4*M zb_buR@2O5k08Zc0ApCTss64LlzH;^=YCfKt zF|^qa$P;$vnR_nfPq~rc8sHiZ8w_CIa_-CID#d8_8Wu^XyqZIUwOoTj-ggNtOSO6k zn>O9IV+DT@I0(UAu=u+MM^*l8Qlmq$xpS~Jch0edImYJz9PV?PHR~K`&7CtX@3n*( zM1t3Q`1XegBE)+~bQ}`Bh`^Duag<#^kk92u#u2&zi;rPGR$M?N+I#_((UuFaqpcS( zCt4T0fU2j6AHXSISEGX0cG6srx!8ck*vPdGG-C_ap%a^N8+UI=~BlgeIZo>clE_tO~mWbfg7kb9BnM`1NDA47Ff68&)o=%zfo4Y%cnN~FLo zf_srM+#<-5a)8e{jCho{@m=v<%43*&csk_+q}*pH*SU~0C<{*G4ENN^pC`0hrkj$^ znZP{3&B2SPp;70zg}7GPGJ)Cv+N_IMaHM$*b@f*37;YHH!nSa<`6B9%)CGg};rh@x z7L8+Zedxm^=9+wwao=(NIMh$q4q!PAwvoq|om5KAe1JcksUh-a?#6vMYZULsoADMS zd^g^Tx6wfS2?;-K%v?)X-H*4E_HwMjJMc3Myp5=%NO`6+uq1#>SlZkA3~soHWz@8x*X*=TV0jr1w^HL30T{amP3*XeoI(}FuA#V#7#FMQ z>$UWhDiU?c7ZiyV6p2y%cTt!s=E*PM7b(au5#FX??#jEl@*b{)A(ye*!Y^~@y?DSA zd<%Cq>gKB-_`mC)39Kwb(aIqMk~4xRVS;;t2;PTt9;uTgrD3y}Wb84l>WJ3j#xdOF zzBVm>Z1M$-qcWp!jnU^Vlq{XEm{ zWT^txkV`N)hBdrbP9Q2M#%vXNg02eFPV;CZ<>RWMme%|e9>ji?lr&oJ@zkhcBsRaF zS?+2vI{BhfbiL(Ld>_N&94S6nB6*&oR;c^&Yowg_hP(3XTv1(us6424f=M?VAf~;<2J9=qzK49c_D^X`5vN}DvDFNPZYC|Q1-%X5pXvJ@GEmWeF!soRLBX9-t zg29_S%^o(IJ;JM6_6};Mo~VvC6a=R5MMt@1>q!RgAC#E1x*%G$5r{t+>ABe>`T&V) zDcu}hF^*OxY^@l>+K(Z2d$fT|>k6#4d3);(q+T`@S+6uUe~P3o&~sy02<8CK;Zb}7 zF?@>k`qM@vw~}ryh_u-gi2@iKX*NAZ>$(~r^q*$bsJmJU)++l9r!Vk}sB@y)DncDjP)wtIZ~Us$0t4 zN(xdpfz1IND~5~GE z$!}<3VQ#nawJ5t=NWMavUqvJTFkv0OUT{<++s^Nql2~lD>@yfOB_*GjE>ylu+9rg> z!tcMrKZpqE#~DxVyS?zOi8hR*{XB<&3O=m!3PY(Yze!hqi^=xw5?7WM22G~YG7!*j zJuVm7M4p=dS=TllYMU*BUfRLf<{%%8Z93b6^}z|;X6!V9+XZdq_2nK^vpyNH`nFJg zXbf8~p{C8Mx1PqfF?8wfcERZ67e*&P=rW>{iuG*+>m%?Ttm2<>Y{GY037+R9@dA4A zA}6u$@nhom*>HcrM*Bm~QZI3qc-g4h$;z;se_Wy+DMZL>e28t-B8?4rl1W`oid~$# zK5POYTL`YK$sM;=Y6o!PH5*;{rwOkibpwKzQBB`mL8FC_@Zm>T{|XTT`svHF;74e` z2!SWcw1hIg@-}o0W~}(An$9z$u^PXZUBVr-6)jV0Qku*&>7YJ&CJk#O)`uDYM8N*%=X>{_T(aAS0 zEe7{))*|3C)w_Zw{xQrZOoEQWJ`sKrX1|g$j7Sh~m2$j8LU@32drrbQFBNz~B6v#p zO;F}wTy(B*$D#AAZR(W@&d-O8r^2NBc9Y&A?BIl{p0ddAJ;qa7Tqg>in(#aoz&NWU zhh@_V)MqZ{Qw7YY={@yXBSKgB*EjUltJq+i_Ax$RM!SWHR}gy{0oQ^2=IJ_+vE!C> zZEKYcD<;rej>CLrd*4UtP36|tGPe7R*w5+e&QBJrsp)r816$^!PHM4O7GRasp;;Cd z3Iotm(5R(Q!CSlvu2Qp(H&YybU!-c5oDcKL%};KluOv diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$1.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$1.class deleted file mode 100644 index 331e5cc88ea38d9c017d887a9833430b68764f38..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1724 zcmbW1TTc@~6vzKV3#DbH8L}6QjB)>Y|iE8e?4;$3zNdhDIh+ zm`dSU3eyb6A-D4ux9x3i9WX3Z9k*uk2DkQvdBArMxM|r^DBm+JM|nYAxMn44RGfNU z`ju#HnW0OwoZVed_#Q*Hy3Y@}8TisR*LZ{GJN!NAo2#0+<;b0~_OYEJ6Oc=} z^OND?RW9worRm}L;*CK0vMyFWSfb%eM};6pRzoq!1tW%mIAnc$Us!&)G`%Tn6vS|& zS_~1HTuXrs@h%XazZz&(9Y#zY%3dqH-s>mVdD;7@EQV?n+tW*Zg{ZZ-NjfbP1zpZv?qhQX zMb}aE3N>uuQ*=izICrLLJd5#w-oP|yH4?d;4lzJAMLDb~?(~c%>EDB1vg)JOE3}0p z{l(HTSaIwaZN>3pB#PgW{7Um}=%=!Stie1CPfDV&FEVoRaK8&E# z)*+j&Dev`4~U6LFQx>`Ip2a>b2o9@?rf1PpM8~1O?orR)(xQsixmQp@GZ! diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$2.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$2.class deleted file mode 100644 index a3a708cd76dfcbeb6b9e52cc79ae897206241cb2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1910 zcmcgtYflqF6g{)8EnQZrsQADKYLz~aMM6SME0Hvkm=raXR6m&vyF*!6cI!S&_~3u> zFPQj>AB>;Vm+e@l3qe9NZDWXmtJT1k((>X8qfsWKu$FN=Ba=Pmx*m2D~sv2sT(=djD1{Du9%tL3G9oPkuDl|;u z0(Fgn%Tp?(PgZG>;mIh^CaNp8jA3~ch(3CTXD;8y0&?rM+ z72}{TOQRJibWx;Q`wUk73gsis6r82s9Ib(}iE}ibI$VVboTr&YkZ6o9su{W`vEC4N zgkJpwR-5~VjKs=BSgL$pd4+=VI>MWYb^OJkf(5&R$rm9b{x4%k!uuDFkM_F x&g*nXaxNWn)_x*8_XTIt^&cS}@?~SJ8@L%^-XfoXEWJ$cphzQ?&~x-ppFa;9Id1>} diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics$1.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics$1.class deleted file mode 100644 index ea8613c180d87f328c8e24f4638d86fb0d4eacea..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1719 zcmb_dZBG+H5Pr5T94!Yewc@*0t=bmk08zw(U#cdMg2o!YZ?8*PxbBj>D<=E^_1_p1 z@a=;?z#nCtyP^ws0+} zeBmnYcve+ww<#6(wOe5r3_)fL5Ki{k(yH)XX?qOjvQomWI^6RFiM4)h6?(w~^NUhR z|1rb6;{OEh7pqcQXGm0aQ!u1!Qi-)ryCK|Hyy1{0Thlgo*10PK`B0e%2xM66*JK1Q zg&AVTXbbLjTrxWwMU8H^wX)jXELq)bRkhlXt;$O22@y+~CxKxq!m+;`W{4~Ont`oQ zk=1p_wZ$_T1fMyUzo{+0%ZZ_<%67CTRg3Iz>ZXY-W=xD=%)~IvK%6yk52k@R6Zi4J zz`TivSTOL&L8;H_n0w0RRuiNWV4z7D-?~4Dx82^ls z02)8|1N>3Oxhp9Iielp>x4W~m`^@Y!GrK>3eg6Sq3iA;}amR1&M$wCV5u}lcAdYMl zV;GO(J|_G*BTCJrh6fsQ8Xhu4cetJMHY_(a$uL`TRL$Vq+}sex7GK}uhG|k1M}o!UyB z2KUX+S<>>JF?>w_PvB0m@|iV;aKWhxhFHmxV!2VT2(`*9Hq9hTj>+vct}I_ZoDBN} zGAws$vJEe(95JKS1$P^Y%;wrr-F9Jy}Q^2N+N5lfLLfnlPJV}IDp z5R%S&23DjbD?1Hkir1EZ`1rB-O>FWVP7Ezb({@}-*2w;bQ`IqmX&o_K(s2=ezW7MP zjE={c)$s&<8uB{k@KnPy9nbMX$4g)H3I$564s*CIf8?(3qG5AcGgc~_!t@xjr!AZ$ z!e_i0>*DFf=PIc-?l6347r}GE<=fD5(&?a3^QEH$muW`U4rdykWgAO;J4o$N+E4JI zU}>PBEHN8O(OqGzT6OW3ijh*UyiizKo-Z>DH&M@8+PXjJ5}N6dFiJCSn!LJvYO2nAj7xL-;D^;B2%Xmrs(fXk#Eq)~z75cRR_*gjZxd>>ueuLyk! z7G9yYhZZ6f!&Mp&2AgBJh9pU}DS)I}$93E|1bl~Xgh`j&MR+8;2eyaqZ|FIICU$Y* z0Fh6hN!x`Z?EV0IByc0y!cJlcH+|b=YnTq?yLl_vc^jkDhv|_^Awi=A>EiU-^-~`u GnZDn302gim diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorMetrics.class deleted file mode 100644 index 3d6c77ae2bac505f4796ee95e356333360b9e142..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4037 zcmb_fX>$}s7=C&-$xN0d7Z3=jtjLjUcG++Xi6}%l14I_Vh*xLR*$nK?EHkrFRJ=vJ z55Tkh5muR`%2@sYe}P*1L96_tR{2Fg_(99Jd-h6pw%DS~R;GLUy`Fxr?)m25FTVz` z9iO&h2Tq5v6KCY?K^v~xMbj29nXak$MXhWFz})T@sf_0!`LF1UNLZ4#}yr~F?0>vwliT`+_Rly!gI{D z%MdwiS;864a@Q5~s^?0Cz7gBWBzc~vr$q8RpFGc#X*-v*tz?lUS=2Geu+(2V=;V*{ zoFGdh=lC>FX1SF~rpS<$kxWwD^hUTRtn@{O-WwW73Cpz|67850-0{W*uSj?2X~pzR z+d3%-yP2_a!t!VbE#8#rt{-679iTsJN+#NfdF;nN1$!`N6)^3{AuK1bMc7#I(fmLT+3*r}%hQ`PxqN>}l?p z^1Zm&B%Q!85`dv0@awDGQ3-QI*lF&+S7u_L%! za3p-0CX|Zap;EjVxCjBI{S4j&~w>7w<)I7bwHm)hly9?!|&$s3O6qGYtKw zrbLua3x{V!)H8GR;OB)EowS`OkNOf(M_ecf*OU8spP{dwu_ElG>v5{jebX5oS0lKF z>(opZ+0JoFyiL+J&2D73VUx$C)+I$#4ta`UgS0_;OtluZCrcZmV#~VGJgvEc`p5)B ztg%gp^r5knzJFTitA%bLH0zW% z2{qE4L5*|(sgc?_(9t39EcDLkH;j9s;XXh+3Y~sn*JJ%NV6i!9vG@!^v3WF&&7nCl ziD+(g=ev329ekoh|FW@SXUd(TD#hNn&!|kk4_Tmn#Hm*dATC5D3ZI0 zyt10SN|AetaM5Dw7uIF?bczD29{9on?-u|fL*N$p1**M34y`vq6EU-6mt8@|&1 zz|Y#B_+9&pEz$mFG3_5V8q(O=kinc#8@m)*&aQ=eaFBHFrR1T&;}8zho1Do+($wq- zrBX=Ej$#B&>?~1^5sGGZ39E2|W-S;cp$H|Dlst<2=`;Hzt!PNm|0y+l0C&>3nl$ae PtJp>BTj*W>#r@$wCtKge diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorResponseHandler.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$CoordinatorResponseHandler.class deleted file mode 100644 index 3753170380440ce9e888f8685f841618b8cce605..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4181 zcmcgvTXz#x6#mX6&9u`2(gKyM4N@UZOGCMc&`^Owp&BkGv06atG;^9xJDmxWgccF+ z_ebCG*=Lt1go_K*r7m6mCD(Gf&t#fTNn1%-n1{@{?D_V$FK6#FfBkd+A%K1OQo$y? z(nNnr1)FhNjLs<7fV1N7oPt-8QjkVQ!D|>+@H*aT#E6XZ3NGNH*m<)VmoO@J-jbmy zXu)X(W5^0%UHtJzj0>5Zj0rI}WK7DKVrX=DA#Ym+&d`~(om^bYYuO1NpVG#sw0PDu zxK%8~2i#}M6YXON6(@{B$8Ls$OTUyQ*wQq2;)Ye^j-{D|W802ySX$9`1W3ZLjAB1S zBD!Lf&S8e&fUOh%HA%zbXG+s!+{tKTCatt4?W|@FYmOo2-eOQtXV~Q9GH2K+&C*To zFsMV8#hn3DD-<}%W>!kCV`(L1f7?QOSMzfeFFpl*iA<_5lj-Xxy$yNI5dlBEjN2cH zcxQ4_yQ0NS&C12kj!p7xk#L&QMvh`xatN!Z4lH6q!JlxQ&2TPSi(QxfDSo-c3&p{b zVA3ZNVlYZ7R*sQy{Um^zY%5*LW_f{ddz@k zWw~i`J>pafTIAaG$c#~(i1^Au#L&gEUgD9WO&isADptq1VdWyaQJ`R3RHC^~JRhyw zGOnw57w@TfA0Np0P{l|1Sj8vU#;~XM0$@*1+g6-Aj_nlU$E(oQvm@hE6`$dA6?8Nw8FesMZjqNJcI1z07ESL822)_0a88Ru_U3nly)TF`K5Sf>=#zlm97gi zo^(5!A?}Rre%&>VPN+5V%RR0JhVAh-4srgAN+2Vhjb(BTCQC1uHQi$V8PQnO(Fa zp@-&Fu&VQ38d1HW9n#6R6LOH`OrbN_RYt=N499LESVpJ}xs1j#n)>fTX>ATYfI1TJ z_R=H4t#`31J<{M8UY)*&H386)bQj_7AJH;$2d!nSy%{6ub!TF`y31HEtO~kT9l~m& z_$?Zw@1ROIu~qsW-O>-(E8W6j={8Q%JV{V#=^idg_c1Cx0GA$OO8Uw5#6C2TKabJZ z<$m%LS>gFaj9#D};U^OZ@FH2u$=h3SkmQ8NB2{_vm?w3pD%D5330iFc`x~R4BM%c? zKVGUh4quUoP}$z~3tDbqy>L-m*F8KlZ$6p4cHY)DLWA@xWa&4wO21>H^aq}m{&elA zuxYQdZ^scFb=kLj>;>mGQcVg;s=`_93C@CP0Bobj^H}&O1y}@}0!&F&;YmsLGd7U> zH?AnD!op?8Jvz-U8UIOyI5A>PLxGK!q0tO-F>yjuI-0<0k^l*g&Fm0H+ci_<*NzffawVAt zcInp9b)zl4x0H6Jm#wTV+j^l6l(l2`tsnMbf5XmM|A?KPJ+CA?4py8?&zV2S()&Ko z@3}wkBYpbck3Ip=gHMA9|_i`f8+FWa1^z^RXOf(Q8COao5v<&!~t2jAr@ z-)q4419-3jKj7;>4B(*veiTFtPBq|R{{Lf+@JJ9p!B6?)=^$>!qa5%tZpY*NawdQ$ z0(eqDy*;T}EnNb3M-4L*R`QBE$->i0dRhsqS&iwo6;=)1D&&|M)^wYhx{|fR{f1$t zG+nU`vzI{nG+ndz3g~NHF-qH*fVzGo#RS|C)pQms!HU8`^ z)^XJZY&ysk)1F|8-EZXbS!Qd7eo)aNL^NMMa zn&UOh?{}P;U{;>I$$Hy-G%<2HLl2wm{sObe?PZZ=CIJnm2Oz)?!3)@RvM4~4jza{P zei|++*GJcdQzPN(zrSD?OtO>0dgEj%LM2NC^!Q*dss|a8c3=hc`tUDRCkhxdO@q=y zNDA06KetZgOZhdjauI#eDdo5l&MJB)JZ5OA-o2D1Ym>I3PLC*gFMk`8T81i!yd>b% z|2Zx`3i3!DZmm!}J~72q`|~)OL+kVF5d<50S;h2Jp0mXYedXcrJkFCA@$a12`w)CA=(QFTO6}XLu!m^AcXg zYZ6|^8v*=W!Y}ZqgbR2>!Y}cbfb9vUGVM4^$yPyCnPsI5S-ISK)pSR|-a~vc8JtmBp4%^A<6NUn z+9vsVUqQ>Jn8~y0V*#y4bT*TxHb!A_dcI6fsZNSJWeLB>A0+${e+uBw68?g}O86VU z+{PnS#NQ>{g@}Yc>=BTt)^6dmr=n(ROc9?0v_e2tZ`r+4O{wK=UyyhHd_hxIskrtT zsS_0VR=?TA?E`t!unpD7hHW}%tuV!`KxYg!4!<5>i-n&pcPyx3DRcctRsm}ir$w}M z@9d%u-(2Cs5k_gG(yaK1pGHdxv+r=XB+iyTywje2-F`=g4@bd}=eXQ=D? zoa>BmlIfN+={I@PsmD7)ep=PqHe91%=R?p0XEK%E9cb8Z9pEu+bjs2l{0WK%GM~J= zE)fR#^hToVV)59Z|0L|oLB6b0omY%<{JVMlZ}Jk>%BFj!!cilWVf?kGiTk~BjRwaQ zn?UcyA8l&f3}^k0p4qTGnp{Hhx&a^; ztu_^nzxjj~&Y^O}draJp=AmeG@IWLweqb;*n23xH#ba|C^>B?&`T;JfhL%}<$-;!C znwoQ*(130;vTDzA3a$!(5=Un&m06-`kco=t;Fs`ERSVs}7XHowy(NtnQ>|ktgWqBAJjmBI;5%`cnd!&)@4p!3#B;d35 zOAs$3a0v}`*H}dGbP;O^UfR3)eXK3w3$J4RW!%u*bOkqx2nj_rU%(~WTSt56v6k*` za;Ua&s`Xc}L4=FBF|_$IzF5SUF5nbVZE~o-!l`bqpt_}q%^_!(=kbG&^YHj>j$ zW4-t&Hi?g+MSL8c;u-7`pTL0lB!i>gX!n&KccW=WXS`ZO!ickPYymaUx)>X6~Z11>+x=;~2Y9&L)!L{uO#%NW%Mspusw zU-%{b31&*5d|>#@k7C%9T}N)h4DMvGN7C(S@7ZVfcK7tx-@pF>;2OTkVginZNi1Y> z0c8t|SUOKHuItea3(I&<-)?I0eG3(==+P|`s}`;Nz{H0p)=YfFFc$60uuwKpWf;FB zco6Lg9yNJ*z+i7GC4#EQ!%&0_w;O)YcK8GC?hEIDe|Nwg*ONj;q2u~0j5{K5q>4nK zxEDHA-w#?+@yHJf1Gp73x+0Z~RvGHWGs2wRWiYCKOE63}q!L?kXHNu8zUR@*g@*5P zZZm|>)KDP`h>62o*F zzV zdb@I|)VZpINPD`jT+a`sYTp&nzTdJ@!@7+rOxws~!^UmgF|lc*j*m?=Y~00`jeGdS z#HTj4weAj@Ha^3yiO+3(fiG=*g|8VV26xHEYj~Yf{Y)~wP$p-N@w*{pSQ`fX)EF|X z4}+14=U^#P)&;Sm$|=@$c8(PFz1tBM7;X-=*(oEmq07(7k5Yjr6;voM5QT4_yFXNB zPhszOI=*t^NK$`lQq>UAD<_FQt3{2lBRx-sg4$e*YMbIjW%`upq9z)gEzy5(tNp$} z+^?VH{vQ@nP6omi@=)}#s9otM()72|ou``@DNmQQr$7E*o>qHj%9e;8`oTdpagDql zs(elbVK^7k8Prpo~FSK9h{Znz|u7{TgMEY&6A-00n6od zbECuFpev6TY4#-=ts+Bzz$(rC0#^DJnV(6@;AOgxkq(%JS7@B?YT;E}B8l`9G&){% zd+TBSgi%@p)Jl)R9%JM;7*8-t@$JiFGee*XZ~LBq`e diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$HeartbeatTask.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$HeartbeatTask.class deleted file mode 100644 index 176aad0080cacadaad8be4c19c7fa82dbf573227..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3425 zcmcgvZF3V<6n<`+X4~yXTA-Cyg^E!^Dx{^nSXz+UQfMtfX{mxQxXs?SOOoBNFSMZY z1^xv50Y5XN#g`d>fIr0#zI{+<939WuO_~jq88^e24>#wW+18)RW!sKnTDoUD4IJjP*d=}z-O zfVM4TNJ9~Z0;ypIs~Xl|YPgP)g0hAREDbgVqHbC-%VpD*)FKV) zMbtT{U(_8v!j3x6*3qZ8sG!+Cuu}#K_ZCfQ2k#D&)=M10wN6euWxd}l}>u6{?6%U)JreL@Ob`~3Ni_2DtrZ5fPUV5xVYVzzx)pU3>4`tF<2ksaHx5Hht z{j#xjAE>N7D&;6> z@9ydHfdG?F+mcuGvkKEQ^C=Ss;g@&;!u} zg?s?+h3dODq20&87L$H}G&7rc?mm(aLpgi4<374Vt~(;_d4Rp4v?I5#v3aLXK7WDz z4aqJhH?g~L=sWaYA?W%x@jOSB4A47H@c_T02Y(Rme^La05q5uLlt4Wx=&dD^m=!4) zq64<*#JW&%SM&!_4-%s5g!$_@6kvO=7C*&Ui~lg`=A4AP^0aH1BnsLy=*{l!2Br8& z2oI(SLQYUBH}nXi0edvPY>*QkrkANva+=ehqQ6SVNJ9e0iA&%FUZn2<+P{bX;JiSR LUgG&P5`O&)CGGUb diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$JoinGroupResponseHandler.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$JoinGroupResponseHandler.class deleted file mode 100644 index d55482de84e7d53effebb2f9aa28ba47f6d4e48d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6376 zcmcgxd3YRU8UKB`HrY;TOrfPfWv!vj)ov3)u}ykxlQi8n*)6-9ln69Tb|;%jHZ#lY zENwwh5vAe@;sG9@s8u|mST-ruiuFM2fwzM9jrXnm;nTijDXrJes@BF>* z_uk*;%-P3J0cgf^UbJAaf|{fktr+q^^c3S@WR{6@ebzdojmg{ZtmsgzH+>qr{Cknd+|O# ze7^?|c<_ENn(%>gJjmYaur6R5 zricoxm~4uN(*<+vUObuBu3`B`*`JcNq`y~J6D`|F3QMD=93SbH(|L2Rh^k3)I1;SD z?Juw@T^MBL?yGSy9v&E0;^y-aHbZW8yP3Ob*G?zy!N&y-&H^z+<|ZQvT;!N2_`yU_yKm7<&WzgXf;p#)ImACUYK8;M{Fby_H_yv9`;a7aKU*k6te#;X39XHQn2PMaO($ohsdP*@BB05n5_*89B zH%4Vs)iq{dg5>Y<2MK><8h_&E&n%O=`4O@e@)!KogTG1mJ8}|2_=mvSnMAsmYnF~f zaV5=c2&_C8Q=_IqBG{Q#Qwhaj!#Pa}sYg=|r72>OIIR#*mv9QZkZia3B>WTqlJIY4 zVLSfA;`^_JE_6%SjUYwA=^V|RUK}nlV^u-QSS+Liep=wzyjW0*XX|D>JGtJVpCvXnq5D!2vkDaFF1Bb+#jYicPwyYM;Xpj~B9tr-f3I_ML( zN=|RVg@Orz^NHuF#zSmfIim~;hyAVvvVd2P$CXT`dh^Dn*$c(lfm)_><;tj(RH*E= zypgX%k|8TylINjuBQr5unw^7*Afpv-)Vz^0-c4@$nwJZ>mclcijAjQiaYMBnoT?*J zA+2m(h*qWn(9=Rwlte@sB(oiABU@2TdMS{|$fPDjMl!Rlf@x(^gU!SKEbBuhLYcZu z+->V}&n#(UG6X#v3?)v9jUq!$JL{v$W<^QtD{x1cRfPzVHPw{AOU-cDS{0|4R#KVR z7nrz%!1BEQ&+VV2>&sc!OLr?oVnC72n4B3AxXta}GdqVvybE+I$I0qRJkX^&7OUY& zK;#JIUy4aTGb{ve(sPnrEv}EIx%Lpa!0As8?^)gx+8YY*5B0Z&L(!h@j!1v7-Qmbq zxaxjBa&^fIC0zaR4P}*o2xxGFcTGapn_qL){&r zSbsDY3B)=&`>22mhM06P8sqsnHm?kZdIMd-_Wo!GVG(33*xeECiSfhwT&?DxrAn}h zvy4BGRws%(vdjAEc4W)gWjhHdshHiAf#oDAXtUFHW{ky-TQ)ET|lNM z<9x|U{Mu=Ps?aI6o2Spw;8G`KB&(U~sB-SgwY~7s4|lEfW2UIAxs_Bb)NSQsm8T6qjcqc;+v2ri9jjW{amMk)P=*h@&BZL1$ec@7m1V5sgmifGC^ zjwSo^eQ6v^t^PR9%VF6#Dyv>ZTdGcAc`R(i!E|ByjJC7VhsBdd+VALcxf>jtWDKN@epmljJDs8rPNz%9b9Yg zT*s#@ka6TstsX~h4su`p?Pl*F~M)c!ZaXme{WIQM0I4csuBNS0Z-<4uetQJY(6RKzu!=jVEdqm1IjAl~# zSzJobwJ>o?$}gA*_4eT^G7z4NAWBA3Dt6HmzK4vi*D}b$+J0KQ+M4nFkC@-Ee9h{0 zwdk`9@#U2PGO=s&2HQ+$i)r3lSN{Y`NbH+uGkLGg3nA>rwRwb%M2p>~td4}fWva0| z9i}XQ8su~b@~7MBEYqjcV-W7NefExm9LR0^$$i-c%Wyqp3&Fly90{5&f`VQ2ts_ac g(4PQ0Xy!T^^S@Ra50n(q9~%suq4^{9eHumo2Ozenp8x;= diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetCommitResponseHandler.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetCommitResponseHandler.class deleted file mode 100644 index 780eb55a82450054f7ae9113cf78e9b947636fa4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6253 zcmb_g3wRV)75;Cs$t0Tr!Xs&cs-cZ-NK%$y03|?K@_>cSE@ne$sJc!zlg+?pcbS=` z&|0fiTdmLfMy;*&fo}`Kl2Cc5Re?S&Efuk)w$;|w+SYe#rJgghNp`~|4XMfZ&EC29 zp65OPIp@x?Gy4w$sK=RNEW~Cv79qx?J_&I*5=gq?!WK7l7{xG=;ZX@JZhDsS@0TPxH)Y-1sa$$B)nR)-Ukmi@dOdiN55YjRs-DeC&t}CFN%Zw4;-2wHZPkRiRr}(wF8c6kZsd~H86{DGH{zOEHbtt;V@0m## zTOu$!XgRTwJfN9vs*xn+)zwP8JErOavM(N2^~RWD7%FXEKT&ln$B;q}DbH1KdvZ`o ze^suXujB04Lj_4iH^?DfWB9$%I>kXxG|Y7{*^U-n-I7ubI&UZ2$fzPc<4pl4CYfLv zA&!Eh*K)o<%1O!bh<4C;^vG~XX;5vHr zyvAWz)h+kz-JwKEkEksgC-~AVne%wFj0P-~u?)A%Sd6y{EF7)ojVW2eE*ZP=W9GC+ z!d@9^^h+3!u^)HHScA1Peu4uM4$3%$!!mw~pGo++j63lQ3BQ!_D;%M)lyMZtWZZ?j z1*YdzY8(GD?%}12N7FGk<;b`f_ff_eb_!Q#8TaFWj9)XODZ@(^_+&hQ;}RZ}@eoeP zco@fJ{01il=&U&Fm}+;0oy01xzNVrlp;u6yu$usyknvlblJGkjkKj=mr}3DC-^+L$ zPsn%@e~|G0C=EqBTS;l|xiVQzmWvqrzAau4zM^AuR zF*H`W7HFF&*DMJmkU>4la9sGPYvN9cnyhn?lvBwh{8!-0F%raUG-(q?f1OoBFBw(w z+lVzXK6qNzp|+}~u0^O%bSyjLiaf`DYh0Asc{ig7`DV=o&2-=wWmlM0%Ca%_d$RXjnu0c0d*U3 zBy%>Ba&hZr#^CncgveIyO_$lYpJDoR-|+bx80sukIH(ZS|vWO=X>k2?d-dRFZ!rvM!n zn9dP~p;VU<(X}LBo2e%?RdT?xb3xu&37JG?%>ccq_V<^q1tUT>j2gM_4c z<_s3{87y^@Kh85CeI)8e?AugrRcIWg!H#Zne}S%OeAOMp{-Ju3A!7Sy23uM}&F$gV z=5}wBx7{0V4+g`2Z`&#|dzLTI;r08P$kf(WUwb&SHRRh!i^b};h-w9ZfF}b9r*AAO zcR_~7o79>I2}s+TvxCJxU5D-DU`he4RDzT!8JL@DGH~IGCkvt+?Et1L&b|$>?bFu> z)&_zb0z*d`_BBy~^ZEVFtGxd3s^&m*o44H;3=n$bs@mPA(q|D0&^}75R{kR%MHG6; zm}-(@W!aK5pk*=V)YN2E*UCCn(C4p3^x-cL)3~h%F|*#5l$LVCPvZibTZZMd<1%^+ zBcJ|(sns=mA*yS{KIB!`?nVACn#{)vdQRpI;vh=IA(V>4@X}m4krNSCqM3eq#KQ8> zLSvZ(*fUFL#umSd=BVGKCktIc9GPCe1Fp`RGz#{i@OFC7v*Z;ry|^<=jyxpCx>3-& z;1-tl6Sus5kWfq%$?822{gC#ch(^U}xNl6OgxJdim?SWO$pV2(2QWoohgd;_Qh~#m zy2LfpHGpY6p1!1jDD!6)3}8kcy_cm?zPON=&*SCu)0nxasH|u=W~DLvR@C?7f@yCk zFYLz~ndD6_*cw#KEJ$O{4xAi%EiX*t!ksw6v=q+zJjWeE;-7G%h3)3`J=fO+}Yi7<2U40D)oaX2rdPa1W@ zvW7`5M~KThvFkC1{C^Z>;uzf|??RKf8ymzu=n?l~i?|QhiTiPrcmTJF zqId)^iAP1gI3-HNX)#+oCgzC8MU8ku)Qcy@ zLh+PXE}jPBAkWSv1;!%dN;uq7xe_wpz$Oj&pne6`B2=cn8hBlh!VS`xR75c-O1+ ziM&WcSi-v{C^R9VOG3m&VO&;PR#P_pRn)7Wl}%lkF+M+|NiQ@@)+RL3 lZ#B7ZA$^l?ByLmbQ~&zHV#+K!qOLsrh}QYfq<2af{{tB~c$feH diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetFetchResponseHandler.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator$OffsetFetchResponseHandler.class deleted file mode 100644 index cb6b6939057264b2484a7fbd57509616f7d5faf6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6119 zcmb_g2XqtX9shlYtdr#eg)hBQr@ zv}u!cH*HGOCg~*IAq5$SmLzSu(l#Awx|8m`o3^j7@0Hj8cPCkrEeQjG=euwG_dY#& z{*iM4y6|r&Hesxq9tkJ9p@~R3aTvN2Hl#!t{GG1B5!~Q}h;hC;T7zS_u?DZgP5kj{ z5jWRh9&Qow8vedj#BEMA;I&S?4!1k-`fA#|!HGL?Cx6`K#2fLZ8oU{Asiy5)`Pti? zcst&~w|8@-ckRRYw3hyy1$z=NFCM>)X9xT+uLk57pBq=-*3*o{#&)wF@(8o#E;+;UQmj4JL!^2j0C z9f_+-!brIzS|XJmQ*^hQFcdu@$5U>Prs+{NAsd?BOGGxS3Dwxju(|n?skH24uz9qo z!Z6RTCX_&WY*^6;<>5G;)cduF9N#DFD&J=hZCrGQCBc!Alwu4hMr1Ukq>?18vO`Wp zu|AzQ;6XPQc!l5|U_RV_L$kD~Hoc zip*YuCGm3TSsfWl(a5@N8aHy$sA&=gqm+!Z;9Nr9^5GG(bQnk*Xo@oPCB(R}@J&TjxWg!}^`rFmTJHt^gYpsCoi^0C zd$*iyGLH#v(3bVBvgQ%IYO_4jVSVCg4usA1 z4C%}&u%(nyN|&sbnao9;WVoRMi3>EUi;BpC4%gIam9TMYlfroC#U^zzC2Cj&2f2PQ z)v%tqWFgB_EuTP@BeAF^mM1d3fRxNaXIM+?5oJK-?Yy=?*7Y1*LJxW+T#ak^iXggX z4kH&X^CBLW@EM%qG)^lWYr$#AjC1fx`Su5cTCOOn263*hYB0eYK93G`)mhc!J zm+*OffuSLfKh+Z?Ji)KlmJ_eo781V5p|_QTu4Mk=k}Ba#cwEGnC42>6W!T`=bxn7T z@R>(V#9UUJaE;KJ!8>UO4J1}8b=`PV5B*7a5?>SXbqU|VQxd+3Z;AM}gzw;K5#N>Y z48AAf`}l!`AL2(Mek|c9cvix5{N$(HGe6_2pYz8r@JkWDlJIN%rh*-I@d4619#N9K zpEE4V2SwP>N#t#5H6B%T?gXOKwl|@SC#k?FQI}$>rY0=ya;oK~o*tKk-{N-?evhX) z@*lXUf0Xbi{8_|bB>WYBlkj)^gJD%bb44gj1sem$06p7Py z#EIgXHuH1vg}s!f#F%32Qpi#cTI`eKX_C-E6wK^q=q_vc%8ibpMvg?3RH~`7a|87( z&YhD=#Yegh(r6TmDg3Lg+RQZTlZ%B*`gxk+O}S_o6{*lzc9I5^}921EV6K;K|6G&Deeo?sxncegi0_RYrFRXoHR_6BwZ zg8Ktg_d`A^qV+z%-@CofKeXK&@P_&ZeZfE;uZ!ayD<#sb3T??R=<=B1^l&Pot7bc* zA>L5P`&(wl(?t-V=t(F_G^EgP3pDnRQ86*Io-AjVlSw5JW#}joM1_NGQ7IfuFrF8aZ$n|`ZfwY7Xiq?H{4 z_5NPaFHW@EggLYaq}I065L(;V1m>Kiy?`wyETg4}tsH_d`18fsMsLPjeU`?xIT~kS zJJ6Ov)dcJh5lkVC8k5FCi&vHgr{=`~QDJ;~pgWRC7(*2t#K3*|{`Xvtr5dFUX6)(Mcx?I-zF zNA=zJVo`@BcgGp5pM-l3_MgB?(kDk}1E*uppp$SXu~EQ@^Ut++Wbh*HQJd*e9m~kZ zwWt&7&@I%%BQzi=%*TFV0Y-#{P=!T^)B3Qm7&i+`aEGuI_Xx{zpRfWC3M+9=aN!AI z6`m4S<5^)1o)^}#Dq#(C2+gcoXkiUP8(SxIFpuD7141X;C2TZ3zk`1N^-uu$u#+M% zNF(De?54ABCLw@5Hii|qei$UzANZUucxae(R?$2U|@X{%$ z19W>GL9K@Kf4EY_%S62VJYphVA>x%HhE^&K(6hh5F93!JSn<} zz#MuzTiYIktv!RTN-5#A#3l+JM6wh%5KW#|_Ez%XRnwE9Frn;OY*11vS?qqo<@^bI z0<1J|ZUUanRyy+42&FeqQ9EK7H7Q(AI{^oA$b84qNAFfL*e3d2{93wcprG)hk$!eR Sf}p(`QM%{(;6jy?FPa7jqFjR0LTN3I(MV@3ZN+rkhPllI_QN@B{oPadr=- zU{9XHn|Yb{=FRud=NACn!YGBq6pjU)JNtq@>ZqL!k0I#N(n7$+)pUt#lugt|=FS*W zEn|HBCLo(L31j0)K)#!IKXty8xI*B8wuu?7UDbLR z%w6LZJ!!*2que>a(5&nHO=~TM69JRIGDee)f#qyyYVuOsNZz6eyvAn2<%|k6XcgsX zOo6HraJr*5oLEafHy8p1Hqm9m|4R8>EIASZcmH&d2oTWYTLAR>0`{QKc)vxEFiy8I T*ABW)21oo3n91qDFv)%a*H(x- diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Coordinator.class deleted file mode 100644 index 0ecbccc5f37c4aa1a9419d25c91e8db21b8d4855..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 19663 zcmd^Hd3;pW^*`rk!kbK9AOs8qP=ZDblVDgy2%sSbNFV`8P@}ClNhZm_FcW7cYzkNv ztXtKpRcy6tRZ3eoGzm!Ex@h-(vwLgzeQ$r(<@Y`Jy_tD4^P&N4e}DY=e0cB8d-tAu zmhV~a0-t;Rsb`32p1RIQ2e`EWmo~Yqm&g^4R^|qT?X&=(G0#q-n>gLS4#h@%|DQrKQ#D9KDtKu+RHzdVSnP~pBntKk8a?fNsFHw{0kr5#J}|NuYCM# z{*A%E_0g?-i8TJ5Tz+rxALPp)eRK!^NiKgb;J@%+eYpJ1;J+LEkG$RBR=)gGbn`E{ z{Eu8-5y1Uh8onx&Ui0$nqOvy(MLxw!F_qLuqp_|~B#?;4nuCd0s58!FHbx@BSbaDU zj|bz1%3~V4F4z$W2O^!p`an3`5$N1xC=b&lNAGpPcwaOU53USEy28O2KCTP~Vu_Am zAWQgp5F`)?lJ`hU2${ zD+KHAe(ewB7P^KHO6#obwOn@w8DC#g=~^M(?($20A4 zDtVZsnhhQc`)x=OBf)X9(y)3ZXBo8;U{KZwGeU{!^bx4QF*4MObgO49Op8X;*Q)um z_+WZ6O9jV)mJzX$6!|!nHy+h;mt)4HD_o`}m2PE1wh7bop6uXKY?t*N!7d$c0nTEF zAFmF@CB^{w_IPt$WVZ5y-6UR<%_RC)WRY}9ml8qN9FH(gnkmO9u^?yTzb?K#(uwIt z76OjbGJCVg{igxtifi$&u^Ga)YFQ@;Y%K9DPp|>tqJ`rFlF!QW=L8Y~6chjh(;07# z6&KEcgQ8Tdw8-srzz0EV;P_SbHWfBH^rqHMQVukNP?+5b3PD6l%{kXc zHd$#J1^9(blf{3fIp$0xY5ln2c>WJltUcJ(n+tgYI zWa@F76xh(Q7S56jcq56V!{s80U=us-Wx2HUNhaJ#zooENXOR{YAk1#w1X5XoV6B#E z`+q}l7q=EwjB()PoM1o}f&!hLSg)&?J!=-zhBsj?sl<%s&J^e{QOOMimL?suP)rB6 zVVLBYb7aie+dJm)!6)T5uw0UZxx&G&{}u;HXUJY2PI5C(xIrPH?!B%?OC34;Ym0Gw``J2zU{>v3G8YcFzQ|08MapP z$=4d~k97u@hh$Z`*kRPuMOu^nTxHV7>1mUmqvuU}kv?P6APt%HRr;h!Uy%==r_Y%@ zhh8@6)AWK#hv=|L&(O0bJw-=M`Wn5=G&?s^Wo6IoX&V!n^dclm1A5GSygBY^V}bjZ@>XIB2R9xE!AhPZ&)f8w?ov2PS)l`_7S#143@^m7fWTLPPpjA=qzCM%#X>koyyWP^Fa`gkyg;iFqZSdK1F zuJb^u*=mld=Bjyyns2Hz)B>hy<=p{zN>_O_l4h=4*5x|&7TF^7I#Vq$)k3w%q&Ecm z6<{wR0!D&chn3A#wZghPc}pF#A1dS*t0ksds?IXGfPJPq8+v6?JD(~)r>QzqEu)tW zRd1?uIN#*4Tw?NgTvP)C{tl$n#bSZ&vczI?p(D2W4UO41?<=ukM~PAOuc3Ev%K=Mor}rHi@I^R$in5tb}XsU|@>hBO`X8{$92c~)_Bt)l+nU>{_niUk^a(zH3 zZ7|g(3acLTj)j~M=rC2MLhaO=OJHPr&SW!F)u4~b!%RId7kpTYMA!x@k6QaO<=s(u zU&_%1G{{obKzdMhGtJFq%2i3uiN6SisMRY%t;_9yG(y>AGh`Y`I!k3AI|B<#AyaKU zE|yY6v6hy362?@URM=FT1$3t;T`^e?lqaX|Dd&VwcEqri5Yx&&Ggn1fDo)Nv((_{N zKaqMTV0E-%wwdGdvM`6e9jTPlWRJuqh2NHq~C~RRC3bH)tbZ0oly)kot!JN zg@p7v1kSK+4u!*^co2~4!sY~4%7FF!%BCi%rMLbk%6*KKds-~Dd{Qx6?`2`pKbQOA zC3om`_=PM)37N%>{6vhXV9C+2k+YD%G(O{WKs`JYML-}Jd7fypf7L_rR z(We0LoKbP=-`-FPCZ!oYlkh=zqc?ytx1qNuqHi+a&aO_-Zd6Y&u_}nu2EI_dPH)M| zHf)uf6)X&87T2}1zUl>y^8(?1M1z7b%If5EDkCygkC>pwU$NCSzRo&fAuQD$jXjbP zH)ELzs_2V`!%R!vm>C{p8Q!HqS`hEYlA(3RBY%q}d0`0L0%z7Krh;H{;~MM08Jb_$ zACaSwsSpkuoRTY=(_h!i$w>HSJRA)6L5NDNqOw$FVVO;;=p@LBg`jg`1--k19sM}# zd78a%;9Tv=#f3OPnRADs9*RTakpn97ZCo^x#;=~}h>1o8`uecaj-tqA5GVRF<|%@8 z07##v1?3E7>6d6QCXHOXoq;Yb0Xb5d32b{!3p5i^>lw1^1d3ALg)N8dZN0JRRxzIz zalBK+v}cSH2~qCZps0_+oS5FYAvsxk#2f>-(@30v%T8A~2gODznrXgZ#SI_Lh1~Fdg%smT01G3#O13{ujQv zs;$XUtwZ_t8v$37OW#JTsor?fd*_a>w#jqf2(ijs$t=5ITAYY>M#D9AeIW!uSi;w| zrm7`kuc_;Z<7{##j+N_5_$XJ@Gh-#xfk56DjRRu4-DJoH*)V%p zbzOlzq`c$gr6huoZD$k<%`p0&qhsqW>Wyr9l6lA)E_G4=<8Wg>_XiZ^WTOQ51C+=63N zt&%C@P_!o=+5tiycm-OORyH{hZ+ozO$m{DzJU!FtBEts@2MN1TR#%&_9X*^m&mKM{ zkBnWkX8H2ghPDm$Ynq!I+i+(6=&X=|jdVGew6I6kpY)be`kN!II%G+zBli{4sZPz< z4Xt+R*eJ4?wk*pV+Ui%j#~c@}NY8?m4R!0zf-?DhDu52A16)DYF9AdmTnG9q%7HxP!k?trM5>z;g z+U3YnQ8_4GfLhS{cS~9RVU>>-YtEjvG@B04#rWrq#WYDaeehHw+dKM>%^iKm-jKdy z$4B3>O{DMGw$XR&+~_;@hV&gfNcxVA4t>Y&k-lTk19v5RKKhQWC4I*}6Yf)qv6J+5 zeE$X!$H<^Dc)QC#M5Fyx161fQD;S_6zZ#%X{*vMWGW^v8r2ITUWBq0M1LX6cI6x)- zGS2{w^Bv9^4>jv^rfC{Jsymiue>VnpS;JlkEsE7R2OEnat z`LvN%(|qGje`I#r&gFY-vIGS48LwvQ%S&-hp=A7>1be~_y7zwx|9xLUOC(ORFbrPu-q zwo)l=gJ!qW*>oASdpYc|6Z+o;4PT+P>PMfmX&U{EevYU4R87C2UqOSVw4Qzq&4_-( zw$4TNQpjqx`c10UZ_(;^_*O!m*XU$}es9npmcByj73i8}ht5JLw(yj;O)VIoiNB}& ztA?ni8T=Pk57I3BoxM*#&UudJK8Bz3?jkR4^Pfc%?;$#4{~O{WmXLmJplk8#IvC+b znC3lT?q)iXZlP1@RtRz%-1L2PCf%V0EyvKK0k%KWU%&=YZ3;-W^jG>D7)^?2wfei= z>aX+kOg;<=kbG-Uvj4P`=rCPFb@b@ zBrXO}oHT@gfPlv-K*M51`*HLdt&h;s_8~f}?{lqvRZ12B`t9m*)}u zdxg+m`PQ@>M?kyDN&9g~F$G3Gla>HHw?L0eY@IJdz$3Z`n*Sgb(*0=l09Db0XnO#y zAEJejqLV%fr+x%IAERsNar7Smg@bfEJ%K0uC5Xi%uv4gbg z9>l}cuhskCcnKPA7V{KV0h;*RvX4q}6G-CbfaIfT$2&uZsKb!rDPZjx=;v8F4gC8N zlxnb5zW{+~DUJ#>Ak;U(9nQrOfi`SvOL!B(QJEWF?qrc&UmY1egp+hs{2H!b zN1Xcxm4Uf(uyPuruAg2?v1-!f6suKPtm2Q+NUaYc z!&J>2qPG21pdZ$!xj{rhv!8*Rp96Zogpz)xIgqG@?)JFyl#Epq*u(i+{0TgY4Gffy z*wQK)tDvIde9Mgyt(EK<+syq7qe%w2%2JBE`C~aA+)F9j2C&} z(_`eT!rziATnzjjH$?AfmUizP^)y}Fo_CNow6=SyTif&T5rDho5CvKfQwLnxh8XDG$+fw}k zjh%UfLhb%){N6Z7o6Yo=w(X|B0atlC+jxmB2e7R;4t!%f-{$*uv=YA8!iuR})7v7-o(YG>U*wu%$Wd5b z#fK@JM~~wjynABxi{!=Q{Aa7HY+^0jm# zUx&ngJ!c7x6yLGgPQryls-n&S+>OBHl>-hDb1|4HIq(H zXs@*(?bC=hq4!s*5)pyMByA}=!A8EN6K(~Fq$;`3fovdnvy>lzc?2+zw&Mj>iHGRY zW;~VKRf?1(x=NXk2dhe1vY~_v8Qh_%Kc6kI{5~7>axZygy2N zP|V)V>CCbmWu+7`V*dM53cb%(=ybY_C!nxVw1=kfL@onRM782MR?pwqJuTJh_B0hBMAxkCivO>X?Iuu-$p^`};PW*4Eq1y;O9v;1zGWu*(! z(V`8tkjAS;RIO??o0emCy$T@N`V zI@?ydL5n;@H)`ut9i;bIrn#vddG%&nOp6blJ5>{T)haSo3l*z#X_{I`GgT|q;Z%*J zd4btXKzKTWxG1&3=G03|5YoR4NrYz|+lmdIky3|Z95}StM^7sz?-h1$ft#YP1k>-` zM`Npx(5-;dZG&|Ce(DwY-GvAT=Xl?KYCl4Ew4>dfL$t32E_c^KdjBAO;9i=KN!{J; zB_G7Qdm($5b-&NL-9JPhlF8zels8BZ*rY~ROaDXk;2=Gej{J*I^pLuk3RMS{s7@+Z zU38l220DAGR)uM$+DsRy2yIYN+NAm@rY@x&Do%UU7Ofk}>b2Cw)jU(1XKxC_d--&( z5i8SP9V?v}#v3ue)>{#I%16ISO$N_;1y$IO(Lo_YDrCrHb#w_x3!!O(r59+5=y1P0 zC$0Wrq-2>Mi(Hmnbrqs6CqwOox9x&l*4)X$+B65&K)Srx%X4`i#t`HR(=wW5{US-{ zmP2U7(qac<^Lf7HM07rb7r1mbN7ngFUg*-faAchqaji>d-^e;I<|R3uhp0fb_|`Hn z<+DNG{uGoP zI7AA*~-3{%jL!blI}{>Z2TdKCA^JPK2d1DJ#=x)S$; qxbMLI3EaDIN6C(v!vRcC72O0n@+V~7P(sY5UxH2y|3B3vD*IoKQTrkQ diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTask.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTask.class deleted file mode 100644 index 9aed806dfd4fd6f9a786737b18eb5918b6f62571..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 168 zcmYk0u@b>R6h-fgNKp9<8m}?q2aH0aVD!&=BFWmA-NcMvQ}_TMWo$alozvVi=ktBN z0W2}jk>?l@2EKEIbhe!z36p)(d83=sDq*@-D^EQiB-D#a9zs|(zF@hFxhmIksfCR# v)wO|5<3e{+&!ubCJF$WP?orkY*orH7cdFF18G(?*7J$;2fwU(K<0i;{+FmQP diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue$Entry.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue$Entry.class deleted file mode 100644 index 0d6c4aa723bdf3fc1e5821fecf425c51931e6d17..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1181 zcmb_bO>fgc5Ph33W8*fVH07IyQowNlAK-!-Dj)^aC<4(&fgU()+$DA6tSf&Y^*?bz zqJTIc&ip9E>^34%xdjfM-I>{W^WJ#(=dW+y0o=zO3kI&5C}LTenuTjvF|cZpbHl(* z1Gg9oDNlwB4?0mi@c04u_k=g(?}ps-!$8P1@%%_8*+|4*AX5=b9wy#~2>GGd?NaF) zgOLU!5oIX@YcrIZfeg}hhKJ6~nC*Ixn6;u^!BFl5QoP7UeGzwgKcvV)C-QmN<8h$; zaj=l?1qnm(sZ8TT276OV5w}8~B!a>lvr>7LiA=0&P*Q68(ddB3q8l+hpIP2CW_^=5 z)0h?Ks?*u$@3|LpIq+Wg_l2LXsl2%z3?xspn6%5kk4!~cnx2|U^M43lC&|`!l*PVy z7O2Trr+&OvaaR$xF$c$n3CqT5*aqr0Tr>>aW_U8=wVIldjU}98sGOV)!{Q_~QO=Ne z)b*L4Bs-BD&@~uG?H2|`CiqB!5Y?-_r@(Z+< z-7jEn;|Mvoc7*gu8CyP(rOvabo~4HfR-48?0yNM)3-(yDf<4bwZ-C`Wr5&bZAZ zs#jEO_Y?9B`o5y@f&AkhC0+Mc<1QVa`-5xaJau4|$AorajCPOCXkl*S2a2CjItjI< dp==Fnj-m3ns2g6wWxZa(BCU#Y5!Z1R)^Gb@CuINt diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueue.class deleted file mode 100644 index 1e0e3719edf023834ed907e33c4a76839bc3d202..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2236 zcmb_dT~iZR7=BK&VNJ3kjfhgMAXN!aj1@4|04g80&=63l7Ob_0a11NSZo1j1T=f_9 zt~YeXYj5gBXDHM<)9H1mGrjGfY3Xw|;lqyJKr@q*J@39B&-*^lIr;scUw#G9k3tM5 zaLvT^7(%!f#c|v)F%m-rquMg2m+?4m;$yvh66K2t9Wtr6pX#?+6H}3IwrU@3$Z4nP z7-n!wZ*Rvji#ZdYnYbem8FzedRlu5X9OaD`Wu>AjH2bo$!r#{HBl%dSs=i%J&3Lx! z+5YOSs;a7tKy=nFIMT0rO5nlU{06hGS4hdS%rC3dihQ^tQ~9E;9KVvvyH2HAQeMh- zeC0W^SV@hkqFhx=b1WWIIUAtM46|CppzYZHus}H3Mcl@yyTrOjvbLjg)zYH!=Hz0L zmPFRg%ibuhMHP<#0jh&_>&+W|7pOECW-n;c8u^#q#tXl0^qv3w2sJkIyY!!aN(Xuz-6OzQBD8 zNpxBGQos8E?G_&3oD%`hDd{gWaY;Vq5lK!4Qb_M}5X|FM9h~8veu8&B z53y#SzwdDu!fD>pLd%8g8CpgV<`dA=y$R9%3ZeN;gnxqZW8mGvwUHjcyD*sN2s6FU zomGp$S)AieP!OHsW3^kw|JC7*W3rv?n3>R?}A96gGShA11em-5s0MY@(ewI9qW_qWB_R>3G z({->R`d(y)5Oei1wvSEqGrt~H%`!mCWm@70|4o=C(w(}qk8q{lS%InbtUGFbHLk&I zM$?PhF2SzWakNcEe@62QdWC}~YV2HRi5p~UGyvN|5X`azu>)5*MG+Jy+l_q8V6a~F zTNc%X4b+Q%kKWe{e)kL~YnEe#kn81G+g_q2jOSZ_YEEl7S^`e8)W;OHkmDex+5XH9 lznw&=ho}1+*g-b12bn?Cef$p5BZOfp4vgV}ppsh=`xmj68eaea diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$1.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$1.class deleted file mode 100644 index a7659082c238d388c579c253731d6296518547e4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1945 zcmbVNTTc@~6#k|ylrD=D1iYc3Rw*sAARub-f&>zistF)IZMfQ{ReB7MjCf`rP$g zZ9!Tt)@B8URfXDDa?3nAjr zQ63MkFeK*eRnE{^GzEX^S4-Sk)=L(xbQf(yw^nq=l=E;gAtPi^9}B^qIZJn4PSfcK z-ebefj8njU2T9&+)`?QKSn>^ny97USL^9?Ao<;6%QeJsMM%3nnGlt>jjvYEkS8t02 z-L!m%VjsvB*Yz!3vvg6`7JcEFRX+dP;G3Rli-3fYRsT^&484sY&r0ju@PebMC9^Da zFCf}(fCx>RyBr~%P1v>8?u^RtNfqw_iEJr8<+m|WNUTiT5FT*^lFyN z!DE(|No!nVXj$bYzs!(pER7}4A%PVdDq*?e*stWPLm9y!J<;tTVIh`IOUVJ$lm6(H zPtd;$-L!j}M$ZvLochjOeg`Z!vV&OeE8?GMDUKd$Tgec3i(YE`$e2v2KTEwFNl1os zG*$z<`W};#3@Kg^l04Uu`wW(+Cbo;jK7kT!ZWF+}29l0Fk`DCa;sKKS9>XOJgd{5= ziMorH?$-P_B;z>D@t$(>p@Bo)<4`GMmk)4+d+AvUR}#2NATeZcE$FXfi24M5WruNwMhej>)YI<|KKcPH diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$2.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$2.class deleted file mode 100644 index 7d227e0138edd748f17951c5857e2de84c412378..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2133 zcmcguT~8B16g^WarR!2EDky%TR_T{40>;E5kq}8VMF=3z4%=y2*v{77EeQ|)C=*cQ zgYm&9f0XggvQUa!u#(uMJF~NQ&N=tay?6KLudm+#Ok+BU1Rko(n1()NHDr)WAg|F% zA%S87j~E7bxSjExhGi~u*SCDjkqp_gUmQJJMKdvn(<6`kR)Q znP7MlvB$QA^gY@uy=Gmwh9!OBN^X0`g7693RRA-VwES6ycTucoBSg?fO3ke^#OIxg zU>GP{QoL!_H-x*!H*8uNEITH**STw{`*1O?Tw~B*Nh#cUn|q$1+outPXR6FhlBYxM zB)g46qNF2No2Dr|g0Gzth53MIMR*Mgs3>(MWJSDh3eR6?`b}4qR51G75Qrhy8R`Tl zomVx0U^ovy!(wO3h;(UFKU3b~JKV6jtQloT)=IM!+~lfNlUzB+u=_t^7@*?9a9mmQK429YYwVmeSEYB1J-7nMsBh z-Rx^ituS5;Q{6y>)C^ZWAuDCe^OvesPxv7;Lp3TPBfHe=Q%W(y={E6H1wC1nFd7e< zNoI0_`tpw`?lT)!RL({U{`<`8CZg?r<<>Du@2oz0tHsi3l~e$Ylwq{eL;rrHK)Fb> zw}=HR1Np)}SiZQASpGA5J_d$qy7rPG*fU(B`4Aa{whiJ6jcTSK=@@CQ#?~`x-WVop zMPVZzP^9vE#IT20q4)*y7K5fx*>m)>*??vA!1{S&iT( zJ-bEEVz`~2Ba1tW2j-2DG=lJZie}sDUL92fE&IAYv!s*N0<=gi=KmGIl4*;&hH+9&H2P3!| zT<*qoq#_7`%TL;Xkn4nmWDHD3VBv&3OzFrP$QiIBXh2>^A%Yf3bP|pM*MO&^7{Nwx z1Jedh={OxhyFATE@`nsOY~T?Cj~aN)z~l1n2?I|WIAh=`15X=x#=x^Wo{JzR*|y_( z120Il7Y)2*;H-g{4ZI@tc~uI1&A{tA-qi6HgPs$fW2Ib%uED&MN%8_uO^W0cpP1sw zR6du>+sObWIpm}EF*NvBhn>PO&k2Us!4rI%C$rqnBu9v!ogpN1e8v;*A>k3jG(+c_ z{6~cC<{hGJbVMreq?c&T3CU%Ud%{l5Ff=WP5AqB{*qgN69lIHNt4PUO!uDLU#&(N2 z;Uq2F6OPTZZqipyiVXK#w&m?-I8#l}wK-G~Titn;p7L*so|R1w@j_qM7(?h_K22_I z8nkRNT+EFNXOxd;m80`1o*m9dGM6#}FCG7oC(iWXaSrl|bPnWGS1t z5eG~h#5*Rg!L_8>$^t)7w6ba8n0OcOQGDYQvO?oIKl|c-X1FHa#|I|%fxO$G^z-pV zTE~YbKElT)`f(k@w$*x%pT~ZeY4l~d)W|bqmv@ar-efqAGKATMDr6-B`B(p zMLJp#PSg=6i^BD!L7z~kTTNJ?b<*`X^@|&)Gvp-`pW-uWNy}vCIHkIhO*<=B%&kZE zy2TDj?i+k(1uZKbcc}HPVQ&dn8SGL6wW;aG)@SPNihZfAmSSXLzl=?gly00n84o<^ z`#)aY|3UIaEjub`nv}fHaB%>Mmvjy6+VlT>O4wzVHTVI^`ZK}A^-|TAvP)md?&Zuq zD<`5JpAvR-BJV`GpCu&?6BQg(sJkQw+C^7V+cOpP5nls8UY(L~DK8bMB+Ff94ax0Z zU!V@=i?&MR&l&1V-pulhi5sN-ixH$Nszpkq;XrqlrgSTVLenqj%0gLEtrRll6f z9nbQtyshwQyhKk~b~=AbPFmS+NaMytSwWllLRRya_==%9QBqBrEUW!%hQpO6UeM$r z{!7j1w(%1p<&|Ce4-kf(mGa6Pxc}Z@sOgk_G|USHA!{aHrPX#-bl;%=QDZZ_d`aX* zot21$v1=_&{4Ph6GwiB25JP`eCD!7z_<}l8g}lOqzSVBZ-Xb+TO-{LU^?A6|qWh{0 z^KzFD3GNmh`F5d!%2(@^U6w3t(j|x z(u?_Odgtr(I;WvWyB0aR)JjuIt<(|KO6^9iRCYjfvmBRn*1%}Q+(hfmbZ90{wE)Y; zx@W;+^Uz{Fv#5zJAar~lwedNGTXp)(!I(u|tfw_Hhx%EVvF~6mpy7CHBkk&1o7zJ2 zXkI`I5w*@?!y@=11->`{ZzS-h<>1W<+!lb_34DnJ_q1+NVzv@^s}ggmLcFX}j1P_~ z@a2_!wkbZ_3A|nL*`W|SD*5=}oeI1w054P52X`uP*P@sO#7O1tdGz>Nle@?{3BpP! zX-S2@dj-31DGD_>MBhUcJ+NO<$9_X2`yCzZ4mEvpZkZcb__?t!M3&aHh30VO zzS_3hA0g91o$KX!%X4$Ms)Dw+k~Z8HE~BOMR!J*e7E&%tpq}*WK)bdR9ojC$v`+MD zUFg?(Fs8-9wFD-#Bu;9(aa!A>Tyw-%zT_IrV2m8Xq(Qh7cS%Rk?l|pg=*RQ{+)dCB zI>-UCq-w#jSw1rmR*o|RO5bt!5N!*z^XS!aua5ivMjw5N<^2TEh}kW)Mu=oD-oOB~ F{{Zhxe}n)4 diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$PartitionRecords.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher$PartitionRecords.class deleted file mode 100644 index a7e366ca1148f4bbbf04d09e662d97cf18994bb1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1191 zcmb_b-EI;=6#h=xLV>ldwW9T}RjVS2?uA~-h8qlt?WQ%Ba6fDZn0DDEvxCt`@Nv8# zCSLdezJ-6UVLSu0QOb?gY%=rBIcLxL&fn*sA3p(jiU$Q`aVw8SY}#YX9=CJ2o5MDt z*pVje?RPqnf>+}hesj0l zY*lOg5wgCDm8lW@^3EJjbJ@-?s6K2FoO*J|>{bF5%e`U0Ep=10BbKZMNhqS0(8|82 zLdW!!P4%<_6J{)OU&OK0^+=?thHu!{c{D<%vf+&2yk|uQYkZch5q*I} zUy-M8Sf=k-rysb)x(XcY0PDEIw?&o!jgh6086!)=!Bxg34n48o;M<}i({iQ+ud)4v V>UG>;?E-G{A7DSk9PY4n;TM~}NPz$V diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/Fetcher.class deleted file mode 100644 index 6073f21311a0b9df5bc69e01fb3156d81e610364..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 23886 zcmd6P34B!5_5V5dC3%y{V+$lWEE+b2gb+3t!lGf5UEJ@w~>CgY~Up^n+a^GFf zJ?DFtd+wcl^WBFZC8D`jv!7~tUm-3%^7OEu?&Ds0eMBx_m&>Dac}y;k%jF5Vd_yiz z%H=7!JZlY<{8~EwjXeEUF2D2f z?~C|#{)3;^^B<+;Pd2~drQ_ zDYsRHpDs|-{i;&Ukoi^lX^NUDud{q=woGx3U(Hn~`_w7&a;mLP^V2129+;=9Z8hId zm#GEvWg(cR7TM}_KV8MWQgVi^YW(yeVd96>Vp}aKr0dkuLbXgS_v3P=ysofStxv7= z(?^9ZXGyaOb1t+opL)%j|h6t+vx+iex` z(*r{O144dOy6OOa)dfBkE2KSYsq_~wM9&HN)+uEd`qd7#(^k9u^eBi@J4AXHiF_{> zu3zG(CyZn+^+SFiD5PR_nY6pyudYy6LT%JlGW8Go)YU@7hvelNpSspo*ZJvr?)9k; zMaqzPxc+)5fOSRd}mewT6jTG7VW0jl>hdNMc>Ey)(o#x~`_l zySCLH-NIB{7maPHinq7TJ-@0hx&_3{XDVoE4~HTNrX~7oup`*AHB_}N*tRWLWj@8L zS};!M_E4-U97%*?kzjkgN`{uA`cPs=G`4Mt?gZWCCBoZ5%4CnuqT9DeBUPP=Fq$`^ z7Ha#pha-!3C1n0w%T%~MxO06loLGhT!Zu-AWB4NUHZ%sN?OGgc*%obU(;q=&%eEyk z;UjO5enN?0YcLUHn&dIECN5qD*6T_b*p7!H@n{UqYjfB4n6{Q=C^p$cqUg1-F}x)b zOmxOT^V$DNzYChqn%}f;{-PP){SS$EZjQIa!W{w=(0=-$#%RoH+?WU^L;!Xy1-9XJ z=LatgYVlOnh2sf)+|;jV!Jvjqv8q3=Q*)fZ2ty3p7TUEm6c5G1!FGr`#x&Ps5QCgl zt)R@lD!Uarn)e?x6H0`L5f_U7WOw+V_6O0Szk&tfNI0>G`KqZNRJmaE0GGE;)z;Q- z@aX2KJb91W^w%AaZgbCQAmluno0&j^#V(4H%6-1UUVAXIrK%Anku7eC*pI4J!H)Sk zd&;sfE2pi4Nm>$Zg+dOi3r9lro!d8uVokx#?I?jgYzeln3j%z2cMJ0pTf?wkiG5BgK3&a58hJ+ZKl`hu1r!d;(oW$AQ)%D% z?CMC(X`u%xN1aSsPlXv7+kl}K45V#f2^4`J`u~W3qud@7Kbqn!5CiOmuK!j|I7V^g zWJP~{DH7{t5Dl13_L8CD;ga^wIO2i%qGb7P;;xwxSRlse8%3DfvCsvbp?D%*rMqr0 zZ$PbIw_Fy1hetQ=y`(SVChKBRuNE^V(}bJ@rO0Ax_3FjTix~>*VJEx_9jY(oAjHTQBI1uFBz|UU z^BCh83H1`I+tkGJX*LVOvm-&5l$=&aJ7|d@y@oKV)MutJwgSC3u*OH40;vRBeHKXr zrj1kkW9Pjclc@**3?){#wc*NCam+d>Kra;WK^EJgyoG6v1gPVQ7&LzijM=HjARaR~ zm>o(W%+U?O?*goIP-3gQpziJ9Ws1h>$4ukOX;>i^w7HO8bqu6r7I%Fhoz3*RDB>^y z#87RtL#^qx>WHZW_1x#ixUE>-8Z7H@Mt1@|5vDOM9~5QUpkn=%MvzMYNU~dFK}2{M ztS(D&asjcwnxtZH(_cFFk-65tClsc;WV$+wJ{?Zwa+&=sbXK$f^|YXJ>Rcqibat$a z>6-rX%dNftPpM)m!P2}{=9f|p*R;0{0{VlJSRb?tkX(0gq_wD9C9apseFzoh(DpnH zQ9wNmi_UbC2U%VQ5gsYsc&ScmT%0BPK_Mm=vU1a(Q7su-OtbnN)6V}B=6jl&%9kvk@Gj3 z9fg9;oo#KBQg)LGk#Mw1HX9b}Vl_;lg zVPw!x%MsKkqKRO;yY*pSmP9*c8)oJpyT&~7u#CYfq}rq9ky)1&lU~|aJ&tB{`p1CB z3RO>b$nx(1wyLo+h>dG(3EJvzTipY>YJMC3BW(79u|emtlcVLD2A&s0kIXW#+lM_1 zS8^5}`{rs_Z$3ZNB54@E-c)ewpDbLVWu1Y8)lRR0)McjB@~ZOo;h(89bno-*YBm;AUuwP!mqvFv_joJyi)XrIQnGF`ZOA$HP3c%+=kn|5Q_s!VfkCF)IR_|QfRIpcGm$k_?C^pGXH^>|)~jq6i0eGc77eTh zW;RAUV=bZOVc8jRTxrgbu)^W(9C7GR^oGM%@dq6)!pR4m2RSIpPxMwFwL2)V&V9 zLkAstlU{dt7hmM4&#Nw5eZf&*R9`}N)=^(pUvboZ>Z=GRGc+M?GFHz~Uz1Oj14%jp zJ{@(xG#Zzykwj^>dcaW+s)uZ~$5DG#63c^u)sH^ySu)9PWXT-0PxUaJIIt5hc-rb= zNA;>l9QAd%JczJ~(XsirI*c?nKyZF+wM6@ii zHB{Du5yFW?s5Mpb;!ADy4M#nxo^sUF>KRAvSI=T+HN##vZD_0K9Q949@GbRic)MJ* zr{eJRroF2Mw)&2vzN@|`Jb2zwFQ|tc^`d&oQQuczVVX6F?u;vP_#TnlNdp^ts;;Ac zpnfQ0zwGb?^&?xo;;0|1E+n}0OtZG@9Q70RQ%C(wz3Ql+3*LUAerc;;IqEg_Ye)S? z{T9hRNBxfLko?X$xW}5)1;3+yFHKKBdei!-35=FUW9h8IvYjmK9+?sJGPHjyeDiLb&fRoiUJk{Y+nW(n1HkeIs1y=b)_)IXq7N2wYU}Iy_6{e^{`y zJk4jZDBxLVWLqSXj*81aA*1`X z?d_p0!FFt;CPHcAah|OSM={FcTce%rVv!PMTZ0`Pp~wtmw2T$Z@;(E^?pVdvNXHr_ zgO*ssm^RnIUv2M5hzrvoa}~g7aUUnw2Dm`WTRTHiIl4I>Z4V_vW$hYZ@vY&GGD+yh zw(B+Tbl?$rBtUkCW0gYDty|H2hFj@tEZS24-`j# zn~;gf!Or&3Kuc*Iueu!T1gp%kCcxNO6Rk-|IAneiMDl;+^5>sG{-8pDr{@IV^yj*nHrY_VK*WBR#zad(-*6-Bb?ZZC?lO&C=0hr zu`3eUVwKm;A+Z##Y2?e=Jj)ZC7}&PpecrOF9BZbSpeHS?A7&iu)C{B%8V#;NKXYi! z7M{J_BE!(j*+_JT=L%~T=oG%Qu0%s?46U;q)2_*HbW1Ye`V8QHI zr&_1=iD_$MvEVMT)t(VfN|=4t#14Ok5h5S9<~r6qxKqfn+Og(a3mj`9y>45J9KO~% z-LcNFY8-2^DAp{Ef{Z&6qOq@yg0dKnZ-qob5K3f)l@4zaJy{~HXRmdQzcynA2wJEq zuISB_%543FWyshLb2w~qXSf~4A7`M}MnXF~U<3gw(_@QZ#j%!J%jB{g>k#3s_yTd} zsi7R}OwqCxGS6DMtdz@H;!v-(>TGM3W7Ug)yvthcSZl1aZL7ht8ZAiS9+bety(yep zYg_9aYrXY8Bn2`WRf|J1!GvrF%o~WJ-1_V4Rq9m2F{~9#pr<3V)Mhf%l!0R!!Kb~j zoHt0prf-rAfkdv>CIsr6B4fJpJzinM=MU~wpsa*_Ik;}NaJ}LqxZ1JKzHygC!^3GZHBQx94=|fvZ z!B(BW2;mGrBHF=?+lOp1&(e$TcHuzQ<|co@=~`rQ!tol{OAP8j3m{z-j_X;Pjj)m7 zc$PJ(4~1IAw-%f%fcKpxr#!*qEp72MWf_HFJ6r8L;i)_V#cb zN9rT;`;B1;7~_e$D3&FyqBu-tZj#dLe56(_Y*u9a#*!#vqZaq9f1$*#t}ic6nVt0N z5!fG@eQ-}er2*wzwXCUTX-yNhck{58Z^zz+*Ti#!tPfUomxb|8q=^k69JqFUgl}=Q z6-I8>5$2b+SqU6Wj?1C$PMs(}!rrZS6A|mABMg+<@x>A&m(biQI3T5?3&?70dX1QN zi*aO3G?Mip*o>Js5eF<)MpByH@C_YuobR-3xaJ1uK)h-lY!HuUbg(d#Nz)vk61i4_ z25c4K_*r_b1o2PGHCF@BweqtYs8qZ?6zTvLaJCL?#K}iQW|>$a=!K0OIz4c_{vgWI z@6-?=)R+2I6S!}FumsR&vn<75(3ckp$~nirZr1${ZY831(H-~=RZxzQjtoksAhq6E ztIyx!H&p4s(b&c9hz(NK%%@cf>m5YL9s~>&kg&{xLr2avyS^*qY%`Y*1q$FNa2^(f zxGl8GW;A=gjwa&zXHKwBwZe;9&z@|(hqSs)G(B}NQiH)YsrfB1H}FMyXb(E`r>@lZ z2|4AIVikU?^uH3j!ADwcB$|YnEleB$!wJdEB9v*Nr{4AlPHqQZVAkLWugyb!G*OdE z?g(RQvy>8o)%MD%t4;ret`TQhy(XKRf}_R={hb|nZ^KP?LFv`6q;xcb#i8|#+UEM3;Harx?ojcaNe znrfSBSJyL*SzCWr{p$7g8=F?Isa>+M@5j6y!B_+$LjJ}aubMu!(JSc8MduOXhJI|& z;H&Zkyy@Ic+#&AIHslop6*gye1bi?@tFUK_pO!+hx<}|in=Ea`iKMj>LMb#V1Fy!k z!V8yXnErV7K%c4T1}v%5^C^ov5-GY?EUR50y2lzOBbLwBWD7(`O|A9POOiaD15r)AD1Nl{w+vPY<#jOFFm)i86p$H`N8PRLPC&ET3$?WEw(%`eMGdkC#o2LmJB zACj+LE59k<*ich{Cd{<$+DiacC)fd*N{=qGqY?uJH|vJUjOz=h{-1q`s40R^*(|5C zj90}#_-S{Bg`)b7HUf3vOQZ!78t!VKTOdFFo_0hf(h@h20bMzhUuIbBVoH6W9#X;2 zwy{tQ<83rX(7}B_q=8k5sLrWo;< zcbxUWIEV18=w0-OlPFRy=SouLY~dXTQPS@Pc*mKP^t%rfGtRa6NpTSoUq&`ch=S$a zlvf@YvX}D9EA~=BdBC@q?D9b2Uh%4bm(3xc>sf0?BKeHhT*!2@@O;psReJX zR6-#dOKns}TWAVxr3wnuEZRn=QadfA2(6$F+CUeeRg8zA^)0%OAP;5f3*>9&3O|B! zHU|unCt<9mpM0G>>H#GiV}@!#MEbS=!EC*xZYse?4!<|zkgF{+=)C%O1L850%3 zTwl?5hL@&hIHexb@>M43#NCH~<366Wn+m!PzwEwE-3>@qcGI*jTIv>+@28ROV}<)^ z+JmHur&p#b^iX9U>6cVVl4c~Ss_XFO84gv`P>}WssPHFgCVh(L&`m(^&9of&JDYB$ zX1Wb>zXKTk3`KEI(4BM*-9;a!&uI<{x5m(Rp32iSN3Kuv?RqZfiY&f?0co_zx9RvM zd@H4bgJgqm2dTp5%HlBx2rN`CYG$BT70+~8`7bb1F!^jR&4e<}62|5C&}^JF5onml z(!9_KH{MS7qM#s2bE*ph1<%k8y)?IZAD!Gyr&QbWB~b7nof@zKpK70e_a*7H050=( zQ$sIRH_P+<$7w;57IxF3>cT)_l1@+38CrF>Lhv6NUf8+{)= z&hvQzq&<@MGPV&h1EF1zVES03?TfS&+hFRSPL`JT94a_OBW*q-?=Wm$q0KdU@?!I1 zo0r(U)aGS?0>MU2J3#z44LJZJjR9E>5a5g(03lYr6i^U@wh`>~RV3-mRns4&6`v>n zC|bF=i^k%uPQR_f+mK#b-CVq;o6derbflqTdXgHG)MQkoVnR2qJvHC$=)~PH`O;@; zK9nV+DKoTP`~k>$8SdmqG@4$anlLwD9pST zT(W3l`UzyG`MOcgCBbwx?!{6Q|$qLxZ!W0>+ahaJ>1EmJmU1mzv1(Kgo) zIY@ocmqFQHRnIJ@o!8hx2609G`2<@g6!aNhjS(qXaPQqWh=~k=`T6-ug{hXhqZT?`A+PmmDp=Ud)oQMCL_fbTHxa)AWnG!aC zD>7!K0q`TZem|V^Q1$^%g)|3|Q4MrxHC$u^f{TqjoGw5Ba5)##)i4j&^C-FvyzheZ z{x0113p|c~%g58}d;_3f+wLC@srsW3oVd|fguK)gY{gjNSkZ^Zo!V!mj;PEX5b4iZ3?B)djtDNwbnLwZ##5tEI>57bO zwCjZ%^m8TRn;A5Xt7ryX-8`N}OL;b}fPr4abLkwI=Z(;oHa-oZ;XH&c)oJ6>Mkl$Z zI)*@TS^W$2C!yqPFsxVZrV*9BbQK)*2a|Ml7d_ugA8N)2_|a=b z1WCFU%+vaD-5&bzDfuP&57PBX`p9Q#egCgFi2Lp#(S}dr_lYKG*ljM)FX|N7N17tn=Gm<$=^peC+U_h znk>b;E8K!xlXP1T-Hz^h=?={2Gnh}t^cm$ki)>i!60S!axtdOf@jZjjrbcd{^SBX& zH_=XBOV{#x`UDi?9&V;D@dkL#_tO)69zDk!=|$XM=1ue~oT8ODy8)Htw&bh5cGu*spPdJGDSx(}vd4swW=W^n0N;jnYy) zG{lBl4X%eSp@NckslnzQSXFI0KoupWZ^OA8AG{Ot#wmOwb+rUUiPR6dR&|vau_WE8 zLnH~?btHc|9&~&(r8=)dZgl^)abZdgukCgyWLk8 z;FF+1%%=<`o!z6wA1Daq_0qk~r4ri}b<^iv8+-Utw{1PFduoj#7<6e{Do9>zg! z3+M{W!{A{SZPCh~4^@t8=b1+@(uuA?c!P@cy2eHfIGZo#GL4o)X|x<-y{-Y){>7J| zXGM2|!b|aF&>|+@jQRnLYWlb=-G|`Dw7fjs@(LG0)5$(WCB|YO(m_4KyMqMNeePQ{ z8P@s`jWS;kAemr*btNE$-=mnoCBG2B8i^dh?XbfzCkt`=LXy6iq%S4u%Zo%szf$mZ zy06)i4~@-vdIO&-xKb`#COq2;yilkc`)yV;un+jlCGSj z?-&07FG>2LxRtI$5-(y}FOR>*oy;EkQAu9r53A9#`GBFMTPe&8eU12@nF z{wSRI4ba$+LH9pSOZXGCoIeSl@F}{8Z^WN_{WN`&Z-UO>Ob_xcu8$Znd=3b@v5jC72Rg?HJRmG31h5Ur7<0n-IKds{Y zjJk~XtB>$=>NftSwmV5GM%HQ?UoAN%1Ri7fLwpUAZJo4`ujT6y?!N@u@FNHW6|d3( z9RfNCHeUiS{y{?!UVfWD0zM2yQ28`M#gF2zACIF)_+#*E!@-3==&XScym%GX>*H`G z7xA0gQrWab)q-o1+gh%Kcaqy`RP(TE{gl`sb$iN=;ps+DgfJ6Nt;pk;Wdp)Ym(EXP z)oVuGfKkl|H^IJ`K2l-k%{2TSsyPfFi&R97%{SY83&R2K)~nHXX_d{lB0FL8ZHH(M zma7}b9)?pySlX{{x|-BM``c~418&q^)6RZ}&N)m|34a_ZS39Dy{^{f1qDiomZ_#m> z4vz2|f6HgWx>b^Tl3v;{UHEz}xw{>9tJUIuTS5Kab`wK(sH>Nd7*o z;|~D&AJSre87}UXbQl9#Q-Z>iS?Efc07=2b2em7+XdJzdKMSbj(>$am?&8nExNQ@;r6pJQJr+?a$j7Z^M&C) zXx5k9#qu&>O){x=dByZ@vP8X=P9kDmiRaVxoJ?}ih9a7ztPOS4-~%`A6Y!zSukzR2 zF30JXNb$gI3I8~|Kh(URA8>1~0lf22!-v6<=R>*I=(2V~o%HBFj4r7IDZ3AJNCfYL z`k#u4_+Sk7xV=ws=_o9&-ABKEh?I1nP0SKBFn)M1C-F^jH~kj(m4I6@vKqfZ*XQE? qJ81f4D3^Z|%?*scn@=bxMCSSu4pRV`sPX!~2Y(k@-minEH zF5s3nwl*JWvr?k@xP;rdQ^2alWc@P=_(Y(-)!yj#9=11n&cRph?t#;GobA2cgYDg3 zySu;95fIw~mo|MbjHDOs%h5y``nEtJa>vS_L`>NcxIB_!w54Pa^_7eqV5_F&{HFAD7e8<>n$V7p$V0#A-YJ0-g= zV0iwwY%^qT8YYRQm`gR7GS8e;!l6GI9cCgr_c0-yNDhu3$>W5Xv9wHa=Co8je^;HJ zh38>SA3Cl#P@SxV0;OSpTeCfpBY}l*Dp+qSdi(U)4HPxL5H}R3%mrG^MA#&cD2_h~ z2I>n}t2dJ}zV%d|$TAkNRz?+<%b3?@R+~9(-ouRo?v`;6p9-x0FI_olUBn(q?}@$F zf2szYGzH;!XI>pVlF^Xiu{=$tzk?ON*Nc2)7OtX zx~z!3x9A!4-^M#hnfu)9{Jbt{_uv2uZTnst4fpYb~i>LH!nJc{Ert5LI_ z{Q#@@yk<3&MxXMn#p7J%oYq+FE_J)dZr9^#t$3Erlz%hfKf}ysJhLmj_dY#~QwHmc zztJ#FQd&wSdrV?lvPDhI;Ad%KniuJuuiobw^%m3Wi?6G9^?_DzIaTjA$!hhAFIzsV i;ATd_Vya;4{CARh(bG5gC41b;@H+3!eZcZLeE2U;SOAj% diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/NoAvailableBrokersException.class deleted file mode 100644 index 41f6f93880d481a63734758ebd287ad4e833a217..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 497 zcmb_ZJx>Bb5Pbt2L@2&a6GC}scSxvo-B=S^ePi_?Pekc}k94HzI?JoPl~(rj<{WIjmvS)5Ef z_yhb=#_HLRVBCNkIZW47Ro8pf)vtg4`u+pJd%Ucpif1)EuVKB07lgT%aVFm;R5#w9 z5N6u`R1=mu#_9c{AM5m3#+Dp9-znkKl6vho+n+ON diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFuture$1.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/RequestFuture$1.class deleted file mode 100644 index da45aadb9a31bf32b177f374499adcc30a6b495c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1936 zcmbtVTTc@~6#k|yER?NO6uhB$soIvyf_DT7NFXt(mjJ<}Z`18q2DYr&C!{T zbyK}=wCM(2wHeyB@QGwJU)tsS+;X_8Sx+mw!VaPW`e_WD4NmM@s0$J1TmC5+^ziLpG!&lfuJ*9 zq->X{g!HBxdbU`XQER&9d`8zz9b8kk^qj5W5^ypcZQYNfbF^&=3`?p7`9t)xIXplO%Io|eZvaz+en>D z=Jm8K$F8Q81;ew>A?$!u1SA3&NWFw!)W^uLg# z-!QTib)2j+5=fCgS)4ooD^49CQJg+NviKFLPf_#;$$p9k-r@wwG1@`#^c^FuEse`? zifl8AYx@|D`x%PXJe+v-zD)5m5|bp7U(olLARRg15y1NnlFT2H49?%Aw{M($ z=LtT63uH~uK{U&I{tw#z!?YiW_TztOFX3{Fwpm-7Hr1lLf?Py<71v0oa2ytcftUDt6#(ljMaL6+Q#K<{?a$hXvuY=iB(bxIpH(mLK))~c(O zNuZ?{dV#i-TU+R-cHl!lzzi)z9D--K3^VNve}jL)Fu-&6U9DtE$;1-&Xy13=%Q?@v zAL$?e`O}{P+=pMr@I}}?7{Wyz8!>dKm%OZO%A=seiE%8-fbK zaaqTAdNF|S%Hw(2dqKzdz}} z4aRJtV7ZfdvsAK58oKP#lvA)YXlFG+&){k zG@fzX3n{Z`X4kFMhPk$3rm}h4DtM(-)+v|658?8cl~ z3?%GYGwr-=^xCCG%Tv;8xMLT^CMNC9I-AA3vmVXQ^RBv zB75|_F&#e)ILuJBe!%paxm?9FHOw>5;}jJlVbh zLYQqBVmy#jmvrOD#OjTcLF>wuoWJH50}bBza&w(jD_mn9YMpf@ntHb!I~V4c6U?P8 z7c@#U*ZS@88i!aRCg0Z<+QHM5;%!;SadS5N1IL|ryz~~0tg7l$*D*jF8mTCGFmIVT ztB&7b!I^NK4R#teAoi6!GjGjV9#es5YKS)nRimr0gvOSfvYWMLSR>gO?P@jd6(2Hi zFH#2X!Z8iE`cCCHHfh&2FH6QZki@8gyYVFhBS>gC-slOcw)tgDx|yH`hy9a*$Am=& zDIISLmA4E`U{c5127ZKhSZ)mb7!Mlw2_DjrZlr&cxw2BA4Ez+Q4Ezj}27ZorH9Xuh zNp?;x-#vC*bo-uxIm{b)R3N>N(*{;?*1#`tTEpqR(4KcfhX@=bR4^;UUMR7Fsyb5* zk%4MXQSL*Bsd~gLE186chA%fVYueptNHq=AUG9=D+93p{ioPmCF(-Q)JLEfDY3Y0q zt>BRgxo(!`txG)JS@4G*rw}GD&cimQgKf-p+C4U*J8e{&HX7_oX>L79`fcRV$%Xl) zC+4OX&rL2YEKbeNr_We1tLAZVo?`e`h`1}fYSNSbb3L=RebgW)Q=M_Xh zfu646GSmQCRs{4IKT$*mPl@>BH)QnLbr{X>(s&&BNO}_~Ca%~mnFYy=!eQb&jv6YGA zT`K1N5Ul(}r*fstM4if6l()*B@>bbf;GW7I9fyoq;J9Ds;y17u0DqM%#n1o64#m(y=~bX#Bx-0y=;tER%jWfAf2S@FKn+S5GNu za%v?!^(0TJka~(mfjv@y>=}-JOb5s^$pjreSr6!UWcCU6;nNxo^+SRP+|ltgVN}CO ztvtXL#qyhE`3#;7SV~w)ed!Nao=Iri_(Gzdfgc5PjP?PGbTs1p+OFeo-j}gfCpcRzyMgfFeMRl>-;FaklBU&N|p@1V4)l zNF4Y9{3yh%6KJ3nhgkTqGdnZ;-uQXe-+z4l2H+uf9ay;K;5IgD*sNiT!S-DycVn)G?yLT>h_p99muKXwcjZuV^I{A^i)iP^^NWrP&_o4X z#s%n`*`5z_rR7LG{}73>mWe9CR&>v(>-8B;&fwFy92TFAp$r$pld`%ny!}r~oa*lT z!Q&YV{p1jo<<^&=K8FjH}wz9_%%?4UBh+*JGev2l)Xv}%YRQb!=t%T z8LBCl8NI^BKLy|GSzD;k@5MY7&@tGw)@Zj+mXWRA{Q~x>NH5ZA(!e#;X>Cjj;NU{B zn~huqEQU*1qMBvOF5ohGE4V_Qfgg~qkhSmM`;3KCe23z|8Z~mW;Mkm0O*stzRbn?i h*l6N9RX50Q7N=~gBHn542E``KtG~T9LN&dt{{(ebX+TA2YB|oah^Du72UsUT1Vq}a920&)pq#pHWHXNE_qG0v3cC-oL&=!Sf_sc z%!Eix{-R$~;LiC@s@v32^{~uxDXH+LSfDI2tG|E`@kV9jJ(^?tpgi5zRxM=m;r-W9|x?3S1 zq<^QxSl*1K@B^((6bj#(FrFwUw2726G6=;$nNdgTK#fkHeDx4%YYqvuP&qAwp>iP$ zm)#Qroe}2R42YzOhBAm%ksT#Lgi=!*9UT$!+dD%-p>0Qsu+Y;+_2bF4au@PCV5Q!( zK4T@rx&9Ul(VY&tg(lm~DBi%v&Q~3_MOgd8;@-V{lvT`Oo^brncB?3%OxXX8v6G24 zLOEw$D9%-sx&E0GHhKKX5OP>g7`)p+1v4OcsXEIiJT%O6USLJ=O1u_#r=aKbN{w?V t6&qOOeKos)dWx%H>F4nl!;+YNL+)yd{0j;Rg8zzhhFIllnX4RDz5wuQf?5Co diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/StaleMetadataException.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/StaleMetadataException.class deleted file mode 100644 index de35577b19f153c6231f18c1dde8d56d1261fdec..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 482 zcmbtRu}(rk5S#@bLRhk)XV^FEY7PhbET}=`OyB9qV~%|Nn(A4UNzV!Ofh&BfZ_FZZ z|L?Yw7oKwiYp4^h{+~)XTQzZc?yS<@J0Dcf&N(yVm3eW1lI&e}h&@!0$XyjyLQN`K n3@ErRidu3YIerJdEn6CbRgr-mGzA;WI>827!sHJHOW67W;0At{ diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState$TopicPartitionState.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState$TopicPartitionState.class deleted file mode 100644 index 32377e1f4d9c842cf01c11549019012e87c56fe8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4139 zcmcImZBr9h6n<{riOX9+gjgk2ln`HniXxzBrJzt0G#0CUU6xH)3A>r>M(KZOe?a{M z{n8KZj1HZ4rv2U@)#*7mn`9ey%2Fr8%(;7R?mf>r=Q;Nr{`&j3KLAXjlEe_6uqbqq zc*^2a78|_xjK4l(@toBcEIw!P1@C>yUth8Kn$>TTH~}?@O=xj!C7>sfLNS3-97Y^3 z1rnNN+O=JMOQ3J{rTR+EmsPWrU$xBAoIt#&JK8q?*ri#!yN2V)gWIaTp_YxUb;~v! z!!iZL3xQZgt=as0xB6P8eY5mLw{?f~Dz62nsyce3St6Ivd6_rfqchusJn%ByP zZaQ|}^`@F%D;DXFG|;$6{={w>rs2#BgtHSH0+9u4OBd)}HB9|cZFf_zK2{Y zorh}0&A>pmonpQcmAuhaqKPU&;Fw`AG4J(UQmTpEq^fDUZI4Y(P73HjTt8}hCfa=y z2<;JA1NjOgKkEdtOk8O1N(<{MV--7r)gB}h(1`FFqc)T&uX3H0jZXOE<@Saba@woh z^eQu~eBTLY-s3?WbL*3;guH8NvS_~Rgs&6-(^-#mvdWsSO7=U$G1XWTre0SCt5((Y zC4-yg(1G@KnFoP_>o}{>I3jTxrxaxHp@LEVx`2xcKH#rmjIhWnxPfT}m-*`?@14Xo z1!-2(te(NyICd12u`4jznlsDgvR+clvS}{vX}WBo0*ec(Xxi9?CT}YtYnv~1 zjb>&~Yl~+wjbe(nrmq!gfKjJ!ZzBT7y}g}ACXL@YZZ5aq%U>dWi2j2B;s`O>^c7-8 ze5!OY*LmC(iD8jK-Q##3+gpnJsE; zuB|D$!!;RWCc^Eu?vbqpX{%mU4O=1Zj|VJpMYmg6A`AC@El|^REiBW@J`56L;Wy}E z;(kCp@*^VQpJc|Fs~DdVD;`^eZe+|#6rt2z(U-tY%(!NmA2u5$d%Zu?M@VvBx6|#h z6U8jQ6}mi!Tkhpqw~Au9(S7u^qVnFZzkyi&w)F5eS0mY($DO8~CRh9D4sM5dB0G0| zgHeJ7^O1ir$AUXi59S^g++bQ6-A9s{IXt5u`#aHfR3{dFGm!|Ei6x&1sK$ata32r+ zxQqplfcheFx$NsiG?)`B_~ZZwZxIO={;Dssd|HPS_z;f{AnzmOAAP+`wBFO2?`09- WGL_c4EU?FzpfyQ83}G8tB>w@c2|)P( diff --git a/out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState.class b/out/production/clients/org/apache/kafka/clients/consumer/internals/SubscriptionState.class deleted file mode 100644 index 5a13a3814f8ef5cd4e90749e631bde6dd65e4eb4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 10200 zcmcIq33yyrb^hkOB!y+CrVSq=go@G*Bp5vT-4Sum%VTYYAN-gf?y2_Xa2CKli@(=FMmn5BT%< z#qYg)-@WIaD7R&c+cogL5+Tp(xJB5j8XaA~PSB z%SRHp9}h(F(HK4!M=Ktb@yF%zi6}l9M;pdt_>|l~9m8kj{*b)10moG)}-tK?ML#Xne@$ep_D0Q^0~t$yW}WXdvZCauq$g9i%wBtMzMUXm@Z_FIRkn* zr>6JcZlAEb%cV@V`><2$RcJY!8OqtEa)J97SHInAhFLo|)P1;A$mE84`?zCPMXav? zMQ^U%oG%P@+oN{+xYIps4-VVi>HNq@KG$8Xx)#|+jj4t+5EK~Hb-z8zlkW^1!I`c7 z4IWWvT{h$7N=2%aE0#x`LU$%tatb*+TkJk;oHpwFbxF0t+Sgj^c{#@!DEcbz&^C-v zu7bKvVZMHSgHuW$-$lzarIIsnaB#5bl<2Kl1J0mb&Xx{29Mids-_V%C#s*Y1GO9Mx zH_cl!xlE}~Vatl&QpU<#6(YOx0|dhK{!Gp}P#!tv6pq-(vYct_&!_F|t#%^g;qu-JxolvnkCANs|$XGP-#@5 zo?Geyl96*xW{M@^-GsCnmNNn%?B_D7Kd0I`W22sfw$!X`7_oi5QPmt(Q6{mV=#XO) z#DFWjLj2L$utBoAWz^#eg=y)mV;6kfGTeHif?u+X!ps#b1KJz1SXw(b1w}*$1@+Fr z6lkpLcK0+L*Zw@YMqyRMChip*oAWxmn0%r-c{%-3oWtThnktT>-C4j>ee$er6SS zluLP!G8AHNaiJ>{b~d}qlp2nzGs~2=J=NAVZdZj2b<&7ZEeiXpg7zEL%1Hvr4+5t^ z*-<#wa5gpI_<&@u681_yx0`&fs&t&q6ayPP6#oWnNVK;yF@j~+ZrGeg$RVJfZx7S& zxLw?l&FX&O=6q52ZA^vPBN_G$xuM$ms5`IF;RCF~(4N^H>ErCqD!ElzK}WfTR5yFN zQi)R&h8Lxh@bqo9YZ?Y#pAJzxQw#t;!)B;l6z)i5irOI4JT#5UGzpq+XSYRP*aHLJ zKA#uZ@gB~X3u)(ujC34xCN*NK1)3I);iQG%gUv+JSk?C3WEYRKMp-z5=UR9^o;&F+ zlfc4lc!7oA!HX>XE?&g+wb)?arLeF9`aRjKGh}CV)!2P9?dU#GVQWyEWGT=8*o_pE zc5Wc)$A`m$lN>3R%FOq%q;oQzEf+H<9MSo9OC^+Lsa9nv%&nYf8ta|qOm@I2&_%l0 zOj>H1n$FNCUt$-NIp>5^Nc#QE0MA}+sTpdfrP@SVd&Lbq3I%&iVlAe=g6FDsHWHSa zrDn66u+$vfWvRJpp2D)mNZi7(ni*9cmYPrP@iL5A_y9=VaX)Ta*{eu9qR!QpdXB{B zh@Ac!yw*|+)Iv)o)ist{q^?%DW%9X~*^4VIIgn4L%Y_0lovgq+IhZe0Vv~%j#g_fE)xqPmxD)+(Dl=tGDmg-Z%WkGk;S??d(r2Bj3tes9f z#p04RYt|?@^=jT_reakz+$P>Tbgdlvyeo02ZvMkVf;NLu{~LmFIbmnZ&cQ(`hkJuq zsnPCQ(QfccXcq{eK2wUY=aY4EsyFD7LG!RPrt>C7f+)C;H0@R~qq26rtiAP0s4HpL z%mep1Lh>$PHU#zD7#wVpL3p%G-*2h|$AIzA22k&u%_8ltD`D1Uw4n*t2?Q{;^G6!b zcU}oyfBPP@S)-j)GOvoK0(?d$tPpf%o_2X`tOM^`sq?}gX}n$;*y+~P(9>FG02`N8lMWG$;-XPlYc!uO1NvcMKTkb?#W;x7bmlr7 z3TV>fYtn;ZWU0XjcSR7Ib2XoHo=$2S=bHHDrNPq|i&E#&M!&ST%^Jt-PX23~!^=44 z22`ErX?GqSM^odNKY^?EuQ~^|fd{(I;W-MYu$T`E6du6BEs@TS$T*VYxaJg{-4U6< zqSF^26U!pnvJ`(32SeXVD0C4Bn+Sw$I7$!S#*n@Mqj(`s@IFQWh|g2pH^A`c`6ePZ z_%^@PwRsQL;>9R1HbuU-Xt|jwaYpR&M72)@V>zeC162&Yk*21%v5leDR(8UEL7&vjU zNiWM6ZMilpX_H-v!`v%@SSS+DVYxzzm|WqyJsLTUh$MK$mTA-qL{tl7zJOvXv0UgL zB%04c=(Z?cevyxgcZ7J6yyh1W4-}kqb0h~hNtEBm2V+CCo_twAY?CY2Fx@Np{#Ez` zUPZ_@?h=2rr0DRKXOXHE^vZxET6{;e_>OoL{?HZGPrD)%71yGYXVF>f*jLl$KhpA& z+6Qr8Ufh+(-FP+Eh*JE_b0yzFx#F{KH)yLucmQ4dSA7Z#f|Rk^g_O${tHcyTUTs>j zJ1`A@%*1^iA@ury8r{Ac-L4wbi2OUa)}(6;>H@l=+^4wDGxwltZmVZ*_gSo|wfK!p zr8fnRYW0t5^(}67Ef(QZg=KARSnj?=Sl-~{vx-gvCC1RnTbc21)3R#mx<3fC z#C>#X5NP)VHhVx*cw?R8nFW)Me>;!AHz1+HW^eKxJqcIW)_VCJ#ML{sluP4E$uAhR zK)~Na10{pvCUw^-)Vldy0o^>bnlySdcfQ4!^e`m~_V>BPzTZgfQ8{*lG!K5WK&*b2OtMdhD- zqf@vVcjHfWn(T8y7Gv}9XS%16G9+APR=t(0%)6sBLda%1!cSvv6mMfaS}pacV&k9t zFu2k6R-5om6}<^2Orja``9Q#HZLTtMOcO#gwu{S8OVwZWZWaL@X73PE_y&ScUM z$y23JZ*6SPYoUhX>oMCs_zM>&-=THVtsHTiwyC1KzD{(vL^>j8vGo+Y853a{_MXGG zu!Q;3fiN%NA2fh}2M{ADQuru)xsQ>aA7r2ParRiB(ArAxu^#g^IYqc?sUP$se$e~b z$VuZ`3A(H@L6>)b$#=n^{}qG2u@dyR^EH2S;#E?lpDqyUHKe}kEc$BW^BKnHp()~H z3bDlOZ`ia*oxInt#5*ZhDi0+SP@!>LS1bARl>BfYNWsV($qMga;@7Wa+iRKX(SWF6 zyggBWi@%#f)Q(!{eB~0N-sLBalrZt%8bb&>CBt_q^t1KQgwVkUO$aP!gvmW<3g377 z;uDcdN|~PQ>-;2m3@h;sbn?EMf6XCngMfDty<}Fr$*rHfWR)cnD<@~h)V%=RC%=Ke$*BrGwXV304AFyA)0Phj$S z-{whr$UISa6od9mLzzh^gmB2 z?m(@$W{QhGtGJN-Ult0Lu;o*XSe_ zIvt28lSLBdhcT4eN~^TP&^Yp+#IxX$&>`=qR_U;HJYXnv%}6p-eWm4n77wL;5W|R= zx^DtnBxoJsAF_~6RKn2qjSab&iC`l6RE(#B2a%FGP58{3Q5HzcFSGGj+TMI3vLuzG z-aL@AlsvZ>+*l@w2p3D1Ofh(Uli5ICtK7!!UvtiKWf`mR7%u*Ahv9HpY~t8xewSbD z*EbH}&`;2-Vh05jDVGR2aH(Rs4~lEVbck~(mrhP!!Cn?UWs2T{-6Q2bDukQgU06qz P5cQ*mjm4}wAqSgZho*y8 diff --git a/out/production/clients/org/apache/kafka/clients/producer/Callback.class b/out/production/clients/org/apache/kafka/clients/producer/Callback.class deleted file mode 100644 index 1d037d57c50b225074b19af94387def1dd690ec5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 231 zcmaivzYYOG7{urE?+_A?fxwHwk3B&)KZ-dbLA=dFxbZ8I= SmCqG`YJ@6kr9=Ir6*S(?VMCAr diff --git a/out/production/clients/org/apache/kafka/clients/producer/KafkaProducer$FutureFailure.class b/out/production/clients/org/apache/kafka/clients/producer/KafkaProducer$FutureFailure.class deleted file mode 100644 index fcaf3b862ea6ea0635aee9df40b401f19ec5807f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1804 zcmbtUTTc@~6#izrbfJ{1peS-tLE3Uz(TbNM#vmkMy+kUBybas2EG)atc2`K$2VZ>g z#TWmD51J?>n&`Vf%6MkGltLQ?eb|{fXU_TNJKvnypTE9-2av{e42>AqFcE_Sy$KpF z#b}t6!(|O=4OcZxF*J#7LliyJu^E!tb^e^|zGqsx;n;@ny2AGKxou&P&#~wBLNg4} zY11~n8HS!jIm_a9fkH#DOmdk)opthpp)G6L;(@=pD%>1jwaC+%bqsDTbJvu1=v2Km zv&7Ic?|Z&0=DBIn#1LPwZQ;&Zyi^jjeLL&81)UeUu_p8l{%nKmhGi1Xl3sM3yl)6s zzbD;~LTl&%9x?B(%aCjk38LA0houPS@?)Z}KB30oN5W~P_OS3`cmdI0; z30Zs8EZuc%Qa^nJszK8P*Z**JT^L?Q!!-@p8M;WTnoSaR$t>7hUNVDG8&IEc`4*>- zul|d%~TKT^S(B-{SK)z9F{Xl#mI8B)WG--8a~S7tK{9P zO4q7%X`~`9QshVgmI%T2CEh{wPf*noM=4oCMUV^*Phm8e$8dt?7%8IRifp;xU&57s diff --git a/out/production/clients/org/apache/kafka/clients/producer/KafkaProducer.class b/out/production/clients/org/apache/kafka/clients/producer/KafkaProducer.class deleted file mode 100644 index e5b26cbb48f8489983f4f5cc0c541db5e1b53c5b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 21339 zcmch934B!5_5V5dC3!EC7rC6O(qOJQ{ z>+a9BT3czWR$K@~RO}+QYF%mC3D6z>9Nr>h`U3nYZYQ^Y5qZgpW^K{KOLZb zW%Li_`Xhn=Oo0By&&uO-^6_JbpLh5Ln_rY+FWLN)08QqX%lN1KGn;=Npep`_412}l zS8aYxI{i|ff93G69sZ5n{?_K-IsE$o9nOD{=RZ39C%OLF;n(H*7l;2E;J@+T<>3vR z-we>Pg40`N{5Jo?=64)^*WvdZeqW$}5TH5yPwDzEhyN`t{|WGi{84~E=1iQ6+22f$_mgjRph8*N0m6rc9avK6I7|K0*)#J&6Vq@az_mb&?+?)fYmTZ?d7PT zqlVjRM1VXsQd&m^)ZS_zhl5zNs*sk^jv8aDedTsPAfm>WQKuRwcOgfOm*xrbFws%_ zgMI1%TOBCVO|sQwM@=s?Jfqn1na3Zc?UX*tnWCpoG~ z8c!}$tJEn0b*eheR;LH(eszYe!nSG-&_>l_t5!#OGJBhRw1ZQs!%-0#d8V|imXD~d zIvo{*%&EAex&pM7U$)g39F=fX(ov}ZJ;^USs@qX(1hcjBvCdIFGR0ZKlCuM9y*kHM z=N5l)hxsSV`=YJB9 zC7G;fyqzhyFrH|yO-9=eJF|9SyuIB^OlKOiblH+QE9&PiJH38k!`#KoPj8rWdgI&^ zR?J;oKbNU$;hEtz;o9z0BwE`NkF|6s5?(A-8&1VLBQ3SF^raz|@(ir9u5cn1Nk!tA zn+bDZx^P#xrNgUT9d274u5F1%Fg#h?m58@?w|I%#rP%?%Wk_du-7@bB-Ci=)7&!~e z;0C5qon9*38cv1hd8w9;<&jP=-kn;6Me%&5p{aN(99`sf#uGheh?DNlG_f!Oi+q@v zv@G1x(%sn|#hQThF@?uukd9!%iC8$AtX<}{#1pNvvH$?DBo&aW(3yw`=hUFyVB40%*aExjHf+Iw>lC{ED^e$iR z^)z~kNH_}qgNBC`9G3KZg+<(Ivtt0Hv@y~i3yVH5P1|DtGnOxyJ`+%et_erGy)59* z2WzEC+1ge;CU4wMPR}wP@7?sS3#3HGg;PB)ewOkbLnm%PE2wfj<#e#<} zjKsXf-JQ)|VtKeZiWaDSOE|hR465SUZ!Ai6M8L>{^N?pC9t%Wum-@&64I734i@@li z1JHo7_MZ!a=FfN3o=zleSZ=6yc>if;lyF0c6_EfkMnR z1BOn|gT7J4f^xX$BV^)x228x0IPdXfM#?|OFky)xT9Jz=q@&A&>GzWQT>#?lKZ{6Y zT}VzprgI8F*khzV2XkZvm;Aq)|Nl(OoAcSzfoho6K;Invx2WVAi=H-IH3jEQTYlq{ zGDgcWKEGjrWAYUj#`?_8bxRXoTVx&X6JA$50p|+aGAu`I7DkdO7@i>sxVxU&;g;3$ zwl;|X0%p$TVKGLf;nr4ha1!s7m`U?HDxa{6z0}%xVs*WK1avp;K^KR`VR4FSWR6{d zCylQa11g5tSU{%w0Z7i+P3k0<`3u!{YLD!jPdM>VHG^i?nilg?wGH@ajJK@zQnL(I z^hD*2DG(11LmOoTlE@s9fFKF9)dpLA(^lV-yoaAgronXge?uz&&$?5Jm^10kYFQ1o zkMuQKhdw@`rXYl5@N;|)FO7Sg7f*DCQ*+m~cwG`hfS>_TCuznZgMv$yP>$(@Jt2e% z38zoV;SHoE9XOjXT|~$PS$h@<_^_Gk^2k{sdtnCUxYIB1`EoL&ce<>6(mxO@gAHti z4(YX2!Vm05qTt6LJ_&YXM143KZ3aVk$qK8b1@bXAM?Q4kR3A?E6OQt|2wZs8m;en`B~%G6XIi!!gxE>Cvj1S1n6)|&P=la}|7&M;MRL_9lta1($&C`-T`8|Mx6AH`#_aC4 zHZL)ET}QY(Ibe7`2{9Py;52t82eGc^JJZxW>li4H87Yxk2shj6UR&LVk_JeQ`h>1R zw4&a^)Rzy3fzZ-E+K@G)HG6``{|^ltvNjw^Es3SI4l!9CLpfh!B6|&-FN`7XylA*9 z>B0OJH+$_7v7Nd;ZAq$8 z20`90chb%@ZWjZEl)D0cvT$HnaxHWyzBa>2c+it!&DtV~9Fr(^A{*J~__%(>p=BO) z+KaVF*fcy6TN7XHwa!b#JJWi{G-pqsnKI+r74~rl|u)S>B6#=nb{Xs zTiwspyyuJ0*r(klf-?hwwW&dL%~(*EvkpJ)UYVzD=HPC5h*ZdwW{?C68spuG7H?ie zS2g;TC2B+xQPUbs{6cU|g5vU69w#4%xavFV0r}V{kC(gjF@1vSC?aO$wXyb4s>2KE zDMLtZb@9?x54!3h^{~tP@_umZgXcDkHwCW79*9z>XtB#9cqF`2q5hqwsOJ~=44x1g zzqtmYSCb5Pc169Us~%CCzMERXN%99cQtH&Ckn|X zmjpZ2!$@vi^<(us)1+MuU1Z2F!(MRN<#MLIGP>_2YFebI4&yf%&LC0C7PnpXqI$`t z9k8S{OsBCm5}P8A1!qu90cCVRT+=2+?i$~`3)k4!Zuj)? zvFax-m#{75M{~lI*K5S{y3DL#%$jxJRnw`f=q?w_gu`;`#Wg9^@oe?7tA46}=JGJ! zD=iy(UjPM181>`B$qsnQj5t+C5I23xRXsgR>TSNtRsT@$xawW?FjPZt4CFv|d9-wU zk1t1L-QP!qlHxx+Sl0Jt=+%J_kI&(AUG<^*$W(Ko5l}r6lfk*nx0Lz_(%pXov+T?iC@=JfF-U|g%PMlYm2cmPB#5Rb zT5C-rm9(v5*DA4WmoMf^b~37eQ30Z1IZV?d5KUmf|MXX z_)sJn@_-9^i7^jBblSa?j={pAn71~heSu6i)U}2|<*mJ};7&q@Dj1Dc2#}|eZ?Ox72Lzg4YPm2H~5s6hu zyU)~7!o4bZ-LbkOb8ui-4(OHT1 z?oONv$zj4cVS*n;z$C?D@l>eUgYvh7I*4Du-+Giw1aTqX0GDgXY9we;BpL;ajusmR z!EmhutOLbZO>(Wt)@WD_jm}Pww`3|9t~JG~WO{N5Tu36Kt*0*9WH8lph}k>{a|!ui ztwk{6prf}2OAK`<5edSv=%`iDZ|xQ<2(gN`Ri7D;#A-szu`drUhGWSr_h2mJFi|b3 zXb-^Z8VYP1F`U>4)CYMhsV!8XedwV}kKX_Mxlw(QlCbASj{qK#99q16^1Yc1?` zOvqRhk<1Q+7$HGOkc{|8OI+(2)i=v@XqNV zKn{@XlX;C z;urqNlLn)oRt{wUh%*c|T8MThm=@ zhIKT;=v|ApupFif+A9%67Y#5heK6y{jZMzf5Eo&6?#V;hJV=CP<1O!|EvnEZi25L)h}8;L_6L+u9f;3L zvkiy61Qcy;zfgu)3Of!3Dd5|Zeib*=t8HK)u4|`z5E=#Pg#o2eT(bzwM_Q0o0puR( zU0?}sOwHb1)Ua@2L*v}~C5z`EFVm^>iMnFPv~-u%`aNDzDOQwL*8_J!R?`owDv5fr z_7wJKO2T^Y5w)+~8tPXI0bwYnofT_sh_zq~34RbMm9QhLTV}5804Ehsqd6kI=aN&0 z1jtbUX%r9pOMr|#8yQ6s8DbvXa%R#=Wxp+9fB(pWRu2$HzotFqWF{|Pb=@d)Ug0q_ zW^7?$#g;Dq=||am8JTxkIC{7TtI~w>afg~sp09WI3MTZ%FCn2Ha6Z(APd7(l=H!{Y zu;}H72bNh~vSF1zl$VQT4u8@#AKV1(0C8r=<52`h132Z&s&MN1$qn-hDtZCj=FdO5 z;E?|8P3{&abGR>lKp3lS33hd0OA2~Ha};TXNCZm42qJK5 zR=#rGfG^PF2G3&z*58PNxyqD9nGVeZau)!=(g}sSWW^c>t2%WME@jGAAYWqgH-t-t zbt=7ArX@E4J6LF*&(ED)bCQQ&9fgtWU)9w|!t&tfhfa&qCtY#`+LzXb`3-s({Ox>G z`NMT=XXGNEi<$lxT3soM}cLfQ1cIT4qoJGe^`*|XfgOGGBab9F)# zStC?}!{u=~T3(7TRji01rGwunZuOeG5h(0pX_=z0wh2YZU^?*ygj9TCd@a7M5tai0 zC8!!`OdM?hbOdA}sX_U^pC`;1_|i;G0u`A&gvg<6+PKQeB}Y7s*~%K;53uo&aN1eI z`~Ad*8IJ(f&Av?=-<+m2jAEU1fjoL!OHMZwW=mSp*lem;eJi??jFBBuX&VWx8zIker8_DwvBx>TKCltH*anka;s5m4~H+7e2T}6uM+> zc(vD!y}yz+{WT<}*?Y7EQyn#I%@|BLj(AL>5|jq?iKO}Y2tJT51Ayewp46E?xS=n3(%YWN6UAHz7_rEJ_OJrjlAJP(I642jqN{-@mB1jh;7&PfRIl=-B$-nX33dnIo=dEB|KDiu_Rc$+7V&J`V54R1!m@D=0!iv>zO<+#$Ls{g*BG0bJ90XWrw`~3Y$*SS_M*{po(Er-9xX>z za1F{?47th~2z|vl3w_1$2Ytnv34O&00)53Mw!Y#-h`wUiSzmD+LSM1xtgkrSqp#RH z#+Qrt!M)-N%!57R%<~vL~ zzrK-%&`mr6&!bFtp2+*-2?O~6J`f-$^?Nrt^KME8g-Shfl|S*-RD!9AMwp4GR&Svq z%s#a`&+Olz;j}@|UTMZ;W*=c@7dWzWu5lZo7j9{cvi%K4(jOT4{VOv4$6&RFa8osw z)%XmdSJUp1=K+vsqegHrd1|;8z=S-mo_Hvb7i5GB2l>RfcrY=F3lZZn5aWpg#L#mA zc79@n$kOl3{H9kv5*!Sya6Zl8R@_WOu{u((S8k?~hsX{(eN?KbvT73rz`L@h&E#&P z@~t!kcSEa!!+L42UJCZo@T%a5UK-g;qaLEYdugA#qN-p;FO9A%t|;!MF}KrxTWMd6 z*)KS@m&QFrp5C3bMWR+bH9lW!^p#%Mmbf%BTUm~ znzo6KM3dMf3WJTE)S}gDTXbDp< zE!|2dG;O72fZS*Rr2zz&3y2kZP;pgorJ1)Xc%tc66+9{3(qwKn)5&!%*1f8!!j)0@ z!|eWgtAeNK<(+z6Rq(X*z|+l*%;+=zotlZAn<$3Yv8rB* z7xht>MH^@cUiZ=$D#7nqwdtP7c2A;vQo38&?g{B$y-ByHM0mUVXpI7>(RD?YrI1S? zv~DZ)GzHJ9!0p-F9)yk-;mF_fw3dpr0eqXLLuowSbU-NYOEia}&aufZDWAr-r&|ml_`a9oAZ}7M1O@4^p;>YQ2eu3WQSLi+d1HI30 z;I{&b=s)Tx`cN&PkJK{yM4e8bsTSIy5^ymWu%#~JB6Ss)sB0Ntz2Z{!5SOVf?5f9k z2+m;+RnPJ$oKPI5UgY8GRUU~mXnSkRaurrO8>0{9!?5o8>O4N2kDy}eQmc3x?n>w^ zbpjvBN0Cii)NrnY2Xk=b>PemsXAs~fPVfx8Eo02YNApZ_f#CG9 z0zLy113PFK0aSK!fwpv%?|M$aYH+g6cRiQ;u4h_hFRkyTbAspg(id;WZ-i`s-9^(D z`qF(1x6=8r62^*N0DZ#?(YeiZ;Wi;9f{!tD9u_|y_d~?|^B5Y#`{HL(_RF}debKU< zSLpuJcqLdwnpmY|4WGxy+I%7~Itj9AB8C$IMj{5gvWk1@qK&vU@Jj>&Ps+eW7fweB zHcqju^4*|uk@C*?WO;WA-WmUUs-EmLGnusoOh+)pcUMBOFNT62wwW&R&BDcKy0nij zvuN=Knq614m9EIp;!4m#UiZ?Md+91X1ivEGd4vLDL9X`Ou1Qnt+HJn=5SZ6#m~}-0 zv^W>&^#F9Mh?2^YaA&t{rCXb1 zWg}rzJ#LfvjZPIoj}RgFzWQyZI#tecQ%SQG7TH0;lXX`2DDcPABNFJ@9Ioz^)|4|g=HL1m(v-Pq&2w8&uPr3 zZ|ZA;9?;hd=plW*nRFpK7JtO0@%_EVJ|O&J-8UQ;u5apOJQ#=<2iftUO2;3VJ3XW_Ki1^hpJA>WMNx8Uhk+}(z|+xafOLkseKh~y{I(R@0e0exKO z-xXt>7KgbRx^M>Uw7E;7tp#mGw25x;@3zy0IAf<^J71z6d@EFeO0QEZw{bhvV>BJa z9lRR0a{*0)5=H?bNyBj0iMtD7Czrz?SbPhW;x4Wg_-^)8OeP3N zi?3d~m#CN6zX+0dn-iZ9e!(fZB6GKcQdFwVQp)CTyuyEmWm;qNTB3>_z~0Kd7L&E^ z6WSY}I}jT7f6=ql=Cc6!Y+sw#11X%hG};XI6^ogsyCFhHLPQ^d*!s7d`R~xQxF|!}&f)=lwK+ zAE0X92<|;dGvS>Z_+eVck5ChD0&jaMfgGxbq3GO4U*$*XYy6m&rsLr+m(e2rBILlK z1Ymp#p0kvyshiKkT@f9GvFGEiM97rpH0~~dSr&tSH>4FATTVXVb6WFvUn(9xL1$V3P&PoqujdHWbptnTX)@;#ZM(RV?bF$l_o z-@}c>jN^jePgCm$+s0gkAv3POqrjM_q&q_Ceq*wP9R+D1I9>d}7ahV+(+vJ0wefQ~ z{-ZrZfOe?00>areO5VYL=w)L8ha8Hxm+|HPqVDk*RV%Wx-6Sk9o4xcjF!lunPe0s% zUliF&8=HbZLhAdBj{f$caWnxmGTfg45)!`&&-@nc#c$I-{0@!d_aOoA!bZJEHT*#a z;fatD(OltEtwB8pVQEPtY?ErXxu*=+Ucpxy?Vy{0vm|H}Dyx8`IJyl~zDTf}D&`k$ zX$9G+rznNFO`~aO`KcskFr%MQIq#rRNaZI0o}_){+JJOwMo-94b%GJA93fbB)w&W$RRBuUup{eWHY_IO$73%)d``PLQG&5|(DIQbH`53i_RDd4$gd<7nu(s_ z?T7@mIR9G4ZKo-6(Xe~zH0uu+Gt{Ey%{6fDsr(u+)Rf^ zpFf&8fEL=JD1RDRatrNa`urIgSPn=uRtI0N0NQ`SZ7lfL^ldWuH+?Jc{;u;aNfqDN zPNPQ_-9jUEy7i`RdXmmu37HD?6kPx&KDQlX>rkSZ$wg8L{;<=f{vRZAT#K15_V7tXC1e| z5MeUCZlz*v;r64zuuo{R&9@!PpU{3>4jTG-jKBfjp9?tO_&Ez>i1(HYW;Y#0< zTFZ7ixuOUU7sIvIl~k1UA(5mbAR4IB+=_yPTu0h>QH=;`I+^$f%(lFFC-^SzMnZ-r z1mDA*PA*>FrU%^zwjV5j@ptJ|d|Zd1w2$7m=|0S#V?R{~7>L+Ioe86~n#QS4I#|W% z6qTT6mC6ux6fH#5brgKxII8Eb!r+6bv_4H#-037jw@Jh$UTeLMv&3w!=R5f>khcV} zVk>_QgcG8Q@iTDm);?9B33;WzNjoGi8b;P@jpE%eWa300|oHKA(o?}sTQmV|jcK25>&Eb!oLN*5J3@Hg>BA&U42*B~&zAC=+{ N;Z-6uhtH&n{{^A7RT2OI diff --git a/out/production/clients/org/apache/kafka/clients/producer/MockProducer$Completion.class b/out/production/clients/org/apache/kafka/clients/producer/MockProducer$Completion.class deleted file mode 100644 index e2dfb9c6c71be32a0d082a1cabeb820125f7e84b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1819 zcmb7ETTc@~6#k~Sg}U4-UWzCvigk;u;$0NAM5CmrDa7~bc7TQLZryH;f5hLwvkBFx ziI18X|B3g+ON@zlW?G>j+S;UZ&YU@y@0|H&U;TXg9KcCr;?R+9$2N?}<)~anA6-B@cJm3stS^ zTev~lEcsrcVhP{O3Cr^f)Ag8fO6H4l#i3-G7NTu85WdTuvU$6fFemO-L^;T*9F(oi z;*OK&))K==3;8DN-W1sxcqQAq&Hca*Y|mv#Z4vJki=J!FZt#>nT(Dg`m|%F_a?sQk z*_+I&g@`8eX(7YfM2_5Hh)j9~!O)epU2(Hg%nN^(=N+;nvK|fk4)<-juNx!5f?Z~a zPkP0YBWgp`Z@8}TCmmic3pyKb0l~CqE!A(=Ke(PtU2ZKi0T@VS7x_JII^3N%a}_tR zi{k2iOO#~UafzLgSO3497&>MGV#+ixsRk6g&uTcQ;XHLQ<5hf1T(f1nJ&l+~Wnmq= zFs`Eu-8wqZsiOx89ox~XV+VFp5*-(CQNx6eOSr7z3d6bPkE5|nL+cFP>&@QEFA9sE zOW(R_bN>vNo7eg`RTzfH>{F0c?ICzsbs=d+s__ z7^b#@u{nCvr>KD|8<{lP97yBEVURwbI(=fJNJwswmXJhIl4znNF-1ut3&{`|@o6`$ zN!mR?7#hj_#$zxe8L1*<3|0{~lF=$6Mlx1K)JRtmd#FPDXx*m*-XMavh~piVdrv(4 zfIav~a6XX)CJ>WJ(T@R=Qj^7*(j?4~(j?G`(j?TV(j?dzc@9!c4dOF8k^UKDg%GVE zmVSvi$?!5_z0lH+&`!JP3k=aguT<$DVowkcV;P;&z|v|bdk91J(5LWyCi#Wxe8nMr z!)biS75q^6ch)Md;m_azh6v9=icKT3O0O@Z;gE)5DGzH%ty08qQ@Cb|48+NJghE0v PXl$TyF}kl)*o*itaEtj$ diff --git a/out/production/clients/org/apache/kafka/clients/producer/MockProducer.class b/out/production/clients/org/apache/kafka/clients/producer/MockProducer.class deleted file mode 100644 index 1c3ef7e66c4f648c9f1b05214a40f36e5747ac0d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 9644 zcmd5?33wFc8GirlG1<+8Baq-CN)W_kNeHzHnqaC#sX-0}3PmXGu$hFF&2G8}O7BB0 zwRfv+#oE?h_R!u*2x+agExqsizVG|KZ0-B~Gdnxk9Y|uIK2M)K%>48J-~Zj;_rEjC zC;olnJ^&Y}i{jXb6ER$YNx8gFF7KDieRBDLf&1ey9D)xTcp#3r^Y9@9AC9BZdH6^i zC-G4OAB$r?PQhV{(|1FNS z_`B@=M+5$e2W9lH2K*Zj8489{hN@GD_vMC1GIlYY%PN@t*{q%K%UFeiT_8;=Qz{hg zyuz~0x%^O+FggN!9!M8DmOfw%XalSBs~fZBUZkc7Dlq8D_}<^ZD)&xu919h zpp>%nT~}8&aDTYKRfTz*ud@zYU8Q0=)3rHWC~_sXBR!P0ilw}*u(@h^{Y>Pz0=L_# zTz+8v&Z~NM?&|5~AC;yVuK6^0zK{9k_7h7sDYjP6%|zmsTayE87>}7Gegz{$f`ERpMLaJ%T#tpM#?NNnk6r38?cT3(Nc%0CQI|FwOu2ozeKEnd&C^AE$ghv$yF{`Z>w*WQx@X+YtcpiHmv z;gln$r`^}IQ@L!al+W{U>)Ke7Lrl6?ULQrl9e5hv3f%!dKm8_O;l5z@=6JhLYgW+7 zRIQbB0&lU4)__&CbR&Q2?DYxC(rQV4Rwi@6N*$Uu6!q-1no^--27dMU5|(&?ho^YI z4r-y4A%xj513u|2fmPPZ6uMlTuwC#*p}5@^9+D!1I$37R%}KE;KP${BPC*9>L0)$n zUlop^CJs8Gk(-DYD}YI1CV}vuMSdux(72<>01@Tr15RVokfFlF*@W0c*TmEc6G4hr z2fGZlssP>$!r+DTf05eptbIdqn}&7@tAY#(^msen9G0yf7_Oi=g}UMhUnCSFgGAm3 z6+&%tyfk{bSlE~&vRl#S!+$%9`7{xr-$WN#M|4^U;LxC3%WP|-U^d8bN$ zb}*+a-59p1NJ?;an_m|P=kBhaJ2h+e5AxY+4ckEp=!kb{wA@H4lPeHaEpOWt+?IEy zhwaA@P3nW%DDo*+qVwxzB=ZVOr`bpdPjFVu<&V&pQw(t(y|rdC$h;ZmwcuYGWm^oi z>DHsYRVNIyf$tMBJD<I#NFQ|2UH;{#Xt8iy7_615@ zALQlW{4`X=P*Fn}hN@TCKYN3{Q!=3KI3oRqkF39{+h^Xk1J~K9V$ZICx!zi`012lG zhNl%BwIf%`r|gYsdBvFT@5`&D`b|6@FEz0VSDAPWcA40WEhetTUK5YQUhb&8mtScW z4zg-Z?8ajmrpm%)`Mh;R{BB|^wka&ETGB7oCbnY--EFFviYxTiep;wDrgB(bZVy^K zAH6KG^P=_>Obnx7sE~=Gw`W2_kr|T;{O?Of_FMnQDPrXsBjWwWvjgI>S^6 zwU}5!2h}^1Om(JOVyd&$*``{mT1~YKZ#3}+ywOnSm})t07oICjyk4^FO}N`s=c<(q zk2=p(ZK|C`XR4%XS2(A_w?C7yhpfzH`JvJQ@vHHLoS^Pi(Au$)@-iWn$ETs58ADHek`$D_pVvJ zdbLojW`lVgM_o)Qny|)Hk5DTOb-t+{sV*?ph4Qdg3|XhJdU|jBBgK=wVuw@v3d!kt zl!4BS!doVG!aO7XWjMqWuU45UO|0AxbQ-wLd5Tp|bjsP{ea_>_We<-Oc}_0pkvNSI zcKe0pgL5i2Y4UeQZ@6rh2^wn3g)ZJS&G>UA-uT4+R2 z-$eoJYPtEhyD|y8GW^iGOTKYxSWXykqzhL#+1^~yy)Vb^mJ;|y2ve`CL7S6sC2OZt zh_vTpBiytxofr2^EeTJ0^OQ<)OOj@V3;fut6(ZiMXGXXhMy-(%{ego?U1>&cw)^dy z(r3wV3VndXtLc}t^`DMieKF_ks0w^p69zs+$QRi1oYFhh$7)8TyhI1I1(@kGyljX- z%81SsT3Xwmsw>QgL^Nuw@no%ty&D4OYGqAnxIOLtSjJz3HbUXis7I3^HU0?JBsfjq ziMY6SHOFm{_Mr z;*^jciFsi?62l@GP+W53<{Ed_>oSKryG4ISt%%?3) z9i8J?P_ZVi7V)u{7AT-c~HBJ5MI(Nj;T z_GxDd)qWc5sFz``4kPQTx0@8g(@3sTV_5i34xQLXb=+*NREtC6DP~+Q@mhMSJTX=8fJ=S83V_72HF^+S(V`OKpn!xfp z?B+0ZJIq9k_N}-ZbND-xY58( z%D~MAo@3xBzuG+)bh{IfTc`?OV7OI!#(h*n%0|KwqrXbjj@@@Z^jqlyj*4@%&=-;Ii?NW~MI}c>v#Ur&sLY7JMQE7nETDjw;Z}~s zdE0oG(vf77TyCi@o?Csz6Z5Q$;WoUSj4EPeO~g16dIi^BiQ8Qvm-0!h3Li&jHUC%o zln>V^FEbUc$H?Ul&sUd_qi|J~?&#F}oj#VThNa91OJVgYUd0R7@Smhm4Xe8}E2o{8 zS;^KrnciA9UX9l<7!^O1*)PLuahGfVm81}AQ;sS++8oE~YCqiVQ`)RiT4pMKcpY`j;`w+YJhxs36PWnQ#*p*3scr)HY8IAmt&f_mXp%Oj3@;gJiN}YVk;niYX zkFxX)QXHcWkK8KVx;xx?5<5B)VU|XBBoUe5{|ds|Xy+I%yaT4(nxl82a}sO!Oky3o z{zXj}kD;3jJx%L5XpWBIlBQmc#_*_dT>1goau1D;B*Nr>*#tI(&>i7KR8RCd6H!it z9g0MRbbX0v#{@1nQ1QxYG*Hve&^Mpu=yRBh&tnn3fL45ot>?=;IKNI{^-Z1x-=ZJB zt-U2Zei8A`+gSGzQa+0OuGe_ynCHP`Uf(~akxv~dZ=;XyWxN{L+@$cseT``^jmX(MCg=8VR1 VoZRX#L8K-J0p9_6%3O)~e*mU0ZHNE> diff --git a/out/production/clients/org/apache/kafka/clients/producer/Partitioner.class b/out/production/clients/org/apache/kafka/clients/producer/Partitioner.class deleted file mode 100644 index b159607326221170ba64afc766f78519adf8504e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 320 zcmZvYO-chn5Jtbs-cmNM2c8Fpi zE{cK&^?20h*ZT*6E1VY?6*w($Mi|zt$t4#9;bn5a<~KH;LuKYkU8u?f@~~KoP380T zzbtBHxMsT&v*G22jg8xF4CcnSO%g4qcZ9z6u@ORdGJPV9=CM^Px6WrYe*Enao$V?J zm&Z!)B&5dF8edy0Y7UOF;xB%mO1L;y{8YuTbXBXo@G>9}I(fSJJDnb(hdz+^07WK* IlO4zK8$JkJ(*OVf diff --git a/out/production/clients/org/apache/kafka/clients/producer/Producer.class b/out/production/clients/org/apache/kafka/clients/producer/Producer.class deleted file mode 100644 index 7590db31e4cd09e8ef2ee176b8890a18f32af01b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1213 zcmb_cO;6iE5S;~rfi^AB4@$4)U|Okr;e;bDNRe6+piPK+ooqJAlJ#!1UUTCo@Ppvc zAHa`79or#FOF#+g!Lr|Y^xo6knZ3Qc`3(Sj@G*d`06qk;&0r~!ro-SzShKFjafqTX z`2f2EKyg+YU)!bBL=Gt)iam@ z23Pa*SDHsrvlSgYty7u7cj31#J$BF|F*3NE-%m7y(jjWyMlpEWGwEN?uRqS?zxjA( zcBQUPlRksRaJR*vJVfV}SJos|>lo~W#aL)GJ>Kw6nciP%)Kuarfcgw=)BM>uwuYbH zYko6bn;&QpOVSbvgN<-rz@N|TB5{fd6g zR+GW!*_WEZdcMy2xfQ-B%e*wHxXggT0zHd#l>|$4fk27SGNFYLUK!(6`o9J*0OFX5aI_Z+M7^b}BDNZ|fmN53iuN(>rqXj}AT3L+_+_@%Y^iy(dTi)I;6$ zF#mf;551S($KT%FO!Hul=N?ddyMeDCMXQM|C=?tCNv%)G|jc?<9H_zUCa|bkZt*+U=+woTgUvs9v=a zpwuOfy3|pt1bZOA!%?dpm3Pz{N3BgBRvfj?QR@Njt50{-21i}iNnJ1ctfMY>)D@1p z(ot7A>S{+_!=SHqlm7B2qi%H66LV^#qUA&5V+V#0D%v~}MzdSJ zhF6~Rx1R83PIz0(VXYR{@ll@%X15N>$9{iikD|-=jgL>3CMO1uOqGfg$BGlBsRT}b zTRlJHczunvnyq1fcP;vEtC=)IJb0 ze&B$O-j(}C#)l41jgKYg95^~Ml45q*lCzo<6UE7i;bKPUisXSzZ{X4KrB2*1G(LJ{ zqF5>o2`6J!?ilT??5{3=fs2N`s?EMv7B+4Ug>~zsn|ql}T2N9UV2RGQ6(Q zli=Br@rg+YXJ}-wRI(7ik@+?}Hg#ZR_~4<*gtw)sBfM5~4yY>HbYAPGG}8|kkF((n z4~{@5&6ycnmz^OBpXaGB`0gJPC5n!fD;v zQv!3HSE-czDDbMmgMKs_YA2!SIepp!{b*b58FPo84YBfUn75UBr$}i_TQk zH569hZY~`O>i$@(HVqWL=_;P&N5ZmKJ>~&(jLpNG_H;3C5GR&m4-a<2U&xSCwBy1F z0@#!6{%kNkb$?#-45zMdM{1#wAhxR=odG?xU99zFeENRB+}yJlZgl8G8eUJks_QYeu`db6LusD8-COb{P-eN z@q*FbW8-Wct*EDwBEl+?DQ4TMl~APY0kSGZFa2pz!%A_%1o5VG(zJXas0YoxutOV1 z_7j_q!D^dx0nA}36RQS(y%}!>ODe6hA8iH6@64S`{2Fg`KPiG&m?kR<>sO%pI*ww6Q$FaZyRJ1yi z;~_7e1B$(LiXMWk^{P$u8b#aAPRAafctw}MM->u(>PI=XxmVq!`g_$DRZz4_oR2lH zXl;fBkTlw!Q(Jr0HnqK%>J$PgwIio)?o~V0u3q&dwLPbv+^cR;w<_v88^fg&Mo>E) zhRwJcc?~}Qq#wmOwYyi{ruOF4Q+m~)+NY@hY;cC)Aptuiqr_QuR0iE@5I5a$#+?h} z=GI0SHQk06M6ufp-B#?oGhyWV$U3L7ZVq{<+nn=VmJ_0t3s+xmGI9Sz|1#99R!u!oi2kW{ZOxIqo6coW1}NHd`^BT-?bD?wZi!An^((-NG7 z9|kq1ApsfBg{^AEon{8;^>e{9{GIEd}+m z&l|zlgmR|sesD`8#AMmHwlnx_Tn(e$n$KYw0&2K@ns+sC!7b0Be(>GOLfxwch}6}E zP5l6$kW=@;f_m6wF*aejnxwN?;Px$qEr@Qxok#KE)nP4Nmz$a&3-u+>`k|zN^Su8m|aOaxIt z`j(k$Fgw?Tbf3|J&x%F!#ueovA2qZe_o^dmFDu-gN$8O5Z@Av{XZ@&GO{kJm50?yA zJy(3aI~)04svF49L+#4S9*`GC7zYPlg90Bc^Xdym!{Eq5F?Kwi@q#L_hnj%Fjj#!| zgAZjPMP5A?AL&O?2(LMYNkI0XxooVAJzN>f2JVV&NR*l_5S(JAG#JS7yva|FZ$TUQ zyC0kso4|}YW3XBc=lPl8jV8)Q%Xkll2E#&ufyJ=K#LlLD%>hPXo=Ml{ms%9q0at)c zRUZ)rDIjY=KX?PT3y?F2F>H3s6MkmsWCzch$V*~?#7x{Om*J^mb~V1X9W$fP6wl-} z8on24s0h2w(5Ie5?kDyhajez&eZr&REP+@o4|0nkisCm_I z`X6r)`3i9v}Jo?u_U)A z2un^=*)GbEqq$yGPH_U z{28xRHAw}~?h6XQ4l&Gv3Tr?!JE*ui)@r|{t&!0%qUkqD(d>gkW6`6y4$r@kK152O zcUK?WX&x5bA&Uy|1zug$c-m{COD|(4r{9fr2?N`PGL^;Y${N;1Q3J1vCAy>#dM!d+ zaT9KzB**D4txeNND4^WdYMm1wM~y2>3o-hBz3Ltma2=mwz9)<{$6|Y}A?7d3jfY@m zoAtr>2pR#soAF_KnA5orG7tD6cEvRaA!tDGTF*3(r+ucXEj#v*silG1sL`BXZNx5W zTEwuL$GIxFi#4zn#JV?3^`)urmllKhC!gkn$ z_VLK*S!O9ARrC`;APmWP8-^tyCK4_r6i!Fs2{gSOU_6Zd_v@Xb~mIOT{Y>nc-EWxU2|>%aZkQQHA0HT^LXyNqYmH zuFtcv=cqN~Z=ztnT280Z3kc6HshsAD{DYp% z6)XGWXiCj{AgHx!u2(}TB2_u>)<{SPL?fIfCaLSy*_vb}S+Ib}YCh*WI5qRq62nX? zw?vZ1nM6jhXaeugQYqtS+Ubxglh+x0gp)jXQpw30JN_DQW3v?m4WC@U>NU8)f{sh- zu1SXKAzf)jb-!XvHCYc22e%U=*GNlEQ?qjZDOy(v0gn$vX;*DY_5=icr18-Ss1GU= z?foqLA-Qg}=U{FuxUM%%g0$32%|dhGCQ^5l)juZ6c~QiT9kz0u;)6oN7G3+)0xA4Z zM6w@Cs|IbAC}y=WbM^-(Boz~<7Gb&Ef<=ei=Ru(gM^FvDie_Nu>g3>aRrlf@SQ*_z z$XnQ10>zk1)^ZS)TQ$CIEt~w_IJ>kB0!&lOwJQdLEoxRy?KEOF^nVZ_IIeq%yiBSN zMdZN1-8DG~gdv);>2TyXP9@WSE~F5ixcP?Ggtw5ky^tXUa9}uv1A@3jJHG;@0#UL# zAkM{_s97q+_C%-E;8cbcK&Pm3m}EDc;<<S2c3d&Hf*sG(LriCeW~7MlfeI6U3NX zaD%O=!A+h-$dEI(!XPA2Gn5FyM#GXD4XscnO`G-?xXB?c;Gm%!@azDRirxdxv)sIO zk?2a5!!0s2bP)#S6yw8D`oMJwu3N;>8zg!AERco+Ukx@?l7^zBTZ)QIfdp_t-)GC> zq=vgN+-Y+AO5;C>Ua65dhXjCD9bra$3y!lq zX=-pPz!fvP+6e$_6ZpcH791cT{H#%DS|b6fV#$mXk0p+y8q{>j+NSs7owdi*KCRtL zi+mvNG*TwAhJd`2A*-^H&Rh%q|5RNerltDIz3N_W#XprhqfhHq_n{0?FS(2L9qO87 zjYb{7P5*&8v_84)gX5|3)rDz&$IY<8;f%Z8&!w%6P(_T4Tf&ggLi;O-MkDkf|zruPA8owBx; z(e#xDD5Qo$9k$nmUK@tTxWGc@gmyR5bYp1x)~BlM9tA)`mx6tXlyr^)D4bK}A^ zPE1Y3Oafi3xN>>N1Jv^1LS60vjh2cYaSsiBRmd7fW7eDfxFg^mg=k?@y6)95OyY;Z zOuH-&%QynHQKE>fo=J{3)#hapUWY?ADDtxpC z%{~d1sh11NdOdP*4m_OG3)0|_mgcAco-lBeV2|VzbuM#ToG;4WYD-)v6PX^zO`9a- zL(%k=Bzz}pyT-sR^MwJtv(w?Iv|a2Jm`yI2VnT@F@w z5*p;Zf+S?_+bqfyUhhdVkcs@OcqyMsjW(n7K<6A$sX`lLV{I+gOfMw3xRKD0+E9T zArerO%v`d9MKxT22m0m$Y9ReijWP&Dp2T_w3zbJrFK)!@;E-_tt%F{Ad(0bIU6Y;&(!mrCYu#ly8ip4aVoBTzxF*6yu8_8q zNy{=cy#S_%ZVGo)pkH7S*E1~?I|&%gCxM8q1}d!iMrK-2FX&slB(}!|u&$6eo3O4_ zo)1l;bar^Fp4IR;ICU%iXvZ;>1t@PxZ`LXZc7UDS#!0gjm+D-4$lawt zPOd?5u{p(8TjQnltwHUfB=yG&EZRQMs?_dJClT8DWn0d2e31Q}*|Y^wUxbRyT?EVx z8N*{SlmJzcsMDQ8Xmo=)7^mR{;lNl23o;XToB3pwjYJV}7iZwM1{ADLb(z(OqlBWp zcq^u$-^wn?TiKY9cc2PM#D*Ti1EcPY3;T6GawIA8{7BKTH8OlTH#e-cOR*R-f81BZ z2dDWRpK^0ZD8hRa+51I!!3i&$!?*_d?$Wx{)UPYzB`Q?2Lz?C_edfinD!)W75AkMK zCpIg(#k|2KQ|47i>y<6@#NKZn;qA-pCe<92!->R`2%J3++(pcG5hnzD#hg7udzO!LRm*& zj5Irk$6!b6`-W|a4eK{|AheumEGs%@bII`DI{RX!0eiqtb@0;@MLREyWtv*JC1GGP z>fkqJ4=?$q?D@;of(RZwVpIPAuOB}>%UwQN+&_GD6jH@6q(+K^V{NZ%?F&QPg@b$R z`|h!o!8`{eI!DIuf?3LsQ1Axyg^7!8jjp3E{A`YB6J5bC@Z#I$)SzdeR~X@Y2d%?6 zw#LV?u4#?$#eB;eKNa&Qt?_-BpSQ+O#{2`;_*Tp>SmP@(|DZL#4)Z@_jW=WdXRYxL z%>SG<-i`U6x5j%g|4eH)AH zS>thxf7u$}f%U&)jmI$mJZpSC=6}^1Z^HbqS>vM^Ki?YPjP<{6jdx-G1=e^9%|EjY}B+mNlLRycb&IGUi`ojhA8kVr%SU{M**p!T2TCcn0I&vBt9)ztkGvkMn-l z8dqr;D4t-PG}8a&z>T ztLRl2cF*!LR?i7ji@C2BP-@Xb~`jl>ZE&T)5Ft<*F(@HqqD_rF#ufsRZyfaww zdU^xKtR#B?jU~166s^Khr)l+R%JX3z)?r=LtCq=O8sJKLBfSZ09C|Z7qycq&93$R# z>l?i8&*MH}*$pQbDFSDvD)*3#8y=-O`logtTJt~*Un$X|boZskbzEC>emET%AzAV43bbNWfzjTJS^U58(@@8JSlZU%b z)06T~2GrHJlYuMe-IBkR7w+bn*0%PUJv=i292tug!&~yV@$M8CM~DS7 zGtZ~-+-ZMa=Jm@?QH4d|v&bDYnBl=JA3ldI%*KEzxF6H`6FjV9_!_!|l}?G$y#Q86 z(C4e5(L15L_d{DFXz;V4ZqJ9py@K9?|8J#F(%a}u^e{a}Z&#PmBkC%8r`kyGP}}G| zY6!o}^60&)O7Bz8rAO8C=mYA-^nUd+`j9$`;buQdYt)|GFUK#^*Qnd^`8j;v ULEUN;pJU6r@e}qn>KJwZABX~H(*OVf diff --git a/out/production/clients/org/apache/kafka/clients/producer/ProducerRecord.class b/out/production/clients/org/apache/kafka/clients/producer/ProducerRecord.class deleted file mode 100644 index 97718a00f9bda920c32cc9a8fcd6cebe14bff294..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3300 zcmb7GTT@e46#fpmoumOF+_ZR!v_eRNQER;*Qt?(%P%DU4OY0#ygg{JUII%jN>FuF^ zp;LXYFFv$06=v#8FE2XN=`ZO2=$q5_TYI0}kUBCG)?Rz_&~@NA=5%;LL^s(%$m3+7d{N&dI+tUGI2xJnoo`#ZoS9ATqgR-L+DAtB_4iJLOyF6X-6(k=duOMsg!BOGjo9w-(4$TQM+00{G+)-&KWZ> z8Ea7O+HR0kjWvi|8!tlS4 znOSc3FQ$ge@i3OKY#^MB*5-+l<(a8uloRAQbC`^(=|FbhW}jjh<2b3F$U+!r@xFof zx`#U7O`>XZbktF9^Keqdq%WJg#cfNG`>j&RE@TW0H?~t1&``dnyf9@=_-gz8FCP0j z>v!<`;6+60LM|euQ7fwh18xIA@Zym&*DV<1rK~&WAQZ@iRWm2ik9mU z&l+8IJGb#CYBZX_ukCY3Lk z($C#q?r2H*YbZ`UZc&{0>`|O}>qUxu)RhclA9r%IpYc>tgGBA8bSsR1Y1|jUfdJkK z;9vmn^2ZaMfkP_Z!*0Bv(Hk$%g7Jir99~0E0`jPU(BzNNJyqQw1&`nf5s8ON#&1DhgtK-todUq zFItorJ<6h>v=?$VNSU@hLfboWj8uw0&Q%KGi~j`Q4`g&A{7QIE84YWIYgDRImxuw5 z6^y!}G^z`%S9j^P)M+n|Q|-WpYR&a(XE&jCjB2BJk92+T{`G3-UQ4Y3PCHlKpG<<( zagMnRJck*1;}Jr_uOd9j)ohTNlubQF>-9CXMYgP>UGmQ*(xG|(Dk70i@(B@kktF%I zif(aJzrHqQY#v?ww=IT@a#uIVoLj3dOt3b)=w6JkVSt=&5M4h(?V~n%K^^dh&+_fO zK$uiGUgE&ikr}7m$R5TM-^1s9ocJ9*58(4WPW*fgc5Ph3CU#^>!G&SF?3$#s~21C_b5f>yRs+3lwio|i_Y*IIg9XU?_1vgIo z0WKhcM5)AqAHa_y%q~_++X&T`cV=hz&3ijL{`LFEPXODf>Co{Yha4Wt;ZbT5KRV6U zbgW}T4&@9g8Q2Wzpwk(Nh=J`gf_(;NTc=3a)l($DQbDNhn6$S=T*Y2G1CFJRhLs|4-+2V|^|^qCqzLt3&W z6`M{t&@Zrr4aH(Bc5}*NPFYOcR4kMex5(3Qn~-ZrOh|^7&ft!wAtCKV9x)Cil_gPg zxEo7zlvJvV{SArFl;iKY)3glKlnO)nsaOS*R~v)3$jW_L0w=#C^W_qL`yYIEil?ra Qjn96%gx{Hox74oHKQrIh*8l(j diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/BufferPool.class b/out/production/clients/org/apache/kafka/clients/producer/internals/BufferPool.class deleted file mode 100644 index a546ce56e6ebaf2ff0dd9b3d0c850e621dd079f9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6445 zcmcgwd3;pW75?rp$(y`9h>0N@17t*&Ff5}eh+(Phi!5pa6sp#jnU`c>G85m-fLL3# zN=2<}sS9psMQVjAAVCDhR;zWZOY2hYX05HQrD|r_7SUvDDYoGud2Bb(Au~JV@vzM7G%4~C6Lr{SV0QqI2Jlz_drXYd zdmk63AC=lE=&P z`ihAAxq)98cvYb^olM*Dc}`o>U8SJrC|K>uWZZ6wI}Nc598S$vC~t}grk~$x zXHw}{GQs(!3VxYZ2+dt)udu_JbSxf@BomR0>pF>aSQb;^Gab^dok-8+>v-;Ib{!sD zUa)7Hb6&>bQb|LsC1IyCF4yb2u20YtiC8i`WmVdll4)*s-0_pR%~)y2(vF)_IHiY8 z-D@^AnTW>381Ixanh^8s&VgaHIcYZ*;km=+Cf$~>-EK!(o$zwIdAUtH+uD+eaF!)J zPqT`B=~x@}AJpR(Q6?2`Bp0~@qPx&y@!^CtIEhr!<*wohu|zCANg-V~XZF&=?#n6H zBV&O|8o{`oXbCsa`b5k40!-)G?c;|nqK#9NQEFT^H1*`MFpx3wrf>;&bBcJAD<1NtB;dQyCDUT9#T#6IZJUaqhf^!u%xEUjiu7MWcgEGwGN+yswD1SKD~~@a z3=TH&)=G^CHf6}v>bOoYmI~UzR@;pRb}AWI=UxtqGDB0p-~$u)?_VMMOG$qbF76od(`uzutq#-bhO@?eI+ObyhPJ+ z-6R1X5oEX!HcDY=c>PFlWvkN(6Zds$LY)4Nk1hNI|1|I~3!h+(g?|eK`-J)w|1q%N z!e{v0!T~{UP(Cqwt~d&{dG}{ygcT<36u>fcv71b^hy}W{sHeKrEY+IK#G{F!X+kDJ zJ<3wR2bNOQN%@58Mc*jDDq=LMVhfjHwZhoLJE*(+QsIcao(r0mIT4m-9UESFH@3RT zmHN6KkY}7NH?Ny%XTdJ8hgFH?Yv`ehbld%~jHbL6g-M+)hK2Xu;k&5L24_|`+jVs< zJg}!3uieY_D*e0lt+ZSV*+a1bomsT!vKvIRH+uo3*?LAZ z`*mLYJuJ`pU%Lb8q%M%GjCF@H^8CKt=|kko^6v7kCtmZh)O1!>ew{EHc_EdwC*$!M zF*il$9elVBb~!wjn&rf!Q&vsYpCSx-$CgGUli>Z!FBU7|tel!lQeLDv9lJVJxKrUy zP9AO0VRLPfeUJrN(c=3aFT;G7&8SP*iDb%=Lcu1r$d)UPYPH4U@mS|{u}cP*dKcAA zn-}lRzo7v7UQQ_QQe$5(qNS8hA4|oNxS1L&(g&y`Lc?$wG;5P{(*>m(8L7T zqDb7aS!9pQ;ow~&`U|TELzm(Nen!sZ*SCBkvvD9*A-#z=vOV!ewjJKc76S@rlJ=p2 zeC#$l-xqR@J%`7~w!<6Qd3Ync3yvmv&&E0A`zoNup_ntk`Jo+9q5j+93-$GHN3YN> z_?PTJ(RLJ*G)M-vqa;t7q1x>zm4_9oCRguJb@6ue356a)?~1Yx9I+kcyHK&D;>c>F zqOv-$1AXgDxL96YvI|v9w&JKAIJyJXTPV91&gHWnefXTo@Az3Lp#^2M{}>`LnC;o| zL|_ar>3Sk?DqE}>w77u?L}}|vwpTWHp2c0X15Jo<6e`M7cr}m`Bzt+3VCnZLL8r*0 z1fOD$a?s4*He^Q&S`pw0m55;(PhQShD|=cC;#|}Cw{eaQfJW0_&J!)VWlYU#+FH8y z4up5%m?gVVLje0#^zXocnhw-f1W5=(2ac^6NV)^Z?ZBWtoGR|X;2Ms;^$2q)*@+?i zzQJ*t9;)y->gH*_VYk54RA|J5dVM&XH|}kn$fwnBwOzA%?X4Ics@Z`Np(UKES?!3! zJTak~hcR-#rVEZ5DM$5MhvB<0s#EcBfxgF+t~ep5^yodkCaHSAKIX*wA`)hG(N3J? zL;sq)aV6jT?8F#_lZ$0~tfydhCcwq=sz6n-lVv+up#$UkuR|yc!9a54AgI!`hM?YG zGif7+g=+doX&aw{xAibiSu@u3EbVK0ruOe^o;zvdft6e}ccNb48(&{sOO*;u>y+Ibv6d+k`JXr1T ze!jl+Q0~$_44#QN$5+cxI}i7H<8Jk_j6TCx>W#I&W4(EuGS6Z(c^xKjGsZGRCo&Yz zU@)G8tqk97JSByTk;c_bk{cM*w__FVVdr`u*SGO+H@=1^a3S8uCD_Ym_Cs9CrtUK3 z$K|R7Yg8q!P&K$x4Z&4v0`8O=>STtB)J0Ob879vxM?2Hi zr1Vv^U^a>=zXdL-0QRcUv`LcrLlvfF87A{SuCJhlMU;ODtz5}uzF6JK-KQ}vnT)xT zKExI9f;pU;hkEe*ZoU_U*?2zE<|zkAWT&Cn`xC z;OR8w8~pXoz(ogGh%N7phkO4x$?;pF+vV z!qnJ^vRqwYn(Wy^;RZ|jB8pqY(kr8m;?Ytv@eEg=!%=u1{qO=pc##GFWlX^<^7|_H zy~=|B8h5=)(_Z&X4H4HVjQwxX<5JusokmBz$dJ+>@yz#m9yd`GGOK3OEu|g^muEqk z!rkHvGc;Vx?!a_`*ymkJrnmAYFMziS;59oXnORASnyty6Cow5kjMQ%_=NzQu(#^T@ zyxKlhuZ3ziXDe`M5;H6SAh(Y|7qS zp%L2Da$fHTo4Sv3|HJ(MtU&IdgXJ#Oa_4l)4O6aoT5cZSCKB3lM1|c=QNck)*@G1O Yi=3;%4Gc5zijHMWs*em!^R}NBXd81}Cm0uK9Y`(- zgr>bUTc9KBxb~Auan<%0&D8=WU0KgE3rnW&$a75?3O1awz`d;J=MA%DS{t^pWv*|T zhE;Hwv}}}oZ>?h4zTrr4u30D>_w9AFQV8ZvKX3xaa}^9d@aDZT&)duu8ptFwgyt#$ z0)mGqOIdgF6<=Z*XejZqSxQUfY=AT%oQ`8!fu3gYJEg+{`kPJ1s>g!q=8at7J8nLm z8fTNDIVbO$fl7Y!Sn|_-B%)~Putb3uHLu3d4aw$KH*G6OuS}gPJK0^YSoBp~h*7Xf{8rRM$O+L$I5Odurig*;0oXQZV0@?|1L=i zer<%gC)v36^7KHRfxgr}MCvVCmfxavc^{!Yg!j?*3$&kT)q*~*^7{kmM+i4?k-Io! z&=c?BeWfLWSDJoNK}F=-5N&u2Lr=rQM;|I8RHt zWtxzJpJMBPFA%{D&a=J*+kOcdLVk#k@CXyg^1FBr&yiClz9Pg4bmBwyCPvFq3~?8w z{cYTWp`uy!23U;oG0S67)PJJdWAWF)FqhK#1zOu6SSnC$T5O1KOL7|)mfg%|Q|vhY z3O_g<+Zn&{*%v_V$VWTlSO1OUSf9IrLV@vRoMg_q_v}6Ap7WgNo?rj`X$!zGj5rb) zi6MzmCB~GvsKg~D-i+e1hAVMIQP40RM;s$cycNZS(vwQOtxjHzVoJlcINrf@8)h)8 z;d&hJ;yn#F1cH`R6G%)uZY|%i?hG&Gr=40&x}ySN Xg^>N9#Yve7%uH}oKYuYt_ z3@*!i0)cXY=k6L-Qwp^CdRp2wZ&5%@(Zb+Y8Z94M<^>{Srfqtc1bWj4qm~Qh!u-IT zKxo{lGT_l^)0VT%l?CbEG!`scm}kYX<_yQd!QpVOa}CWtnBQ7d z`Tu?)Atz?ZL+=%cjuD<#GlFTAM!e`W-HMzv71BM20W_qDpra2LboAmm9Vc;0$7v;= zM@q*FIK$@IP0JGK*iRbeR}?tD-(F~%R#mz>ZsHaRl3KXu$%fj^>3AQvb(}ztz>J?y zYIRZCsfM(xY+GtYHX24vraXs#QZ@D2=c!fGTTBtp5pJ?6pOI?WvP;!;wS6YN>kCV= z;%O-9n8$4mWgQ>jLxIccP5d7|Y4}LT$M{4b^UtAnNXYk;E*DfkB1XN=N(D})rv~;k zu|yx=gzO}1I7S1Cr@kgt zeNDFdnuPT=IZGX+Umsqi?C0(``h}@;nN5hy;5q`C>^g$U(0xQRxpjmxxyR4~YY6AI z))CGmL+glqdl# zx&eQs5)S$+m1+nXuFoU*gyF&(1~p_g*$&z wj(?z?vg;Pb?`%^Pui-)qFg8$m0kK5#2Xudj4wanXArMLRshje3>HyyO3zOB?m;e9( diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.class b/out/production/clients/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.class deleted file mode 100644 index d87dfce5176e4d94fca3383e3518fe858cadba45..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3048 zcmbtWYjYE26n@@Zx@lNS4c8V3Q9#llYztVFh6=r5Yf7;xXz}`X^R`{O*$tcBf|p<7 z_!DqO9cgFmjDGWjzsYetXLm_c8Y$?P_kNyp&T}rifByaJZvd{~+br^!?8Zq<@o+1P z_c5KN)eNsb&`{AZn?(kb8QkW1t{ZpoVU~t@-c~bM;Q4MA8t$>oM;boP;vg0o<-Ueb z`0!I6KFi_=KIhdJeD@^}Qp1u4Q$tOl%Tvw3_65c&uD4>yhBQ}|u_l+-q+!~Ya{Q*z z@Z4HpD$lSSUwMwSo5r2!XkI-GRMVdiF-8P>JY`GYT32(+%S}b7Vunl>{iPMq)q(<{e+zrUiR6rmCr}oeyka5jWHzmye zsxQs8S=or$)z_sRs5x)a^IT6L9d3z;;sAwypVb@>39}W6(iu_IWi=;UhupjJ67GfW z+tOJv=9V5P)29#|B62$%cUP?yM{>{v%-s(9bxN3v zY^RPhSk_U%>pI@RX$>nnR{7*z4D!knIQy?rrZBaVLg47O-&NnUoRzV_vTKTH@8^`D z#|61eC0ckSEuXFn%dS_@@c?TYY#ntB>u})mNrR$as5dX_cnFV8Z0hh42%PI^%j9Ea zQWm;S%Nmwh*KrMl0)uFm1$wu+ zic&}5ct_VFae@4{z%=(cE^1Y4`5UG$^`2KUty7z%8j&0yfzx{z0N0CbG?YW-bD`Ll z)=~Va0c=~nBPA$5Pj+_gXQwNb>FVV8-0g{~fJO~wU3o8O7uEKO;>?S*XM8s@ z==q9`Ef+Z4$sb}kP26)U$?qui`M}}6)rto>c&r^@VbyaVan~Q=FGP>;HZ1dw^n5A= z*AXanY{_oD?mEW8j<;OdOS0Megc0q#v$3P#O!#r?Hr;x|R+Kv!qK|DJ)L@Z7jynK- z{&U;}Xx>NXXKRCtQyPjr;PIT~}}b)Lpk=q9Kh0xQya zfdokm11QnVD^?@I6)Q^6`aIr>)eO^|pu0r!36CLiSNQsDh;}5pZJ{M_0T=0Ba9PqF zn|dwW1A6HfBsP#LZ6N()3oRQWl<3Fge2A!r=z5GeMDM!j#U=VzVa&@h=J)hIO?PF+ zJau6MUBjuvsqawQg0{GY%;F}xbJ-2-`x!c4@83XA?f~sK(7TD(et%{|vW=@jqId|s zr0Xi_VXjIchvk-cW%?I89okpM+ZZM$S>h^roWeVyrQ$DK6?6vr+e(I3YqX}u9ox-e zW3KZN7r8%>+rq)cc(;i|Y}sLEC$g-EW{`#~;r9`fY*oZiKDLVW5@uhhG)3SC>42In zt(g1?B8nCEZ{bL+=4hnt7){Ad^lgVC3hZc`lKxnUpd1{D5e;&M5z{RH11;LKVysO# z-4ZV2dMwO&#lpMu=~kO$q9r-nlDrg4W+LnRpU=lin|vmc@4ZPG^IpvS5XFcKMk{eX s9Dm;HzBX|!R=*>RA=r%=%!rLiQ?yRJ2Ab#5Kf3F+@IeGYG?%C6zkfXga1l2=jN+_|a~?); zzKKJ)Q2(#`$?l?<55-)Ph>yg4EatM9D=y|-Tve!dbslXfoa{W(Kj^T?<8BzGNmOK+ zN%C+ZEt34x^l{RWusMZ8dD@Gk`#Q_xJWdmZb|=l&L*3KSh6y+I+NNgy)>fK?%P&MS z*IUtjb8l^}Z*m2-$l@l;(oEqB#=<1KUR$H?@g0`m@BjgvUPrci6oK2&fP z(p94{+KCf$x7b=S*|J{g($ngsk?ua!SuFd?>Es)6U*YDEHoGw?_QPJ5t`?EW!Z^uI zmgsIjykEMO%=d-q=S!wvbaOW1tfmR;+}=0m#a9s%E_MA_$5g``@a?2je(t`O;L7&3W;ZqzWSqUlpbbIzyohbwi*!NtU9C)v5aA4$c*Dm8-t=)D zQ$F4;%?Tg#Xe-RWOy>VhUt!F4W$);{l}9F`+|xU*J92ljqDK)1qvTlHcVx;GsCq^U zjdhdTSA}zYo1=Wo4tpmL{<|qG>`F>@B_Xa2C{c#?E$;3zLvfu9oDQeqr zf^pQh;r_w15xh;?VkB^qc7_#>5$io1p~o-t@eZcB_wWQ9*O%ggpG!(w;h6K=wUqoPNcUE)rn z8fCTDcB;uY3%CpyS9URIR27c#OpuOSv=bHDZTf`2Cum1cKY{-%ccn51Y@G#*HABvf z8g%PXvBUPh3f?SWq0)nuwT36#j-wvFmEFYQ1Y!RG`YX#`p)VY0)MnpvId1ABF*@U1o50x9Su4IL5?V z{NV%mP{y-c42htTu*uHsIeX4`Zr{w0pWnU%xQEI-vbf}64woHVz!e9}xSGSY9Ioe3 zV3-SgDk!WnJZc!*_4t5`p7i#4XPFXT&P)GGuC|Eg2RXO3SBFza{NU-ttMa*f4_oyWFZ+9x5|&84T{W z*3#B|9t4uajoI=jtWj0DzT`n6lf7niM|G?9(qn3hNCbw; zgk1jN%#haRH3O?NI6Ee?LT;(J{H3whSN8Y;r%pZ5!Z(4^UCN?o+Bw{CaTB*(9KpN` z2Spbplyg{daT|AN;xi4ru*Q&&BYQsAUGG_IPl}MCJP!J2m2o={NBMBvcnZo<$yRn% zK%6}6J59X#zcUJ|jZyMDG(FuNYNAm2UXu^1F$>ehcy+iCGN5C!N(bfbq!3f2T`f&u z!r%QjqfSTK=f?O%#{1JWY~)Ead=<$^wv&Rk=?ciwg+SZ}%jaWy&{dJ5*TGTJy7aY9 za-u1(d;%+fLFz+dbBxy56F9(eT9<}?3phbij1Z#{ZLx@xL##U4Wayt;{s{IJ>Cec# zdrxu-BeXXO+A`9y9e$&^IEB+mG?yZsA&)f9CZ6YTp7spgXcv*EFNdtMwA1qosFY_% diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$2.class b/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$2.class deleted file mode 100644 index e9d1845380a8bdeb5e531648e5e2eec37e46ce6b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1280 zcmbtU+iuf95IviuHFXUQ1zz6VAh_M?~i3CIxOWyU2XU^Qt?vJ0}z5}>})j7=IqKjEva&aD)T`c2@gR2g% zIVdsA#sd|V))*eNjP3h;kBfoycX@A@`yxjZUAX%(+CY?i0Qs%$e98m2257FtTnr%Bk6_9gEGq*-hk!GmpXRVojag|rL? z?}^sZHUb_+lEjVa@+j4)DkqdYN-PDgjVsH9q0xRQW2;2ux3j%rbWio`&B{Y+ipT_p z)iJsJ!EQ_G zJh&)(sG#a##ltP!rio8A@KTMTm`3&kuKWJ8&W;o@Lv)lQlXh`{P1PkaoQ? zh6#W7-%NTvX`dSt5E<`I(y&8MhWXgUJZQ_%*!`@kCfx%wbSn_S!HUI{BXnQn=yfqq zT93ZgNzOFYl}}*RFUWn!Y>v{JdIE27jMk;0-vW-4lp>^P#9b`n#1N}VHU;|URX>7# zMgB7i@7|M~!y($+3~d?t)DFMVJes5PrP2D+_^@0plG86lgnWCWiP@7}^F%R4pKe_T6>e)>~p%P6FB=K!`ts zka!ax_yPPV#5pUf#0p|sq}aL3-E()(o&DqIx9BAHkv;-H=F>qhrfztOBdBvM2s zFw`gH@(*W*oHnl+Sd+oqFsT*tv2x|l@1efF#b0q^>X8V!%7yZ>6+>q-0En2?6??tTaxdlFOBC>1)^+cGkCC#$SUcR_(}1{!j(QpwE;-5FE# zdN@p4pT1T|&NQ{fPhhn#nEH^}9HG^D0&j4X*7=d&ERK=n5F8rKFXnK3gw-UQJpJ=) zAHlvN_Zj(j?@6A*80~F_wt$?o!*4VnCvY;0=2N6o)7Ei&(dG&eD!1F_R!GwVe23|B_;w1ww8@R+^y1o-MTb_`DVWfPU z-{E#!x}IGTa*1KM;?{jG+acY)arAb%9E5e7x46>~_AcMq<+kIw!k3ZV3WHkP5uxq+ zQiMMDBKwALg0OalOQ*54N{I)TUEh_f49R@qCPQjHsA&MMFRr(nRS|CSsz;V=IdHgl zlZUQ4ca168a3h9^jYI5x=>^@SH+)}&>mH9Hb-Z-~sB4bXZnixxgK+i{b*PObuhkfq z^Cu_}bD^3as47|*E}b|FL#86Rv%AS#nj)?4WfItc;yM(~;0}>##2Z4)-ci^+k0F;Y z#JYOiuiG1>mm->?-ulmr|EIcH3EH6}UU!wWxmdJ5q8HWprIE!L!Ks2ahJwR(=$lou!5kbo06$7s_ynXzip|9~r zO&G&yufaE~w}m4aE*`&>p0$5l&}?lr!rpMjotwzKCPp4O5hAq+BQbt$H;O8N!kIU z^Lt?P`$%l>A^90nceVF8eNU4I*w;w0Z!p5XMUHIgI0h4#q7$RBI@`_n0o6#6ZF>Gw zq_#Kjsb%4F3?y)?*!dVk*NdIf!Ci7PsQd&usuXHXLKVxZVD=ph_C2!f2b^U;>Vi3n zprqY`YB!B1spL7bK90mgx=0$Z3_N9E2F$?If9XbN4}pC|6$Z$=bQIWN0;|&>BC`); zJ3F+8;R6NJyqqd_CR1IYGnk{mITZ971D(Pk0sV=9{!BoBL5}?ji~WWo`&|RJsGMSc z4$mn5P&$NO>VrOl_X*w~7-xU>iKf_AqABLGU=JxMp_z+m>yAl*GS8Do0t=X=)g)wT JT%;OP{{WFP)LQ@m diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$ReadyCheckResult.class b/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator$ReadyCheckResult.class deleted file mode 100644 index ebd6d293674be6bbe55017bcb8f4a0e83fa19b93..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 915 zcmbtS%We}f6g^I+2^ogAX_G*C7AR5~BrK%vQYxVU5>*H&sk-RSaZKW7;)y&n(5(0- z77-E)mOT1Vi0g)j(ju|w#=Z~xo^$TK$3H%O2C#*DJ_@*A!UAr%+cl417Mj-HdIYi?NUc#fRd}P;eQKMzP_p zT78FpSB*MK+YX>HP;&TGMPl-jaM&)xYG#HeerKALRyKKdq?42~2MmR6ZFIWBP}!VE ztnVJ&-)Ha|RNs-+Mzyk0SJ^$$jVQ9xjHQV7h1Jgg74*`9CdlQ#G} z5=r93zy61wNGp>jd~D-hCY9ydq{lc|w!%Z&F*Q zl>N7tGv$6K&a70=wNqO=L*b#DUVtUs4N$~|02dJixP-+bHW*rG$8{cg)Y6k>iG=C% zSKUJ;sipZ%H=xJoLc$~3&jM|39%Uy3w1wp5;FC{^$Sylkbkn|&+GWa>$$L*f7U-^q zAHl*0Q{=++De_^}o5K4*i9A;5thq<*3q1A}CH4)A>^my#7^{>E;kg2+;4&PFW*#nfB_gfs%#IJ}M_N;lyjg!bmtd zJ}i5c)&v?7I!66C@WrlG!l{fI-{dBD?+71u?oT-!?j8}sZjtc_oqlfAr|Nv9>^Cve zTv_QCQs^ULbD!T9LpRP#e!Efselk%e8>q6T@SG3WE^?+3AY{LmT!50j zDS~8kibk?EMfj8J0Xl5g85MMakS@`tD=gAAR_Pa(_$|5VXP!0A7^&u^I`Ygx9eM5s U#}7FR7;lw*#2V)C6f4;H2LzVfA^-pY diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator.class b/out/production/clients/org/apache/kafka/clients/producer/internals/RecordAccumulator.class deleted file mode 100644 index b2ce89ddba98d74058a3564dc6fcbed1e6d546b8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16641 zcmd5@34B!5)j#KDX5K6>I|)2c91t;NA&3xkqC$u)K{f%!ptcOjOBk8V#F+_#OKq*T z)o$9)){53zt)+`?X$yf!Ef!nszG?S;x2x7JpS6{I|8wt~CCQ9Jm*4MmdH22h?mhSH z+naph_#>Ypq6Nm90Nuq)g4Dqsa<^12ohDxxBrh+MhvjlvA(xeM>5|JTxm;xO>L8Wt zeQW%@*5q|Ts^;}3UmT=bJ-dA?e5s!|2WbjlW^#9sPT?&=dFH@n!GJYXUJ2Jtm*Z$mGx?V4rHwchhLwp;*Q4V-hkl)O2G5M`>%I!k+9dgv$-e z-Q;%!X(M--e3$peQT>lVduNF6<~3sz~l#Ie6Pt51^Iov zSC+kBzj%H(eb`CI&LrpaB2p5(w_+{ri_oStN=cWE@!v)xHESzU>Qlj@8|(`k9E+TcWc zcXw`gdUkAZ(nIkKQ;puyF*xWXdb5*d@nk<3xht_gmF!PBX&ljb@Sqv7XirHFPC1j)q?yRhW!{5ZEU4Y>Zv&FolFt0US#vnC2-Iqk~b9+qxs# zwa2^_TsnDHv6ijq8P%EN*_!AeHUu`t`V-O2Pzq=4Ie{(bmBeFHaxm7jKAOtJGT>%=0jvw1 ztB0Io8%wjGK=ax!z`xLrSWYdOHW$I-<$!hc^b8FQ#i1#wStV2)$N0{RC1RNinC@)o zT2C(P%EWGn3Zk{+JiuTC=JHekI>Y^T%DW^Y{h9g;{)r@IP!wCyyB0t8mP${4*GcA$Q zUP+%XtH9EVYoc%piT+L7QwSN5b@K_}6ipAMM3u2WsI+hC?uxlY(MhC}DW;bvU$4Y5>6c&Yveor&g_<~)RVN`bpN>&X50^fEoNv8+ z>E%m{aB#dycShs!ZHPtf#cXo?;(YjVV|anPbY}{pI=JQQ8`5FpL{P~?U#@m(h~Q?l zH=2n8ng>}05{Pq8fUG<$4MpWA0AZ$-;GrUOi$EFUxYsxX$<%JWJ&mni6*ifS-m8&z z6o^uVEG0@TF94oE(|Tb~wAS>++?*VP`Zx1TW`@ z8&_g0ZYJa9mJ>45>hXyGJ24mTcVh<1gTSL4DiTB_$|n87cv^}}c9l4`vPAM4@TgZ(35?=Lz_?5? zs9MsU$Q|Ja+gJVj%0D*Cg_MJ0Rq$u)`udO+i*s{&B>|I+&ysv?Q*tvVzkS0SfHdiG>a*!7UMXJDcJlU$tqh50D`j8cqpb{&~EY%P5zO| zKSuG0dJm*+kESn4K#U->tYHqy7_Ta0m^xoxH^&~{-UVIFS2vRJsf1n?6M)+nO9?a2 zEGflY^4*o_ONxewNX8HmxSpW?@DU7ULZlPDOLg*`o>xrHxBL0cVKSj09^!Uk&u-M63k6rzlAmaZz zIdC${X)c#!sJ%2n;|#4(6SF~(;jwo9>kx`D=S0N}pEte)L0Blb}AjEt$&X3rLlHh?BI`i`=7EwKjlv zw5JCRhFSCG&12gAui#x!xSnJtr_&O*kznis0{-)Q(6LYjZg69ADAnUEk4g1duXCY- zqvnbyvgl#@utmeX(4zOy0~WoT?njB2cTz9Yf|0F1v* zax%LoFsoEU!J_?iz@m@PM=kmseV%FN#ACF7$k$5(1R%J?28IU0(ShVpA|r&1!xL+uo`|L4o@hd- z3^!b}-2u+2Fo;SdGvOHiONZUAUwpT$>76t8-=<+cEWkxOS6yaN-zKOlVqOCAsKQGy z+~y!LRCt9OoS4FW2wgqxxq5|;SVF`Qmd;=}j&SuK!2fKfDK7rGRI4Q`W&RES78%>b z_sW%S7V@Gyom8~n2@g7{mNcrUL~nt%X*gj?Rnx#6q()OtxHpwVJ=h!W9TM(jCF!27 zJZ0lVz0$hwnrtg_5HvtH01cCWXYud(4;KHCpJ6&pIT%%4cg5pQKRQoX>g37`xS-Bx zLbxqRg`<5)^c7FDUO_2|YAw$d?R?ha7M^SIpZGbZMHA6_IdxY~-QqvPR|AMrC5eV3$&viPq&RZ#w$#s4d;m_6}0%|tDgRt*pJ*7FwsonJ8d9~S?U zU$p35bRSbkhgLjM$!Iv?Tobs;c7bKo8k0m{=37P`0*XakDkFHUkiq*+P zzh&5h_EgK5W}Jeg-ZD-#rX#brjMKn>Bh1es4MaJUhm@;TEMtbKQAxp!O{6SirmU{a z15()(DhSIs-I!$=XBe|B<4k@IVlx^nJ{!mL?`3X|(ZIC!WO(K3)|kf4*xFSj(P$Y> zJk>OsEu+PN>m0`|(8+Yl#AEYmtYx%9VU2mlSz{Q~kxE5(OZtuM*)rxE@JVNs0yCy& z5Tq7Z#zFzwCX6^2>4vdLWO|Nqu7HBlOv*D*9i+Hu84+2~4n?3J8|PWZ`G5iSy8xR6 z+G4pZF*;0RsbzE;7n;U0%UBNRe2!!^4~eK(@In;Z_`syT!sH!?#L*r{3ijP|En}t8 z1;9KXjf{yjmDb(~5;2WcmT{4>nyH9)91kbk+R3t6IYjhOlh^2$f!5BkJ*byRr(BOQ zruPNX^M)biR4O$zh;BxKt1ZuSURhF2{<#b5<$DS|_C&5|g>gFz=qBa2DHYQOsk#-S zG-$CkbV+WWX;{_u-}s7Bx>u{(+AAA$5jk<$iOgi1E(+Are>=$s8_q1n&}{J32_x!` zK(x14KQL`9cC;3RUZNWEB^wBN*tzDrh{T5|t9t1uyYkxNet{|0UehOLp>0&VQOTEd znPXR?00T+*$oHQnMv@36p-Oy9OgRHcG&>T#DxUtbDK5ZqF9d+)3=C$n-yc*{McLK3l_l-eW%l@=BY-`nyoS4DjPD1ECCgD=dP`3m z$PUEfs8du~0Dl8h!7m-Q6jL&7_hS>F(jU5u;-s4WrjerMNmyV5PN4H8F7lD9hldI^aoBHlhh05Rd8d#@n!J zm8eYFKny>!ndHhUNAD`-YRp)Anb-tC<~i7zMD^FBJ}3#(`J&_o&$qG(4{h<+WtuTJHQ10Ce?rY(wA`(IQqQcY>73S#22SC`2UEK#K|gZ1>o4 zYZUsGpc)|$-Af(*fGS*>xYj~6l$z-;?>gDj9$AzUvoH*|%+`^E>Eu2DyQ%gD`I48$ zHzfJx7zA=sRbeq0@Z_pvi?IiK50&@yCKEt6+_0*E?HkZ~f?LZaxGEu-JU%QekM5~}cPT0rl?olwzP4pqWjd+`E zkL?|%>1x4g%{Afdf*G=4X2d58eAxx3H`mO{E;vIL%#N6{z{G;4nlp!~A>vn%<}}wd z>V>x7#&Jy$Me{JVGU=FI{6D&D=>}$hGy+IaKe-HlxD5>dhCYs@KcTZ$v;eUH)HhNP34EF zwYz5CK|1S5W79#JAMx7W?uhTG?H!@B8CWien6`P0R@!EBi|rYvg%Q8)KSt*qrnYYI z3d;rb-7+q`!ve*ah;4$hD5TXSu6dA!nG6F@&2-*HT8KHf)iTy$RA2>uSBf-P8 zpu5SI=lT&s5`K_YMM8GSJ@F7-^cbyb;Cd`15Msqa4m{G!tv?$La^5nGceUk5FMai`n=DH6QL_5xzn#g-ck0 z4^dacjIW26y$v6j?tynb3O)QXeCw0&tlxuI{RJ5FEd1(UX#}eO2z%+1TtScWIryy8 zNssXc`ZV{`QQk$L;n&b-`8GPn`|<7wJpqh9uPo`Ez+x%v;hXd=;546B(YNV4WCC8N znhWBDud2BK;0;Jmn1XM+>Qd|2`{CWUh2s~<-b$yGLxQrfz0F7%_jXBXOGe{ z3QfeyeVfCKEBsvHVs=u?>E)q&09r%m7pQiWwo&;@BlMR%TDST6HW#gbVG~k|6FUhg zx+n1{)fQv8%P_+D+$c21I}WgV0i7r42-Dy2|NqkSxFWxRZkDNF+Sb&)pVoP{9-wti z;RCe3v1vbDEY8?td%7duqqe7Mgf7kBQ0=C@H)TiXg;~zzAK=bT`kGyDKCjU5WS+9HCdp{VU;UalcjWqYs4MZFNT~j@s4;q0*7iR%urrql0!OxPKqj%eWp0SkdsE zIgF~pYT8*;T~~Fta{AU@q<#A-w5Ybew!Ydy3~(0J3D9gH1en4=*fzPoy1w?#d$Kf6 zshhlqrqxZ^L-lq3JyZ=?o;`qkkovN*v(2uv@20xCsy$St3FtRy^Io#*>*~FSX!|0s zU7=KADn@(ik-f*icbH<`wx`}(XC9=h_Cb8N@ETqZrmyEqWOP*{A%cMT0`yyCkH4d8 z`U712ACWCSL!00{x6*USCI1Z9`4>tet8u4OtT2pK zpY=4dSA9YaPS7xFU88u2>Wyfw{PQ(N289q}A(Q96Gg`QfJ?e;EHua_7ZpdyniL zJzDlKSaud&fJj5M7;#(1Wr*1_YELy2#1KyctLvzl>**|>Oc5|-G265T7_R4ON`-{^ z&B(dMAC*C2o0NDFiD`o?-p}H`9+c=+Pv=(EU{1W%bn^a_D#7^U)T($d0a&>7Jeis! z(-?-BDCd;T@SG_0iyQ=G|_s}L(ns^J3bnVtt-%GWU zgwjAYF{W)i_xw< zq6qVB&5{Pn>MWu4{?VDXPcZ_CyBzxJGPtngU$%YwAe`Ijr(ieHFQAAR`4pa^p0!%+ zMCc)&1KVnZ`ZvQOTJU#vbKzlIX(`X6i+Dc#{{l+zLio8hx{Vjn?MNi=<#Xvh96>U9 z9umm&@w(FxVp^c$R6ZT^qV}6}o@sN=Gi?T56;MOG{HJlgK6Q5P)H9_H2aH)<3utO5H(;lj!s{qh{vuVGJV!aF#aZVB-^6fGG7F~! z3v0xGaF?UNze@X&;!8|!$|bZ!;;qg=&Uuzt7gB-0)6`LzlW!~p;GxP?NY{|xmHOoDm$)Ys5C9YnqRdw|SqMJ4`phqFy^fH$$JstlEctT7T-n8{%$w^(|C` zHqIozf@brTIaeaw%l1ANxaoZbRXT`aJH~?nBF)- zZ^B<_KqpjGnv5Ieau=*=-cN5Hrnl@J{gtMmT~VO9Q3H)2l#^k;i5mH}G@ozI@j8t7 zC`qqy{pJ@GtEhSK79E!Zwfy8tMQ()t<^kv!uy_F?=b_lFBO?9&KT`8mSvE`(N$ z0>KN$vbYXyn96*ByVOY9e7p!I(kAuu`MKj~sI9P>n!66sJ3m2-1p_HsfRTC?fe2f0I9@&Et; diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch$Thunk.class b/out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch$Thunk.class deleted file mode 100644 index 9bd35ae8e366b2250e3ea25a675dca706d4171d0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 788 zcmbVKO-sW-5Ph4rR%5jF`x`uX&??4*;K47jAP7}KMesP?t~NGq>Sp8r@*si-e}F$q zoNYzDSgbpkGoP*GaYn)@q5l38_u8 zslparwzs4systfR-fQi)}D}iGTKbasx&<2A-7591i&>@#5=(=2250~q<&3IG5A diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch.class b/out/production/clients/org/apache/kafka/clients/producer/internals/RecordBatch.class deleted file mode 100644 index 52bcf3ad70c7512d0437381ea5dfc23f8a530bb7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4607 zcmbVPiC+}w8GdFDW?2SA3{}*WHJW5u!Bt~T1I7fXR47CRYl@^H!|t$*?9QySvmhpE zdL%v5D^0KTh)oZBCDlZ((4=?Lrg!@f^zX>eU)$&VW`P0L3jK-i_sw^_-}}7xzV+`{ zUk7jq|BPZgoJt%-K@eAvq97B3ObT*AkSP@xqp0+bKC0reC^YZm@hCooCj|L$6d%Dy zqxcw}RPj_4TkvrepNOI!pRB^C@M*#4GZ8$k;Qi-v-T?z%=U?~+et$}x1;NcVBB^zb|F=%(d;SMxEq-8(Aw zRgW=e`{cWNUQdk~$#H#jTu-L#T+X(V-jQUVk+YpC@67^bs&VbSnL4REuIZY#r67Jo z4yRX~oph3>24v+TL^egq|$AW;WSt7Q|lVfSIv$@3B*>wvTM+aZdpqx|DLs2ecTDkJS`4{k zCbe--nTKwyTOT~a3b=XtrDac2gRV2xp3fUrT7erM>KIy&%{ujLc0^B&w>51@2+u7% zQThy5PwTEOlsx)>xyuc>1;>pkA{^>qKu+k{q6i`?QOc{Jc^v{Tp`$|&xP;lK=cVqM zZ)mKvZINdz-sAaOR<|*kZNjXPTnVFsgX?&9jb6gi@fF;+j?2qY z?l%Xu)6_sx%f+DA?({v<4&7E;A-^y7qwUm=?3# zE%7Gb4R4hejvPs4OMw)O3)Msi?4rXwVG1v=^@+Ssgt&$t*c#rAdo;8P(k93e+^gX} zLGDMph7NQpxL+jImC5bKF;FhQXm}5fDcG`{S=t>(pAzXsLl@qw;>Q|(f}d)55ihCu znT9L4s^Vo0ui)AS-!JE|0_wDadpCl^r!LRJg2{}ESq*cT*KinjD>zj`n~vp-LP5_M zg_vvekGZ`iznI)(k3JroG~Kb7uz$>!4zM_IOSjT7!*Oh;g|T8hF{}13ijU|(n^ol@P3@p@LT*&;CVyCSq!OoQ^W7^heZp? zx0k$ELH98sUd*^)q*xZLOsrTioEFxW2{Ub^V~cSgv#mun#7233tr}W2yoEn%co=_D zuyq-y=fWLDGn+OX1v`AYkFPA0^vf6VXF>kLBHA2tmP@2}a_pmMX zAH=4oSvKqAzMSU;Mw`&CPZhnk$d0P4M<+(<&G8<8q7>UqleI$i{uYsXa~(a!92xh& zQOc`BWu6t?L_v0rOtjQ@Fe6fWeZ1TMTuDYs=oneuH7AyGKQFPfY$>N(&%IW)d_B?@ ziM1}X+N6t?V0Ua=opvstHN5V!r`%hsZ+kws)H3*RFXaMhps>tV?2XE3%Xg`Sf9;C! z|KsHZDi7YrU`=rptf&hLnAIKZD(+xcUzij)P0Ges0=IkHA`Y!_vk<7QE*<@(Xm z$f7ORS-0iL(k(}pUXFF5&XU`4&I9O1nDZXIkF=L7FCoYk;AmnNN+LLofFMCZLWy0| z2n$k?Xr6|e*gK6#VgZ$dvxrWkic^i#>S@$mmAv~%_Yr}hA?)T)ygjJF?ffN}#5U~5 ztwebnao&l$NDoo!ci{n^PVh7(zzLk>C}yXKK#?ezuua-V;o|I+S}E>Pb~^o3_(N2; z0Ck9>wGupwu7o$d1aBA*;sX-iFa|jj@Pb#$uuAP+svEomtw1C{@P zOD|(vT|>od*ghDf=#hcJP~#kS3=D=_2IjFdh~VK+eMo?s$E^xpKw8Y}L-TlNfc-;# zYq;f2G%cV}s3OwX5Wa@k40g@pw$=*qvAdx{na7?WE@PAYR$5A*yei(lL)>c!&*RPz zo(w6M|MM&tgia%kEi;H;rMw3)iHij9sjUOGl=~F@U;sNf#;B451l~^g$0YC}j1l@c z5l=$r&0#!-NANVyJ%bdUM_Q`%IPnjmnmalwc>6F$xEto#TAnaC+YK9|j1u8bmzETM z2^ktq_|ql6W0uss6&2U9Te&Wp&~tC2szb&2=C{d25lw`rbXqhZFUA+pL|;$T?VUlh zfI5Sgx_#a}FoR^>e$HobK=?{P`bveoSCoQl9*>eh5m*${3%^}*hc>S}IETAl!8Q>| XijToAvdc@R=NT&TL8iStb(E z2Y-M+%6MjNgrFtJCOdP^%(;AX&NuVp=eO?w7O^-C50h?~@-c+_KJH?A1P@Sh(nAlA zJUsSLW*9%@amk2Co0fg?@eMbLOr%x}`Q|=9p<2Q?|L0Q4WSL>5e3<}e@Km-La&_Gj z4ysh*Rkyt(%og8?X=SXbBOY&aBc0w`%sKoFK|?8F>M^%gkoxQrSV}cIj;~c+w>z;& zDBk#o43+-*&p1-bP2Dw-Sds3GBfkqf=h{Ryc&VaTTd8)} zMY5+`9x4H%jLDkX5$n^n$jjq7pQFti@KRD zyT155ii9PS&lAnxozv64*#4PQx7%8!SILH3vMngz^xwxmYeiIT%Q%*nZl`JyRSDzT z3C5bk*trzO3sde?LERg$R@ufBJt{+B7)GX0aJ&L&CX&&^lc9eUdD<A$5NriwB#)=gd0fK{(#R+rqz;KMglhD31WUH89jZ@)^f5I0rfg**?;ua;#;5P2ioTJC_9uwT36#fQ~EwBjGdcRd&E6~aUMg4&>MrxB*w23wSx8JLs7%5hB=1hHn$DGD?OvkP-{AF$K+ew>I!p{uWxeGvZYYIXKuMpyKf2Cl*$*b z;~DoU8$shkKuLx z|KY|!s2A24VhyJ)7&1+%#FKunCERDcWz)`N)3La{#$6f6!_8PwB8G)AyzNcjs89j4 zp5R{JC1=%96>xezN144r3(P?)8jf0*o%(WNf$C?40z+v;IDau_h$`n516wBEtU7(y z5{oiOIW_1|X@hTbD#=4-*^Vbwhg^4^wvHT1I&@@pB#{b4Q%4yo4HX?#%xkFWxP{w9 zkE8Q%e|Z?PK^UgZRmWUuZ3xR}C?3S!pJ!0eK?Cm}R6gty>l-g7@8b2x|dflG?<0xpKpT?#EPQ&u48G_H_iPzDiN&)_)C8p-ti E0455ifB*mh diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics$2.class b/out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics$2.class deleted file mode 100644 index f4935876daeb5799561411aa3b9e46b2e97dae65..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1501 zcmbtU?QRl56g>mT23!Ow)NiGV3be8m@hccZq?$-jf1pkL=dcX0?J`Stml|KeKR$r* z8H`D*#veX_4`sZ&P!n2eE$(J_=FZGL_sqR#=I5{PKLAv4FM?#xXVefO9!&s zUBr;0JHl;^d`8`o`Ep3Uq%GQK0~xx~q9CCmkVtqU0;nZL(Pb8(ZviQqO;fK(2rMc+A&(>OyGIr)$zaGb@t31AB$dVzGNuno42 z&^H|Wg0Og(f?|E5i2%Xu!hZB&Gd_d#AJ{a`ktEVOa diff --git a/out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics.class b/out/production/clients/org/apache/kafka/clients/producer/internals/Sender$SenderMetrics.class deleted file mode 100644 index 4678a4241066af31874547067ac1dd352be1471f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 9145 zcmcIq3w&H-@6v}p#m0EbitBRYgyC<6&F`!6?Ut;Dxj_p&|MW|q3r*B_uiR1$)peX zZGXSs$M>CczVrB>^PTTbo;>#DM~P@D57$zLz7nD=9g(L;HTuU8Rnrlj9t+V|>1%cL zb^0fr9@puK5PgHbDMa7W=t-TvtNYuy{6Oa8XXJK26|PaH$t=tvYBva);S>eDv)xuPWyDOfr%W{S<_k9xmM?p z&UNq%8+BaI4R8-P)^QV0spDooO)gV4o>oU!@)V6*YUyg89^x4s4pD|r5Ahj%rp7a6 z_$;`K&z7f1h-dL^xy+HVxia=Pxtt@Hc_D7)`8v1hyg=i2jiXHU{Z=|**;}o=os8$0 zjP7*Wve&1~T+YI)HsVdp@5$IhF*9q%hppHybH^?-7SD`~WYRI0B<9Lk#Z+g5YZQ`m z8Jnr~gq#kgFj}^qu|3Ad>k3xE8c2><-pCZ&!dNcvADeDl@r<3wZL{n?XV#-`95nOs z;r`@}MgD0}IcrSQ{ZU&C~5B^R7fK!rssJAD47Ox5c%35%(*Cz-Z-3nPP; zJzx%|Fw)$UiJPfQO*<*??qIc8#?)0|i4&a&KDIYe+ElTF55U;n==A+fA?;vj%bii2A!WJ9+KdR)MT&BI|( zUA(oo%;ImO)U9`z4s+)`;)+tX^s6SJ%``;~5YwuGVJl+pwrq3AilkM^ zj%0R322oUUksLOcNY=6=S<}uZ^VqKBE$VK69b#Z7)BMR6c9o9h@+S7Eb-RZQUd5}+ zmB5zAv_h1?)2S+O(ra&(RhoR;Zq!B$4Z2wzH+B5DGd_}tu!@fym_^*o??7x+ zrx48pEC$^mqNh1D(b9w?dy`+b&_|N#$VhIS$Ve_L0l-uq{&ANxUDC#u1P6k>NDy~c&eqBnab_jjA&Y{~%C*a4nlf$s=vq)xFj$aqY2W`h+S5nEf z4{J8KdP|#%%4SeOtU+@rh9b$Z0lVfb^NSS+IC}fi#gUQlKq9U{u?VV07Lu>`OICxf z{z zg5I@N`k=+9t^@Wy>`OS&&Rx6^C)Lew>ZCc}>0fwCsWKA@ zrpqe1cfujH!f9i?p=PdN%Xu8fuKX$q{b(i0rv|`P71lkG1hsZ8Y;%0i)L%)Dw_>4M zA!ugNP{D!G3K8*x#R-yID-l6GpYif$YU@NoP}B}tc_-jZ%PUBqEZ$|&^>S#T{WKFXdh@=Cbn{Vw-n3Io?OU z0r^x?q{K>-R`+sBra6-p498cOh+m<2I+O1$q*4&<5C8Eh5x!4KL*#z@2s&>txpjl2 zV#lPBbO`aGb0lMr_9+=T7}P$Qo|WivT$SFcJ1t+2pgz2fj~tD@J=Sv)pI>sJV%j!)v#&F+{Te?ev~5kdY#2*`4BbBIT%kD3*FT6M=6P2JH@5;Zl+$r|rLV!D;yiR-(Fs*-pQ5(T!mEqIu? z?H~o(+8?H>!&JTfAl39nn}Z7;A?*<9RdiW%ZD(~`dvj=v>K-QJFx78wZU|R5H%1Rq zQ)f-v0W!ihhiM8Jo5$!hH9r;e)5_;tFh3phkTgR{3MO`3cr7RFW1cNsB?bq)55QFA0=O73DHTxg3-$ij>R7QJ$+PS1QVL zLD^ZPTq%^E9cq3R=2w@`d)le_^Dw`rIA10FYXO*pbp9BvQxsjGSnp7@%k%~{y%Ez7 z0&-RKZd@P2Pd9{)t`-el1NylZ9-=usO!IgrE#h6Yj#G3okC4e}x{kAS6JJMnuuXSy zp6=xWJGdJ=rd^W#>V|*)G?suXCeixe1cXN_&<57Mu zp7!uZ`F(sh-_8&49ejj8z)$gBeuh8Dzu*t?%lzSh#vchx=Z^*E@y7!z`OZK$-xau= zKS{Si@1;~D2}19o_u@^SBq0OT5?D`r=zXwmUSKi3pKiyGW$OYn=nnb-IQ|l(29nI7 z+6yX{CfP~5z@wmOgg0(EOq+m{3z|2NQTGD;Z{C8-7+ts(=~UG|Mm@dJZ}4_E9XqSx z<{)Ocn3EXTy1-4(UX_HxRoAPc(iW}>S0AK{I)l-0(2vM*Yi@Ja?9^n@um=84T?x3j zC?KpKS3;jFp;k$#6$v3FVMa+pNFd>_U5emb8&OBeFz=pufAF6S@M5PuQh zBp;-`{1A>kU!wc?0FCh&eU}f?Q3T~ze28A*!}#g%EBO9?gxmR1eCK-%N2RagWb-vn z^Vb2`#}%wT0RY5!tAbE9R){JH)nJWj3PM5JOE1!g0V|F6(R1_>RB`;IN}s2XVyu>q z(x>QS7z=ST?WB)mtd3{WHS|{)!{Llt>8~+X&)3jQ`UJ)r1a2OP@w5-K`1J${c_*F% zu=`573v^XQ=sMc%g^&IQo~k2$op09YZ#DWmF#bJ#5=_`a0(S$;O%T2leuob(IYiim z*8!B5F&%>2NML)2a4y`(o8<}J$zyb7XHC`eV7TTGT?Gm)j6FtC%#bHVadoF2)(_Cz z$LN~Q+O{x2b*(>JWT2QXogo!;Y)w6nkog!5cGiXKggd@xj1m*sJj5G{%_y-*c6TgX&a+ZpckPkMX+e38 zU%NcG|Jd{3oi7q3mwrEe4pais zFjc=!r&nwA`C~LoqX!sIuku;1K$S0GIzV3p;XlxWxB|Gyr5fZG(H3~O2;45bEIM<~ zEJVN$8&#XV8YR6Bc0t6xGZ+p&L3QDv!%^Do&JxM|YbgI4n$Ew)2KOSZ=9jR^{f@f% z_sXejr1PRJp22u>a#F?~f@1>E`#fcO6)l2slH)bB8ksCP9#ke{i#bN!FxgGZFP+%B zWs_Lv)z6c*?hml;kFf4fu+nmVlE!Bt~B9^+7mU}@QLA6jpOuO5J?4woTpac=s=Lw{%=5m&zI9t}>*U_#V zL#2F!A_1b#0Mo`mfVKpxs6S9sw6zlv8iNf%+K31pgbjkNfnszA=n>4a8at%m+tWX zt=i4HcEu`oCAQi{3ki|p($d;$7qzWzZSCG_TeYjTRgnKX_q|z?nStQ{w+Xy=-+lL< zd-ikgxi62s{=kDoRIfF<>1LiK7oS{advKW}t+^gDd7iXRm&+M)Ia4mxa+xm|zs@xt z8ppNrus|*gU0heji+FJvFX6KUR=v(=yLhQ|E_3lYGU!|nH}LW@ZsZlxS}CnnE7nbn%fsE=BYj~PZ9wY^M`c)u+ATmg}{|0 ze^lo^!kyc7{+KLshcM_)o%ed^O8%%oyUW8L=TCU(8vdltpOT56*7-9Yx|a9pJRo!L zcJpWXbHdDfr2AfB)8}>mf`@M4FA6KaB<}(`-{;}``O7-*XByXTMnh)&LNgf;2NO(2 zb2Mtkn<9Zk!o*`qB-YI|abqmrU6+V-)o-uc80+pfN_Xc*inU|-`M6x}6g~`OtOpZWs2PoxsypLNi zw}gAmSPG=+ap8+F!jVKD)4T%f#Cm&U(YjPJ3^ZjX7SiLXXfzxJ>-Z8TPgg7+G@Bx^ z1R&@MGn$CS0dh?NSc+ANx&jQad4U)mUs zhLbCqUaBs>T&|dkkIFEz_>cm4=L@d6y}7vn2#bvnsVHI{7L)t(q62gIolp1tiy*P4 zSP1+s-x!XXt*PElGrlFz89@)UIT(ms6o`lA+3s{Cd%}=OeS!5(CKHjjjW*N#0ssnA z2IGR3;94!%G7ziGA!Oa|pxKuU$D)8fu007Iy)e+HaD|fxan&t~#;BrkX3<4k@m}z= z6`SbNwVl9{+jHD8^g;{)uMPHKRxT4BnPUV@WkBYu25CB%BrGU7j-q z48~}e%__XC$a%RjoJitrSzjy?$-t=^sY(5PX>F>{nVH6jqcW+)-81oT+WG+@(c zJt~84h9?xin=?XCDg^Xo%v>YHFJFmYriQV@8>t>Em+L$T5$imp^FahHsHu3Ga3GoX zbd^1UXeeTCwwyV{-xrID2W%XJ4q9V#sTJucop&nXF`Zk0pJOAPNlW$>HU&!{TVj3T z;O0O)33`+Ty87@S6fjN5^k*qrTL2XcQd*hnMsZm@c zlaUEECz_1+W9~ZZZu<~r17O)a)s1oGwrkEJz2uaU1{hdJT8xFTwE3S_6

4FvDF ziNFtbN~pRvN9{6L7SiQ{@;GJ03vgSMKRyzR|I7MgQ#>)66qCC>+#L-hQ*ju>U2oJF z{u|gTdxPwH`yyu2RDOf0=L8trN|v9vSw5$z;703pON(0C(-}yZo4UFZCUP-XFc57+ zRsdx;{|hM0N8(l)jGORho6{alWsV38bE-#b4jfZXhaD1wUF^74IwPSER7EFZe5K{dxYz=+~ zX^iCXDzEouR+3#sVoiU>9V0!AtF6Y!&c#JlH{Zjj7k41jhwvAke5##uNp3Gb7TIZ} z6B!@VSq0|W?6c=IS;h(aE>O`{5}pmCICcUd*nB8vu;L|KbD%+a3jr(arImU|!?C(G z{cs+st}di6;sNi&wiP>8orjtF3rV=p5@U6+r=T3Ay8=7R6hgWO`%*KJn73%rBBo1D z&9_+lvE_x9wYZYAPeIBPinI`irf&1n+2v_P3W2h{o*Q%GbI{v@c%4~dLn~lMUy`lEB$5PnVuMf+f zvdq$hg`)EY-A*4f=)?38gFZ{2Gw3Juj6t8IPZ@L%9Wdylw8x-3=uU$^O`kF7to zcNz2vx|8X=q9hlvRcqCeMFFhB4;VB+cN_FU`VeX(*ghhajduIgKHryy<7=+le8}Lh@L`=FH25KY*r5IN0Mn#wh{|k8nHFq~1_Pcg5m6@MP7j>f6VHgRdu$@TJoq zVbWs;KhEDa_&fYvgTDvq@e{J}ll+v<-#7RN{6iFxvOL=ui*{p!Wbo5`#NZ$CkC~?D zi`ViKvI!KxKQZ_jq2s6gGo7C`_~-l!gMZ1tGWgg08-xByuNeGWdf4FK@uS#}%d~`V zS2)?@lS&wvh5feBE!=DHz5ObKf6vb`&1zDqOb9(`F=4CglTD0AR06yA{={p z!6=k61?MP;k;hI`q#X(;f-zJhgUF!_I>s+!qiOIz`4yf2#k8y_l4 zGx#VU6GMI7;Nuz@8f#kG+D2~O46Q`erhrHa1zkgP$mlhvickIq$>= zi}Bf>DccoQWXzdkxl|?O)$w?sUs76Vg62bMAxoFNAuzPrV&#@gQFE-}uhCIMo1@K5 z&rz=~2qb!tpc&dc?Q~r`!_dyusts+v=1*(Z$Xv|OYP4FWx?(is>#p(^g6jgN+LKOe zvkUF;Lx#3ctJAebhPGH+VrXY+^@et~G5~EUYOeX9if4Ov#9J(AER)MQOiNB3fUccu zXbl?D+lg7Gx1nfnm55tU1>(s|lyJZK04*J}%cf~Olpxa;L)1+aa;g)OWh$U>HmhuI z=WPd=E_*}iP=O57LfEc!XA1OEhOQ9ma*u~lj|!QcDV%>)+FQ`v+)$>>DaH^X);dSQ zVN`E85(y`;AB%>t?M9^>#b|#4h*p}b*4WY$kL?mEE|+CiNxvHVhm&RRua@@nmx}om!~CB;{zQ%2p88i!PG( zLKxd+qK=F7uDbZFaPSx6g3sG<}ziY{L?w&tA76{^rrN=0|zI3c4_On ztlJH*2lcjn%+@~SF36%i3Q3jHj4aK!n^Jtwu?BP?8JDuZb(mNMCS+|TZ9tf#Lx8R3 zZj5$B)opTmq|z2EeqbC*I{8Ffd=Y%W6v!SY8@;ieK(yRWOt9@JrXsl)LcS8Hyv(WojypOIn~g#0qbkM-*P*DnQY+l;Y~<5ydO0k9E#F zaa3tYSxcp`s4a71qe#0DYUxO*(^6+L9teW^Gf!-RHc!R;bVSY+DtJ)&>gL>CV!Ggk zGX9u=Kt-*1IrX{=kQO}lt5Iz_(wrX7ry-k4Q|xnbP;G#!NHbd z+1f9M+iJMPHVCe?M;Qs?+9}MvK+VDIh52+8Ud@^TUEUQ)tj$Tag+sZb${r;vVL~0I zrr)c;)C8^~2ZJeK^~qTy%=Q;#u7wo*N$!CL@|o6kTefX#+wit^7dNfjyrp?lD>m1# z;q1AEkxtOp#6mE<#V24=26_@la|t;VPX)!1EGeWnqYxS~MReJnUKbpgFoFY?y}1x1 z;{D1ojeKo6uhMQaxxoa&E;KF@vj9kyCtBy!IzAGxTD8^fg*@;m{8TaY}Ew-Uz!6cjkm>ZGYYE^HmxWmu9Jmv>%_r2F;tjAk1RTyVk#8d zRI40aVs9a;o0Z_r^jSQz zUwoS&q3}e9q(y_UMYF(yk%&kwB>@ufzUXy@qiNME34~Ca9+91;xsO1H?*&w)<;2x% z8G<~pHx}=2v&Mto6Ko9f$HUoms=_eSl(b2W+@%NyyCPxazLQ1_u*%8M{yzCm=bT){ zvgycScbc;NE^&4vP8h_N6|0glOY?CSpRe;GuxAe_zo7Qe^g6;T= zrWvDWf$|AfJXXOW?XC+}Qb+j)d)*UMrALRJO!#mqy5S~Os+ipy*;E(Bz6ijg<1^Ie z`PucSX6@F7aP*e~BEqifmP?ARKN?>Zdkc)V-AtPmq4Ty^N>`?cBIIs=)(3+4^aQ@% z4c}~iFvoOyu8yh?>}>hu&RkBj>1HyC=~fy?r%3@9pN>wHVzRoUcC7BGFsnPt$m)*z zvAUy}tnMf!t2?UBxUZ2)v$~^hs`^nK#$A*8H0~3y-aT|LuAe7jH#w;cy&L>PRO+ue zK$?HS0n+`o2Z;TZjsxWKIcZ~X4=Fs4+sgjKp~-A`X8mud;S+^Qvv z@qi)w2~;b#>;zOqMm^fjWCTis0PwqW5$B(;nm?kj|(&T~TFKm<%RW-vjrGy5K zKPyjD)8oCj(>WNf57KG3Q#o4NFim6B|3^c4dL)!bG1R*U{8n?PTN7t6%KYVO;7n^d zS!pq}60W6(Om9&jQ)Q$UfbR*q@dkKjsE}<%1uK<}>VdT{q`%a902hE{tXaU-b z=z40P_hI(?G5=;jxDkT82?Dv9c0xa|f_7d9y}S|U9k<|A<5n>K4*c!|V?|tkFnNNp9jg; zD(qM=YmjC)IJ}PiG-nTVQN5bGO`his)9Lug$vK(MTvqCZjOH_tTrMqC#U7@b2A6Qs z>jKYe9iU@|yh3@asx;HT&_M^OZdvKT@ki8Z3NBhMEW1d}FZa3zX)%<2)(|aeaLa(R zs!C;&$3~)FBwQoHIos}8T0-);%}MsGZ-L%- zK*KMCjwc|bo%8_2{UCjS9)cK;VCL8H<{M}|O3%&mRE&o3PG$`YnA6<4S?U zO<3u1z`KL4q;J!AV3T*l`oD{}9{L{W@&Sf5^9jtf#y^=E{}jfHLY+zaF`BE>_jUTg zF{;q%hmMun>(r&w(>ndgagdKGM8Il8b^jLwIVi%Xf{FA*oF%x#r_kj~5cu!+j6 z{1_|$gbZ+|f_|!aVOSGwx*6P018uZ{dc<>j0c~{_+IY}H^eaI7HSqpT4kV8aNs|*q zK!9`B?!p-1ZcEJ|tr?`IL0UUV>y|q4xPA}4Np$M`n#{WkluFeO6N*m5gc3tTskhYD zsX5Y8I<=ue_j~pGY2#$Nu+n+`>>=7TaQvA8dS0My&H}4+LQNG!^u1{h-FArDI?!w% zqAdb=khTsJ0vVXYLk{)Xl&tu#r8U2DQF|xe3&k6@OVA@>CFe}Ee&P0 zqW^Ez(7mAnF~#tf$-B2@F#8;YQl7;Mt;+@}FhJ*ymCnu)bY^WPPSYWYKTtXS5w7u1 z(6Sex952!tXxGq7)JlJbBm66(*54?F1CiZmUqvs&3H}q#?-fL&e<2FJ0^R#JZ01#Z zf?k8%j?!~<3~ye?({ZlA7qydcaJH07@bP)6lJ<3wXsgnqpCKY0fh9akKZl&3plbRB zG{XZZGoc;7fpt9xSN&7^8T9TTod=8gJ+%H-IPd4^c`D_l87R<@jQzEMray{qqEGA! zQ4B4y$r38(<)H3PXrI_s72a8JUjSTd{EMoO=yY-&gR4>|Q>`7N6*|479xi^BrW_^1 zdI1(mUhxU=IYfJF@2Aes&|Ca9wL=u#Lt8|_Li%AcJ4&jcTwU!Qj+*_{ z-QMAZj14qBvK#mc?ED4r8AL=mjW*r0;UM5^G)G=nSYY`D7>Xs=-}ZRFD^z%yt&&!jld z!e43l@aGkCaL_&%$M^G;Kv$|1V{cjj@ckk(@hW?j7$uF^D<^8o^!Gvl4?_UbCqmJc z^U|s0H44n107l}N&0-|9Pg7H-32Q);@kyFs5t7(Lx?_MGCHDf3MMbF+xnE)Av!-P* znwG(6nvIb-OeaF&dH5_uPML9qzk&9;RF)JvM9B{17%7%O(cnNcW9iOJWL=qwtlQua z9Dbkp7-j9d25C1Cmgww}PG$D}Lv+P3ye(glB#gVopQ*%Sg|uv! zs~lc;Lz!2%{M-Kw($$0Xj)&=*VR~l?ZS%UM-)avWf9Mch+fnJLbb9q6x~{=}h_3IL zagg2xc53(2yNBsL8hPD=^xh$QUxVim-C*V3?{6qOM2#J@DjjL{@Or#uLv&-7)hZ!} zg;sDQ`FI7b;+3?XSJ7MfJm~mp>g6@~3#2CK_*%M+*U{~~p7!yEjN@95lwI6a8Lfho z6g~F{b25TCYzKeoI|fH1!R|0L{Vo+m55p~r^O3&C5a>(cz&=mMAxa(Y7oWqZG~h~F zjqeGtokE5-7tXT;YwV*b>_8e(mJzRl0hemSfTv2sco9l0;=POGWxg`3zJ>&ya?}(Z1#~mUlMkVvF5ARBr9&9Z?Y9P(5i%sDgpE=I7N%VT+~X& zhT2qk()`KV*e0LG(+W)HF-!*f(=&3J4IuL4o_O;?y5-AMAzWc47Kh%9a>7DTbeXXDP>E#&~%}JG;K*2iy%ysNjhXQGt5j1 z0e7Jmal-{=5v?diK?QA-rd32mKm@G|h@vR&0xIrOYvcFd_vX!;N#-;hdrr=syZrb6 zzkk0odE(f?hX5>AW&q2u$G|58@ZjzMK7~)qaE~7gu-Cx70T|dLi_etfv$(Gu_v3Ri zeBQtp0;s?P0X&E=mg7s0r-#%KQQpa0M5pqGCVDx&&cqjGCV7XKQ6;hWc{3hpDL8b6Wx)t z!sPa=!aKvEflM?OYLBKfEeih5Xm3268AwGG78V>YX-}kjL*Zn&Yeyv17w+i`hq@B| z{fT&}gZx^S@%qWha4HkcL=*9J>)@tDGTOBvyP+`2#cy3WN%AZ6cZv(hXVp>^h$F2n z-jmRXs>UEvxU5jDHbX2N?+tZkQqg#Ci;F-EQyf^KKy3sXH5vi@&z*Y0i|Z0lBNoq(+4dfB%m8jog{DTL}>>{cdXu}GKr z-VN8aJAaFUccoT&QhPKW=@{tW9!YHyLn&0ZC%VG1E#XvD*7l}XkSokA>Tex*3M#2E zuPBk@1sxQmjw|?*Y0LOVGOdG)(8M!Lj*Nt@02&h&3Kfp`BrFta#&I=SMBAv7!3-wE*35eD+C;X2A*ek);;!dHbRAK$1gI+HML8Hi8!dx&Qh&PBmKz?GiX-*{0Stbe|sX4 zNoP`FR-?1rpmlGZ2nQqF-JOc0nYuG9^zle0)W)GR(bX5ptgucQM1Gr4%84(|A}O_9 zwJDt8(%Q;SUB$-dBjLEQ5U3qW#coVQjcS@}%*$fYn9bvQP@zn8vpUyl^=-vv>+DkS zL}>f?y!6$5?9m5>5RD+F;%nk!tYUpM@mu_k71G4-@dpzx z;w6@fVsRu%Vd7@IY~YV3{)9gpc*VqD@K+Om!^cCNV3morGK}I5gJsQaJm=M5 zDkaKK*8E(~FqH={nP?NWPAvx4c958$gI`)Sk8um0sY~si}OpL*b-Cp{)ba z7)uV}*qrO7Zl&v%)|tvsWit5r{bxa4Ev&9Hb@vojnUh^+77+t{fXZVFzm$f?gTBD3TG#h&Lbe6}D<6@A^_u`82%i&)dq>u5AD z1=32jZbtS)Rv(miCE}TIG@iaV!ned!H(8cD6))+H5Y622PRGI_IbQ(m{Z_A2sH1(0 zMZ4V`XB9gam!w0RWuzVBiA={pEXJy*zs0ap=2=d!eZ;DjDdS{|7+#^A)3u3orX$?X zmBH19gnXYmslLsH-)$FPUu4i?7fvQ4ajMhg*4`#{Ti)8TvyC3jqtDuMjIHbm8qV{% zgd8qYG?$t~n7U0k)yiO~) z+uqT%g_Rz%us4dc#$a9O#!4=YHr^am7t1FfWXyZ9f#W@x&v6b{^BAA?sNy<^M$BfO zw4w>yun<=^+u~J3Z zq|;cVcH%;{3v1P_Xj2Dpk$MCdtHWqlFJPT|5gi^s)_bb3!808jJu`8MCxlMVB5d-s zVzZ|kTRb=7b=XLhL(H*O{>*v_I+;zoaSAqJGiT572ih&%Gbk~Dt;}jaaV+Mo3s6RM z+clj*In~r>Gd=JK`uK)qg3S6ddyolAfoU;HiUV}Fv41>~VS~JO9^64T9nJq^* zvs-2n&OE5_45Ej3OYoq28ok&-dmdCzBFb4Q?FS^8FBZB!2VI{dU-m5deUAK?Bfp=& zAX`%7j?{!BwbhZDbfjKQsVPdm6IG*hWT}C)fsFneFt8Iif(1u0jngAI`xs^zxJLhR z*>n^J_m5(+foqST^$0vi(a4=q#*3NH$i8G?(7<&D$lBsQkC#SK1K(J3;3%eRx#!rG z_j>s8M!bodu}(7Dd>nykhp=Fqq*BQ+nzEn6IO74HDIK{!u?4;3$D8pMtIFbe-Y(_t zW;+SnJxd#O5_UA%DcHn`&cIN!xA71vw#~S~Gb-7?P=+&yu_&91hXnyiEC{w}D`{%A zkY1G0y#YFT4qd#8AtgcDKqt(>47?R@1-?`ILjmLrjX(FnxO?+GD~L!tH#Y#NN~Ix@3C24O@<|886?XZ zb~Mdz%DM%Qp;SGvfVEFpmuRfj+EO>55;rnMcQM6xYcX{e14m2{@5TGbz7pklKP~Ig zRw{!!iYbg%p4@^D*y3AljVklR7pilMOPxwbPNgkJ;e+@P`G|$w{AOjN8!W_KbsGRbE?lB!g=KC8^Zb3&DOkT q1bhn0#u9D}VQHbT5tpzgTbKtQCtD9bfh+kLppbd|H`x_F|Nj801bayU diff --git a/out/production/clients/org/apache/kafka/common/Configurable.class b/out/production/clients/org/apache/kafka/common/Configurable.class deleted file mode 100644 index 369150543fdceae40ec6c5f279897c8670ec8036..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 232 zcmZ9H%?<%E6opS2hOxmjNc=2xV@GTxR>Wcw`#Vji%4lWM@o*L%z(a{D3$Z&V`R@Jh zeLQb>05f!Av|}U;ZDCEWRaHuc{%CWgEAh(df^X?EAMZVoYGud;VE6|tK72t&@zj;t zsQF|WrjpcFrI2e~_`SW=*Fjk!r*8$rz?O>Bg~UPf6Xhr3!k*98@L!gp8x9JBU3Qeh ZH5nLc{s(|X40S}lH+-w1=~En1>jgoyJ&ynY diff --git a/out/production/clients/org/apache/kafka/common/KafkaException.class b/out/production/clients/org/apache/kafka/common/KafkaException.class deleted file mode 100644 index c5ac04e3f564c81f0c43b865dcd7b3f374c8cb4d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 821 zcma))%T60H6o!A3%akZ7A>7iCmT*y}>MBwf zeE=Q`an3NJkfstd(ixxQ-=F_u&&}=cKLGZzRX`qf4zD@9;jrPL;b2ps9I8OeQC|h2 zw&wfsM*;C!pwzV{jHHSBaull^arhAe`2%fqbSThj_AceMbVt$*+>d-RB+030+pWiXm?2-C{K&oXsAPhX3qu#SB|AORPL3$ zxRS1Cecu}Q3nxcEJvENVbVtDPRT#=a0{q97K+cnKNRulw4N`qZFKoN|xc;=GOp^tL z6B`Gf`lPv+rKh$&u#qB)C~;WhQ04GS;O+Ao6R7=d=38tc?W>2bv56dABwY$>aPWe@ z^OVh@fGmw^TZ%8Ka*PbUfYWO4P9T0Iq6LZt5(Dc5siRDz#=Q)Ss8B`=u!LoT@Ms0A zq-N||ian&9*XLX93FfA$8)UALPCtRz-gBkBc+jVNC zGgg!8$x)|Lx7>D%_8wWedw8w+wJS$xBUZH*stW3qLeF8R?zl%3PWG?WUH?h|aA2I~ zk2j`?E;HiP?X&IqDZ4dhP1W@M8)d6DZnYel`-?HR>hPw!M;fh4$!c2Vs$H71X6CF? zxiLTAsF%*zq^f+D3=;a*krgJcG&^SM{}=r0q)=`-@+lootYuZqCFbla<5sOrnl^P& zQ7|gDJEoIjcYha!0ncm@2!_{DNRPT!d5)Q*$!6*+oL=T#|m!q8ByRNgMxw+qGJ#0duud^K7SmD6xwRK*Gi6O=6RaCTpfF&DmwjAju zJuQ2oU31x!DBG9YR*hw}B@F)Asad=19vZly5N%A&u53X5Chaxcs@Wx9jdxuas@SDempmNWN~{(+hdGk)N{j3r)N1%sBRR z-l^xCY%U98^A*Nn+2u)c$SN>#1+OZw(?5-qCccO-(ewZAJ0?bOMq%4BMvtf`+D>g+ zsOwv%C_f>+e%Qoo_%b;i%8E@8SnS99c}R^|r&vfBo4`s-aA02<* zXVW*Z@o%?~Ie!DaBhld;_iArrd$Ew)SIEUC_uWKa6fb-wqORTfNAW#ciG|2JPYN%f z4-@>%y2$8WBGQZW{E|kTr?30LhN+iEwC~?Y8aSf< ziJZ^f<2nhB`d6v)Dlw|2$Xzx&qND`}8U-RQV zUEv4ws{!ZK4?Oh-o_c*x{LaIZzWowMNz-`Y&xlKKz0~W`i?3;eULMDNgY0Ns5BID- z{ASm~{dmd`VAykV_2I8}Iqb#Z1d;9SiojTHe{8F7rfapZ54(^%;>4D?YCliti#GTP)s2Ji1s|!j>Zc z@5rS1A-10JxG@LG%SYHk4GJ<@!nPmk6HKPtko|aPKQ5EdKDm$E58i*npq1LHrEc@pIgTU$A4p&7Axt3HcRY5$~{pyvu%m4JYtx zUhFq`7Qban`W;)+B4+S=%;OJe;E#A6@39-dAENqdP&8L@mgf|0Pvbdu{)lc%y=+Nw z^IzCZb~F6-QG;I#9(kChmXy-bTiC%~dRO-DCFIBqZ=K8D6);{|mUdP*VT^ diff --git a/out/production/clients/org/apache/kafka/common/Node.class b/out/production/clients/org/apache/kafka/common/Node.class deleted file mode 100644 index e5012ca4112102603f06530b7a91cae91f6608e1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1881 zcmah~-%}e^6#njRb~j0v5>ir>go3rL`9X=bQL8}F($c04&^8sNXs1h9lZB9tNpSpI zeDTR24?O6M1jo)eKG_-n7ykpt`rX}4NohtOa_>3ko^$SZzH{#8&wqaX4L||qC=w_{ zVBqa2-od+4x+0hN!FkEsP_R8MwrrmmAZPsatm)wd~U$a_{)B|IvQ*+t^XG(6Xnzx!(rDo@M ztj~9>e8t_}bsPB=W+M!};YvwhDxLX1Up1X!VWV^$P*1pKwQD{^t=V#RnNn}r?Y%my z8!C6K%FeRY^uUo}p4ngQS#=^zr2De1Zf@I^&TQtPfa-2;zerthD4@BWn%yGB9k)ku zwCwJ+D)y2i35m#xW+X-vDcm-38e<0Tn7D+JiC6HdiE+6kF(#1uFM`)v`ZB*{9$G^j$L^7}IwMzzEWD zVL)(5-qRm^X<+d9z+mlo@RfnV$$_x-&|`)%awO--%GrR4lQ0a; z{2QaBV%U^3$L~u}`Xl-X;we1A(E1@x#)rFzu`Z>Hk$BvfSzoLt>(YLThAu{rPgbRG PI0^wKfw!1l!R3DefP+8B diff --git a/out/production/clients/org/apache/kafka/common/PartitionInfo.class b/out/production/clients/org/apache/kafka/common/PartitionInfo.class deleted file mode 100644 index 1b86de35bddec66b0e2a96884ecd719cca72571f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2326 zcmai0T~ixX7=BK&$u4oBG(eHBkX9*S6B=!+wbG&%#fAo22?ZkcV@VdeCG2Lx(i!jl z30`~QO>ex=8G()+xpl@r;H|&I8O7&pLik$D40FzV-aXGb&-=X3ZusNhU-tpr#A*y0 z-fzPtOvvGm9PY|tQVt(z_)x`1G2FwHhL2EKT|O;5b@lqT@r}rpPA3iY{hievjwkaJL?kyZB?`8*`Dn>0wO06typH+stF|W zZf!klR?X5=E4yK?ZJ61TySeE)**Ujt5vkRzYQ-*@TLQx?tu}YqPT?D;G~a|o@7Rv* zO$yXfXKCgB|F{;nwRk+XD4c@TD&A9@ zRgRv^>d~QwIcLqKG~KDOvqcGnQ)5&($%ieZy__0rQ3U1p+^jUy0S+w^3^a%EOp5;}&F5SVE~QoeDMcatMqnPk(aa$U|O>ppEf zJhWnCX>Zm1NTBn`@()*^StU=!qK+jj(*us{SSpG-K1WK&+qh2IbgbZsK<^Q(E^?zn z3a}#if1x9dFLiu{ujODe@25ruM$YSPvn*+n6U+|#*tK#@**aaCiLRkzFCH<n+wrP3-w*y|q07}PiYJNw00|%ltTM{IEt4mkwhTmnQRP`plncHGiht zIBVc5Nu0nHuD(MEQO-$Y55h?AAY=^gAZ(29Kr#B19Yl=2$PS`Eb596wa7;ku*bjw1 zZsQFdv~m}-eRT6MPkxh6C+lZo1XsD@={`5%N0MF0k0igaA4!IaA4!f8a_hxtgJG7l zL`IBX5c-KhokrA8UiGij>l+%^@r6Z-8Cl+OaQMel;PF840LB8q%ITYL27-rR1b`!_ z!4rYt;XrWoG7Im)Sci_PVCsUhK?^n`T9@+)){;}`bOIUAlFO7AP>ACSnT zhpzj)FYY3~luk%g`>^TaRHATOPQ+@P1GgO{65U*`S<`NuGacknLjks`Ds{10>$ Rx5VGLGRVj8F&V<`{{Yp?=FAck2pjRUy;TVdO`C&{Rl9 zLCdcxq@ZUDdZ}C(_3rrRK4)A zt~Pj+uLL#Sc%nmJ?D*1*>i(`mu@Z!vu5RgC-E+6~*W22y1v@)I(|s(>JUZ--9T>$D z8y+o|PS#gQ3#48H$yVs^uxu*ycH50eVQ@Xtwe1JG)h$hq=<~j5>jvwO^`m)M-STSD zLg_0O3#wa3nTVaFDTwM`$dXaeV@Dyo9<;-n_k}NTM~-$rD`eUj!@P|l4BNax2G3FF4U}W-BjFVGVL0V|m_P77fe$Fhk)$lZq?N%roFd^P zn8X?0e$fXB9}Aug zHord>JpHf1T#PvQrx@uxD<@tcEyv!_65Y zY1ioTF@-esvV1>aV}^{+5~)iDF5xO?Fo!FB+|0*N_{kw>!|Z1mCGsZRoM)a181`F& zG%QRh(xcvTNU@QS3)$5e#h)>XHK+UqdtF~~3ZMD}6g7i6Bk hIh?~l-tJ&PfORm~!Kvd5aKCQ>Cf;pNiFp!m{sF%}5u^YB diff --git a/out/production/clients/org/apache/kafka/common/config/AbstractConfig.class b/out/production/clients/org/apache/kafka/common/config/AbstractConfig.class deleted file mode 100644 index 6dbe4a27a56c840f3aca920a1aba611d81f41e8c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8093 zcmbtZ33wD|8GirGCbPR45=c3^Eu~w|WH(0-Xdna%1R|7>+J-}@wN8>r*k-dE_khw` z?|Ro;Ypb@^YN?7^4+0V8&|2^NzVG|KwPNx8{+XSf4I9#@J`b7s=b!KYj`urf`OIS< zJ`P~H>WyGNPRZqC5qumEhjAS~QHw|L$y$60pRUDc@L9Qh&cNpld_f4mD6hX1!j~h6 z;45r)9g&Q1u}-Go)t8*3S&pKuy$na+$4A zmC6h&H1uS$!)>|L(DIwwdNRYqc6PNwu#mF{73zC#vi4bRg?uvA)@SGWBHWi8PFwjx z)>c?l{C($O6AJ=ZL`MscCn8KcPj9R3zM3CR;pl= z!-{F?eOg@8f0I3s*ILwcCez9MT7_#GwY}DuHL%xi8?lB)thRy7=x8R*TY4xt+}5RU z8||Uh<+9vIZQq7WCS_ac)lEBzRaa(^Xw>&4({^uRwBOEdxB64uA+iHjYNwS=%Coy! zHDVu-R~lH&%hi&M@_UmskzSxf&Y|G?{#-t54dfj~iB4qD9uo7CnY2QDy5lAL6siX^ z^tr-$K7ER|DafeGPTCth(vwLuAQn$^MyMzw%mW)U1s(3wA>h#Q zkd~6nQL}SM(XrJ-DjrxlwK5)Ag~}m?BSHn;fSv-}J&qNQE2j6O=Q8m~SIWvUW2f_o zCZjTqQ`?~{fu`$p3SJ@Yl;By7ZWQ^7aF8`bi=Fgp3Jl6wICVETkQrkgYA@qnxd8h@ z%U@90Jl3aD4F5(+f@(XR&yr#FCg*N$NvG|sHb~0$9EVADL`3x~M12ArbT<@| zsX^U`CY504Tm>_IEIb|Aj_F#exx6!i2`R-0X~G_icJZ(9t6 z&|WLI*O>*F6I)n&`Z9&=fZd&xVc@*-uH7nuVdCky)5Ocw`G$&`>H>A4i5=L$hW8ET;#zR%)h}nYbQLWe+!1qiQnoG(45ju7ztyZl<{$3Jn!ERYEnJ*oCKy<53ypia%jAOEE?t??#jgJU>8)Dn`(u+vPka%!P04} zcC}JrUP*dcH8a&Jg04CgV>uji=Gl&zp*l^q2Dg}MEndYaDZxvQPPvmaSD9*^_`WK( zCT6PjYLRSiFjSYRHWDJdQVy@%zSo|F?ya}+05AKkTr5`@8%rha!B~GGA4}%a%kuQM z-zBZ4+N8P}kGR8BPg0vrwM87abgJOmD#NocsIFE^Xg*oT(i!qj$C7S|jb(;n%=4-E z6l=0f$)?(>R+_3uEj4jFURZ|6`fS!ZAgOPvZMZ`rRzkXREn@EK7Zg65z!Xx0S~CXv zkkqVJg_fywa)Pw2JZVjQSVHlmIL=-^%}K{7enlwED03=@|I+i8yq&f3nJn!rcH+_` zz(AQNQvBa$-JJU~(r9J!az584%^%ug*6+zD$*qQS12eWvBmEpodV0()>Y+oss2>j- zEN{K`%{Etc;phl~V8F$K< z$+Lpl?|dNcl+RXTr_=o!#=_jjJ^o3e^sQCPh|62d7}jHYYABN(m8Me3`0IVPRshaJ zxXK#jH+m@SNXzN7K@@w`i)I61U~rcF0vB(A&fK((|Tv$Na{wb1{|AAdRUE?5;n` z6q^#ll43I_ClkXznI7VV7u;`U`O#g;O=o-Q>>Y{G+O)mDG>n#gOB8dtROq~{h$M5R zL-IxCU>|l0m;QYwIAF;~mcdw(m$-yKXwBzOZ9)Eg#W91c2I<*Q*v)kSd-%+b?>*nZ zXEuHB`9{pq9p(7X#PQXyO3$3yW zajQC7eG1{baltV}c1NqnQQI7?o`4xZ!;B-SJ8|&L7vjx_Q6EplKZN>m%sh-)@ndM% zeFWzn#q7hF(=v{^cfp|8dGFWu4Dfn6OkU%(ZUO4ij5hwL){a*6@_sF!cH&ZMeHq3` zxgS^HR{s1p$P>$HMH?=LjUldR4TiCocUdKI6D^jN;1bZQ5u{LsQLatzU7GJQ+_?zB zGlaO>z?gw&7|0sP8OQ^dJ&Hh+3v)q(xX%F*H&Wp$9<7ZhnuAAi{t-mQrYeX{5iPTw z_mx`Lm_zDmSnFw6>uOliOLjyr$G7__h9eY3({p$(49$5Z5uTNpzy$%^jq}AH4HpJK ziut>%5`9N;(Ge^-L5ew~xHy1aE-@YTkf&h-`2wpk1D#liHN<+Y)^x7J-_vv@@x6u8 z1*Iyeg;aM)nH7>_+JTZ>gNn9ct zJM3ym0(ARWg*{ebkJYpBoGDl>oXqO#X;?iM&zpkPqRFg!r(yMcykH7eizlk` zt_X+(!(^p~G*j?WAn73Gs;|Lw=n zLb61g>z~!Zx-D-4w3kf<56RmiH!RhR3mDSE6MC z{Jc5|J_(n0ed8uSt;VrxSHfj@@XT@Y3^KhtCa^kyQX{yM@Buj*nrFi*;6m)fB9{D3 zEPy>MfqPhpELNWs>rb9FV?V3L3-D|Wh!ogOYzJ?|o3xv5_n^5AZ^m0_co7K;-_ zNvlKPzqs7My{z_+Vk7D9E5bzH)tXm3(dpD(v4;g!o4=F`ZGL#07}Kz(nYVgk!Wuh) zwLu&_^UYE>a3be+c9|FAO1#MP;6;odsjdG5ajd@^jmNNIx9eB7+b(*cVIwci&YGwn!KM?@;3;(9 zg;~+yNpy?q6L?Ypn#X2|_@FM}OZoq5z-u@wyq2Bpb&RstGXmd0$J~i^gnBclgPm+Z z5|Pr4shDepten0ftNZbG8c@TTLO(+iGJq{{?OUPOjeOOT0(t8Y7GpM(A1Q~lAIAv^cy>`E>9Jh7UL~D*> zTZeHFO&uX=powUW3tPiA#4B1O$UT`C4O!iZsG-r08UlByIU3S%H~HX>8dKvPa`6uN z@s84KWV3k!*Rs?cJhM9*e2A8{5X^RtKaLZ+S#W@RXyd&ckM85!`}zI>QoJ3j2atP%e@6-1@NR)`vZ}9%l}JfV}E)7$4-yajOJ`BjG_$f~-B1;digho^GeH}i+#^H>ctSd$xe`%IqBwTHl^DgsE#L33(65!>N e&j~HSnO!5UfB;Tts*`lNOr465KZ3hz!v6!Lufsb4 diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef$1.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef$1.class deleted file mode 100644 index 905e40e175fedb73bf8617608cd1698001ba7ae7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1667 zcmbVMTTc@~6#k|y-F8`MEuaE|h*f%nauv0;f(2q@(T5Tqd@vc7rCr+YHruUCu{C&41v6EXipo}J@OoV%HQ~5qc!97=w2d0KsD{6@g(uo8hO%?u;ghhG^U9|&(Hw)T zyuh%0k!TUS$lzA1H3mkh1`0M;%iFxx@rrN3V76HOv(2EAQ z8-j`#=UqCMhOJkq)q*i@n^yWFm@~w>TreA!s%`0_xq~OOGbgZ8|2GY_V%Bw|xUJbG zb6>(81*^ELU;u*(0=TE(K2i$O$VhmoU=8aE9w94XLnxaJi%!S&w0#P$V3a1^aU?Y@ zc$Qp-;kM}vQ4A5MX&L%$^RZp6bUQd4iS~4(U|U2BG3btn10$Dsoz5I@RP46tIhpio z&9rQ8s6?%t>rqymck48aN-a5s9zG4DdFmdl9g|_GcM3O+Ml~fS&SUT@VGR9mdp*Q< zw#+cno21J_G>0j=vid>S09Y_6288ZAH$4Fa$$E)a8Mx>V$g%i0u-JFFzL4fZh~8fE z0eg)hT3_yXVVG9pC%3cFxe|lX&GnglJoGISZ^3;GB|+i{eXgchkS@%mdz$gD@FeJI zp)WIWi~#9g(!DK6&50&uaJTDE)9nlP27~M^g6th8*?UZ}4@j^h%(IV3vrlbou};+! zY#EGUTwtf&U89wYuJRBjFiCl@lhp@zFz^TCEVxeSr@H7@>8s#oAQt}xUxJ<%`~u8* tGKfy$*e&S6jkeb`Znk9@w@7+$8#5T8)flCfNs6D3tXD}(kbJN<`VR%tr!@v~g(-Tk(P$hwaiXEW6FN#dv38 z{4HLqi803T1Nc$KbGC$X;RQ6^_nl|`C1Y_V)vy%-?bA zdLgn^XGgDoY&rt5t&%`$$@W~|Z9f-?=cYFWltNIWmjkBjJLS&)wi8z6wnsCAW?)Ni zQ-&_{@noOXmi8WP9x)wVAY%oc(00~bPU=6|bZ%E3NLsh(+g{Le{rbAoZUi+I_f*Vg zVc>oi1JJT~fP5AYF)uL6DTXKgx={|Sj@?*u9j{gl!ywG!5r;iydBU>5@|5Kn%Odgu zGgJ_xTM#2Eh|w(w85hJjIh#(w*^=)&VZoEFmeUf@dV5#4cOAPekUQ&g0<&igJ-!O1 zdn3sM>2-Mbd~SNnJl7hF0_sYoViuRn0>=4sgRtiM(u=x)+9GLHH%jG~gr#y-AYr|# zY>+Evg(s7%mA5Nqk-22?>w>v#S=u-?az8b;(1!Ucd4v=$Vvzh2!4Tp!0>1o$IQ*0O zf!KHc#(zRlDqo?zjxb4>=%GrO>|u&f?O~cQ)x!*-b}IS;^QrJGb2t7Q9W9N^xQYaZ zF@Z_)DdefI64X}9SS2sw4J`61-oqxZp@|lG8=vr*JWlyQGWt1FGAEQd zsm!TXY{zNkjVf~{i!qFwIBQ~pV0fY;37e;v#a&S>3vaPF69%rgILd3`y;W)A9DBFf zBUn?OCxgkds8(f_P@MLI#iCdh&XO#yh=mnVbo@%i_ZYkdcdink ziu6Jey1vH+OL9Tf%Hgai*EV!ho_t5c&Zr_j>WIJmc~U}0(qzA2C_t;IYRo3>$%}L;d|llqm93XKDTuR8mAJ! zz}npLbJoqZZtI3D=6YvY50ie0`Pww?dh$lCGB1PMV!o^y^c_*26@jbb#zP}qa#=2! zNDkNJz5imanWgkI6AEYLx>(hfwT3USKv~JGjfl%7QDwm?Q4x}bUZx^DqHkY{=QA#2}fgZvPS$L-A2so<7DnzrB|&5Qg1*w{4aTL9CTD^rVzIc zW=uGatX6x5hK2~8h2C{4Y+L~BkJMM>6!Z;zf|-P!PuJ06 zcbRo$-f@K_de{zff1vv?=m9eH5IK5;E%X?9dV+3xibM1a!}J^{=mo~;B@S}Vt*8SW zLV+XhMNdZBQ?n(sr>073Pt9d$PfeE6o|-+aJvF6?UiP~|zc7pz6Sj#y6a6LzObnVR za;VA^hIlQAQfOa0y_=xOln8q6z$+!GoG+LV91ijKNTgpd!qh+;O3W?kVagie& iBybd)8TWIa?fl#6n8@?LXbkyu2D=z{bKJ^!7}?+9aGsFEOls7&PM< zciU3ZS~@dm4t3l6L^X~U9bvz1H>$S(j#pj!q6NthY7B*R@tB`+)8(RWmRi0e>bcB- zlH*0s;%=Gyj%4Xlse(6%G5#+Nb#2`dPHTf9d5eldrc5Ji9XZs4Sjnji-fH`lNlD8< zblq~hmv{S8=S91*K9KwknxbCv+P-DKab%B1u1CEfJ7nOlT)3n`TezZOm0@XsRWUGy z*9IQqk%3Xf4UEepfjIH?<{+zJ2wScvNJt``>8tRddTd)Q5}W>CdfMv#_6ED2>&tjO z^_v26n~U`4jL>hVZca$C09n!-Sdt8kP0FT71Y)>{`(T(L?=e~V=pC@H2wg(?Odi7| zoe>#hGlZYQbQiyjP;f(!tJ!Y|UqY4m&<*?p`eIz6FoW43PF*CFWL$(oFQ8@Tr!Nrs z5`_Edde=3QN7C^mz@!kxQ>3~=K2jm_+4=#s64#UqM9&cm99<2Zp};Z1fioQWNjRlL w>se7d7%El59SoPw!K8qzdUa9=Hj6o`S?Q8K!yLs!NaGG!oscS?V{CKcFL%`>nE(I) diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef$Range.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef$Range.class deleted file mode 100644 index 2b8e70d0e5a9c10938e31fc1e457434c83dd0b07..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2019 zcmbVN+j1L45Iv*YO0r@H5e$ieh&T{QmLF^L2g;`mg{&-C)SUY3%GV>ynMIBv(W8poYD?#8iZ z;GTi|0^ycv38X7e<=3**l-6!(z1ylQr>w8!GlAGG(=y#ffh*aQMkh77nm}aP-c|w= z71L5a+GAO7Qj@OO4cV+o$JFOQ8FBZ_jzIJggVQFpW?9NvZpuzaQ9W0&o!ydb%f_B6 z?aQ5gS!&p=mTgg3JLYa_*%K>j=duUDybbAAl6{!PS;U2*)+FbDs|O$Ol!A16lSCTR5jePS87YPyuuFp7uX93`*>pmH(rp*nW`%r z`wwK>>jEc9Anw|}ufVx%ZYW+KFWRsuu!%={sitrh^E#o&0#nDWEj!b)9hGtSq?O^|J}DN91y=P-{+{q$;F)_*Qa?n; z?|{Jksq5kmgmi7@oEX}Nv%2}&)4~XhZQEUbavl+W+1cC*ziioVD~sbcJ3Q4vvrqOv zC>{spn0;>;Pt41}{^kh$A@cxXNYl%qhWGf+O@a+W2x&cBd=S%mz9^&AabS4+4o_P= zg()ZUhY5@H_X;u z5E#*;%^5=$xd7{LKL2C|r|>&s>-nD%`UCT`2Qc)O;=AuLeT4YdOyLNLtwSW!V+Tn2 zmuC(zeu#qiCVXN{3M*Y678uP&P2PKqG#H0o9*qs0(lD?El2Y9--F; zsio80lht!gX~^XD0q69&z@+CseD}r(s6Nd}rU|hQLu_DLJVah>VpUX!nr#IXH3_zO d`Vup~#(ju?;g=}W+(mGMGo9c(-#{gn_z(30)l2{Y diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef$Type.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef$Type.class deleted file mode 100644 index b7a23d9897f350ad2e467ddd4a1e96698b0def7e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1460 zcmb7ETTdEM5dKzHc3D?Nyx;FFU=^>uVXcT*X*ZC#XhQn5(iK->F$gC8L49vAsZECBNRrhO<&G9TZWIWzOk*|WcXfBgYq6^0Koo@#i8=L%*OtZNv+3pN`n1~qbS zaKuX$ql}v>#u#HN#u>L%Ofbe(Ofqh(m|{$*m}cBj@JfQRl};OpSW1GNd4Fb0(37rf zm*WNN{MsV$)H*t~BDqqrSaJ#6BPSn;3)r@gW_W*8g4fLKB~z5z zlT2kKw3@r=Jwn+?^P+b1!!37j_Xu5B=r5Gelm1GQ>+|2q}yjYdDp5`%aq9zWQV$*E7Ui~ z8d4|RYH_{E#y99tHM3ML=jLCPBZ5U8 zgBZ~<%4UqsIGYJJlWeBgOk;#vxH}G(MDJ0qP;#lao?xg!Zu;=V&Q+-Y;Qw|cVXbl2 z;vPgBGTt8k{RX0&OLV0M=-&D^2U!I`NJ0dX7J(#3ASn_^q6CsIfg}y^5Ai%HlMEjo z;324cVpD|tXf;Tm(Q*yXM`98}WWDT@hA2LaxqAF6TEq@5Acw!8^%^qAx7>?=L@WAD z6y~uY{B#Ty#TA#}p&y`x=SHsJ{UqFt!^t%9@*(%}`PQxLx*NS7c1Rgi8$di*zWy@K`C1N;5zcdF&7(Y?sZ zHME)8)*9N)tha_vGpp3lWoFeHy3MSgmU;3(a#3FBIs?ii64daC7>ZuD`!(;Nd Q;0cBaHHwrG#n9&9A9qze?*IS* diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef$ValidString.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef$ValidString.class deleted file mode 100644 index 303c5e03204944ebdb32b882546f08a5b997c49e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1935 zcmbVNeNz)h82@b&a>;R`21ISE4GK1dmunRT4bo!3mlCmUAmycwvm}?eaJkD|E{=W# zzk;8lovET_rtRPTOpfETcb9MkI>V5e+h=#5-QV+j-ZuYz`s;51Q`k_^kEtZ0xT#=T z#SCt%w78=n*Na))RWPUEdlgAcCGdlk^9kHjk;j5uEUHN0egY2?c&Ol!f*%>Q7u*z; zqU#8&)?_$S*yb;|-g1Sh7ev#|F(iwkW^uRW7!1Q*>REqna;v8M7`ZvR5}y^8aOW8M zGhG*Qqa}t|-mVbX>4LC~8kW5A}ofb%;w=xjGmNL{}Esm0mEui7CZ+S+62R~4^eoswQZDLKin3td%^Wm8$Qp# zY#H@^MR1K26xCBjmzQ@Q^M;oRDl&$IYx{X-7|4uXiUDI52NOlV&x?vv7n zhKnd@IEO(6j~S*;k~9U&8lK=MiS-2)>^^s&f`Xql{DLCGlDLH`ko%(=*O4g z`mwlm>Xo}Tu}FHU_vg`!ti8-}VcI0T#-4W^{)+M&=S|t;3?m1sw_Q4ZcEe9WpBJFc z4wVD7m&=7kVCRB{@#?D$u_28#8irM2mN` zX2b28w@s6ZpK_WzT%m_elC9fEa0^`x+zIWp`Ij(64{$%I9q5BBZIb6{7QqF?XrwBo z@k`oSBN-v7W_Q7|V{Z}pgH};|MPnaHunol6CX(zoT%x@}|1>U>4f#_8^eZ&iJZp02 zLdV`!yv8s_Xf3VXB1FkZ+e36c`yR1f^!!fih-WV@b?ljE$H$Uwd5!e+vLF*+yn#YK z05jwNBEFpc6OoAzP$GDP%pMZ!W8-^BuJ1xk^}d59)u~hO(MR)lIK7KA|Lpshk~pk_ z9%dlLs+eZNvz_(vJ3h`}6j_h=48|}{YVeZ;f^oz76B1w5AmUx(U! q&UW!hrF>G|0Wd=W7z#*4aNP^j4Wvn?h(eTt`8EB24v<`mCqDy3yy4LR diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef$Validator.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef$Validator.class deleted file mode 100644 index 821b859405b196e628066cdeb9d84074a51fa07b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 293 zcma)1!D_-l6r2~+#3Z)(5kmFif_Tf(QYZ*L2;QIRON=JFvWfk;9{K_OsNxzyJ$iXF zJZ6}gzyF_K0CPNt=!ZxMiRk8NWzALjMR*-Acl^z&VqK_JYfD|Ecc0JAPO>(gtO@bT z9Bn4=WhI1{h0%Fgyh%Pi+d~WpGh++I2hO&l_MGooWu~r;c5v3Jt7X(>b89Riy5oZn tt;H@XZW?I_!&|XS4Z`@*h=4%w+;D(=LJz)U9-g`zAa)o#1tCNfL?^N}P;&qP diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigDef.class b/out/production/clients/org/apache/kafka/common/config/ConfigDef.class deleted file mode 100644 index e4b274882ffedf8cf21c8a4c7c1aa616991a28f3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 10278 zcmds73wT^*nSQ@yl5;Y1nxxYU1X`JvwB*{9QZ7wHp;w!dwn$1!xlbpPWZKD0n3>QP z5fEK=*$b$9!QzUjNEHwZv}sB}6joSSMHiLAvbw0Nvg|78y2!o0-+#``oTS6^q$|(v zKAR_VzH|P|cfb8-=E-*+{3?LO^3f_J@y#Fx@uY=+2%;QM>d3c3_%@yj;yd`R-u*f?LijO$qL-gm zaml~x#Lx8Z=X&{tUS72EZ#G_14F9gr{}IA4ere%XLDb{rDx~mA5WDbe8^6(!{|sUQ zerw~uY`m&0{LaE_HeOdr-UwnX-n8&m5R34(Uf!{B*b;JknuuP?I4I>oER}#I6$&dY zv4S{L?4VRhP%j}%91@Z#)PhvoQlk&GL76IHTc%kuom6CoEi)}S#+GA)I0w&KQfJF? zREr#M$t+uD+wwk3P6%R?L@b#TM32l>bSK(UZ_7Me=G$_TEe*Cb+R|i8a|p}vf-Nn! zw5qUFZoMq9rHu-ag+c7VA-y}efpyIq}sy}YY8C+`SqPcX&k|k79=ipE}lZ&QeoNgBc zHtiaU3u-rPylmC#HOtTM+H_gZ@~-n&3&LI7qE|-SlEl=uaqG5tEVq=~V|l{lqQ+e@ zoJ%Cz)<=gpUDciFPepUX8KT@gezwDh&gFDzS31+*79EPl2I6hoqkY?>ZL##=U^+#h z(zUrlu8Q~77sOq(64;se6~l>4 zyjLI>a(XD5$;Nx>-^x3myK~Xlb_SC1R7SqwjsIu9a#XJJ`2Vi(K2!SQ&2!eH66D4A zK#)nN%Ayzlx4iZb6m{SKq^R$mh4%lSVf*N^X@yB4f{4d|p%nY;lv|oBm=rZRmT!;m zI;yl-ti3FRLz(mt3t^=}#+#YrODyxZS7xWVj+AGpqp3-Y;HeWvkMuU*h*#Q{EwzOI z9qc~^>m~?l@J9HV0U^r6N|4&XuDVKB4j;XpoVksL0s+N z!?>P}o3W&kYi%?;z=&}0AzVYpa^y_hXvs21&XThoSuQIC3nrCp3(6N9Q*@MZ`YG@< zoG~px#F3S<4=zxgCjEe1RjYF-C4>jf1aXuVB%nn4qx5=2*AC-;uS_ z>EO@s=MMJBI!n%Rq)XN_fE?K%8yz_p-*E63xXFX6dTs3tIKFLh7K9SAx z7!k?$DiPN^@)FDy%$^(BF_4H2LXXE56(&=P8%C4)WB$8+(YmQ4Tk*&kINFo>6L1Cga(B%?!WGlwT zA?r-#;{EZABQfb!9?9DYrO~D$DRTr_zypa-@ByB=9O;vO-96w)LbmDMc6vy0RaZL2 z`J@bvQ<|F<97##qk|Eu7g(DeZxVl>l(dtM}?K>=2GNv!iHns5M$PU?Q$u38(k`L0l z1*W64>);@@5N8sMs)~~g`0-quT;s^q@*zQkZ>JfLBvM&(j8q%M^k5p%)}9znC`#0s zOvd}8NjA2*xG!F&oS)jBO7BQTwBaAWqht;g%wWo+d0gE_(tT#938!l)Rgqxn^vHtc z^30qz>e6uVSNLlOe}kKp!#0;iT6eNCj`K`6BOi^17P0e~EGhS- z>gLnQ=44HW=GDj1e~r_W8}WA2r--Iq6}TWIwER+)8p0G zMkeIS=+IC+#bau#e}QX?JW^jLQS1inTsm*Cak%qa`L^ci5-2C%Gnlr@*N%8hPaZ+R z*zeuz4+>R2G)zmK;TJ8>nX#U~11#AT9-D8_ObRO6nH|>jQub%`CJL@HbKK^<3vsK( z3cVq|ldSPgnHJiV`_Qj`HNOP6C3y=HizJ2x+VxUJ z_1!xXxtMlf^^BT&_w8T3dHYxIzWpQ6w|`z+qY>>!P1!}Alqj9)O{AhpBN3l#`9_7k zLT7&}or$lEYSU2C@hH4ydWel|cxN{7lvT8g*C|k6OC}|bZ>!I*Z7S6i85WbgJ&f7S zpc;b#@>aygw$|6_tunK^eU+bpJ4_NGL!!8uF003r6vfo#%&t?C{Ee9~Q-8Lx35Z=<< zQfJo%>MH6gXIKX??I5O?V{?^KM>V6fm>#jHHyvY7XYj_fv7;JvjHfwR#2=wlSHs4w9JNv2>a4NwDWizL z^%OD8af`DxP0cN>F7tsA%$TUx%MFuj-K7O4YYI%(6qu|jFj-SzvZlafjmMn@@G8CQBqb{P^FWyk}gH~Mnib9yR<;Kw9?fJ zl(4JdfD!ce(t;|v85Jdre9KV?ny2TArz);BUZ*PlEI#MPtn8(as%Z~-;lJLE-I;=e zI7YCc^&pNF`sgTc>I7dyxINHZ7ubj6ypWvJWS;V4$Cp8$XKCD3?7?Pzm~DcxO>f_~ z8z=c=_ymTo*{^XNDaLUfwLRxnOf`FMMU^PBph4DNp|*EGw#JUzR$S{_u)3&k2|oRGb4Ba2k>`HUQbgA zKfr@V=^9y^Xu!`iTq~%+{N3F+$s21X_B-$e6ZD4)&Tt5KniOpF>-;^uD_$b)#6hxBSr_m|^O?Ji zhHe?b!V#RDH@U&Eh?vRuqP?CBb)H0Dp6XFL&0Tkzf8FV$IAh{E;;UACBUmzu zc3r*H-MQ4ivqR@RS@IOlba$RPf@PyP%fI?;eV(v-xr=qVx9)iL-W9uHjbf$hY*pC^ zR*zy$e%t#!G1j`<)_U8jRNT(|qIJ4x4CibKcOAg`N7@4}&AbXXQgJlvxu)*59x;8M ztKoCQ-9-gsKCOI-JjdF0DUt>b@{Nc-UyoZ1{3ISX@CiI);353Vz?bokfrt6-n(Jk@ zG#J>(pD^h266rB;GvA@RK64{Uka5kh02#g<%^I(u8qr& z*x7)z%NZv@d=t~~B!A=bE%pcB#uhw9uD^o;e3x85&7X~Y57*)Q-6TS($|+HW+$EOWCpM+3l7}To zsT|?2X((H*JkOufye46JL#7qdTe&>LM0gVd@;sB!{Y--ac>{BKw@;H=*{pn0Cv=z%Rlvdsmoc|l z%(-G=^iqpSSD19i+}u{Smb97MWd0a7Z7Ecz1315{c?>;U*t>2X#RXnmtZxZ#p~r7- z318@fi(GKA3odcNr7pP41(z#ehZ*&iyS=`0SHQK^tT#2r-5GO1aeUSYdPfj{m)Y4* zzz+AZ1SyKy3|_K%1d9F<47@vv#5mN3^F4%4@_Dq&9at`3;BTnz#74HtJ@Q3nhA&~K+=Xl9ZrmXE;A8S-M&G@-O}>JA*(dCm z`|yz5&#!?V;8%V7@MGCe-yESY9-!}y(#HiNQ*`#SHmAg)2+9rRjw)WTz|iH8wKh4CA2fE7|* z=dr`;DHA8k%%)Ghj&-#*i~2WQufi-s{bPBgY*dtqboYp|koEC#MzDJjnMa@2zbRv= z-xBUSfb9=}su;gz5t)C2V`WScX~Gpmks<_%hIuR-V&b@hFhifu66Oep39qcI;_u3m R*m-sxQ%4e45%%al;P>!h9P|JH diff --git a/out/production/clients/org/apache/kafka/common/config/ConfigException.class b/out/production/clients/org/apache/kafka/common/config/ConfigException.class deleted file mode 100644 index 8b93e6b83bc08652e15826c91f100d5a4a080a83..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1261 zcmb7D*>2N76g}fCHYRCH+NNv;m(n&{;{s*rqEb;nY6sVk6$F-i$8+b-x#yf2|NQmk8-PWu>PTZwM+!L=c^xqnG!*4nl3`v$ zS%axzLC)@}xUb>?!$?PXHg9eVuVcH;`_1PJY>Oei;W{0kJN`Crc1403$cHf`S8d1k z*BHigl|6pI%_esmX4Us>r?HaXW{7XNb-^%Pu^sWI+o}og9j`Shlc~5CHR9Bc=20^4 z@7f)Pg^KGnOy1_!t}yrc&OSFSx7BhT8k`-wVGQX&$3RRSZl2gmz-*2}9CVpIiH{c{`A-Vo}9IIv@tU z>UKR#ytJh|V`mhWC5seBF(!k-FmX&P1b^1Gn|0yIqa2MNZ~P_$gspP zYV5e45gOU`xb%lnf?jUzNuaD{#yG8AJJ z$0@T$Ifce#p?C=PIjG4})Pw9K61as4nrFi*T*Wn-Q9a?_APk9hGs4k%L79w?5Hvw#WZ2-Aos cNsx+ZDNCX6yGO{JO2%YTt}@KfHU+c40mefZ+W-In diff --git a/out/production/clients/org/apache/kafka/common/errors/ApiException.class b/out/production/clients/org/apache/kafka/common/errors/ApiException.class deleted file mode 100644 index 9354d483175a450f049683f1234c5684de96b6fd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 944 zcmb7?-EPxB5QWb=i49Ro)0UF(SNen6)F~JV?t&_z0#YF2CL#A*vrU#b-e~Orj|Eqe zxZnYJD8!7hq7ae`Y-#KnpZWIecz^!-{sX`>JS@Y-9)|}U_BkAQIP`ENP)(H8GU_Xv zXcND0zZ4K}1U4>AoTM^N`!brU5^?wv0`7Ai>-1c}Z*)G&Pcn#PJP5j})$yRUQ0Wb= z`7AFZ)jI796fevNC9v7iv3fThUn<+eGSvn9pm>8SZ4~GRJimE1gZP$-HYt`# z4D9j2LY3b3|H!a~ZOW(tJGe;@e%r+@QZu%eW6vq)e%Ei#;4D)Q$;44c@N7Vcxu9f zAHWY~ymz}Xv|{vuo#f1)oco)zH}m7?^>+You~9_@D=e!lYb;Matb1q+EX2xb84i?- zwKd1R4+7$oz+BgwIFTk9$S_r9;_xQ~D(|$>$*w?g^Tj}*)U~HdV7{-7I!L1v<&NY@ zNGFSZ8xU3!oc8%)DH-Wlpx3u<=*x2%jFf*Sznn=wuu)`O7&iI|B91{#SV+{ZN{rf0#IJhC3Y0i&PxTVV2|%gFxq&vKyeI zk+YY4RmLg@GL4B;ztthId=oZZT0OL>3)Dx=J$d!N|FzADYp7w4Wr?N1(iGUgqt*$$ z{s-x^qHa#L8*P;7?a(_w3m)##e4lm&R8gWe#U*_}CvRvLG?zPLh^x#pPg*4_&?IvU z3$&_DOn8Vz+A&~>lmWZsWN)RjHAZ2Uuu3u3GD2hG$_eDhBh-n(tVYi2p-2PpI$JNt xD9*y#RJ4)7)+e^#FfWHuDIQ~)u3~I6$4*5o%~GFH#OHsB$SjW7qEH1q`2}4B#+3j7 diff --git a/out/production/clients/org/apache/kafka/common/errors/CorruptRecordException.class b/out/production/clients/org/apache/kafka/common/errors/CorruptRecordException.class deleted file mode 100644 index 3508b3bcea737c07cc681bcaa043e44d0a55e4ac..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 946 zcmbV~U2oGc7=@4fl~HwUbhNDd02eVaI{L;~t}72+dE|k6Xfv!^X_S}@ zq_IlJpHAO0un!EYfsSn=;$$EusVo!4U5ugfTE!|kVJPlD8!(gtJ(LVGWKh zzBr%IO1-N?!U}@ZJYOs&k+KZ0y4s9B&%= z;kz%PoF!!Ln4z)Xy%e8?KN0cBKTC{?N5{^FHT4{3e=8U|i>>*NOQp_6^-QNGl<$>GV0-x~4xK-SH+_nN=eBtyEWZ%cHsmUKYI)EL zIE53AZ93)JaFXgIlIBWT$te_<&S4ETY`D^NrNwacU$HX0_(xw)QqnH5n@@o@%Jdh} zAAv1+SfTL_%?hZZM9(#>QoTznN3_Oh^g4BRjl$PVv_`c`FMv&gw6IQ3wYd!UP^Xzg zY~+X&QaWv=bFhT4Lq>KpLUXRl38cq;G>G9?&1J0OZ8w})?Z2=dkk>h@m9tJ!q}v_s h;Q2L5w_#7nz+ML9bJ>kCFE>W5a9Een7J&-b{sj&q$ie^s diff --git a/out/production/clients/org/apache/kafka/common/errors/DisconnectException.class b/out/production/clients/org/apache/kafka/common/errors/DisconnectException.class deleted file mode 100644 index 80a6c42fc07ab51d59f784f3192e4bda1b98fc3a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 873 zcmb7?+fExX5Qcx}v=D`Ih7F}{Qh@{kv7+TJh#L+PRm26*+?&Ns42g~GP0MR>6^RQT zfQPC&vxyK*Q&ruSe%rI-Z+?&W`RC&kfIWOHqJUMFcP#H&J_M))s0z$@<+SWHmGjz~ zuct=>;)}r4p*22{CTYq}Pvwcjj}R#AYon6`f$YX+Qy_O}ua&@bLmPG0>s~2$DX%&d znQPdXu#(`k9tLwsTYG`MhIOq--pRPFqFedxRz|VycCCq&bJqFjNc-3tqvGWFK2~=L zIjalQHX1kbyNo*0w4#f|Y168|id?px{lUfRY%J(1?`11>^2aWLd@OsOJ}!<0P<%`+ z#)hxszgH|?*q)2kiRR;nV~()HK1wKIie;W9WGM@L`u~Oq?EYiZxuPC%LtdJQD)Mys zbdgv=fC(BWX_Y|{IhvC*(l-?OOrxMNU+Y6Wq$$&+MM?!iN-krDW~D(AW-&)A2F#N( z;1pSUfuOdv-A85=vC2PVil_|Ih6vh#1uUl6083b=f{YD^*aKR#g+gtskL)P5LZ_>y i)TKez3$-w$GOIj}l^bQP5o`S~EcV6ySs^DGy!{1buB_$& diff --git a/out/production/clients/org/apache/kafka/common/errors/IllegalGenerationException.class b/out/production/clients/org/apache/kafka/common/errors/IllegalGenerationException.class deleted file mode 100644 index 48994d15e954b903907ed6a15881939048402275..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 894 zcmbV~OK;Oa6ot>^Wr$K*8rr50NXn|D2n)dm1$9#%3W`|JlKs>Sj;S+KjUC``v5Lfk zAHa`7oa;nY38|2wfFBH@pg_UDxvs~MJsMCq5k|UQ=s69_?k{>CQ4;~B^D#z}FQCJ-$*1VmJ&x}9S zXDKu50~Zmi31>V{R`OAjDf9-;$DzK^(a417`s29{BR3v98yfGO&%%D1npmf=jWu59 z^!4&-Rb4K%PK(iCZlY%4tb@9`)+{>RyOh7*(V zT#gj`sPd2C*ToJ3+~9hXQ3-WaxTittEoPo@Ra~pxDb%;3WR+THDbQr`9jtNJm=)nR z))@()NfkgJW!^y0J$yJtX%VqS7p(%(nU&2EJV5kDfeo;UEjAQvYmPl)EEa0r!zs#( n)Lp*aUP0ZQRb5hRbE>emm$52~tTtJlYgpn-=Cea5CG7qN$||)7 diff --git a/out/production/clients/org/apache/kafka/common/errors/InterruptException.class b/out/production/clients/org/apache/kafka/common/errors/InterruptException.class deleted file mode 100644 index 46c9f88f85f42af9b2fa8a1e35fa08909c6b1a6a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1055 zcmb7DOK;Oa5dJ1F)8M3`X=w?Nrlc)-6f6Xsg18hAg$NEQkvMLhB{5CxXl(<3iwi*@ z@dNl#h*>9fEYYgMhqY%s-(zNXfB*UM3&11X%i{`e=TXJBUTQkrF>oi3Eb0b!4eSw0 zzEqAF_oeb3&wY3Fj6kmlg+tHv1K|dJF%D&x`Se4C+!M!jf~SOfqx(^O64qF_L+i*5 zq*CEDkb~!6Y&i{>tJCZgGKbzk5>~s8E8m2Z6RD2H$(S>1UC(A&!Qvb*l9^!S_=JaD zPYo?G74}G4r{cq@uxxKK@mz~VJmoLtmSTjgEkd7}YnS;VJWhJ=1grAzUDGf#J+*MCK_m(XrZk$2ZZWUDTXG- ztgG6D2md`X!u~~=*Lvo$J;GM9vEz~QKI=`3Hn_BO$3LVP@Y%6LrKB62#p{h`Vz~2c zhp^(D9U4Nqq36^}%KW=6Nnz~3;N#3k5*4f<&+n@orC=h>-$ktQTH?%0&ZYQmw%Rk$ zcci`{{WYq*#%qzQfg5}wDvZC0HIA;&>ruu!M~uV<=lEK1_2z=L$2g5FwAu$VWX`9p zB8P2;*CN{591znMu*JfpNg{z%Bv4rhyhSFmQEs(=BKv-Z+=ViAme^&9y@dO6EK^>{ K5O=5NbL%fY74}j9 diff --git a/out/production/clients/org/apache/kafka/common/errors/InvalidMetadataException.class b/out/production/clients/org/apache/kafka/common/errors/InvalidMetadataException.class deleted file mode 100644 index 7a26813ec80afa58a4c0605b3b1070b3b07f6131..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 888 zcmbV~UuzRV6vfXZn`{lK{byoR8%?BznidyP5UKQ~Dnz0Ww&wkIGfmcRX3FfQ`n7xt z_}~ZdLy7lpmeR%+!Cg4>hjV{>=I;FZ{o^Nq7kFGj8Ji+ak%uCWe6)PD6_yk2B8vOk zC6P6syKfZKdxgbAYm$^q+NXG;OU#i+DU@GFCQ1(!iaSsG3f`d|XoaO-Wc0_$_(Z#7 zI*EB_wP!=d62p0&Z+ht{N)%r8tQ!V&PT@!gXLNc-L1@QgYXa?@bxF`Q7ZgW>4?3j* zrS$e&sLxZ5x35s!>3yLK3Su(D;3##G8SYCEf2@;)hS`_@GAMLzD7ykW9y$9}f@Z8z zNRxy~wHqA@>sMjdODjiq;zIo{k{&c~JI7N=SVa|!B5NXbk%q#{|7?xIvwu{6(%daM z<7GR&HcEUK_`hNUJ{EYq!>a-+@OaMgSnu-Aa~>6srOp)UduCZ;t*{kXXLAF~Jgd%3 zSivf<1hB>`fCF}lx7^vCqA*X`IgCqjpJNv?Y$L~Bi|EZ$w>aYVEfIw!5jQwg0T2EFu8Fdf diff --git a/out/production/clients/org/apache/kafka/common/errors/InvalidRequiredAcksException.class b/out/production/clients/org/apache/kafka/common/errors/InvalidRequiredAcksException.class deleted file mode 100644 index b356a8b1b2171d216a3b62b7b2180f165c1b851f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 526 zcmbtR!A=4(5Pd~p;|hW(V7zftN!*J!l!Txrn2iTf?q#Vz<)?;oBRx__t#OZj~RbWosA zp$EnuiYQSo6Dq)=ie+vnZV=Nau0Xkb*nI|j%6lpl-JETcau-#?tF=(Z>aYH({`>g*4}b%F$|Hw$mJOComMsru58DEBk#bswE#)F@ z&1L;qK%5CoA6XN{(!?zp_EnZR{0M>EcWrchD3IRWYYAkI?6ne@ZD^y;``s($n(`{7 z%tFHkgp~xRc$Cb<9UTdLYgpIzWlshj<=@GhJLw0u+qK45&RG}vb#pI6eciM@9i03O zR4*oPHG#@*<5u2FKa{5JU&KzEcI`!`*>U!P+tt{b*Huv@+liO|P6=cK*^lVs@lfb=+Zi~g>h iP*;Xk&(z$A$`)5Au`=VV&&1k!g~h&jJ|%LJ!rBujUaXP; diff --git a/out/production/clients/org/apache/kafka/common/errors/LeaderNotAvailableException.class b/out/production/clients/org/apache/kafka/common/errors/LeaderNotAvailableException.class deleted file mode 100644 index b7a6a6fcd5c6405f7aa7424cdcab07dc57dfeb7e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 535 zcmbVJ%T5A85UfES5kU|IA1A%}NZgBwi4YQlnqXEgDEDS}Tt=1|GCL4|%ae%*KfsSN z_M-7(qKR|p?y7Wk*UY@Vzq|rC!DbOl*eIZ0KtrIKNUJCwNSi35@46QP;##2GF*hbe(#-^^?vuT*Md_m@$}sknq?6a~)y??~YE4{=w|DOTZ)bQF@K z@4H|R8U7u9Y3v}29Opcv3<_L{)ELJVX4=eUIOh+Ko*|wVJtdCC1-r${ZIl@|KD)4r R3M1}E6>E!GHAWe%e*od4ioyT@ diff --git a/out/production/clients/org/apache/kafka/common/errors/NetworkException.class b/out/production/clients/org/apache/kafka/common/errors/NetworkException.class deleted file mode 100644 index cd0221de0539ae908693c81f6a0aa86d926237e1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 870 zcmb7?-%b-j6vn^lzs05q0>u_lTcZgU!p3OgMGZHoNloPj?EPUov}3nBWOfR^mRC)@ z@Bw@%<2l=gV68FkCg03Cd**k(nb{vdzkLVr3Qx)?VOQj#$Rm-*0U80C3hSwMhT@)f zsj4lUz}3`X9BYK6$!1 zL^M(4o9A1^4g#$3c!#4L$|&%h&9UC)%u5~>kHyv$>T6cA##-i5pvJ{@tn;igE5bc& za3p|DRsnR7=NAZC`v+6x77@GhCuWGotZa_p1=vD0!v@&KeQqe&+8letu~;az_NT}% oQX9OxW=7qfRozfabE>fF%UFd))>CFZ`wL5a$@%QClN=uW0>$R5(EtDd diff --git a/out/production/clients/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.class b/out/production/clients/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.class deleted file mode 100644 index 032cf1185745280a96c53d5baf84816a07bfe2e4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 918 zcmbu7Ur!T36vfZ&ztyG)f~~X)QW6!5abxriLt?5Vn1%`w<0pXE*sP<5hazht>mrW=Yy{{MmJ;oanhdp1 zjI*Ei-x25oVWIDAk}8`H)g;qZ=Ey^Y+8bj{x=Sc;JsT1#eRrY>ivwfzK^7ltf258l zyt6!T5n~m@d0lK)(y>Vh{eknNP@Su2tiv;PdZxn2#j&%Y_TKpIe%iTN}bJJA~%e;IsOo!iln@@G$kpj&>wc5bGpSqddc3214(evP+<&v3Fl3 za?YwoDodEuyw*Wjy$rirT0eA|kMy3AZalsDL@y+30}U*QtcbKk+Jv|N-7dn*8yFuo zcTmoK(PDH_<(tBP9UTO?#p7*Wl~6~8=Ny#v4)46;k$9~3rl9Y+Ws$YcR-n!14wiV< zn3-@F%e)f63abEi*(u&yZ+nW;JYkJvtmlN*%vBKBk9%k`Ls+eXwU08tK+xNMK1F#R u*5#tj9JVpDUBH3@#-+HARsNJ<+XePY#L7JNF-LszkBGvOh#d}9!h=6B1HW|u diff --git a/out/production/clients/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.class b/out/production/clients/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.class deleted file mode 100644 index 4a3e750e4a57acedf4fd128ddde169b36c55f98f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 559 zcmb_ZO;5r=5Pge4V+BDJOw@}v#l*(M8-|4Nk!TtYpxl?Tl$Cb3*|x^N<;lc@KfoVl zoL1w-gC5+&%)XbIH*a^|-d|n;Tw>qG2KFp8EwmWwu`rScGht$>)!p@gf!#1v`&z{b zSILYAsVGrnb{IqXN-CLj8JZ{VJ%8X%z?JVz6C;)1Ih`>S`+6Z5YOYjboQ88@ZuvYQ zrr~Oj25}lk_=_wiD;YD4Ty1=ZN8DQpXU&&u?sz&3wQ__p+QiOSCqt!Ee>D+NAUz)U zmWeREC=zNheDp+=koyjU6^c0Kem?GpJqCNKQ{#z|%nY^vBzm67RImw~VelU=87_X2 znF#vM?32DGBTso@1M!8AMo_I)|l>&kVv*}W70EuWhB z;2+>0Wt_P-1WO4Ccgc5m=4L)KJGX!S{`w8zJ+^A7VufXuWsPM$#72lsff=uymT5;h zudVrVbRZx;3ry`><1=Zpj!bh^Ar3!6p!!i8o$U#fw_kSzf_;0T1g6{CsN=kMuH2bC zPsuaewlQHP!O0#ugRHB)K&x$C63M=dyDGYrUoT}8+g{I_NI7SnkB)72Xl$NzPgFnE zvGj*OV%5*6!InUCyZudmmr*KB5}jsFo1}HabJlhCifgvGzp$sgmr2pktt^2`EOSpE z=O+TlKcN<58#n#`SFD}d+{NmX<|(Y-eUdlaP91ejvCOeFSQZ67Ja{|;Z|})HQ8Y0g z&-F;Li3;5a`ny;{h)1+OCaZ)R0@_o9q)*86j#fcyr8$83S%gfJ)+iK|8ekbSw5tz; z@D#ITF<_3A0Y@m)3xv&`-2qDDh&8^K1!7|uHbPJWo?*VghFHLJD#+Nz2)jo%w@_{F r3{W1YZqeyp6x4-b)HSs_qB3i75-S*Iy(HGFJ6PNo&u57`DPj2^2urm~ diff --git a/out/production/clients/org/apache/kafka/common/errors/NotLeaderForPartitionException.class b/out/production/clients/org/apache/kafka/common/errors/NotLeaderForPartitionException.class deleted file mode 100644 index fd87efd21518e6ab2b73aa5b00a4e5a38f6cb084..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 912 zcmbu7Ur!T36vfZbKdVg<1Y3%r6!8IyaUV4C0mTQDq^1%RpzoLM*bZ!W$m|sTT0S-L z!4KevGTyu0U{g^a+)d8>$+^Eddw0J7`1%dNb3Ce`jCGL*A`eA20yG0`DlDej8H&5w zrN$Y?`Z02cWB)Q za^85(_V#n6FJ-c)P}}aD(-nm=CH?T&JCpSHBu+5YX-fU<{ND@;jp?$JfwP|^Tvl0% zXq0kYZ3_I$q0pKx%}>11Dlsa@cH|;`XrvQcfB%}NlCz2`=0%o8>LRNOul}n|3NQXq z-%hS5Hof=SQ;$4dHyE2J@g3p+jr$02i^to%nnMLeo^w>zJG}FZN5x~QHHP|+`uBwsuL6LVu@D*SY{Q#K0C!*ZtadSH%nOK80#6KK5^v)_TwIE%n(-nCRSmV zwZW|BA6P4#bHZB9S#1>f1%lS@lQD|3uq_VoID<7Ow(DR)9*j%50S%5VJ(S+9{REF| Byix!F diff --git a/out/production/clients/org/apache/kafka/common/errors/OffsetLoadInProgressException.class b/out/production/clients/org/apache/kafka/common/errors/OffsetLoadInProgressException.class deleted file mode 100644 index aa1c89d97411fc10f9c7a798b6e166b57032496d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 903 zcmbu7Ur!T36vfZ<-)d6?!4_HtX%h?->%vSwcJ~i>d z55R{a+`HWnsHhL_CTITS+~1tJyI0pge*xGOgbsU$G+tgAm+ z6Itcl(f3eYCX{hcpt>2H%dgTON*(#fiH&u%$7#F~8D*3n|C>Re{Xp3r(8<7>Z=5z| z6+=08M5;dM5LmejyI)#9He(yAcQH4j_1DY1;g}UvFvqgQQe&wL?Eg=b1m6Cm{IjCI z@ojG#izbTnYtUy#10EjH_?T8Xlu@8LWh8w sRMbjg>r>k;%*$X@ie;?OQH-r;*n1HRv(ycW`05W4nZ*$s6e@@3H@BF$#{d8T diff --git a/out/production/clients/org/apache/kafka/common/errors/OffsetMetadataTooLarge.class b/out/production/clients/org/apache/kafka/common/errors/OffsetMetadataTooLarge.class deleted file mode 100644 index 34760c143c1076c28d56e32152dda9b713f11108..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 876 zcmbV}OK%e~6orqQ$IvLHr6pC>P$?Nab!ERe~VQl z7W@Ey6yn~=2%->##Ef+O(7B(FufPBJ`VGKS98^%o1D1y@J1n~a_5$n+tT|=19CVd+ z+N2-PUkQk}0xM@GbzY{vD+gm$A`X8-p!`Co+Mfy(jvjRdif86Z39LpsRqw~cOJzIq zazH!lk%Z}EilRZ+zX%LYrGWlP}Mf3-!R{g1krBmMd| zRwGX@jD3{o{pl^SjR1FOT&C3$Dk##NVv*jZonsmWjiuHE;!9>(C9RMZsFQgUYc#7) zO}K}3S}|aQlmVyYWN*3Eo?vO7uuU$ qt(ENjKXA3p|Jg|t5DpxpN+yoJB-Z7baZCi zsXiNWXQ^!iVm0Ah=bOc35XK5`+t&4ceW8Pa@y~VdT>F6?jcnu_=d6qUPOlf6q?0C{ z-m#AQ=HPo^E)wS3Rj6&Zzv!>pAM$~JlDIJH?@AJHWa3!&v!kyW6dKcIrvqodNv^E2 z6zDW&U2O_{5}- zlpY)QOWymkq1Zx+zXLxl9>T*Nj(545Lj^_7nU8vpJFhq@j-}QZ>PKc-pjK!FR%mWu zk+bT=g!@?HN&w4L0qoH!-g0YajJa9D8e^WmX12mub*(z`q3*=JsleRqHR55RY<=8?lP%V(A^EGrI54$1;kk+NF08p=l6 zgnthY1jLcR#J&lmScY*!wz?`y9KM7=?uQO_yd#ia-)IPA_RXmhnDlh0es|j^%GTvc zi!#%m@d+ylPVpd_iJLkS*!GMKT-lL+Q@I!N>_WP}X}3-2Dr=36+&{`U_H<9iezR_j zCvBj9Ui+#Olf{}qW!*cMm(p#?FmR7!tHYr7C{u4*bH!b2?9geeD3U>9>DeiPtS`F} zJ)9j0p!|qlj4eLSKkb-5HeKthUz+Doe(ha0*-!xmOt8$b6j>GoYVR14K=qB&yB+;7 zP?V9SKZ3q47U5uw)^Unb$Rk60GDrG>GF!9?TCwUxbgC zricMEqzpJjnr^_URKN9*8b&OUi{%7S>X!`=bO4_)n_wNxQL9vfEe^0d6tjg~rP@P! on7Tr5S4yaJ{i+9QZa`&=3!_+>Vb)h-t-ZivUp${ha+1Q*f5djQZvX%Q diff --git a/out/production/clients/org/apache/kafka/common/errors/RecordTooLargeException.class b/out/production/clients/org/apache/kafka/common/errors/RecordTooLargeException.class deleted file mode 100644 index 461e026f1a4519a310dabeeee4f33a75d69a8efc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 879 zcmbV~+fLg+5QhKF$utUwatx3{pj<$L@I@S(9X5yBP1P(l716OvX-%{?ayuOvLZ#o?ly2@H(BlleS#$MHp@uUsZ z@q@3rF}bS=RJOevc_-br3O}JhN^iWz9(yUEfJv4)mLkit!2bINBT)U%!S7xDGDd75 zOTPepPOQMe1g#$^N+FL7?a3GE6lK29Drn7C`Vha8l4;UBl>$ptT*M6R3WFki#4JS& zm?LGtDbjQUPNllrM`{$YN-jzXqC6-YBIp3-v5;ULEK;jff-MfQhZM7gT&3DadX&0O lZ&yyJi-W2cYHmnni_7C!nNikfVr{*_VqZL;6>^fo>ND?!uDSpK diff --git a/out/production/clients/org/apache/kafka/common/errors/RetriableException.class b/out/production/clients/org/apache/kafka/common/errors/RetriableException.class deleted file mode 100644 index 052f7199407cf7df1e1928ff6d82b6bfbb838318..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 864 zcmb7?QBM;=5Xb*ZuRU#wAc(DkQt$;@!^PlR4T(WbFpUp_eP52XU1@Kdy({vycxvK< zAHWY~{LgtISZa)S$=uHD%y0iQdp~}D`wrkGwyUUML(CH~PsMD8XolEQSWb<%de}2Q zwJ!N|^hQCwS6DoBNt)>->*?Xdl*y4tDO6tB#AXKyt)1={eW9bFPWsVt=55mN+=!eE zy!)z8ho-aJQwR>-nNe8k+QfXAj82U|kv)Z#u8VoPrUv>&FvtcrRoL%3-;eZI#{(0c z>(A#pirr}BlE`@Pd>VZ;JXY%7UdCpeQGZ7v9GNuL{hWPEph7v;la!llbG0%)r!DEA zaWnpBbEO?gs2;nCkIg$PZPjl}+Y>Q0)UYU~E~X*ok;04r?o*-tkDAwGdp(t43uWFp z?+=d=;ts!e=`Em&fNM&ky~oHAN_>H^)qXyOy2_K5Xsb*H>V!0~%vJMu8t!9-9t*IF z2ZYFK4G)_^I@X z#0NisAIdm$UI-$*2+6QB_n+VXXMmr-zW)I56h~#a*yHek!#;-t4~HHe3RDwiwT$}8 zCfdaB+b;#g8-cZR6DO&R)4q%*sze-q2?6(oj&*t_;5Rx~@}mqQ84rSPYIQtl%~g6s zYd*<~NVQJ-0>yLlK?$sPbgbS@#uv)=_?f^)$AomdBoCD5#bP?tiNN!Yv4cQf%W$ZI zk-QwqAT;B#i34S=u}RR)tm`i_CF`#~hw7RSEdg(=l0*(N{y!!KN}-%2^l)>jU#c$| zji;#3*Z&@{&|pFJQb$oc?xr#v^{fo(j#}en#VzHonb=Uh*1U(E<+D6y_Z3vI#-YZc z&S6X7*?%oZ;CNNvTi)<&i#R}uekb&uV;dfB(C1Cc9F(ap*b&8BRB59?Z@}}Lr&EY; znP{D2nZ&>rLAFt))BYbBZexQoYQP=bB?zB3agWrDt>xG=%DLb5n^QQ8)Lk;Mmr=Qo zR5PlZQ<+s?#wskbc+kTYEcQiz2;O+kdXnWp$)BRQh&2G~`6ot>^&m~G}X-P?cplQpZqzDVaE~uLVQc=W$hU}+iNQTCqYHSx?i&Z2R zJOB@cIM<1)5`qx6bY|}Lo$nss`SJ7HcL2|^Uq=l)BKJiei0lSv2iQ|sO^vsDJTN}B zE_r|YQbD~@SUzz{n&~7P=y7hUV z`@ggzYybcN diff --git a/out/production/clients/org/apache/kafka/common/errors/UnknownConsumerIdException.class b/out/production/clients/org/apache/kafka/common/errors/UnknownConsumerIdException.class deleted file mode 100644 index eb1f6882e00dde1164f96a6ec42274a9fe5fc167..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 894 zcmbV~&2G~`6ot>^&k&`wG_*~BAStVoA}jajp|pC8R>amd?z*zVqGVYybT9{Re=Tcw9vVdm;}-9*Hyqv;yoatfj`sdfYcY zjh#K~zEM!`6;_U&O*3t?z8+6ZnH;&4LgjUA2=}#jaMlMO54UPBCr{S3$S$Ac{!z3}j zd-3*bWG*ur>?pJkdY|a&G9}Ooa ztYamMLi-e{RNDxRRLCgXWi=s-XfK(K*Nt?Ypn8ZzDN1jQ`-(nSs z1wVivg}8SzDp3eRVl16&Tjzd0WB>U1?K^-McvwaW`yvM-_eCBAXar~yHe&6K8uhe~ zjkD*cZwd5)uy*2XoG6?0)M%=U%#n`>rPs!q#l8A`z!6= zeTnoW;c!QU>S5$@f5L%0^*@-Ky5~JL8Qy=N~Ms8ql^jY{zbAn%OD%x4dfN z1wVivMLcJ_VpB0(xSPB)XY!tB-#xom*FP@-yu?lkMLc48%(B6<8DJ~Gw!lK7oR)D* zxkOuYT7N4bJ_yVmTa$Qcd`redRUi&uLZJ9c8|{w-a(ho(0{LUxRs!=4ZPce>|4g~2 zJd5dMv0)>^N`h0J9p-&kCjxI8)^$QTkWp8KJ^8gK!^rmg)`ZGA>yq%)^o+eQO*_!h zXZJ-quRW!E_dQYr-W&=n?=`;3a~a0cbixzwwCNmjprEgkM0U~ye;EX-lWonevlkpM z?VR<$OQKJ}|F{HdlkM44Z?$ri(up0qNWIrQk0*D%({oN+Mj3M~ODq+ZRe{(4IWB>N zI|y}iE@R#PsJv`TFK-5oZ4~Hlp)Zd$1h_}*ELAfoAy0b>OM0J9UeGFNE!0L3KT^v) zX^E^rh0H5hpj~-v!UHT)#egMJ1{{%-y~W!82s6`!HHxvG5-MX?Mj$^PVwo7ss@%rP zO|v$LwfPU$3aS}vHDlG0qZfgc6r8n_xCt&vQ=rfSrBXlAG>R|Wh@U&L^Umm=d?Z4 z8`&}16Zu8jsNju_5-b~QC|h_d(8>F~4+2@1WitXb&Ff5MFmf&j6Xo&_m<@gQt^eW0 z9&cWe*U>;-kH=MAPsnV7w(x|}=I)vZleyab4$(ZhhtX`^!@Oh3z&c}{M8E;#A={7e z6b-(>yLlU`MChlQ%7}EOg$xRmu(^T+GQ!Y4Ic_wC8{Yq2RR7SkW_P ao6Y5BXGODt=c&XCY%o_yQ@}QADE|R+t=_l* diff --git a/out/production/clients/org/apache/kafka/common/metrics/CompoundStat.class b/out/production/clients/org/apache/kafka/common/metrics/CompoundStat.class deleted file mode 100644 index fefb0596da3cc191b44e42e4d2215a27ce8129c1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 438 zcmb7>!A`BE!|XJuhHG)~^V{xQr_>k7p?&fm4#^4peGZtL0_RqNo&+^e~* zceXL>Ofn`%B-2L;NHW{FTtAg9+4#_Y-@*{?{u4nM?N_MS6l%HJ=sXZ6rzW&;gxM+E mA>8pUeFC8)I+H)3E+N9Xbg^_DB%+J`X@I`eL~gpG9^zloM}X7- diff --git a/out/production/clients/org/apache/kafka/common/metrics/JmxReporter$KafkaMbean.class b/out/production/clients/org/apache/kafka/common/metrics/JmxReporter$KafkaMbean.class deleted file mode 100644 index bbee68c3b2c4750ff2bb3203e56ca38450184a8b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5120 zcmbtYYj_lA6@DjsnPf5{5E?Lr!a{*ULRdhc7}7|&*+4cRp=m>{I@z5h1G_Wp?ktyz zwne;Od$ZP7?M1D<(+UY8mDXCVwrai9yRG-T{_>BHKZ@`7&15s%-4Oct|zYlz?m@o-};j^cgda!g#_ui^t5 zYVko0AHs)Kd_=`ZHPqusH9jWp9~YNTXqbi@RD4oHqcHs`jfQU0FdH`u$t@~w)o>d= zU5(po@fqA941ZR|o#K91Ek1|O*WwGfTU_o@ac?aa<31H%6!$OH(C?Qu+>ft_FdoqG zARf{X!^0{b5i5DL8jopMf#WJ3uf`J^R*Oi!s^Wx-lL|Daf1jCjyN!&g;G(X5#sT9{ zJY(3#pqVjkH@>s@dV>NrW4bvjnO9IRHRA=>O2-pMmhWnMtU=pw3pv`^Rr^I^`CzEmPxbGppLkJVn-G&aDkG5O3<#s4&Tv~2z zS+9cdW+z40^<9>2b{8`JW^RwspQfat%SjsPUL$9TXI~h0hb%5b^W5~ChOI#Fv~4rD zIc?szE0BwNm@8;~T~i6zU`a4bDg@)LCoWU(QHk)> z^|(fIKWQVAp<+bEDN2LZi2vH9*OWt70iPlqT~^+u^>CVZ1+j7@U#qDQ!N!1(Vp3=& zA~M1#W>yG7md~{;XV$HULShs1-DwXv3g%VHVe!2nhcS`A2umz?nBKN}cR!l(vRWj? zA{sG=(Jb+zDdC5-U0j+7f)f;r|D6hEm6HE(cDw+3y>*v*`u_`6g{5MYoU-f#&VJ(C zRe=;iNPSWSMShE@NS$cb$QhZ}=B3E2P~I9;FyHf8f>IHn4TAD>&q0n2Do&G)!9t@y zztSq(l*~I1YHB@BA(u3_S)wmZ544XJqVMYHz?gy=V(fL+D~$XQ%S1;TnA17uXwG8p zn{}+mn^Zih<7+sh<0(9?6h@axpd?mm5-Sd*bSenA&10eyHO|__2( zdli4s@khL%<4^driofXit9a@hn2j(;eC(EO)@OC}s#Y3B=BxnZStX3d-- zHlQLj45i0(WrmD&+Bs;ZRw!6E6>|b44(x#?%krv|h?jx5$Fwz4~ zE@P%V6JkRRggd`nU$o(X#-cKf-R3}=$DPu3HwZPa;ySsP8?Hmb#>?h*+6RoZl`8VT z*GLzF@r0I&)>1cNjHRm}lr50A>w|63Z&!hx$Cp%Wd9_K$b`8tU?=ZPjlpKG?r|qR1 z5}Er$1veFqM!1v$n-F8(Qj)AKPC=fURH?9YKzQ5|G~p?*9Yu|7*0fX1ThM8R=Ix36 zPQZ!PuH%UmT;Ok(-T}d%OUhQTSOG1f9WqjkV9Nd$U?~q@MlxyU^NUulTpYmb8CT2Qo<+tcKfZ9; z6DXq30CS{HwN)hJtP=Hcl5&nZWaPWeL$t_Dk_KU0K9u6>(}XD_NhFJ^Igs?_j6Hq3 zwlbgqzVZl6s~E@xiOYXh}`n|QsM-?O)14sOLJN@jUBuo>I< zL~D2puArYzj>Eif=l>3L@w$$&#ViF z)jZ)(AG{HctBu_i| zPU=nS>qY6yYZ_X>R|*HXEYUPH#xV^CecOj{*rNbpo67b}lh_vIB)4##T=%<4W6{p4 R*f4Ly7W!LE$Oob|{{wTbai#zO diff --git a/out/production/clients/org/apache/kafka/common/metrics/JmxReporter.class b/out/production/clients/org/apache/kafka/common/metrics/JmxReporter.class deleted file mode 100644 index 5083f39b4245845e4afb1a04127907da501ea8dc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5932 zcmb7I33yc175?93GH)_@5i*dl$Py4GnF&nl0vG~T76HiwOIV89mdE5}GBBBmGcOox zsoH84>|$#-rO@4Uu`OctFNMd3;139}Qs|J{H8sWygc^_=G6a0pR6A|Dh+A+*-)-eJbKS!PCqu_lwT za?8_ZK5ykURHU;>4dM1|E*ZfQ zW_h-YXO3%6Wvq^V0?chNx1~8l+Y@GblbK7&XLr(X_oRq_-njtanRSh)+aQNmC9;|B zR8m9ozn4u+%hwMvqw zbbLX>vQwEjSKY#@;ecP>!+c?MwKSd+zNSq~UKStfF}u31v>D~w`fch-o}->Ko;pV6 zuv$0zJ7MM|rBmoed;EgHAYP)-_%^_m$oY4uteJ9c$^A`l2j}@l{_VP69aIiGuC+mh0dO%ripdRX`7ez zr_x=jLrGX#gCkB3sXEjU{%f<>}4Hu0CK>;tux`s)m_UXi`8*Un6D|5MQ zt|5`LOq-3fK~>HMvyk)+-CQgsV?(QfpW)~7_=SOA3eMH|6@#PIW|LTA;5nIk9=|s5 z0)C_8w=C@?=M*!*!0&|r@u#9P@O%8hz#nB%Yvw;Sz6QH={K>$d@fTs{uZ%APf5YDm z`~&}FCOP;^ab7gSz`yWs4U-*#LO2$e8_vT29uMkx(ZF&1rwp}o6(NaZ`Js?kPTj~C zt_f`TC05ojIZZpu%yOtw6=g$DE_?{Vb0$7>MW&0Tve%nkkBx=h-g5g3aLDf1th3WET zPI1{2UP>lFpD3>f(pDyE_oyIcF9>pelSCdJR`R#!4iGP12d**`OgjcnJ@;IbHf`3r z-m)-t**)BZ8MXQJJuwrMc&q3`t#=2o%oYtr-iyqYtYQI2;3=#j8*2o(=0p|yO5 zxBy&?04_m0-((xb5Jv|`QL)Zdu@MzYwk|RRf8;PKw+tch7&S&8L$IVUOpG92KVle% z%Q2D2MRC2e*;NeGDctN_xCSrdnxI_8e}SWkx(W!sfWouHqe3gH(6C;a+(|hHx6Ewd z4u^z|eA|T0j&e;b4N$V{VN`92AB_$pq@iQpFse1&hsk^~G(3r#R)1uQe+cJo9vpct z@+kCSjMFfT@gCtx1>ss(I6OG=eDkqEgjy?`k4>pO@{l@f3$EvR*JG<1{mPyh~;8wM9D!%WZn?Mj%9vY z=65ZdKzE8ur>fEMRK420mfG9tjW~br=-@9W>y$+e^rpdev7avO1gSGc97m`v>{EWS5UUbb?R!& zL@zR2rAbBiCOx#A58p{db!2t)jSz*;;YY~1f!laBcA2@t_26FDns_tq4$`8{&EYv} zplKJ%3&)FwF<0Wc(G%BO3vt~f91SBP2|?NB{UguH=Xr(C`w$o$d8Ro$UupG<%2~1E zY+*wKlW8A0pM8K~p5 z#lyJ7#~N4Tez??!L5%B=xDQ{(B)NcD%^L?pYeq51ojP?hXLr#(ujTx0T)lxyoe$)J|-m8^~y-4IpSMdw`Zy@mD&_8L^2!1+3EUB<|*<|M!0xtRA;R8%aKveQD~ zQ3v+|Chs6FD9h1>3@=&rv$;8TF6yOuCr(tassz9UTM{`H*IGvXKmQ;Hf+oB3I(rd zWe=<#321rG*${|l*6l6Z%G($D7&Z)?^ihH`JYQ@c$ttDE~n$hiYwPdg489f~s z-RD>)i-t9^QO7I3b5{jM&w!IT&MSL~oNms(}bEenhUdT;Q4DQoU0N}cW| zQ~yQy#s|_4odWSqWv9`1WXsN1nL#+2dM?oZx?(#US$k6vZnP)OwB&+(kIJsfG6k_9 zFqS+YtJJcN8TQ(1uUc|)FI?GcolTjJ$~HETz^s9GbQp-C)4(NMHqeD`0~Zn3F=ya0 zo){RwkdCJYo?%{KIA~l>(33Hbj*CEFkON;UflG&&rL~gxzCI~Iutr=T?pH-=%$6Ap z0v)!NfKhPsdAFS7Y`vx2tPTY?gOG>e8gnV2`y)+_$In?ORvKgcMzw)F5J5;*gI~9} zEQs1!3AM8}+(WWF+@t&@aD}KIo<8zk-VB)A5N5iDkU3mK*c_=rGkdfeB4$sdhUiz? zgm9H>A9QIC4L8t&LB_nv>ffTb+vvv-uJLuw_laIy=RavJISF-=EJNxfd4<(UGSk#a za*puS#W+4gi@QWd%x{Q%;lZa7RhKCh&hQQD;W!^8avU5w3LZTXJixqs@D|xge!Yww zev{cXxkrCM7x<0~!+W2z!+Yr+($UF2LAwO?k$YI_)S<8we4XU&R8y?pI?+Me#v%6k z9MW-9r-(wQilAOYbq@2?MRJHSwk8$m|A7__pLx}+!?beWt%Qvo$R5KTBeaV diff --git a/out/production/clients/org/apache/kafka/common/metrics/Measurable.class b/out/production/clients/org/apache/kafka/common/metrics/Measurable.class deleted file mode 100644 index 9e0e25baf186b328edd2fc603f53fb0f686baee2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 204 zcmaiuJr2S!6odUrX@CWe0SrV!Y>XfVM8&`fUZb@1Ck;wl4#&U&I25Ak1z2n^{_M~9 z^#-uRR6qz!2*PM?I;RQS<;A*8@`Ke~%aYTQrLxA@S{mKDTs1Nt{9x-e&-Ullkuba2 z&MAG$tDZ2A2Noxl-W2>~!umhaulojJ;XAV8I+NF=(5j6HgrNTmP)Had^anxa;~DJ4A}4rRo-yB*&Bnu7cmNM2ZYr?~0|Nu|VcwtT z3&0Uu1*Nc7*eE1~B)6k8a;jP(?9al(=(uPl_w2KpA56%sD>dKQ+(b8gZCD%Ib+*@? ucvqQ0|ECDMDW_ZROMNM7F`jUk>zMWk>74C16%znlTQ<8GHq>wa z7eDKacErx;jQ#G9QlEQwAu*;C$Z8>415e8PnjhdcwXniRS7X-9JyK&&VhXSG^Fzg<=LG;WEo`)}~ zSrgDBui?gubsA+_ffor(S6|piw$+L}pMe9X)ojww+M?}K8Zhp_qQ}hxfpoF7O}~}! zxhpVQ^#b=vtFhxYx9lCCnz3r=*#5TN^yE2FW}-ch$;y4JSXb4ogu$*?XVOx!)Ys*t z@Mp#UY1%(z7YbHP;^vEMXZ9R;nJ}Mgvo5e%TYd6$ZyK^a{hj zLO;d`&%?*a|E$6plvB_sXOO`)3}F^|TqlSdeBQ-v+~gZTj}y>hmM1}&*OTNHSD8F( z{({s`+@xSo4k)GB;hgPozL7XD(Ks7k`Q!IM9It(L>bdy z(dGn+_G1^4J3Wz1u)NrL328nO$+YCG8_7K)x&Iazjo=8eC6C|5V zl_8S6Y~1u*(zI?s580As70FPz^QYA05d&fbdF}XCaGGu<(L39f{-KMc)t=A_oPwC7 zuNh3Xt1y}Q9wWbF^sBLRuW|m3IVs0YLQJdXkEsGGEM$#;s$Z~I)ei0or%(%O%mwC@ zrx<0LG8=x<&h}vC+vUfDwb>JC8Sf>|{~E0Pf7-c4XiqyJX{V4h#hJ`JugH1-9}-u! A#sB~S diff --git a/out/production/clients/org/apache/kafka/common/metrics/Metrics.class b/out/production/clients/org/apache/kafka/common/metrics/Metrics.class deleted file mode 100644 index 5fd276c9ffad082c37a744967b97e97268594b9e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6409 zcmcIoX>=3U75>JOJ+?FuY`}>j5U~^+*+zllhGI%eTu2GFp)r9#XviQBvOtzpEd)1h zDBZF&UDIaiPSS>M*|CM=me4JvX}T}JdQSiJw14|Mhg169_eQHNWOq6C8O^+T@4Mf3 z?|t_^>5YHAasj|b{Mp3)J{UOV!%g^ziPJuK@lkpBn2C@3V9J}7IMu|FPnh_m4}Sgd zfQ&ro!$Wx3#HZxp5xIQYj|KRQAD_kNMt8}S4AeAb69JSWDVm&*$#UNrHNfrfA*el!{}u&zImiiGT>9UgH)$L*uX?NB%| zI+}=wMx9J58cv4>)MuZX-)q2h#|^CRKW3k>L)lC;#x?P9HkESXnGk2Xw*z*P3z~+a zk+_}7rW^zJ&SJ@ys*E&~L3`BctvV;qdY4Gi3wpN-?X(k5CsJtx_soiek`}Y$k|=y>C)9zT_J%< zhxT$7Z8mI)#-o|720R^W_Zjf^C5E~9n*M0q8O)9zaZ-EjBQcJ&^e4i0Y@eNq%CkG@ z&5T4TWo=bc6+CILy`$>n5?(@Oef62;ux<*G4ZmAa|WLH@8ta_`Bhb2g-4nq zPDW?Gfh`@{uch0jcFwr_hB+DM&Xx?Ew)dF6di$zKPkVh=RkHa|xHjfM#!T8$%_Qq* z4R7Gl89Y#v+(2o8owcz!Xfddwb_TgVGBM5$#jGc0fh4e+Y7|n%&Xf{R-bk|%|+Pw5hfJXuY zeugC90t{@dE=M~y-@1zxNEpmdIT5xYC#C&r;KtfGrF;aVK^rj8+_AQ#l;nsMku3pZnngwB>SU8NZg}ZP_obodZKgVkpMi4j9Sqq^gp?E?n7WCamjyd6sftA$&iWy@dSZz{K zW`6ez!%G%U;9d)_;}`PyhJ|0^O$*2HD+|BIn>1SFu`?EPB6e(BDv}*#7ubF>>?AYM z1f^YDoTLwiw`8NSVJAfy+XC8fK>FBlpxwf6@LRe3j(fg6V8>F9Jv}Ip&wbVJ9?wx*)nRRooSq-&~T1`lYL45-dnVw{p3O3CUgc7P( z3rlXMrPXpkQ=Ie3;Qu4N;e;K_I!wmv^68P0Rfmn)JyuJQfgH0aJe=oCzHZJYsm zkkGrUK<}nn4ZVkEsowBXeFJG64(8Al>>P)W^!vB7@cy1BlsUU=qk?IfL)V~**Zq9DlXnM5 z_8@LxD-YrjLU=#jc{l$KD*+qn(gB9+eYl->Bw6s%VM-J!GA2P5#~)Un^I(YgI!yAe zkMH(!WGU*dFo)_*++kv$iTx(-G-)4ik&@1!g9PN6!gcy~E)>}2;5p3er20v;)luG( z7dTj_v@|FNkrHP&7Z^5E%iVl$kXIh^W~vrQf3VXqE@J5fuCD}3mVo&SV15_uG{Nq1 z`RyQ>hhU8a>zV=SK7|y}a|%d}E)v(l#u3svL(H&?`A`1nD30C~?3}+d@L+$NSJZfvYg!e~1H^lf!FUKAco;o+q!3ox zC~(?T;Iv8YY;n;h#m>X^>oE+6S|x~k;JJe3ZV->EI4(62L46}E{;Sw>cr%NJAk2%@g~+sZu}sNA%VkC%ZUBs$x^|5-oZ{J4Wws|M8Mo5&p^bFnk6c*xXtiv<>75Qw5OM40~?cw246Y43r zv`10%aGi++lCDd4%Dz|)ukzgoluJ|MPb%TOv=+Z=_BrO}d8Hsoc8BOO zDP>+^S!1am zk0%~;9}1E?sn(jdayo|rfca6*1XUDrO{aGDeEg< zO|k|J7rqWkzZlEZ_2E?YrguI>9qyVR3D93QkJbdnZ4{P0(tNqmiu1YoO=Rlb$`o~w lORM_=0v)b|V@|ppE_#gmj5 diff --git a/out/production/clients/org/apache/kafka/common/metrics/Quota.class b/out/production/clients/org/apache/kafka/common/metrics/Quota.class deleted file mode 100644 index d6510efa609c020f8f384959eb4a7cad5f66d2d7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 959 zcma)3+iDY06kTU#avP^6rdltxwN-Nk9SVI@L^Svm6=@r2-%h5-WSYrLn9S5q@dm#7 zBB)>~DEI+>l(_fF5VTb2%f8L5wb$C`*YEE?0c@gSBagKbEUa_5S->ru^6dg@0=aZD zQL%v7qq-lZ;Xc(Jf&6_x^pgiPY)!xN;)Bp>jv^Gn{D0_|p} zb!MEc^*@>kFOH*F%}DGZdZprQ?)cLkwwbkNvqf$xl22QDo|+^xD;90$I~>X&C1(p; ziS$NKs@Z$)YBsV6?$wzcxR2l7xlIET7}5^M(r6a{K$a?LPDpu!YxAj5(Cynmp) zL`&;ogWf1l4``sutW;Hb1?4;hLn$s$w+oa01j_AWh<4)$MtkK5=2sdSI8U*vKQB|f zP6R77TSb}XYXl@1D!4$IE2bB^;)N#lRV-#}HjyJf!2SViw{9IH_l3II1{MUYYgU#; zf8nB*TEeCOFqo$>)c#>u(hLHZaV68>QfBsC`y0$pC)T(A0_QSQf-GE}gN-@Z_WU5~F-=gn{7G$!Um#CmFf<*T|S;u6|?o8N+_*fZ( zw&<|k#LoWXh2Zr~kw)@d+3o8y1{_#n74z^2NB=ZI*q)`lvVZw?i!C^Od_E-BP(qn= zg^>f7E3zgyE-=$&&f#3y+kXYUOna&vy(zoF%1zW5x4*lvh&m(gM*~aKSxrU`mcIa< Chkj82 diff --git a/out/production/clients/org/apache/kafka/common/metrics/Sensor.class b/out/production/clients/org/apache/kafka/common/metrics/Sensor.class deleted file mode 100644 index 5c2083c5bee7fb0b0dbff55dc20ea0131445253b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6638 zcmcIpd3+T075~m&vup-}!JwLD;4Efb85Mk2~e@K^-58U;=hU zxcG1lK7zY++^yrD2qs}y7`tWssJ#4G828HJz6k2DM|SS7!N>6lnLSX0PvTSZ__U7C z)F6fjW&0r=pN(J|9**DnWm&FqiG`hU^3$wdw@FYGj zv!`nCG`=8@FY0(Ef`xci9$ykRUyk58d`0g2s*bPe__~g7MDR^~OGDVPdhBe@*`Z{Mb6CF84cU3O2}%;g>4xMalZ zWz}yW-}dDqm#2n?Gme?bc>;&4zTy>1X3|^j9u2Kk?{`miD0)f}$k~G2^ij`==GnwX zQjt2e%ud_6M?K)FRkF_EHW{-S#(0OP%&62AD$L$Ql{+%sRJnGIownBH`?pxm zMsrI_xlbl(rZ$<5Eo*Nxkn6SS2i5g0xz6~moSE!fYYr$53~3$TW`OHhspC63zN?{z zE}iTVkQi9qMR2n9)87A+JfuhVyhm(%MDP6Z0 zw_G+tnqxRDav$F~`*4HPaU3g|ak@4583XDMxDR^F*o4rZL4I3_Y6mJzDWJW`dSf7qKN`3Kf70=11AoC^H7u^iM`g@Sb|z&CUM1iRyaRs|0RFC_ zLFt~3Z7Wj6GFufEDlN;xz(4R$q5hYc=8en*V$GWj{2TvaW-#zy{LjEkIKY}yO`xNt z88HT>+JTn^_Jb_8242Cda_J4j9CZDbE3725@zl|7D`y7{@zIFfL?g{-F>UHNWH6y> ztVPuism@hH^VlfekUxsZ<<_PcMXLGT*z07j@|v1*#B}B?$INBuNmZ|}1f2D&y0k?w zBqM8>HrfGK-V3CsWN(wsEOkz_2z(t zFKc}zf%i>N$$41(Gy8>e8Fy<0@8VIGPNUyWrR*#lnshg{U`MlB!-TS6lD*ibWbLb& zK>3zkoWI8vwzKl;l2Y?WuPSA4tEd2Oqcn%Pel(bmc#I!FV>nXmmZZ&MYMxQ;3PsPV zVJ0QLx;__Rdq*S8Jj!+DSR?4ejC2I^k2F$ni*#8ET#7h_w;k4VdKq0TYM0!8^04w) z3$n{a~dNdTd6uT?I0u(j4KIG0;}H z(5TeqgrYt}Cb-|@mg7prrm4EGR#YLyqrMy(9U0c-q!bkf&cuAWKhte*RbN|f&`kA! z5-HrzZO^jh4b0{52JQS6Mdz;?>_s@Mlb%D3>?YL6t^pd(<=lr=e9u;*v|i0M8wy3o zo`R#nf9vr&Qk)NH_3-lsU_*QmTD)m5eDUT-;eQf=y$Hrzo<+p}I6`|6XxR%rz7OHe zgBZ6Lk@B=A9wn_2j|TT*Jhvy1qIM7yA0k&jHt-ooL_Gs&!ZE6S!c*VIgO_oDa8}VjY+0cs(@k9U{q*-uV>$#k_)j7N3L&kCVAv)Y%jt!f`&_iJALQw|O5XQPbn= z>I;}0FW>|Ybtm#sKy(l%?f0KPCH@HXCw60!j8o<{5tE%hZO@?>1(|?C<|L)*V&b!e zm@MV9jruL6726dOFS~v1(7RA$W7g{(UO=Jba423|z{#SFdpe>- zTtx$PR&WSaaIifN=aNPct&Pv30ON-+MMD9njAXTeKm2tmR+AO$+LHFQZ1MWYOvFmL zOtlHbGEAnu@y%_)r~HdT^&v8^8N#U=c4Ml{>qA30&Bq_@Vq6)s8ivpi!p;zF@_f8u z5L2VUr}vjfaOVZDs?!FjV9DCf=~PrUFDWeU>I&g7kEve%Qb(%&!T z+siSTR!MMNNNcsxTB~TI3t%Z*_0S&T?+Zv7rv1grAx7&IH4E}hZ%N}`Ph;8ZN18W@ zo#{WXAgm)pOE>cugnjE+AglM8}VY3TQ5%MFl|X z{+)iJbgS635|w_`QuhA@{u9%7z`buB}fui*}? z=463J7hKpD*oSGGnLG5 zre>O>ZKOGFqmH*LO(Y|N(Ne4pVSc5(lU!tP2d<~oVwf8|!-RQG;%L1Hp(zy+O~HO=mHB=~nT1>eK}AJ|Q3@5N%=#}EBIEJgR1 zz?YI+6nT9~U@C^L&Hn<`UkWQpBbsP{j8}LMNC@o@3DKC}KXGf`AXBkW(=- zyNT$WPqy=jjzWwoMIOcSkp92#Yy88Wj zz5px`D=38lA#z4rcP7jyJL@u*tLTg2Qqodb+q$+5*T%bC-+2Gy$~I}9E!W%WkuXf` y-D$JQD?^AAFMd|YLkQ!4W1UMFbv9ShWPCUkM*E0B2-=STg@hi$uA$#%fanVdVl-0# diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Avg.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Avg.class deleted file mode 100644 index f3f3baef1d1e21f8b65763f156486fa5db4e24de..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1615 zcmbtUO-~b16g{tB>72a9P8=K%(HdPH;^in#pbn#8$i=Ti{5^aqU};>b70KE4M5343s=eR<>l_ zQTw(t7Cdl#fn3R}mrYrd)&twzllS+eX?fME=bBYJs5_Q#`hg65^G36rXU0ZtM+SsU zY|3g)VH<0%Yu8sQ()Vq`RWm)}wc>X+2|m%X^Iao1LPjf|d*3PNSJzd|3BS!_xodQ* zjz8ojiWOoC%ydQR;2;oh%1VPWh*fsM^&Zk%7tko-HoG8D%#?OzQ<{yyshB0l5B{;r z^(=t}g%u9}YlLnUOi%woWU>joKvuX~Uc(Ip20I)7sP?1zr30~ru66%ay=xqc!)&9q zw;^lo+0r+ivMYl|oeDfTsK9^a>dLdDKcRZbO|Mb6?3<1n*F&7+8PyR3Iu;BJVOYZz z1BAtpz)?rzbFJXN%ENruOPJt64QNeAzsmn zz~D2<6&U63>>Q&vJQYEO0X;@hVodY*YNi4qHoJr_zd&YTlS;5<1^Qm`rH zkii1?c}$1w@9^9h-<~GtBt@KKRcE=9&CiO8JaI0psQ9y@VpvC%_}4KWikd)yxkf46 rc}BOyRgsg7oT4X%$XDpmnD~NOl2BdA@;@C~ISg=B9gLIEQ5fF=%pHqJ diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Count.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Count.class deleted file mode 100644 index 5421b5ab2c38e269c7ef568846500e5a7106b2c9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1532 zcmbtUOHUI~6#j18=|DRaM2ki7NzpzmBZ?@Nu&~4gQgl&6;_`MF%0N5Qbfy?xxbl~{ zacN9622G4>|Aha;#K$+&N^I4%#wNXU&b{Y+=R1#kKYe@m0l*?|#-JiQ3I(}1^0?wJ z1r_rubQM$ozD8)=c!N+s%vr*n3t$#$Ow%rmFCE8;06LB&rh=exzdNMqpH4Hp!t>LZ z;h9hPJ%f+%TSzP`;UzVx<_{4Tc#fHrl259Kh=lMF3aJ>&$}?5G`2IPiyre@U$ZIMf zFiZF}j$w{#8aZ4F$gQ$QQR-)KHpr-OM6*Ep=L4FBBbtSPT$o5XRyRf5i;U8#lW#vu i`!si0$*+ip#TV|(Q4cX%2$usXLk_+LFE1lbK>Gm<<$s6( diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Histogram$BinScheme.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Histogram$BinScheme.class deleted file mode 100644 index b4d3db1cbe41d84ba5ee225f9571d4683e5f18a7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 312 zcmb79yG{c!5S&deXYL3g`~yYO;D&}4BE(e``GSq*98S(QvJD?c!3XeBh)skLZJM2Z z&Cbr(_s1uITU;qj6)p+Y#)V86FP0r)QsTV}(p;<#rP`+Ge;eI$MQGRYEgAdh21}Sd zxm;qO*x&5fW3k5_&RdQZHYk6V9MpIz0)w8=! zdfjWM9f87WFZ9wgEb7fhW8xY|r=qqaFf#3h&gEXva^kD9<+D?nj%?{)ld-4H{ic?7 zJOZztiozt7VS3IBXY7s>aK79OLnof{Ws*1?nVODb*OGIR|F*jFL04LK6a-Od1x^}! zc484AO{|Mvl16SUgPnh2o)Q?ENu}MrB2@{v=MY0Y;ePX}wt=1B!}vW(TJl(9yIwOCiiC@KQ0enU5+9&I#0QD^_# znLm)Pv}PSU6*%#)uq!^^z>1m%^Ifc~PyZ%A!!H)Pr&Qni$1T{-2Rs5YhK7oAiEp@~ zjLI^-$};U@);PBw8yGk8^cFe_%m>Vm5LNvRhCU;Aqmus!?W`g;NX(bQF;Zqq}QQjk3?tr1pRE; z=n~+vxN(eU;%dC1Ay*vY>Rs5~Cos>RQe^H{yPm&F_TMn)?LLRdaB)5 z5?8XAqty>Fh?t%vWYAl|!dzxyqE40~vy&_aN62!5NI8Ctt<1{wVg5jFA=)bDZot3WS;B5`@VUhKvD5q8&z ztI}4gq9Wh|ir}KEcmSjbi3kJ|+;hbv@CZCX)&E($PHs^x`JCCAIWxbRIlJ%wdHoi^ z37pSi2WGP<;+R~%k;`!%C$!t2Xy;Emn{0rNZ-?+*4mo_Uz5CcxYV#j@LJlWU(=n&x zw2mJXs13)D6ex8CX;0juw4@*tS;DmyXr~?DiO*1B)R!u&^j-{_)Q>DXzJ0mtHSF+; zXh_fEa=;_2B6Q?)U#Z4zhp`iH5uS7W6{~G~^v>6P-wqdD5k)o~7M6psWr{UHv)K{X zI>NL9&kKChv*XaQA~T9b9GMrLC=Obo@D9rS+ak^>(EYB*42+gGDlA6776wv5W}{wV zO+#HjsjA5m8w!S3VqtYIiM2j!CJy@EOY*oj@MTMKD?vB3>-sSse>`oWr6H7B>Gso2OuOi|wbzb=x9t?Q?@} zn{IZS%wh1_j2p{aDONaAzcnZIg)QwR5w|(3=tjuws<9+TW_~6FCizb6fPyikDRCye z^F=w9WanhKtF%v`#B~p!I?$*A_Euj)sg^d8sv4U}SB+9hZMHM+daw!gIUQ2i%XKF* zTz}?~+sNV<6!0s<{DythmYI}+upd+0rF4M)N+L;w6y?bz4OIxP;0;y*ddI_{d z`Fj#D-*e~(RB#AiQzNP4KJ{cto>rSrp^rh^D6N;)N36~mhJw05i`vB5u0F#HC&tHm!~ zBL8lhdDD;oMy^NpOg^i@k7F1nR)uUM)+>Bz-Ipcz$R-#Hc);}`M)4~Wk)9Gn+<7APr>9b)qe1P4+`Fr+mkG&E^=l#U%III(AvvCXEc zsH$wb0I@^rq6;?NRHBt2kUsbUmHGqvBPw-Sp_cZ$Gmf1os8Uo*zIX1q=brDJ?|gUW zm)AeM1aKV107AG}j{~?=hi**B;XOILFKsCemwosCq$hvUa7DwU@7^D-LBo_ER|9Co z2YyUzxE6qh4+EINtTcWkhdDp42YB>`dhc^UT5ihWmWGct+!hF?a%R@B&*#kP%$;l@ zE1*x9X4W3d8^vO_sNoZVJ!6(xEE#60>ZQRDsH-mdoUR2(q%3+t7+W?LmW^1( zDika;R>+p@T&5Ufuu?I0DOW66i?&fXSPhgAQ18$zpxj6bcr#YnOYzy+jAhRg zF+47Ty?2d#nK*LEibRh82?76VncU`N2A*QMK%*?(F$y$HmyFEvWn-n1j?bZl38~?J zm!dq|$CF`+;4fLzq}W^(XbJbHmW;bbEN_^LG1tnRL|UL@$Hw_`EfWhtl5)!%V1C2 zYZf=TN)Al%Q;Bki9P|wyi7#vmI+;zGv`68lORpM4!R?y>s|5$2NF5c0ffZn$U$- zf_0(?gU+541~J6BVn7F;Hv~s;v;uaW=RMr(jXWj5Q+N+;z~{luxSqZmnR^N~m?3Bl zuhH?|hh~=5!!Jq=r?-mJi(@QH;vC~A$}Q?QX6Pf%6fxAsM_)l}f&WQ;^3cOms%OaC z>XrB})%nLb+$SGfz43Ii@4oc5+SU7yzDAJG^xHr{!IL$w_~xHqTcAH-Ob;Xz5q_RP zdh1AT1L+Nt+-Bx+fI9;u_cX^O*D2;Kvp&ws+c1O^{7Vj}s`*dhq?3OT=Wq(cjNgni zJU`4D67+0<+VC2FMG>)AJSs54ZMV!AXF(UIobULcIJwCNgPcoT-F2v-@q>g>T4du_ zi6lGL>lx{C=y;4KId|1xT@$SLcSiuS7jDIF?}zw#K1oRpVK1}oVV0XcQCOd>PGWC% zN9JD#>2?y4eVL4g_NYH2H0Q{q+v&p# zG*7=k%Q{+TqfMvR(KZ_mwO5@y-SHRLTXPtY4r>Z4`8FErSVw2|kOZj!#a;VnPOx=! zNfO<1kCHl&=>REfCut{``v|k2!Fk-GErW3^;R0z);tO2Fm$>9mv5)6<%sNy^dLLA% x4`7Wa$5_22MThb?bPeO=Wrl6IK%2ryZ*lHqgdSe3UQ)M@^M1|;IKS-?_~6!`C-&0ZikjhKtc}pD6R!flj2` z(GH^nmyqeiWn2k{X$@C3%xJhK5M8hx+g}t2C6a3b;U#y&66ns`j&-M2Ubm|E~GngKq~K6i-xR7bIUSzm-lw!8pfQE(tb)L;M1EWDN~7l6 zB_nTp{y#>!k|eOO`7IN+9xT(`(~L^*Ww|FXaJbYDDnH7dZ?afu8R=j1wa7+@vvB54 zK~@^w)V^vL9qHGqjQLS>%>U)9CC@_@QTggvb!%1Ax@`vy8e`|CgBmbvsuU&Z6pfr) z<5ke1V^+s03~QLvF^+i+Sse?wu45551ZIzRIeu>{f#lIl$xa}1G~Hj^@U|y%!3F60 z8B;;}TkOb0E*CUS{otVQXyDlonGe1dY^6}q@&)=6%gK6QsHEIo-?mIYi;Q}Gf>$U6 z-f+Y@c5yYM)ii~^dbI+Ythfph|c3Or7CL&|MDC?Kf?r>8MO3qk9%JPSy_qx%@= z3^~t5LNi*wmfD9V@B|}+kyO97k60Vtqn&ifW-jtr6HmW=9E?1pQCLMcph%2z9AhNo zNN`SLLJ?dqGG)0_Y&J#SZXDiUZK>Y-Fn9nbW bg~VrSCAn_HB)^oBrqHi61pgd5(FgrIwMmXD diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Min.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Min.class deleted file mode 100644 index c983cd60cd7b9c710f8e4319e33f979c9b6a6ec9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1616 zcmbtVO-~b16g{u)bf8QD5fHyX1+k^dSVaVisL+@|iY_qhAHz@v+L@*^#gBz<{3UK& z8WWA;LS6eG{0;6+6wjMh6x0;sBz^C`JNKS@?#G*c{Py-8fHBPJFyoZeBN9mBVk_E^ zP2dtP#~***37<9^UQx@qj;lH*bW94wrybV`W(1#Kc?CM>) zQlh4-;90V?EUS(>HWP9t=-}GP(|C9WT1L7Nd5zc$>LNJAHNhP?3LVGLMaiHC?Qj^~++*lLUjxH} zMk^qh$-IP?euMZcX#0Ws2`~@>jnSOO4Im{QWe5eH#BrPm!zNW5NlobnaEcy+DV*Uu zMfyF%w{XvPL!Z>5<9d&t*+#3tQ}p*~nI3%`i3mQRjdaLnQhTC{XFH$zwC8k+h1vCn zBnG(-k;qx3xQ}8uBshaO;fFCuoyse_OEpW&;gD)}mufa77$wqqoFi$CSU;+z?9;R# hp@#MRik7JOLS2UE2<;=d5K5ys5?ToUS0vF3<2%c8m7@Rv diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Percentile.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Percentile.class deleted file mode 100644 index 48ea281b6fa4c8b8db825329c4731d32ca815740..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 684 zcma)3%}T>S5dJo4Q(L30@!zUgP)RGeBHk1)ilduj*Ch-)WiXDsz>AL=Z2Lb9r{y|Ho`kmq&sV*` ztquH!8(oWrB($obBf51Fd2&B)rs9^@XV_1q_m#mu@5dtU^UE)TWOv&xC>rrQ2m1 zCU+GMXuLBuUH+SCw=M34i5yl;7|58IK$c-I>5nBDVkk%=-W5TUUo_gT6El!<9VL2^ zG(D3BOA1hBNe#+6$QC9knzXy2eTs6v@&Z=b8A7YrL+DS`(lABQQpZJ#s{~)eG}dVW z8w4a6eFe(0k}^zHQb?oPz|2^NQ9t`7DG) zLE?cA;G+=7OT|SzM8T40c4p_Mb7sGN|MC^U6SNoL;C3dKOPI%<;`&?q`IT@Bbq9AH z+;ebWz&Vyt{U}HUL{FfYj>4q=NMNOJ;-N2(WH3_xq1-=|eqhF9qy4c;<1k44B$a96 zZ>u;^It?S0tTO$1sKazk;9288@_Wr)fkM~pDS>i7)aun_JW%l)If&?~_Dvw8T^Wa& zed;V^^9r~dTC2Dl$s|!UpU&RA-r@MpSjl7(v)SY6^KV%kx<>DZ!`01Z23}wdf%eq? zeSE4ALzCZ4Hr##E?k^-$?h2! zi)gucfQMvpW)kZiw#{1kk<>%~_258dyWveMv}KX|$y8@r@IR{~FlT*axzX&~_7m=u z-w%_NyyP3%X3TL@Cq!jsR&ILD9%_6E^AK1-u2RWh0;q`@pG1!TBFen2@YF>f1?sZb zI)U&SC&+o9k^f)=&ruc`2&|z>dC?}9aUK_FG0B!BmAtuhT5y|LnmaYH2t7!^7q_6 zaLgXx?{U-i`+Z-SeJ2WBJ2b9Xw&4{s+@*TtY;zG@|#bl1A&4`>ix(Ht;j6GO2sJ7*v#Cv@7tQxr8kW7xdf zbQze%a|3sOaGASxKofM&IozVBg!1moF@|@U?6P3jFi-VA^K>3juc*%mq z9b!H8f*xaiSD^Yb2;69~uj1H^1s*rY#cqtix=XDUf#PDgPTZ+wFK}P&_-k(TR;~5v zIk_A|?RHyWH}7Hnjbv+FW|U>@wI${RlDq_tlC_jGah80it4nO`b@Xso z;Ql{h@ECjH4(s-uWfQ1nGR(bKyQ&HklT}QhqAgp%9<$$d7*B{SpX7_oFgV;r64d!1 zAIzY6*L(c>8V8MBJ}@5U!#JW;0`vJ|voa{Y)O?nHk_-RSCc5CAvD4>{$s8H#y!|9^ zL*5m-__PWzbT7~|#nmcGyb^(FYqGwNg7qE6)!BWNzQXuSw}BeRNd^HSp|%P00VlYs zr@m>fPjN*6oW>b~YU?b{!Kh8(JT45=*qrrmCF=`}Pc#O=< zXpXPRu*6wfv*alL@XCf;sSG*dBMCI%BXLFZ(Ezwg-bLEiQVKIT&UuEOV|=S+{a4W> Pe-oGJ*N=pS+o=8qDu%*? diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Percentiles.class deleted file mode 100644 index 7014548867bd8056d6070d9d0572258427d9fdff..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6038 zcmc&&Yj_-08Gg_1W@dLXO+#s#q@^~{Jh0i6ByFLDY)RWBtxZVNKnhqY>SQzBF5O#q zXWMO6&{{7jDqavlMJ-ANF9li>N~GXjQM}(#!Ta@xKY388?>Up~ZW6+7<%emXnRCv3 z^IhKWJ>U6uo_yia(*Q2PKMbtHTMb->w*@$QyS%(3fG}>+@h$@z-Yr+&BUj$5<2FCu zXTZSwb$r0U0^F|SLk1S&4nICD?{~_}UGnk~9UnDNhmRS!8z0y433>mdj-v*a;vNGH z_*6AMjeDi}W75QDg)7%ja}_K7f_@f{x?b&Ck(S{z;JsovtlZ))Nosrm~STZW0kE&@_K=cb#r|nF{PCBu;&6Tyg22+`6yOp*g zd+heT*6zJldnA=eq>}9k+sVWt+4ihsIobB%!dqd1z-TNv!WR;@j?Oiu;E73wR5 z(Oh_$inaBzWX$On$OrrTD%4c;EZbZfc=&$^t>sz)UvFxRur3^oCGDYHV${y;utp`& ziw08>D}Jq&iOKoYqR-hAlQ^EevkbR78#`q8CwmS$G8f*2m1mCnB)%$RDtjBxUh_;C z1P&6+{}-CxL@X(=V8pQ^d$(C>g_@%CV>&hpR4H~5+hEJ!jn;mvJx5};50c}A$D58g z0!ERr0&C714eIeEh;Hp%{Yz#h^k8~AsSAg!ggv&+wz9d5oBQEeZ?7-wM{g>TPUVtg zjNECSr7zqa*6}#=JQ9m0Ehm>DyRM#tz{|Elc=ucJ9F?yPmi2d=+U~unf5Wg?oj06{B=CqTCi} zD{Ep#n50-Tb$~hs1pNDQma2+GEyBdnhM);=HkaUvdZ^dq#kil8n%BO!`th-dVmrpnIh!lr5Fc4qKNr*Os!oHJ)ln)|AWP3SZ6H4F<} zpek|PN=DoJ<8eD`#W!W5xdfYr&H0F(c48@(^i}eFv@%WZQPsqgjK@_|#@?5UW$bJd z%YKuSY8thhI@&tgq=~N!Y&cv0T^l9dt5ey^6jr3-ZwS;EJsx4#l8pA`V(~FMBOiZL zpzfOFUT*Z{w6hLNeOHr-Z{bOK`L>Dg;3gOvXAWk0GhyyE?vS;(B~vV12m= zUaIVy_yK;%Vq8wg+>NkWy741<`7sX;6Fukjm! z&dO}eWvGc~u#dzn0@|0#vE4E8EPiX^cldq9G%VqEbAghsMH7F(A5HuTe>U+K{ME!5 zXkjz{X5#OX`ftQd0;?;Fp~OtZ@07$%CYQ5Qnp1)9a^iE-SK5{eTwa-i0%XMwS)is! zQOQm=PQ`_O$Ie(zD#I2w*e6Sszjyo4$c{}zI|SO#(k2V6Y#OB&ZLoi6^QNo0i>6K# zRT$iD7M224l$3XGP>EaVw4EFixcJ4y-7~I4!_-FQfi**Z`{<2!#0gI|g)b`acT%ns z0*j}6rxHPG^acB+M;(>LUgVJqg=e!b$u>~nLTaB+g?m8A_Kx;0tMn);n#zV1bR+Y> zSz=*cFA`Hf)ruXfN{rVUV;*LtZ5r3>30gXriQ3yov#Gf4*zS(t!MbLCSBhrjFT?3p zQ9|sG6}Y!#Iv1*wpXMad(f3%{Av;gbk|?zAORCAt+)eq%uHW9z^QPBr#C-C1l6sFn zV7*bBI_j9)5;7Ivvmy~Yn{Dps=n&XnK9;$gwXpA8NXWtYl0S`ytE2E9V`W3i4w7j4 zf-SO9)puu72_9N5CrK@*QqKxni&Y>zP@zX9G;3b17@=kCNGdai^Z8ZbGVqvy7mH+% z!EX;!oC>Q}HiAFJ6rs==WIAYD^8(Fpi6JmPvLxl zJF#4jE)aMEE4%#lTBH9Ynr|8H)Q+8bre3SQ{VwQ>apCdmPQAf*Tc_WIVcOgf5P z-gQQu;i?P@97UfT*BPhK;-R)sS9N3cBWNARsxGr$Z#3%y9iyFky+>;1RijfE8V z*9A`DVh@fVF^J{KBLQBIE=JpNy6tjmfGh%0)QDh!D22FBI?0g@K-)F!J`ZL6dP=|^g`jN_7GWvo&5qO`k* z-Ue?;k|qzr_WNTmS$7 diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Rate$1.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Rate$1.class deleted file mode 100644 index 5bc44c8841258e27cbe21c5ad6eb7904fa923d45..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 978 zcma))YflqF6o%iSmu<@e6)zx)AS}=#vR(j9G$vR|Y;3oNEf~WWr_+%xYXM;k z`*9%TwxbRecX%XMRU?$L?g`fq9q*teqrTtMu&m*RjuqV0F^VxAt1xxk!fl2rb;NYJ zcVJq+9gF_9Bi&v-2>d|D9Tj$0$(oXNB^yfa!DLwRg9DQvbJ3UP5r1{WP2mp)zGn_( z6gVO@!-z+rsTwn=lHTw<8Pr@JhB9O*r-j?yLn-KG>1x z#;VXn~!dy+kO%L~>{n Sm&vo3BCXIYQA(X0<-&J-$=Ndi diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Rate$SampledTotal.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Rate$SampledTotal.class deleted file mode 100644 index 125870e09c50b7e1b0a5cf863e7e3e6aaba1fb6a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1635 zcmbtU+fEZv6kVscfwmMxi>P=@u!nhto7UOEr zHQdl}Q^PHR$h>7+-hx0dmEI5tEjmS0pg(Wf=Dm8QVAj@Up-jnO-Z5l(L)I*{Zwf=+ zw&e=M*JPzyHjC?yC(GO}+qPL-EKAold78^RwUREY(%3fjUAeU@b;GGt99yrLUd=LG z-SwpB>JOx6jvhvuBfEOFNHG_U1A$7aOSmS|8nK(Ga~&;LeL{-}S(D`{d2i{CO~8}ZV~KszdX#CD#LlGmv7P-{y(t?su)zRhyJW6*z1-2$7S zk2N%QS7o)?b-im=$(CNd#@L^?$NpcwI`V8Oj-R*anp3YC=3Pr&l3vxrRAot1ou0-SS_q8bEYB3-PjueHIh)9ZgHu{k8~cEuz)Oq_hsToI z2ZRFnf-tEVx$p~3y!!QRIQ*IhAwRAmpMXL5B4aQy$~l8ApWFg-6rtoC&ifH%4`^m_ zff-bJ${x_n;u6UP`On}o35RL9pC_e%h5A=1VJ3ed5)|L5b&dM~QvA~XDuaH`gWN|j Mfds3fayf|jZ`nAa%m4rY diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Rate.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Rate.class deleted file mode 100644 index e19ca5339a2150afcecb1bf8e827ac8f88a2b907..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2865 zcmbVOTUQfj6x|cTOcDpM)KcpeZPXByR7DZ-0s^){KxzO*TiYRwFp|uq$pq|8+tR1L z^rbKT3w^P5X;`bS)xP(S^r?TLtNVO2hS=JIg*D%8&bQA#*U4{x{`?DoUVI(LI3^NE zV$#6X1e!1 zmOl9HAk_f+;2-0`V(7~1pgI})?)#D~49pwIC$JY&F)YYz%Wy}A#RT?X$$(?vu7PC( z1r27_b?x$S!Kzg33b#3{SSr{H)1GhfZM5n-zJ~qdcdZpGQ}vxfChxiVYPoE?erDP! z+A|awpkT$fD0p(*D=%iOl9gYwGt1VUWsCB~qUUCcwqJJgm5j($GNG0nH@vK$bBJ<5 zLsP12mRQ5yf~}!#+;Q#6YH{8!Ph0Z^p0tm9d8;sMl^xlKhf#mYA@QSI)O^kIZK^z& z+QcLwayqp|&_;9Q;>>2~{}VSR;h40F^u>-;S71TGau+i>I@etkSC;L(S6*NwHs?N} zLNe^Rcbvt6kuf2zGrHVq=-HIvbGJ1#uULgDd1)g=@tGJC42m|U!6@2RrCR2z6E8_= ztczGpuD41MyI_?nHtjOVaK$b&*PRm*w%vLlHTm#Fth z4jU+%aM5j|2d6ajZ7p5pda|)Hx{chSGk>%B4;~IJir$+d}-n<^b6HB4e70$t6}?wuB-ER?L1xHv)TO# z6>F(ZVa}@D>_SDg1Lfc28uluc|JG(0{M_j9)ya_@gYT*;Xy6$5gUMI0-9>*%HIpnu zU#ZSlRCQ?RNM*AxteS1UH}0*n4O$gu%kiy|ZYX+c^%#Z&;V>d-mvZB` zu3c)6bByP#M~-iCcbXEKoTeW^TSw$4t|K_kF~J!`GH?(lxH1DdoaC1q3gc9WF^Z^? zZAo`OMbq3mqU&h>JJ2Bl99BRbfp7=}v?zk?f24IfgdF0J0Z|_pTHP;z>!Ka$2Do?~ zF0Skh;QB(iIf6A2t}VR|J$>{EjCA)C#5Op#(**EhlrD;sM=Kp9Mt26+>fmi5S3)T@ z;=D8H57mex#F@L(j}faO-W1L?B$^j$*j9u2BVPyp)s^Up2H)TxwzvLy9O{Y^dcvb;Ni-skC&~ z(ApnOcglwAw#V3Ruj7?|eP?v1?metq(5jqsj@r*-AMu4jFPa6X&LCN5ki-b}MkxJ; z;u7T@9iwEW^OVg!u19jH@I~lpnQoxCiqu_-vcX diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/SampledStat$Sample.class b/out/production/clients/org/apache/kafka/common/metrics/stats/SampledStat$Sample.class deleted file mode 100644 index 1029ebf0ee6e19c620452297449e3cede828c2c9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1134 zcmbV~O-~b16o%j18K!hP6{L!S9~4C?6*_{L7%|4A#6$wRsKLbDWw?}~?M$7S7C#by z1v}llabqHyxbO$~qm1vJHi&USbQb5{bLO1)IrrTAV;0v5-Yo!0`h=@nvgMwz^6{tP0qwr-Gywc7ue0HRiNroILY`?Qnl178vUF zbFF*2_C5pU)wQ`z0kalvQ)6V^57gssdrL)6ZR3j@^jNe@kqlPOx(%V&TQ|>gS>xJ!h7`SbfM7|fhaUzr0?U!%Y>CE)^D*{Fk z?z68*#foPz%V~SIAJ;<7N=REOubl|kNI|tQ*zp@HbI;jp;0l8{4sKy7i)Dd3 z|2r;XHOg(tpy58*+EX4!W|Z}X&Yz@td_ufIJ)Yn1gc^K2`SdmQqXLU3DgJe-nC8RH z@m-tX(?p&-l2Axjjw4+;5v~URjN&xy7`-O_A+8gZuMm~lLu4wsLl~9mLzvnf`oy0Z z6lo`6(RPS zXJr0Bju8gDWQVUP_L?2PNm1i$iKR!VB4Gsz3tYsd0bq}{y1Fz)SyL^1fpzd6BXkVz zvj^{xH9x>KJ~GEh4`{Ixczf(Hr6WQKms0}uFqIy4nz-UwJJ-gEP*G%%HFxg4bI&>7ced-_|M~e>07q~? zga|GK@iwMpm^LvJf{u%__#njLQV3yOmf5TfSAtw#4dEKDx8Os}$=(f_eI(~THt~tP zKb7s9GTf4Hwg{y~C}SdP!Z9(g!I-u4rJSR|nk*C?Z!Bk*%TAd~x8PP>J9o{_RrysL z*I;I;pQ=jY0t?Ny)4C}o61isE@Q>Q zyt^Z~Evz76;V$lJIMlRax1D;cM%)bg>?7%Agv8T}I0&57Y!uUxEMmcsNz;QoC_8>?VZB(%XQS_^C zK91dpVSsO$2@JL^3}UEm>^qqJ0f|_b{upM`h!|b^z+-Ir5>{-WOJ74UX~ZJNKvcem zBF5Y!w1n(cvwa2!z>6Ac^*o{7^wt@6|^x=KAKF>=usqb3L`kn@dcee zo)VZk_k#5q0W~O|YfwC=i1`#9KeVG1VP~i_86{5%b?Z}v=ZMqavxcpL$fq1@C?}0> z2xe=;LaWj%!5krmTIS5sEjp#cu>q3XChjq0w*7~O%8>H*dD@*&YI~Laa6@gl0X&S; zr0=6@w!FH5J5(=t2ODnw3R`6!>!GV_E>=rM;t4u^_zmH82e&EN3$QT7T%6uOYp`K| zuue;&Pj_|G@R^4BKWSc4Exn0({|nnQ`Z2vzC4kC_hv*l{P9{SQTQU#{JVMtHpD(^) zmIP)V=-T)H{xP%;x{vV%E)n!)f}SPlE5vgZDfzR_-wm#BVNQ{eL{Fgu@2Zp^C(UW8o`JuMex8N=V$Bnyi0Y2 zG%07;F|w>`hm_|@CPVX!is~5#LaJSgz+^ETS5zf96Ah{pja-_j=aR)_eP7kNkZyth aIofsYAQ_Y0q~e%j`+(|*VRq1NSpNaip7}=r diff --git a/out/production/clients/org/apache/kafka/common/metrics/stats/Total.class b/out/production/clients/org/apache/kafka/common/metrics/stats/Total.class deleted file mode 100644 index b18a94537d022333b1cebaca7127b0ed52f331e6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 888 zcmb7?U2D`p7=_QIo5W_A_6vok5gSnNTD;k%7t2;JS}I=7PSb7LBqK?-{wxs` z!3%$YKT13^8x{ni>_w7!XU=)vkIC;pKYsyugPi~^Y|LX0Pkn6q*z&P0;MTg9nSkgC zct>fG)^7#o!u@lBR##sn0*iyRNZyU}VN#ySVaA=6fmSj*mt|_=X0ugarIoA@Dl96LeyM zU0qzJqr+a`U_50bfrHuHY`M^%IfcG}pC_^!mn`Phj5YN4ImF;pk4u#tr)E-tZS&w- z-pUT{VZH+ozQBtaW+bp=>~SWGQT%>*ov51a-32(YJ(%k`<+!oSTW^75=Hr+e_(zzS zCGj0FI-)}G8$>@KqJ{5pzA|#q=6iu>z#1Lv)Ni1}s54RHK2Kpuiw)_RYNMZ!_WncK zBx#GJZA)5Z7OESPSuL?X-UXJi(l9@;a*wKZ^zj>jR$y0 zkl8b6ldg?NShXW~j3-T!E>DcpjhN#L7nANiYYi=W$9CpU(e4!OTA)Minsu)K1vB%n AHvj+t diff --git a/out/production/clients/org/apache/kafka/common/network/ByteBufferReceive.class b/out/production/clients/org/apache/kafka/common/network/ByteBufferReceive.class deleted file mode 100644 index 7e1f4facf56562754b0a86f65907114e91f9a849..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1322 zcmb7DU2hCg7=F&ov{Tz%YE@a3YK#41(E6-2R7fp z;UErm!oy(~M_e3rF(lBw6fMWT5-3j0%N6N0WjO0iC2G@Hs6@^}?lDkp7@Dtsu%2&aP zBJ09bBK?KSa?t=9J%(Kr1iH(W^|S>#h`HEQ2?^Qd$_+YVCGT8}qn1ED8&pf<&&e=U z%_VQjmr0_uj9RVpse7a{PEE5MfvnR%)Nxwfj|vkYcj2C|7LyuXQ3y?Xr(oOiv$2GN6k{0*Xp{pe=2%4ipMVk^7byw{nrjaHvD zLt##uqlN7}_rUtf0>?$k#SRxcneZ7qn3uz@jI~Fr>)NVsV7=nOGIZLFvufJr(}b*< zKrhLrBahMQ;sT4ECvcr-aO@X!q%)-;-cKNf3`DbQgVlLf*Y80eyJ`t<(WxaY)d*<6 zN4te{Mz=?O@-ap0d&yu*aHCORm{T942FFd0KI1EWSEWQf``MtAY7S6Gi)!ju6V#Jy IVzA2J0DAx&J^%m! diff --git a/out/production/clients/org/apache/kafka/common/network/ByteBufferSend.class b/out/production/clients/org/apache/kafka/common/network/ByteBufferSend.class deleted file mode 100644 index ed2e8d1dba04f481c65b98d62cd14f8ad92375fc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1585 zcmb7EOLG)e6#j0{qce{{2w?y*&^!V&lQ112$|EsCVnSgBvq&lyvf<9mO{U4Dd#a}c z>dsQC`~fa?!vC_r1WK`!q6h@dI6Qq1n=&Pk#k+ovc~B7pvma{OP+5&+}^62a8=javQSHy^1=8JeA3!Fl{5A0-NdGUD}g-aiWL1Y}?Y;bYz8>h~$x=s{0y>4pG zDhYAAtzf&|^v^|3OLscPuSw-ah1&n4NnvPrEtl)}Oe0b_vsVsj_l3x0p*fQOdb+vX z*AmI(>LqV4?srTvm-=iRwP9rDd9B6D93Sz|0rWPN{=c%hHwmT z5J`&nDe7-hiYBO4tO;%lqtr{Vo-!b9q@U6)AOZFm<1fCUS)CVhQ0oS RV`=gQf>4;??>u!2wP$1OQD*=E diff --git a/out/production/clients/org/apache/kafka/common/network/InvalidReceiveException.class b/out/production/clients/org/apache/kafka/common/network/InvalidReceiveException.class deleted file mode 100644 index 5a8f33b5f7d55aea8930a3e176f14b4264e220d0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 618 zcmbVJO;5r=5Pe%ds)(Q{s0k)|F$%_BOpL*BF`5t?4??&vWrdZtOWFedEl(sK`~m(b z<7^?sC|umbY~JjfH*aP?zurFpTw*VSBzAPzHBmHCVo2XeUq*F?;;H?@$J}zc-?N%g zApKtLtj&dTPf+Ezi&;zREAg*D*M z18zCW^OSG-BATdRU^VB8jy~Xcq6f<#lpO^<&CtRAbD8E67_{d_DG_+Q3uGOF(33@Xt i)$=Kg-(qDdI*i4RX0(NvITxc+ivN)f^23!!ai=yC?46(eqoW-w1;7S(1&f;}Bdc(kPa`-KN zXW&hRv=@{V^20$`Dn{Pap0mZ_pj6UfuR>;0N3q_X zPB1$g_(fmG7lLr6I7+>}{Zz9mu`m?t(2j#pVcjxSOQ(F+cbDUzV8w7MVW@o~-}Ydx z5#MCsYjKnC)oDg9axW8VW1s80algXWuBuDRW4O%8Bud>UC^ryHQf~FI>+9o{*$Ewv z*%Ka5m>U+NIr|`#O89;>B(fOog`F6<#VYDP1)h+s@1}yPOyy zRSdjk;HrV&D;&E&j0cXhz)g2`lQ+#mGne#m12Sn{SNENe<>UPErD4(w!tOuUUhn%IRWOq|3i=F7yNFl^!-ysOYD z<$b>z6Q$3Zcnn<%53Q{E!xh(?)GWGPvu-qN$Ii6n`j$XujI9XR~$8P&BWjEzKQ2B%!2zdH=mQfP!FiW{xNdi z3d+(67-7p{)`QrZ&{mx%7LV=v)(Z8x3VF6u0!edw@N*N#aNNWPVz?kIi7cTMsbSfX zRkp*}3Z|@CDH?%E8=hxX(fnPZV@w)g4fv8=6j)Ps$VUpB25b0hPvRiat`iOa!21f_ zKL&QG@6{P#;GZV`g*o=tTWJ}YILr9nc2w=oU9~L-{(aznf83T#Flxg|?#yed(q2uDxsMUe+37dCihoY3c~69E6uh z>XMGDU(FS*e!#P%sJHvhk1MwDQ@ssr8pvXklpKFIq@|A5nl8&HL&d87*Id8B2PDfG;vp|E`(jfL%t zFcjvI<+G;h=J<|9P zIK;JCKaq$W=E(-6Zc}(t*9z{xt&V~Yk{F{qZ;(B9p zxHoaBYNDa$h{6z#)`i*YYr+lxSs~neS7A1`n($TP6X8R&oSs83r<6GH!5nscf|f6^ zdi(~~h0GZ?Ci@{PQ?R#?D8b6E8SHjL*t-oU1>(g)N1^y1;3MYN}|FUfnV zEyI+j+OKkc`=Hz|qC=s-V-AMUMwDEK(Q*CeH^j4up9y*PE`zq2aM~AO<+t3zm4DLDYRde#8e?3AdA4ygTXkirQ@v_!Fs&F3i>6PHHzIBBit7mfD;6M63^olULw2R zBY%Wndk9ggZG^9dU)SpVy2iIJFdVITow`ODlJ&SkZTo0h!qtq1wcjDFR52$8Bz}rU zQu2M}O30+Slay{{FtW93X)9DW)ry(N!ymG!GAuc%lPacs;>l~|@)>OAKLed`mZfa# p?J3)wVUHNEo7|EZ1e)@l|Hal%(IRU3qnBfq!;f%xn!6NU`T+>w{8<11 diff --git a/out/production/clients/org/apache/kafka/common/network/NetworkReceive.class b/out/production/clients/org/apache/kafka/common/network/NetworkReceive.class deleted file mode 100644 index dfcc7db953920a719efa0ae2de7e54652b56f64f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3055 zcmb7GTT>Hf7=CsWk`PwJAm>oy83GzSp!EP%PBt2e5)Puax`YK5l1-W{D81{2GyMtH zai%vNy=kZIOo6t})agZg(V1Sf1QDx76|ReBR5WofZixP-A7e=PFz)AMLQE#bXsQOcFzv-{KXxG|mhXt>t{3ll z@xC7c#C*6X`un0$eV7q_(vLlu@L^V9XkN^DF)yKZV)WW*d}{PwB0e!TI4mI$TU1xn zNJ=&4BMCF78}k<=>>A7%c~dpaNj0^sNvNI}jYY3V$A^a`NKw}t{z>rUGt0RocinmY zLv{@|lA&iJ1FNPsusk=XFg=fZEqw7W8(9M1c;g0R}6=6Cj$upRlCC~1d zZfK*+=@~6IuFj;m5{PAzYHCu=>0<6Iy3GYWFQF%v$<0U9teRZVB1`Jrk{U^7(&>y5 zF*Nf*CbtwBwfiwGsp%^k18%4gl;Cgdh;ENoY`8i^ySJlr`?drh>9Q%!WSq3S&EzIb zHMw+M%{qbla+*3jlFOtebUVxSh_PpVMZv)k(r24JE&oQ;03F3*f0=Vt(V3UkagICAt#o#;!~X<(IC5DdulqUNuFr|$DnlGB zLu_>r&(Nj{a`Pn-$yvP35QK)#vlpfGguB*J^94ucp;id;>GI$l{UltVwVVA# z{`fXp1%~vPeHC|Jx%!-dKf$vuG5ijSjQ;kgsCj{1(;KLL8En5_FnWTIZ1deyc3*eK z-Qa$TLocvr`UL{CuMg~9M?(X1&=_dqXdTTPXn9FrUaJWTcam9};&m$-*@+{cel@e^k7E0XvPb5^eVDT)E4@eT#)ChB8c;>bnYI%mYp z7Ax_J9G97W8Nbu(0DDp9MtJ_lK}mXzA*t_A_+F!3`imH)dJWPF7~)SzaG1S>5yol1 zE9Gi+ZB^@HnYK!5eYa44WEr;gD6#zOXnTUX;3G?0>c@T0-~3UY!vIY|2bmwtYUCKV eX@PHlj0&f0LbP~FFXY#drTQI?b(It{uKf!YMupt~ diff --git a/out/production/clients/org/apache/kafka/common/network/NetworkSend.class b/out/production/clients/org/apache/kafka/common/network/NetworkSend.class deleted file mode 100644 index f4721fcfe3de2d5f3e6710b49683ca58f070a76c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1061 zcmb7DT~8B16g|^jSXfp;P(-QH@*&%$bP-8>C>m0YNt1#PHRZt*dO4J@dudrai_&tV-sUGJ0Ew>+A{eF%(iaEqMp=Zx<|t2y5=aU@i*3l4Y2xh*l(h9?8I0p8w*rI>E0p zXm*P9PVjo2N*R%q1v{S_*$LD>&32UcDX-(9IFriLoIQ*kbPQ!;q!&`AI-kWH`BAH@xJGDGRqMiY4`Y3NPTSQU#GT$7%q}|d?Q>{H zyRT#H?T;@`Pe1wueLt-kU5pje+VAjzZi?Vw zjXn0*Q)7P&nx+F|}C!os1O} f2J$;FjN}(YfZ!hD9uthDzc1ZLa4c~k`2dq2n72-( diff --git a/out/production/clients/org/apache/kafka/common/network/Selectable.class b/out/production/clients/org/apache/kafka/common/network/Selectable.class deleted file mode 100644 index c3ee5986e4874867efb4cf3dec0c102b24e68f7b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 832 zcmbVKO;6k~5FOL7VHdi5m5-LTd>l}gLw#sZO@)L6NL#InM5Nx0Gb|=vJ4zhrZ|k8y zpg*eWI7t@-NUeJD^Bc{3Gak>+U*CTKzz5h1VLOB!2BEaZpmYr0N26rSM*d?B-a%#EugrygCZbgPw&fr<( z>8#YYL)Mz$Dpkqdc+JQJ6h;L=Ve)^KX#))pC0C^7cZAh%i*M4vtDQ(CUk zVd@h~1}o8^c|zrs<|9?Qm_avDV7_)F+oks zATBh%^ZpDD{{9tz6bzQ`dOEwfuD5En4k^orXVXpTFkmo4dkD}EXbB)7K1;Sb}AWtdPeHtdjD@J-$YK{T|;SzWIQ^Y%9KMajUKP+9Nr? Ish!{c21ymt*8l(j diff --git a/out/production/clients/org/apache/kafka/common/network/Selector$SelectorMetrics$1.class b/out/production/clients/org/apache/kafka/common/network/Selector$SelectorMetrics$1.class deleted file mode 100644 index 504c1ea1fa9bc9c8a0475c9b83533f3bf3c333f3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1346 zcmbVM+iuf95IyUJx=9?8l>5DaDT!O+B$SIt1wsL&Rw;cb%KOIMCT^VF%63|bAK-Hk zQY$VGd;lMX7&|Q$fk>v7?A;mfoS8Xu>>oeBeFw0NCmHCt6Fu%`F$*(;Jnm(&fcrU= zQPE-PSk$q^klo{+8TQ@4tT9+^<@ap9%bmWk2Yh?LZAT4|Yn5?>Yg4u_~qTTOJPrLFTi>$mRr8lxfA4Zup#`S3W&V;>c#D!;o&N zEkS*?T`AsorZVPVMOJ4WKX9Zo)q0QY>06ReX3(kDS48wpCM3ZS0s#ZI196(PJ*2a(AfFuF8OLeb zG+pT)=}6O^&UB9y6GBUO+NSB=d+*(JkEGvwciK`Rh2KxQ``-7y@&CsAp7p@P_niT- zkuGn+DBcpm819x&Z`JU&2%2zr1aHSXEijJ@9_?nKd>-dI_Z|eA#hHpo( z0e9>8j*jnY_+A8Cqs*HC8MCWJG&;K3WqxXafPx*Hnf*xpnwe=JeJh0O_X%QzWtJK=C+A8NKW zR8S4ps>6h)zFeB^Y~Po0?EcB|qjrACI+|rlbYCuIWe;2VjC>D>AnNQl*N?3m1C)3yw2#FyraTHequU%lkwJ##7nn4Tzr&|A;#Z8h2h zwsMQNv)Hyl&q^I*xD=$S7&aVu@t2a#AuIJ}(ImPE`E#qMUrNY_6 zMX;&ThI2-zJ#VLS`ScE1aejHtspITk>20&BI|A(`gq++I;k>fVrFyI0KDVfYt z_Ap^EHhf zJlCKY%{OQPU7*oIgBHy*2!j@26d@z zC1+eSr5az7)5@S#)NNoK4q<7dCR$2+&aILhUyPaOE=X9@SIk2%X1S0tiYf(WD?FVX=g9|2AnjIljhGWt@%sb z;RE9~_1jbN(y|-pwB@dqamVa*V=GymN+HwbK5laEs0PmJCZxLdph=mTT%cumh&{B9 z+j(o$jytMX#&aX_fG?h7I4pVWxbZHwJnhDF`S?^m!Hd|q13B4Y|>T`pt zFE{CU5?$WXS~5OKSa~)9{t+khAz|PK>0vZC>5b;NlbTXe;N;HwP8L|>N-0O(@ydbq za>_l|&NDnN8)xKg)jR^JiTY9lCbwE13+N~aa5FdArgfZ;owemcBw3;_Q7ww8%tDs3 z>TIa^8F-l#U#W52;v;2))W}bn@yYRcQKK_m(CFJ;sr8(d?w1`S0q_xUbE#vt$GyKTA9VQu_+^Ez@3PH)g4Y4t@JANY;{$;6-<2k` z1~k9?cl)z}L|ihts@nG?4#cz8G%h-T)e1ucp;aZd{oD-blOc^iB6z*?aPg7RL zQ_4)o^<(Rl$2My_quy9SS5cKWrhIlMWb#Q7jE~Z2#eM(TMJqk6NL2+W`;%iiJPvY~ zNXrCmE102bmatwLFp{^_;CN1|do)^W&^lT#>v03OdX^Xf=t%%)1@A>4cl-|@C2$fn zzbVMMIL^~ow$^T`B`{fK=ee1jmSrv#aGIC;q?wZKgcDX8mYU?zmVqrbHi^Y?UMMww z#x2>XqRuwcEng>b+Gx-w+HBAkdERZMi})CqV+6PU)*inUWNR@fN!=VHQ>_K3Md8d< z<0g-*oNrfH?if!}QZP?Bn&Qr+4cbN*OB&D3?Sa%n@)I_eN1^0wp>E3Rq@qFVJy|}c zoD`3}JZAX4Eui=rTtnr)VbCRXsYaI>w1fH#+DW?zTmN4GE6)D|N3Z7`;j7KLN+Vp@ zh_!GU`(pq%k!WcI*~1g4#dkCwvAWdEWTjHJ%NLooe3jY$C@v+dYDZSS-dGb{%seid zb%wm!9i@3*_j3LvfY0!*T1HNWNMPtXX=v}GH?(l6krR6f4o)^Mbk5NH^q|e~cNUZ@ zb|zJ_D;}LEfB?fe73>9UW$KPuZohpaOJta@;Tx2rUMhX_W6s>+QQN!H=1kQ2;jopR zcVv@tL zwLC3?Vk=AZfKuLViWSMJSbaCEbNw$1a6xmyuZ_~LgOA3qh7ic?a8;JclxYN1iHnEz z@l2L4Oa5%6S`c5U!+d1&O`H$tuAO|}Mx6gjIL3cTZ9!DtfcdFYUMtl%-_O)HU!2r8 zUoX@*&$8;92TJwLa}m(dA@9=s-UPxFZsgBte)$75{*Ta`yH}q8b)SUjUULGW(`Xt# ziRRV!B0K{vs`JMTTKWTWvWXWWr_f3`1%s!P=2-J7v=Qz^Qa;Thd>HM?aI7g7K7|hX z*2xl%#G0Z}|69_}E7ITc$Va=o-(S=kJ)fCkO{~yYJA>ILFy}Pp4oByi(U{bI67!SI zyk$!|%iHlenT9i9K~J+N%^@X-4;zi`gqJ zs!+7wjP}=XlqphT4w%t{XR6l!QgvU{ocf%XOI9y3tB`3vqURa2m?K0 z7KTJS(xL-d(TSTy6t{@;ahI5ld&C@kOw7YiMGSuv^GOpI&?>Qzdc`6d5OKOiETI>P zrF55AM)!#2^nh4F-w`Y6H{wEiSagY5qFeNe)ndO`Bd!ys$cY~D46#<+E!K(q#Cq{X zu|fP;Y!nZPO`&G7IW$jf4JE}zp?xA5x>odt9I-8QtJofTvA87kCb1JwU>~+4j3IR4 zW;}_XxCh}DJele8N%lH{6`@;kE1rT7dO|nisdyT9uY1HpcsibeX57ZC2KO;v#xq&Q zvzW37;t?rY!?QI!N5gY9JWs>(HM{`$CoX0F7ve>J_Y=S1J*@zJcJ&N~UO}JYf~0$K z)tz)E=ycJ;9C2RMqDT0HJJNpHr`F8i>VET+^zwkq-=;d{u+NrkI*n_FeL6R*$H)#D zy|!8?%R7bZLO2o#oy`rFo1Dpgv-u<*n+%(=a8(f{xQDKltMiMi6$PXOtYfD2^t32F z-1TQCBTCQFlAegvQ{hLdYO&x(sYs4vAs5MRbc(&`7FS|}*oQ4*KL*4B92Ez-I0o=c zF^D_F5Z)-R!U=I0pB6{(Epat|E{5@UaSb(#YiY5#j=IHT=_2ts+ASVWR||{AMT%|{ zX}Vq5Tr?x}ZZS${#27s&j;Z9H;KUlBn^gKX;V4~?+wo#f+B;~IO5ZRo#yjy6PHv65 z@p`aNOlIE9cRrJs_dnmf1+apvK3rU=qJjluE*f*m z!=i^P43^wts5Q6w9*+{PI#DavQgv=KIGIp%w$w~@C*nIi?uw|#pZ0hZr~Q7aA|>)? zsqRHBs-t>Oi&)4#!Ej@2%le@8@t}j6-8)jr{4T?6{co1#O`=&%w@7-rDV12??YD*A z;O&GmwPqUgWRq)Y;xg&vU70b28=9-EFSAUhisImbQbMmLJj(=yx5jd|L?Yrm)r-Fg znPs~?GkY`G_2nltsg?7%_mJHzzbXUh>O`3(mgQ3^W zr345uAHc&zfGYd|lL$O4G29$~Eu+{OW{&RqQG1)#W2pbH78dB=ap|wJLEiuq^u`QU z;faiVg)$+g$WBwV28$lRZ@dO;T-%5Bk|GN;WT!|4>;oM35mojH$0#>9s74L56j5I> zg^}7EPL`VY$ul|&8?Ru$gXcVlW51w;UC6rTkbN%XfmOC$oUu1w;_>m6w%qyM9!Z?--+nQkk|o{g>$rp Rh4VN;-lv@^xQyfQe*won(OCch diff --git a/out/production/clients/org/apache/kafka/common/network/Selector.class b/out/production/clients/org/apache/kafka/common/network/Selector.class deleted file mode 100644 index 86040208735854fd3f6323292ae5fa2e34583f61..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 15681 zcmcgzd0|VRK)_!eWzu&p{%@!sK{iC+rci&ykJ^MNLyvf(z zd-6*}v`}4BNRRL)X-=1Bvou?zIm1Jb@m3FS<1^*;EQ`;!_#6*;_*{8EFCW|SU*7BT z=C^peheq&Livu1i99EDAusQcmrjert`#Q{d1wN6$w(YEfV1H9~y(Ifz7J9xV%`4J;TV`LC+3(2gcLkg8S^RyAf8gdHy7@$`Ib?_4FJ$!8`|Be9jS$wuzm?{97QbFZxAX5U{)3SBM|t^^ z!2GkD|KjGqiWvXq=D*9(8^S#B?GFBj@aRox{wdAF0_y+dG)Vqmvh?4A(SO|hmYd%e zm4_NF<9DQa*W&jqe&3@=v89xy@|a3ng5h8w5sftk6R}Vr&SW=6BEeWg*dLDv@mjno z=8wd?Lh*Pg8o_&BINHuswk{fLuZf4-7VfB77j173#_E`gBB5v!9Mgn#JNy^;YcN(5 z=)0(2wo zb;BM4Y)=jDs#^|-C4p#HcQ}{`wzdQ#tvGZ<2H16>cmivd3>jLc3G{dS10BJdPJdgc zzXnjcqLG?NFwq;0b%K^is|G(Z1%6{N5DZ-)Tw8g};F=9s2ZRP}w}#?@C^#1Y>P%CQ zfT+_!;&Uah}UewD!{Sg!ALwB12MJ3ZbjG#+f|2nIT3mZ@M_C=yC6XG&Bxt{s-^&RI20#$@Y; zU6>)yBYB?9n!0T>Hp3_zqOCxqbX_PCZ0_mW9*k}BZx3Sx_8#zuH~V8Dd3OdG6Jrb* z7h?gY379=Z}8Auz0ZSlouH4)%;n^W9;46F^uFxlA*L1vZo;Oy$`K@9It> zUEGrJ2Rflunh3Z4*l`kpmFc@r|IS*WVfJ=?3OiB98ok_(>!fX$KLvl*dv&^cHmM*K(zF>6V>5ba5T z@9~5`mN2m$2}UgFZ`}nxK`#oy{N#8(`i)Vhc^Tkvj$^i2wq&+AY9}H0gp55&6pmt} zrEKK)PCB^sNW?IUnAItt*LPuUn|sXzR&KM+9Fn>u&yT&Cqm<3f$gs&h;v z+j7)SKuLm}Cm`#N9b{6j(2FFf`2s5WvOr#D9}8UM;K|JkW%Gt(**vU-1kco6$-OKP zb~1NQOSC5z2(H1s3qc_%zd0fco4!I{wds5GeVY!_Q#QRM%{S|NWZGs<@y9!0eaRCvzw2W$ z|1NQLHXWkFHa#oN3(`C%&5LvxH{{`tPRG~Wmq0;gq)lI=uj6W!#shbZ&S0y9jjSoN z>09&^Q`K+>8J4%~LLSm3?!i{0)M%T2K(E^LL!olca66MIW_wi}Nuq?oT*ad zrd7HTnLxWgjH^N-m||)Y`wXRD*%N}xgDaV~sUsK*`a*G^Kdi%}&+kinHD6ma<_op@ zY*nGg+RCfO;b<-tagVpv1a+KE@2H7LylhpeuxgT;Y^y2KOvNj`p{6m-(%5BL^Y9s| z&Bbg3Ut~4Y6Fw(z^*QNjt;XYcVfl2XiATcCjLtUat186j594f7(X$>+>}w7HyZS#2lMP@@QL~sFq8! zf@%7Ztj?Ks+VoHO1a*=sw|TiZhJ2e>@JY6+S1WDRphjD2m918*HMTlg%|#Amt5d*N zdW$jAD0;P4oocIf!nG!8nq}W=(TFkW@a?ZYPpY(M_3UHHG$6{j05np4?dZP+yyhUj)y2MtUDr~DRFq{_(!$+Ul6LA&= zTYV-?1w|1RwN$sQ&R0>JYk02B^U$c6!o4P8s~(~20)=2@!ay=8nv}^_y22OJcNPFar02%^H*+7mQtFIJ;hiO&*;brEg? zzAh+tC~m8ZMY}Fhm)hzw$b?sjU`|osbybRqIbQhBgGJrP7u<-pvHcaDEI)6mT0~f>8q~QY?U`8_U^SV^o>aZv!?4%azlf;AL{**pCd6Ub+ z2Uo%rKA8t3d5UygqzreSyzgX-N+;R+f;u)cFW=M=i}ngH^%Z~fG30o|rg%eGuK^cS zHLlTxxT1)Zgo98r`J#Y2#adIJNccmMI4;LS>KjHW>SiDrE%0}PKKK)4v6@#I2}O>? z+Q~7H54#gl6ZGI#lDx^>D_6c;wT&$81{{2_h_r>;@r7k&7#9NA3M5{&E$3FvZDDJ& zk^`5NIwZ5kWXFSvQ-i_o`Yy= z1;c)DaeVTklR7L}suNyju`tQwBe+*cKTXnUtC~RSZE9yv{Ltr~&0B z_$}bB72OSYtSdqAsaac-MveVehdI+4p$7OND2@kd>jOl=5*05jE5pX7IW*njVe#n#fLG_Z=0}~bOOU(&ayW`C`g2&F&vy_-045oNJ>@yLs{NrI2%{>M z;v9Io1u2o?sim`{O(?F;%ti0Xe#wz?kc)4@&X;c(XIy`+EX}|QiVQq+VWhRPP-A3F z%nHRfOL-Y^di^n2-_jg(WQf|)jw45O%`p@;wpIdXB?*b+(oqUbi309EV+L$B7e*}3 z97pm35c*qR3Pypp37Am|*iiqL1U6(GIuVMm#sv^z#|_rRjOTRY1km0Cdn^ni(RTU; z7}jyVBos$=E)ZND5n+Kt6S7UwXni^cNrf>0UPxBjXYh_vQ|kT8ct^o0_5M8GQH9ES zM=dJr9VIn=4$4dViE^5LqPCV=_cGw1w3K>(1@9<}>2;`)=_jgkcq*wT=_d+Gcuv9| z6p1kY6C(a572u~`M2D*fsG#}@viiuqk4kF`50YoAw{U=p4poouBfHu+nuvw|jo3OsBl{>1-7<7X_0i~bcTBolp6*tpyJOMyZmssZ254L# zjmO)Bts_R19f$XceN>5eAAnBkr^$UZrMmh6ngdko@1v>JW88hD9tBk{Dxwl9#Pg@1 zXd9KIJsWT5VthMh&Z80JhdcvRNv$-Cf;6AnXbH8`a_XQ)3ehIofz_R8!_-4vbU8)n z7V4(^=zQ8wF=#*>(oWFN0QoGOGy=SPjed?EI7q*sU*ah}kpS!7ufTJqd%vdN02i4R z`N~i84ojhw3Up6oYw4ay%}u{W-*#ZUL@#6RcNnQ8^*#u3Tl6}+Ec(4gf3WC}7X1mo z%dqIrMUGPbMT_{aM#OXl&=b(5)v9u~*X7Ok7J#eAKZ>C|%_)z@uor-LYRZ;toJv#l zl+xcoh0s$wFao}kh8{X9rES_8}|rz zKb?p{>mV)PI&*+l^wUXOG@4#(km~bj%cCG-6P-!t&|>|(5B;S_mboqg$z-Zljawqtr|v!>--fe>Wi94#m6!%6KPTNq5oJ;KKE^7d!7k?_Sym zZAXoe`(S7(Z8XPBDOyI4|*I1$~~T(1V)jY6B}p z;M5d>Q#A<+JH~0m^OE?$7MR@(ti;Dr6ekGr6^^E@R4~0|O|xDg3u@4iHUxvhV=x49 z{0HDb`ZNkYvpt1Eg@b}P8{wGJ#_1-0hsrJX;P4_A>Vt%LbUTFNEBfm5Eqj69DK@nWOM^)8LI zxYXhiZ_&6$?YS7GEaY-&+Mbk zV$EBu%lqjJSp0FduIkwbXd9>p>0=FqK{`{3)#3^Or@x4{;sjB;IY6J_u8Uw5Bk>t) zES&FT1kC9Ok@G?BA_U6CkmCxhUJpWr0SkfKVyt|PL5N_P(u)mJ9>*C)wA#^&k+cHU z_;Q>kgpZ)ScWE*-#NvvL7LWBpQ@lLRn2qZo;Fdx+_97IY1=M+v@YAi8)s+Ku&I#g| z%)sJ;>T(2LiGSk{(7ELWlY%#x$@7@{$v-ClJ~a;mP`2&0Zm)Zg@I#5(LT{l_5j947 zT1S@EdZdr^uG;GXcWaCEGgxa2Y;TcfVjKFEx{uD3C#c=nt=H2}9eb$WYxh%Vi`OIX zTSolrQkRwjmX_;1hVKII0PX0fPSL=yGt-qvvMaKO9`_dYQ*+SMEguBnurK&P0D7ZLBS>_Syq=28urtSk+XR`pNG*-p2)qJxrTjwFHho6@$vZK(R3c*Dt->1P+rAHl;87A{tM6I zcX*z1@qAUx3)BQYQC0FHHHlA9GkCFD%1iJ8s#b00I(05DQxRUS&gYZxv7lbv&MVa( zZqUBq7AoON#P!#BJQb@M_)~)kaEm2s5;bYJSWKNNPx}H7Xnj-rf+A4-Cmp(MTBoMb zLGaT}rvrzHfQ3|2?M-?VwU>Q5^(s*}@g(%}Xbej8li{uM$)_&lDHtnI7t=(Zik?Xu z%i&K9+-WJe#{;f-peTO#2Hr2mPTPs(> zJHJa_w8#QVir+_U9=iKKP>XaXpyW#xoS+21tMQb`I|XUnyrx+%QbE~`kHA^q^a2&h zqn~b;*IVR?WyU9ulk}6x*Eyu#sRu@NRGlm z2%=3x6XlXW21?#_JWeX46P%nTh9b#*FpD>5@V(5QTgY~@~>%sZ)?ccl)U45c;+CICCfiy0S* zay#i0__W}^#p1RcSX=@uE(I2s9UB%(J`Nl1awjZU9Kti;h2$wyci%??cvcS5$0a(B z%DTYsqr1)AZs8e{6^Bu`3-9ibWcE(5te@_}Ds$scL-k|H9+U4P+6~fP0lzFGL%17B z0g}d9W%r-~fGGkfh&o1>{OhNC%kDF2!yw(S2n#B3G0_oQ(r1y7$&QyxYQ|SWX|JNO zd^P@z?HZ(Z*Fv_}(PqA$w(|`X<{PP(Z=&6NGd+aoBYZ2I`)&B8(QeI4$(+Nqi5Kb2 zc?Dg;C-4%mtP#7Ga;>(kXB;X27AFfk&8j-|MDAtyD)$~;Yw9K?KNt2J) z(*ygl(3nCwRG#=jD&U6@Tldpw{sR8;@Zpp)0$|Dxr*XZGj=~xjsgmN1FeKL}$H8$8 zW)V{;ru|zJ4y}aVd`76ots>a0?-sQ#uj>VxfXO_s3%QqB;KlXzB5AIe#Zm=;`ZPc@ z=JnHO@mR8t%4_o_&)TQ=c=HG8b4ayx67_lHyK>`LL#N@kpR5n8!99Zjs(?{0<0q&B znb$;qlBV!MF!m{0%unO*+78i5K8!owv(&_2#uexVsLgXwwCAavU!?Q-E1DltsM!p7 zQU#FvQY^)pnB$fUpm4^4RG``*aWW54auct_oy4V8Y82(YM^Go+NpP>P*n&V`2A#t1 zh=97t@oqk>+O>YaWCd!QIa)d&n4?xuW*deq&y8SDPgP%1Bw?wD~1S z7==v!7S;1Bnj#b18&Y&MICPZYCN|`zT`xCnCBJoY5M73+1SdDp_`+dYJ~SNuAVY`T zsT16eFIVzgqa2XFc$gj@3d!69O(;A9(g-)7jI6~lFEic99|8tRv0NvO1?3~j`6X_M zZu~vgsW~>gj&Ae1oSXBGZgUei=Qvr3ZDwRi!=SyMH{{sNM{zQ_`LvV=n*<>8Ut!rJ zPtc=}Q?s}u)^~NCqanFayb)uHTTq=LlzwoHM7jO|a9oaOKa!K%@du=f3kv@qb-G8@ diff --git a/out/production/clients/org/apache/kafka/common/network/Send.class b/out/production/clients/org/apache/kafka/common/network/Send.class deleted file mode 100644 index 94822e708fc0976138c4ad5260678ccf83e724dc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 325 zcmYLF%TB{E5S%TcAuW&c_yi7#knn|DZ=ixeNIirTH{&dco7AonLxE4@zz6VA2s@SF zvX7bFozZ^(e0~AAz)_5Y7!$&{;Ld3yowkN>l4kb{`6@*tO(}BMYEw=LiL%R8!;TBW zR+>E$;&rPXKUl(9y4f(=3ROvCxaq`=bQOo?%Xi0D8+4l85k|8&#j7CJ#W*BHo&Mnd zXdrX@(ki~zjfaw)O>rK~5Ke7d3b~T1Vo}TIS_*GyX^k-K*0!xhm?cd9t{ diff --git a/out/production/clients/org/apache/kafka/common/protocol/ApiKeys.class b/out/production/clients/org/apache/kafka/common/protocol/ApiKeys.class deleted file mode 100644 index bdeb0e6733fa34eeeb3675335a5fc1fda24b766c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2633 zcmb7FTXz#x6#h;#O-|AQh5-vHtsoaoTOoiTmtJT{r*t4`N+ty&YM3U|HZ;w|$%KpI z9mTt%Yxx6w(MJs;EL^g9cKI88)_=egxX(#gRK>NV5Bc`~&OZBmXYYNczuvob8^9iX z6^4Rh1SL2j7$M9>uov^`#D@;);JDuydq4G;~+;} zm_El4M?;wBIEFdS3-bcU2**WXzREGm@tQC%aU9}!U6^lhjB&gv%(pla9B&Kr9gf2s z?+WuWN0Q?`VZJZaT#4WVd??(HLim^=G@Z?i&m;|oft2gb$L*S3TyWxx_S~W!FS<)h zZZ%%>T;DCam3X37KIUAk4>5F37`fymgPJm}Tp=?tVHr7#fv1gJVmy&cFsw`&iE$%a zNKB0vOe;&FRxUGL$QsirGfC@fW~M37C_LCvIA-J-)+IAjxojqtGR6zm>{`e;<;HQP4xg)-gsI{vj;R=jA>O6u zmKsGztj?%-9{a?jbNu3hiv7X~W#;DUj$c=CK)5{Z_;$(mNy|aut0~7WIbNb#GV7j- zA;EQ7zFW&WwMw~2VZ(x2Ia8xe&cn?kg4L65)%V;=#VJ_}4Zq}`uc{c8XBCU= zRUDFOTU=Y*n4mODP;V?b-owNR!H18!3tx^lt#`>2vG&6hFDf-l=z00RGS+VQ&p=Dw17rA_=$V&UV zm+xzbnPRGz2}s-i0*x6B7|Dp4X1_VMi$0}$>7Ro>78P9r2&q95sWTF(BNC|#5~=hO zsn`;!v=XVD5~+p~sb&(XR1&E|bW%i22w_yzhj6u+9f>qPfYu|Xk^D}CMLEHcA#XFG z_=_~benQ6`1imMaVVZt}a;=-=i?@hTTTx?YX$~;Zy-3(_tkZT2BOM!@E>HTI9jLjfkrk+BPwQ6#|T8)O|+S|;)J}i zuqNrUElwPz;3Tbg;1r3@63x)Lc(vM2NhjV>{ z^n0b>rvc!GH8z!yYj&0XcfJSWoskXoQ>S z3G74<#;}3bdT|DQ@bDxqV=5R@XkYEVHmW^UrGLj4^*+d!3V|%cqaYiEy zG}&~s?QVLc?KVg6D{Y##NwckJB2u+!x<|6TyZ61@y}I|=J=)!D(*JuiBM-*>3?bI1 z`M&qQ@BhB*y*K*iYhQkmi1x5tJGIiEDD z^f&awrS!M-cl`11W%`Gu^pEsU{NbPZ$G<4_uk>#+JtEP+%k&=#b0M_iUF)5j#XQf3i_23=N%OdprnDw(ZTDB-d?W%`7~*2rwFLdRUzI+>o3 zSeMM!D|FmtZIJ1c61zrbZ&2t?mvya7pOTm+vyBSfvTwoPW;3Rx~|yG);v*bbTPRLF5zyJY&T%zEHp_C}dLC$ruB>y0u! zEwP(qwnw1{+<|*#dPZjZxb$Y3J}tzQec1WRz*=-0WJIp_# z$O3jmVttIH(d1Avbt1`VTf(v@d-NH-FlF>i>l4#@Pr)jcta8tcZ8=uKD)z)|+p;SM z7_}x-$rz)K)WAR{md&R|@i&l9N0Wo#uINjp)1xEV{BSIjiE>VtXC$9Zjl}!%sdRoM zn$E_v@f7T?O2nf5v2;F}%I2fTqwz%aNCI=#@@G#paPf?l>DaN+SSFj##)o75+&ZJx z>DWjj-WQ#(W~B!Lrc?PuG|l0x^I#9hveEu%HX4+zOvbV&Qt6?5>||eT#6`Hq8=pu; z`}6T+ek7e5Ok-_~Hua^FnbBdO?@Oi9{qbZp3#ju}vVlJb<$O8*L`G}k$>Y&Pyx((9 zz%JGVSN7)4FTrilh*M6=8*ghlk_YY&WMw9P2Z9|c zT9a6zNbX8;V$Z3bgf%&7*vR5=p{Q3XSxclf@-}fwKdtvv9kbXotj}OTd&ZnB>rT}+ z7$q;3_PUQny*$~|$JO+9A3AWw6?6#7mlmvXqiKBT^6E2FmhD`f?pCgw5kqX-jWY#f z#xbojqn(!pXo$-0iO!fYe>fmY*CR7gX^gt>yzJCVyTg*xda-I$812})?P}(=_6gK0 z5@y**R!d`soz=&RScfDOrY0C2+#2wE#<9)vV{7K}RxS;9n=p)%2Pz6WnhC_{aL{MMIQRNh z1&ldpTYMLQ^&tiIv>dFlu_2;vw~5npx87AQ*Gm3l()kdn863V z5~j!KQAYb0!IPSps2EPF>ZB&p=$l3$Q;u;+o?ala#7NDoXl2XMW=#}L*)`4QLF|pF zXeHe&JNUz-v9n@`yRcnhdYofg%tw8eZC7WUv{A6^@c^Kc98d|1)?u$;l^vt(RJ3u+ za3Pg+r!XbV`D6OheVS!o!~%Q^{<3Gy3fCbF0oVIDuG<#zl^46LHDeZ1_K0pfChy?^ zggsew5V|o;iXF)^g(XEXA$#Rvt(FRC*YM%o-1DRY7WFm;Hrb=e2DAYzKFiy zp3CTvt81oc7IZ)V(TLngc)o5LgQ0ng2-AnT&3%j5bkjX+Sqa^~s18pAmtvL+#p*bW z=*>zf^6@<0=oQ?jCH+iORx!^Rg6XxES+iKw^a4M9z!x9kxE8BMw?hmYjy|qCx{r?$ zLdvVrWor|rQRF>B81q_%@BUe2onu*=01C4oy^@e74QJM}rvo7s2EBERn-w&GvfG}= z4CFhq8u6~oSY;G8w`dy$!#r(_!{Eob!QMsu@2)dp>ErS8h;228z zM13KPix|usu)E$0n)SF=(92sKzdQRGSOJKSa}a}zfanAKYRSN+-!ycYvfs$vKMxOY z%+17v$xm>T(M3!`f4~~**T)c8{60~>fx6|L6VAT~K(px}GGY-x@B~M&XAuPPGR~6b zxag|^qep^yuRD%Wn&F$VV`<<=koe3An=um}LR^p5cT z!_V&kj$9B&v;O=&W=!ll^J@?;m@p@CNg2m(Fj>MzrHN7mNC$Ci?uVb^@NQWI9wHPg zTh+;_v>|#>FfE6M-Te^BhZ3-jQ*KG%Tu=~&&+7tz6UHoBgjwEMIa@AZn5zd^;4)sO zCXm8gPjWc@R|_XP;TU#wX2vK7ckAJckj>gAB8b&n!U4Dd?$E}nyy0IdwohM4B)wEh zdrOKIri0Eph`JuH!ZChls^EZUqIHRLmY+J}Vk6qR?-_K=X7N}g_RyHrAA@+RJN ztXoQ=mkJN_o+Dt2p@to%&vQLHTY5Dr5E}2kQ8GhQ;~Sc*WdM@ydf+#zk}<7aEocmj9jx_t{CqVRS=K zuZJ=?y9r$7Jtz1o2)AAufmNMD<#KTw1q$H?ycfO5W3F#-ja@)H zU-z2%cn0(WHvO}BcLSB~+bXr!PMue-ZoD$5?t(=1t))p zzUxgs>RDdKEx$}MCtu^s3<-}d7tkLdiQbH5J{VB8%tP`$F_wi9gzwXDdRot6S_=eQ zYt-I#fs_|$DF@Y>*#0uLZT~`z!jWYze|ZQwv4StzHP|U!{}8G4Hd;k*r%rk&e&%{u z818g+dTFE5TKX+`nGuH|6ML09CHigP_#OIP561*%a2y+Iq;@|~E9Y>$LhZ2o6t!-j zv#p4+kHPli0ec(h_vrVbLxlt8?1ic2RqBGh!`!A!f8cwx^&Fm%8=+?!)S8P#LM;vI z$VFDwX?2}C&(jLQUeg+4;&H8bT-VmxpsrkGeVveU7`j7w zowTGu*XJUe>vV%)ZxQURvfQ9;xkz`NwhQ(S!QQDZQ89j3F49w{HwwXSA-GYMRH;EX z(thP63e=c%Moo*F^147WNwyR2m4(1|<>U5hB92SD8s;J=x9mz%d z>eMd;F(DXGm#WJeG?q#7W4TDWP8lJ{ z3c;wlTvZ!%JQq1pr;|dE6M{R`73#_c-IPoevK}IezQKv~Em=Xe0ji{>{bSf8_u2WG6NJ4m=jDUf9~ z%7QGfQ4VB9jqV0fYg7PPS)(bCNR3J$9W}B+R@G=0WOa?sfppgBevmaadJtr7jou2f zu0{`mbk*n`ne)UVtrzH>=LuQ%aIQ}8I#0+2?_2u?diPNGi@l+P8`eKT`?@#Wm_*;% z`8>S`t!L*8^h+T+u~W1+m{s{Vv;vR+mKOHZ>))Ddc#qR&)cQtRgN^3~x(>gAZ>B75 zL3i&)``dw6PP?#`?WXVGH}oIUUV5EwW^3sdwwn&H+vy-H&~5BKin7P3k3CB<_6iNM zA5fhAgoauoG~BYCQZ4;-tmQ7sw4A2VmUojDO47;Dak?XP58W9$MR$kpqkQOLx+nA~ v>7gg_{4^E(#tYdt(S<^y>!Cfr7OF@TVr=)1(K%bZ-Q^GIkFbgs`s1Gh*9(cG diff --git a/out/production/clients/org/apache/kafka/common/protocol/ProtoUtils.class b/out/production/clients/org/apache/kafka/common/protocol/ProtoUtils.class deleted file mode 100644 index 5038584b1f38a71631ae865df10d7ca59cb9d9f8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2456 zcmb_dT~pge6g_JkWEt^UAgS!8CTW9hVw{+SHjt12HHix`#0FC8evB}SqaaJJEHmY| z_f`~FwG3@W2e8V)9GDFU=!#7oendr-RpbLJ@@Rr%b$P$dI;biz6oOx4ow1n?w;=@4dQ%;Gb#WWyN1oLHWU<(aVM!uTBX5*CEzxrDrg zMTYi=x~V%;3~lkmDnoGA+TaY`3%bekjp{nLm$h|+EPV@BSu<8OTNh{F7<69gb%xY} zWmnQ#O)J0R=`C$@OG}rnYSl8+HQREmvSp-~#Ae0Ojrt^mTqnn>HfPxk8}U-<{BUQd z#_Q>#8#bBCB`z7kAbBZ??A=S}BayV4KF@a;SdO5!xn0*SlOg2q2=&foi(u$1I$C+_ zxmI(xH>*=6&3 z%`)r!h|<6@_({7`)9gC;Q!+dw<_oX2Z7pr;R(fW~;WLfRO>R$KoPuLF$__PA`??#+ zFmNUysphBl2r)d3|6i%YMXO<#`J65Wqr2JKqoOheS8Fhr z_n$+K`$4@&Fz`$7v=@Cp^tR#BZ7?jzSLgCxhY@wK2%#+|Hn%7Vv`(h6M#x=~1E65cv@)+`vtjdI=xVExqA}5AIpo!V}gyIAE F^dD8KJ2L

4LUo#8W4A3TcF?5BML3}T3H zJ8*^<--rY&FKL(L*xxloaw~|A_i1r0u?jVcM}1mS+YxkA+RmVx)^-Km-D}+rZI3@% zLoC;~g1r+pXr%0%?2o;|8@Z^qFZ&Yvhhtr_8V+#c%~f;_+8_ zCl{^Z5J%t5=4gAEy(8IG9Hl{`D-j*i-dn}72)YunA?Q5|EU$699Fy3m*|!% zL@5j~iB3L`bB!n1JIUL3H)ZVM`g?H&eYis*vs7Y+I?YoZm)bnTMZDnb&qV)tyPsmCtsAY(Da46Zv5id5fsqw2#s14!zxFKZ0?t1~|&m!)#?- NhZ^mHMtjh2{{=IR)?NSr diff --git a/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$1.class b/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$1.class deleted file mode 100644 index 4e730fe55b04e3f13726d8752e0fb2eaa7ac1a08..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 256 zcmb7AlS;r-1bx*61KmTNOg*HEe0T%GTSdJ9 diff --git a/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef$1.class b/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef$1.class deleted file mode 100644 index a0ed341dca1fe60dca3bcdd075cf770804b962de..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3821 zcmbtXX;%|x6n-WN41_^32vlv6wiF?NfZ|fHD&iJou?4YitwR_>B$-Jk6O``zzVEiR zdux|Z?WrjBoF2b_*El4o${H5DH@AF)3E!Vst_%Oo?ft3`ywX zVaQ08;sKa4(m~YVXbCLw&ID0~X&EO=a7qkk#NvY`cu2e-4&o6!Dy%&wtUVs2<0oW1 zDdQ;_PfL)ahHh!6ED4+X4KvoJrqt-9);6V1OsQ>Y%hc3lT#vP-Oe3nL(}vl$KRfDm z6uW6-YI@i*Oie<&myNWgT3XvaZDv$WWHeiY%B92cn66qGp?>f>)wlGYP^Zj>caD!NoCJ;=-TFN zM2oM9!`HT$E!Hf+#Tu9K9Agni97p1gpzALb#*NU`b!R^uo3wD+P&6+gE6+Dj@6trQz!gfERpb23a zFDiHmdnIi2fa#ogL?6(ShB=cBtlKxs5~7M*Z5d(sWd+S>k@1RxSMeI5<%+;fZUwL7 z4FzxFEd_7mfP#1Mu7dXjr1x=}@mKHxK2-1#&PZsuCN4X5WPGgP6MQP6Cg;zI){aa( zF)p%NI-=k+e6C=dFkEq^oCIG>oq`^m5n^9Z{+NO<@s*6P6`aL43ceL!hXu~N!2a0d z0c|n8w!WhPg_AE5)_Ry-pjXNGPQo@1oy95{$0* zkrdL&c1fO!JgKG!?awNjJ|)x$$|tPo=IC;lI{%wugZ~*8~ofLQZnTC z7|j=}sg$OVOK4pv0C_G7s+Wod@(Tp{O8FAYaF!$#Vs}<)vC`&Kh|s?9i8rDD#}P40 z)*b5j5xWMwrQwH9v|xT?`$Wg&TB|)?TTvmJG{?pK>%<1`Y~*Mw z<$Qbx!_DU)h0nwH17|*L;#5g?WyG|am~!6g=DpR~oP;;_NY~qC>WY6S z_R&$Wc@AYhT!J##aslNRu%s){dKn>mYXvbZ^|RTow5P4Js8|u;<2c3a9f5QX%OnV3 zew%tN|0e;dE+M@d5?n>cVYc`l>_8uSZIGRg`+1Q2xWAp?2#Ipr8Hy#;+CgbQZFFKM zM*-~Oz5?H0;+ove-)d^?!Csft@01m!Vgl+MLJ|gB=TIr(G)6gCE|FtZSKvofUqsEJ z^SCLrViqd}omH1`Gl!vD+QMxJ_`h#l)j_4!1KP|C|+s19^n^@LVIP zB*j|vV?PEF#SkVq5~mWTY%K{yxG&BvHcvf9%KfAe#4!v|%TFyQXB_$cly~;-=AH;a z8GL^tDC3?#aD{=3B%t=tQ92ITa@%gLZn2pcDseK#C*jz^Le9v=6veHqnir zR?5T8j7y6hn>%uOm!H{=k#d(yz^#{29HhDr_u>%W0kq>V)^QZ#T05RW2l+H`&Yyn) DJOmI8 diff --git a/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef.class b/out/production/stream/org/apache/kafka/streaming/examples/ProcessorJob$MyProcessorDef.class deleted file mode 100644 index 0f9de50b0529c007e73862e120a8cdb4e79e8ab0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1180 zcmbu8(M}UV6o&t^+jiTvLPZ4xQN)_GCb|?!3@O17h>4`4Nez+9={l5w?QS!>HTYI~ zWdaEg;6oA4lmZoN6JRgq?3tYN%|G+c{5d&12C#@1F4CBGk-@Zs$1ZHlIC$z{*1z!9j`X_RA>==Vq10=Eh+`QOGRrDd$qK$L&PwLS4&c!6NS^+dDo>c2B9 zRR?o~x20j~25+}gHHT{*%o7%G(oJydQAhjoy)tIp>p#oaMtUAD?ywWLG@@8Bpyv*X zgm<@O5ek=w`l+!e{e(~%0v%v%_r@*^>G{ub6V`9w(v?eTEStaX+L-0nm;u4z$7#bd zzHpZ3%;QKL(=UI+`o%vM?(%4gfigyT%%2rt6a~)MXpC`{X@0{nQTjcC&;+7mK#lYAsgY14v6D6|{j`AiPK@7-?y5l1s8=vm19e4EQVb zr}RsiHq{xO@rz&mKaS^SlaMqalRDiGdtc9U&w0+vy?_7v`gZ^`I9Bl`mSile=n(Bg z1>Y%n)QuIaDp->dRj@8&L&2tuEd}4p*j5mev7^F|$1-+R^kYfEo{Am;@PmT5nEFw{ zzKka-uH&f~A1HXH;JJzbmQ-|$Uq6_Pgo;~8ieE|wmvJZ~Eg`sKn%s^U%yGCQ;nMo) zmTe`u<5>0*Ka|jQUpIAkL4t2+cvpfyVx_o*-l%T!jY2NL?H!gdXhw@#NoMRaTNmT< zq~Fcx1UenH>~x6bSu(>zS$3FZA;+~j%jsr1#7|f*Z*V76J$Kbggz2Ol(@hBjL;KMq zcFaNsqs=kbCYf+a6K1CgyT8e@gwDKeY&hgZX zly;MxoG9oC0$CFjRaGF#fqHUJ^A&>c z)Z2Hzk*b=R&BPH;y5R)^RF_YZJn!n3>Bz{?IA#^>B!8$2S6;65#DpL}h*6BuLdg*& zG9NJ~Lxf5gY2n(c2_B0CkgzP5SYY`Hhug>877=HNz{yySJ3lu$H!VXC;s{w<$nsOM zO=pJwbBXwdqmouGXPKeOCMFCj$4IP3uGCwU>%fFsDPB2KD0Z(s zFrtHkcd1TP^Hi(Hn`;lRtXw@{zpLlC<5Esd@DpM>=~|@WVgn+1;q3(6V}_9=Z)A5W z5X%(=5demSnHDe?Mn2_SZ1eGUWLZMr+nG%D$F=f$a@;A!Ul2J^&dXcdn~~*MtWCXZ z!*#H}ehIy;AoBQ$3QBBcV~s++e4uV-gRnsnINZXj`cxEfZV&~L43jOSn%1;B^SxZ8 z()g|@gHu%V&S|XXv4o!U>?~!bFJEveHisH3K;z7C;}n(jvS_t>in1GEsgzeNB>Yv2 z?L~LEv92{u(%PW?UouU~{&3^j=TUxPcv3Z-9cm@n)R}`*rAsJCY*fQf!Y^$R(oCr; z2wl42_i39f{6(=Y;h$#8ok!BU`R}|Uw(GqbiT7zAZ|S>>z23^>L`B{0FXFzFYO3wx zCT`O+PJA7pTO4EbAE(h*)JoKLj+{YyQCgm$zj*6`1pF9AhrcND@F8Bcc9$(f5V4ypY~DgC9dv`YuAdn9+%GWae4j<&3{7kH?&WS2$0-o zMRZehv%G$d)<54X;#Pco>~LH0BYe xGvA;K_whBcy+C^f*WUg`>U~2qermrZi3im0K!kJwH0ljei#~-O(6=!H^*{4XiUI%t diff --git a/out/production/stream/org/apache/kafka/streaming/examples/WallclockTimestampExtractor.class b/out/production/stream/org/apache/kafka/streaming/examples/WallclockTimestampExtractor.class deleted file mode 100644 index 7b7ce05340eb763872d400e3b92b93c9f8139e96..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 723 zcmbtSOD{t~7(LToy;RY9zaP7{3wI+CAw-0QTr4WAr(@K#b34u4sqmvLtcZml;75t? z_BKI?jm`Hs-<*^0F>miLuK+f%l81-sJZ3QKW6sCCj|GO@j%q8n%aAQsE*QK6b0rzZ zLe-Y1-KZ|@Ij?I%#zP~xzTj3Rc^~xLjfxreLt`5O?{IM=gBHJT@gR0q@<_ECfxP2U zN6R?4-Z{&KGj^N7mubSVRSui{mIs;>yXGv3Y=1${>P;!!FG%H> zVZfP=5=pl|Bxu5xy#HS6YZ333>n=^1ubHkD@<=5kul~{TdZJ%I4!)1Y0+z7Mu=md` zFcd$#PFrKx{6n;3jgWC{Z18iX-^6P7v6B%)Np!81ZI?VhQCcU*qq`qt4O#jev4)(a! dw2qWZFENHQpogZL6pfQ{XI%IR zuKW=fet;j~SNILa+YgFH38?~f1*fWRpZhp<>wf?B^&0>@frkd%H{bz6zU_J;7YJ{Q zyMgGKY`~l@H+!tzW9Bdvob_G5WA>82=1aj_uERoZl~j&bYw;H>g!P)+@mV+$9KqA0 zvg=i=-mofKl+uW(lMwetA;Pgf8z3+x2dez2!MqWF!3F~^wAd-&A9l6;boYS0W2VO_ zgL};b?u1rj)?#G=f6@Szs*nakCS}TGnNq2p%&{S*lTsMxWr(n^v|_vyhXAI^=M>U(vjhqP8w6&ZVKi*6y|>MjfyBT zeOu}Lq;dP21k%Zp4w1B?ixzf8FzR#>3&o5+=p~_t$oe~2C+PPB-yCw`GS7X_LYGi2 zMbx+dnZ*ttbE>mU+30A%LVT^)cP8y)(bPpnsp#h~>b1&d*@O^Z)N2(BHfFE-AZS*M z4FaLtUa75q*3c0@mgJ2JN4Og2bDIPi6K` z6W+?~pCNn}&dKxVb01(EE=YWl_AkL@iLVg83fBPQ-|KSy2H~4D+0Jjnn6 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/KStreamBuilder.class b/out/production/stream/org/apache/kafka/streaming/kstream/KStreamBuilder.class deleted file mode 100644 index d3837ce1ab3fa72fc2400635ad3852847305ae4a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2434 zcmcgtTTc@~6#j-5N^2G6Dqg{BFVt1R8?D94C23kU5v&s9!>}Fd!rrpon&6v1#s}Zj zhmvUGn=k$m|Aq0)ZmD2f10^w=%$zxMF5fwO&Y2&-zI+2PgQW-t5Dy^{L66*%{g{$u zDvThmMWEoiJl>GY%?OTRIzksSlDZYb?GWxT^kp(}^=!v{bH_xTDE)gBf&RYgLBnqUBT+ZgcguP&WDcCReJiBe~<_VU`(Ev0XEwIj=e-hDxwE|9D~X z1w+zTtGSx4sFtbL97mY0!d=VIR3+zadCE}5n~rhM;C_UunJU7~ndQ8xI>I2E-Z0NI zoQdV*Em;2y;C|Qg;WG@!S_KJun3v0bWiY&I6ZSz1uw#13SE0=qtTtbf&YCR&rsX^K zy$#wf8tuP~+0L2kA6TJEOffu?_Z#h#m~lDj#~_96j>qF-%m{ z7s3ce>GTd`I7t%HfK4(1+7n;Vv$~DIU~e6LpAo8~zm9Mnkw=NCWF1GIe?@RLF}028 zN0Rov9P>2e7)KHlNO_(nd}|F)WS4GFk@sUs+^Q@p&f#N-S=>wEbC}x9+ z@nCW)RmV{u$#}2A4PW+|& z$5h9aRBi|8|Hq6M1u{}|=?`_~MXDi2%4PBpi>>5_g|PC14kxMu^LJAi5NoU1NY28t zwz0l7Ip=$}wcD#r)ClhIt3Wv2UK7G)|92(`2qUgNK2{v_iNZcl6P`xkaJAh5OdDkO E3-BH89RL6T diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/KeyValue.class b/out/production/stream/org/apache/kafka/streaming/kstream/KeyValue.class deleted file mode 100644 index ca6a32441857600fbc1b06db9edd0fabf48051b5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1045 zcmb7CO-~|05Pb~@Iy;J@;^(fqnuwszW%sfYE}M9P=m9dx?ybRIY(XI7V8Y42qz7Z7 zi9f&}W%l(9ubXW7b*10d>U9S7zK&BU&yFkZQNau%6W)UkI#PQA<@dEb4k3x^`Qo?H z!MXqO6$X;VVfd@1{QH?k1qoQ?cGs(^V{QI!dNBJ#vqUzgv27!PNgE$9DUciE_e}DP z!0f-<$OX*ox!B^`#<|lG%ozgkYXW2ulWT}=Mo<@KXp@Y%p^efevkws2+&v=M{d+|3 z=!#;NXNrY^6`rdsxyHnGmIOA#G``Ohu|S{J+zep^3vZ$phfx*h9>iCdyjPdv!p2sEU_sDRm&xB9Ib=09(ic?#Za=7xMLL*`ZXWX87s E8#dSLMF0Q* diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/KeyValueMapper.class b/out/production/stream/org/apache/kafka/streaming/kstream/KeyValueMapper.class deleted file mode 100644 index 97216dafcbfa19e46364134b4fb0f687be4669ae..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 344 zcmZvY%?`m(5QWcB>Te0be45Xvz0UeQ|#RXv|%qV^{BCnIiQ%>V!Z diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/Predicate.class b/out/production/stream/org/apache/kafka/streaming/kstream/Predicate.class deleted file mode 100644 index 55d6391507c09470f22cd1ea9f900e4baf881b96..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 295 zcmZvX%?`m(5QWcBswLtrR3h%iO2yKKghXRuH|nbXsOBn(SF`W{9!gv!7HTocnJ;rD z=iDFH8-P9Pl=G94T07pv^_K37 z4~LUM(Xsr)M#VFNy*QkdFUMh`|2F4JZbF}P*?XlTbPp;JoUy-?BIa$(;+QMR`%tHi jKT>=>gw}7DDuGba)d4aHWf*#|=&gjRo{ut7dlTykRw7UD diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$1.class b/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$1.class deleted file mode 100644 index 191903b206bf5da232a81c0aefd4dbbe8583993b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2184 zcmb_dO;Zy=5Pg#bmJO>QU{L%3H6jFL!EXs7ifBOvJwU~a9)@Irh0Shiv%!z@;N7cN z|ASV6R(Y^I`=c!1Y@)cBqC#O(>6z*2dHuSlr}N|I*KYu>;l2(Xmz5Y(!b~EGaUB{a zl(?edT9WGP8g6K~DR5*NDrL!>6L=W|(Of7##XHEe$jE?qJb9 z3~$aV{;qZ9^I!=3Rz`BktNI0tUYL^Ok(bM-Jf_|nU(}piaJ&j1{h}4Ddqo42$Qn3` zE(7f_3>;CS8&d}EsM$2`YM2q2X`*}VTWbdTF~By|1kPUS(Jqvp5V+F>$}XF|dDn1H z;9e6D`xD_yS!J&k$TX-Iwy_3{fW_5YH;#5a)${a#U7qc}R2FDqn^)bW>)%Qgm~4Xj zZuF6w5xCmK;k|KVfPYI#{)NRmJ5`kc99byg$kI_e{B@%TtUS)jTx%RtnUNic%()%J zGO3mw#51E`koXkZrzl%#4{V{Aav;P=;RH@{#j~m?!~GQca0)S;4z;mR?ppQXEcX(s zyf`ZFZ@E%d{h4jVMz+!N{sVf{$aS3aHqKi=ijOo&&<*HF(SJqPuW<%%LOQ7$#2-5S zIEVA3vrD2svUh=dX`;j-&fp>^F${J-AgfE)6U$RFkNK4`G;lF=Q~pF@X+@;uvRr#O%N=bP`ABT75KJWnB6%OjcxK diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$RestoreFuncImpl.class b/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow$RestoreFuncImpl.class deleted file mode 100644 index 6917de2f94cc15bd9507e472e4b931b65f653ec5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2396 zcmb_eZBrXn6n<_t$tLO2hEz~cDr&SzK$gBhEkU#p(a_izf~hrHyGbr#o6YXd?xqgT z*iZdK&J@b%j6a}1rK5gv939`!O#Bz~9;|hVZ7?EJg4Z-j3i3mc_CX#yBz&yn|J-yeo9?2`L+)gEbjfW#nYM z&u~gNy$pA`qicr#Id>S+Im@Z4nyr<$xLVUTYnocN>UGOh_pLNfr(M?cc$GVym(vWg zYgpd3MxB9W8G_!H?#9o_xW+J)H}r~bR&VKM#rljv$(km2Rt(K`xy!K9AJ+98uGQ&^ zT5E}Fmx~PFjCUz0d~i`Wb#IAbHSs_AOBNXXD^`UQPdVM>gsa4z4XtF5CYH0xno-b* zIoh|CegT-Q*SYIi4qt7U_l<7xZGq(l%2s>TxJNAwmnhNptZT_a?*x9y-s{+b(^|kjSdWMR=q6- zsm94mEO$@4uc?Mcg}z?8$IBie3*6U?2C*zrPGT+48p&G?r_5J%LH|fslGF<`!i$12 zOei>m>k6L5a}1XUZe|4|7-fKh8@MTBT|pijGCojHz%2zI;I~6dx)^4U2xB`|nY*s#s2zo5xN^jmBOS{2;GlvIA0lZh8A3FW3hmk_k~BjWj*%$c;f|>puG*R|`G##< z4z-?KbMWn-+l#p#{V6`a<-$`?WFpN{bz5T*be5xK56p;n`Aerj)~5W zYa4cC(AKOa3z($O=V>rJ0|`;_deiq6Vom_jsF)nI^U?1NUL>nYl1gMLG~=nM9kA46 z_)^jiq}12&-+qk14uW6Osf3qkjnQ`)_!eP2#Yuce{=Ub{q*2HhdB7`pm7cG4Y0lD) z5Fpu#$1(`eec(YV^$5Y3Jhh9^F2WR2TAH5NL*$bGp%fC*iTU6|$v-_Y5+qCZ2+AIY z4y2QNIJF?t+1=)?BWq|ViBSAZEd7Ewe#Ke* zh75j37JuRfLCiA-n@RYB`J1@c5fVt+xP_F$RFjJbm?p_b9&31=*q6v}1v5~I!8i@o zS<(f`_X5u0JY?9!=LM2NP4*_qzU5c2*Rr~JU^PcpKFrgJ50~%)J!2$|(|@1mDRzvc HIDCHrf5??9 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow.class b/out/production/stream/org/apache/kafka/streaming/kstream/SlidingWindowDef$SlidingWindow.class deleted file mode 100644 index f884a7e9ea8968e4af312cdab626e2859da2d845..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 9359 zcmb_i3wT^db^ed`aaX%nwyfB)7l&9*BuA1Ta%{(T|Wy^LFId+9Z>+98(wAN}@ z?5>fY8tYB?Stl&<6$DLR- z?4R}Z+qL)(zN>WqV&Gq+SfwWYTNJCc{azGn@O@qI?=k!bUNG>37}n#zbm4z%?|*dR z|Hklt_@R#dzkwe`aH$qsRP!I>C(7?7rFmJ$F6)D`4hr-khSWqP5XBw|8WJKZ5{_b@ z7*UBxtszlEVuqN4y53Z>KbampnoRd+?iHBb>9n2Qk+O0*J13ZXmz~RJvi9z=bfSA? zG$jb8GKpb9L(fU;l+}^4(t{m`PMow8`Aq^NkxA$6`|^SfJ(=uahc#*?hU|`EYhc*w z$mO%PH9}DxquETt&gC-MjywF`4tJsQ$fYv*gJUBC=~ijU9N9Ix(#G=1R7X!TJ#6>) zaGcb&y~)9}l^@I6g2feyHXq)I>B^In`Zp6$YY`Y7qvy6(kXs1{ z`f14FuVx@pE_@s3;EYQ(unRr;>^MuKP6olCstD?@X$nFzr{!OmBWPnYC$k zu+?4VQaEXh38$JP6l1CASlrr?QM;eOL=$8A%)o$)$Chj4;{y~-TMF>}R?i6>$LP&x zdBRXsXdpF~8=}bmWHx`=Q$6IhUDZU-Dw|V6P!B3W&9N>)gxGhT&g+R8Nn}P&FrgB@ zFm27*q>0cU{pqCVrqsN$^M>qnKFhrb?Q+`%C1ebcW@ni4>xgi#%@mzV-md|2Te(qG zWh9gC$az#&KFPOko>GH$wz!nMt~){Hw`a4~>Ef`U!l74@qnb8jF?rS|GN}~nf(&)L zyKwoH6lp=rd%oDc8%ZO}rj&Fy#hW!a8ggBCcZHEox7bDNE%>6K|56Olg)D6ThWW z-+_0UvP^C^rB&KYX_vSu%Q+z{)X0^x#FSM!)*-76A){Z!FPXAN*0S_3Q>CAmqtc-?^X8$K8&lTFt){G(4TfwqdcV&i@$4k}i7r>AmrtFkm%6GRZdt|GLH{)(o_DZ+Oa%MkEnPSLvGt~}S zDIT?XTaWiqP4%oZ{M2vvH>K?KV1B5nKV#>b(wThIh?P$asR#53R+lN9NF|v)a(Y+m zA4}NIRn6(Rnc0*B^bIaP$P8r4A-O}ay^Kk@)zv+cmHI4rTmd=Nv*di|JPD%zx_+O4zFWTtDg{>a*cdK~a19;m8I z(HzH{_H)x4o!>jhhH9D`c@Evrz&m)5KW z{qL!|H&;@3`jNG%r1&U6xX&r$Ygvxk*&a*3axAE%6Uzh|TTCjRV&x+hwGUYRta%r< zc9+vT;)6NXsZKeauhxBM&!lTYm^OT^vDtlGpo^+@Iv3l8VaNal8pjOJNLhuv`{ zw(VhWIyB&t?WEJq!bh%O03;#Uy#Hn3U~E#WJLL-=X#uj=@SdsNx#E>-$%*8JBf$Bg%uJO)rcM#cQb_rG7c< z96#OeC5^d{H_I{m(151L?$V9+D5&+cFC*n*eL_dB5A8aI*5D_Go=kd>z1Cx0-TT_~ zgDc1M7OMltx^!e(s_@wBVN{c3xc8Liiz~}6N1=Q~c?NU*Tq1b5!k%BHAgORlU2VV1 zpNF(hpr(BafxZa@&m+Wcm|f#MBJ9?(8$FNM6C@8{kk3Yh z_}tH56MFbt!67W6g6;hEVaS)gXW6oR6OSs9vDFgRj;!iz-`*50bx&r$yM+PfbQkjizAUuN%D zO5}$N@;&i(Qw901^HRe{zbIEAioL@Crk8LRKQ_T0f$9pI4G5igHn~^P@{KS zZyzS8he51bC_A=%9IsN33dh6GAtaAaVR2u3JUou;n;L@?c(rbh>sSpAc&}qzr-m3WZ9IldL!dHwIf8HVCR%v$dt3hyjn*otNk8(LF9GgUw{ zr59Ak-BZrS2OJDFjNO%rm%Gw+0bQwR>e!p9gBzeN2)u;t1|B4!ml-Z018;TS9t^U9 z7WAzMWp31ta1mI^|19F>cEO|+|0kN55x3^ z+m4>S{9G|n-QYOkyD<;%L7eHW1;2|nJc8TsJ|?T*D>#3gsplal^jqk2>OC6rTRepP z)Z|jBy)IR*K&5D?ABCv=#u^Uul8d{1Ge4}n3dA7e)i zm;VkCceiyXTlJC%ji-mNoSBZ#IZC=mS)>H@;)t>r?w;o;foFx=7^UEZ)Y9zBO`PvZ7*Z10S@i@SUtoZ8{C zG3(8E@L2U#kUKOqHYR%`Wp65 zp!?#=DeUX>B(X6k!Ea1dBp!>`O`vB5W97iv%llOS1!rIWLD7{47#+YQH~J~Y>(k6B z7kH3gWI6H-dhv0F>a)zqpJKFsnvU}vci}Vmef~R*r|?-kj~DpobNB{6kMH3NxP&iC z4PF!zUy{}MvaH2dWFx-H&uw3GcKPi*&d+f5dl`dKwr3cJk0M51r}*|Lv)l7b1@C9C zj(ihz@IJ1%2VZxxl;P;{2b_)Y{Q_P8gA9ZH_`a9dw&6$kLk2-eeZ%vlh^%8s{0aT1 zo*zFygby=77t2z71b<3r)#%>J5OsC>D0OmWJwOk0B|Kh`p!v_0@I*m^@Aw}<_%f4= zo9sSj;Li-4V}$}-W+fMOQebh^z zubMJmI-p$V8mNg=e(_G8n0G)?TJ4Ngm)m{!7(5kDu+Hr?9Zt=8+( z@=PFsgoOkW55Q~i5dDWZch_sLvzD?k^v~>_@tkw-J@?LJ^4CAV|4Bq^^fW^k=m8JA zJbcMRnTI_d9`f)gOF8;Vp~qRuP(`7Ag$@*|3VN^BGL8-NF}Vq z1D5Gmvengx#FKH=JualRr~4suVrvoO3Zf9y<^(W5p^I)pLBdJ-yrfbk!!(>7L4Oy* zh2F13jC3e)@tq-pIY;#}f?2Agsms*1U_j;+f!4ujV8Ol{;a?di`@=c!@Ccn4v=I^yKbOW)DIGOEj0GS$aQ5SLtg(yP;Bq zkH~2kiz5q6^hD6=1Vn;nhK~$SJV7@j5DjDz6ff{~FzLCgBlrt=C*koz3S@yqcwf!3 zK<*c6xnJmqJs-l3Q69fS)WQ5R=1_Vtzk)gBAIujh3;Qa42JGixcuR4-$cS#KzfoGf z(W9ig&?7~iOY|tE=94{2sQFZn;%Z*$QB2LJdnA4VAWn-Iuesn2O3*!6n>0mRC~6TQ zm1u#s@%s+lr2F&*_I^WYc7(O-__DaUdi8HSf+%3UnGNpN+r zp`Qf1AFi-_T=jIjukAzNd3WXy%6tcXv!^K^2PiWkN?u_~FH&tLP0Mr#gj_ZaF8oN9 zpRf>f%i%?OJwThJ6_y*|RS4yShiRsMj09e%yJNtvMrV88Ky!pWCs>WM!JA{j@o3;J kv|`wEEE>2p7AzuwQFpIs+h~7I@;*)D%O{Wj8JZ&TFY2>lLI3~& diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/ValueJoiner.class b/out/production/stream/org/apache/kafka/streaming/kstream/ValueJoiner.class deleted file mode 100644 index 409d1d1a22cba500380192f96c4e70c752f8c24d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 342 zcmZvYy$ZrW5QJy_i@)GwXyc(+h!IN*5ri17U&9$=ViL|o@YO7Q03S*`wGm^nF!RCe zGJAhqZve(hi0wIXzSt{mE~ z9NB%>u?fyh4yVPC9Q}dLA2u&$3C&HCYfmkL$Xx77j&ik3f>`Ons{&z=XkT#3-d>52 ucOi>YrWGFpQ!V^R@s%<=eyi6Agp&CbKovq672~VMl~6PDQ5NcN(s%;6_h4B7 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/ValueMapper.class b/out/production/stream/org/apache/kafka/streaming/kstream/ValueMapper.class deleted file mode 100644 index 3018114c2b2505c17d71fba4b0c3a303fc5de26a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 301 zcmZutOAY~15UlnagN0KF8!yDlh@};=AaCC$&zKJ;&y2*?EF8e0L}MeyqLZphSEah| zkLwM<2rUN<2Teki(=@pdI^B81XI2Rx0=3*k#?QQ7Q6jjjFgS3QTSI7c*LvXf)_Uj> zrV~9F7ae-|N1PV-ByDxft#4*wBGGNlNp2R>&e;B|8KHl)f#Q_?ol!AwV^*ik8s3M} kIQ~HO<&uowi)sWyNge=Dg-}LCcvV;lHJJ}`P=71#6GF{VzW@LL diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/Window.class b/out/production/stream/org/apache/kafka/streaming/kstream/Window.class deleted file mode 100644 index 3f62f607706c71e0527a8bc88d17b8b540ff5f0b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 627 zcma)4%TB{E5M1{an(`>GGuPyTFPxgH5+EVbRstziIGf-EH)$L>4)ARp_y9f%u_l5_ zX;g8r>|M?5&d&Pt>-_@&F5uLHBMXi#I3dsxna~7A&RnXPvYds_oF*(v7%jBoY$dXo z=1PXVD5RqIzprbVY5t}?H^3T^$eswe&V0#U7)=>c?jDys)ZWysa-|k2z13VXEtN;0 z?~6ENx>TINnX{NR1ILTNn;5ida2X+;fNkDaJd%dm&Px>UG&F9Kf>LeM3#(3z?$vlU z+QbD7xydH%e}4IzY3}399LKPIS*noVh!mv{9&ir%dTuO3@3y#~REwY$fxxitG6A}U zzFyv1OpgG87M?bKh;9d02wecYdthO92eU2cV{F&rK`q{ey+8auaw_Hl41X#degp7e Bu8sfz diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/WindowDef.class b/out/production/stream/org/apache/kafka/streaming/kstream/WindowDef.class deleted file mode 100644 index 79f3b9e0908b7502e35d3fa98c4b66fcf0ab5d95..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 386 zcmb7Au}%U}5S)dR15`r6M`$9v1{&`|MQAXgID++ZJdVe~JM(zxzft%Be3WtV3QeTY zVzZmc%w}dzAIEn9eY8T{gt#RHmUo)axx0Vb@GGk=+e{_i8Jk6fy0LrDHWfx!7=_Ce zzi_(ND(6+sYVV!qop4oNp6c1yW$`A`Mq)C{{@_GA_>VT8O`_R68p@A2o;?1rHC=3U z>ZADuZ1|f-XePzMrFv|(;-OVK8|hW=9F*|z=QN@rw7=^qebfkqinIw-0zwr5Ao~h6 R!NW6(x=eXB(ELKKKLLaLZF&Fz diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/FilteredIterator.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/FilteredIterator.class deleted file mode 100644 index b8f43246e1ebbdf2b0c56791c7b4be94a1022ef7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1639 zcmbtUT~8B16g|^!X=xWl!TNRhs#OK zq>;dM1~#t9a8w8x!%xDH}sm$gU}6h*b2%D!!>Wk*I_qs z6%G$LQmNO<^K#C2UxW*~+2y(Y=3Gnaj4??UO2CCFh3wzxv-#M z-EJ*0$lj{wyN|l-i*8WYi_Iu(tD&0iitA9m*+5UrY|NlZ)$h)V}#30T_Kp~!7)tyIF zky_;o≦#p>C`_)a^f1MPe2zUl^osV#A1^$QO*#fxF(?pll4r;HhQLUoz1xFLc+X zAT=q-awLU9u9sYY)dm8k^1Ns;o>b~{yK|`x*qh&dY(V`*D`}>5t5vGg8suB^A*j^iN5ET1}X6c!F+7!wSCoFON1DDp+u={dp`1NGpK z{^gt4_Kj|*a*P(`;wXvKIV;X1#dnB(L(`R2y7Lt{Fk73d7PHE)B&!oPj*=6-cqvM=jICNl>{{k&-*q#3zWxH%0)X z`e|BckmD%S?hsb8Pklz>=@y2*(l)4u)pQ@L)Gk)CBN#Dh#a_fs-WMv*i@!4ZiQUyy>D6%tjHiYQ13Aqu|0SL~Q6eF`IvK|*-hNbK iTf)o#4YFqt9`wy?bTpw`7d^!;9<}$iQss~$vi=1+qD_7P diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranch$KStreamBranchProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranch$KStreamBranchProcessor.class deleted file mode 100644 index 20b817da500211e3a4392e32163b433f1c616318..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2147 zcmbtV%}*0S6#va`>C&=DMa9o5YL$M`r3h3h7NrJ5sER4+K~KZ7z|ykYY_|vpZ~hZr zy>K?sARf5u#s9|`^-WuvG<>9Kv+2y6*EjF|-tWDc`T6(zA%F=y^P?9r9}RIod>Hj3 z0j&>XNDAF}A0A*r!K8vI2CuteRzhP8GdbH?*La!h8-`ZoYelYAT*u&BrnRmW>!M~_ zuHjg`RM8gmwLj+^Zs{9o@-t&vrki1y4R_-xQeg08?Nx(eAZJ>}%j(vO;Vko&5@`l= zw$4ih?wDfTPdMBIU^`l~ zHre_{0y%jG$kR7c>^0kY$DLJ%a9bD6LZpCUdTV_29zp@0441{QWb?NR{2ehOuMql{C zZ={K4i6(vkKa}y@Qgk@h4fmqCy{G5=&cpxq{`~dr2Y|;YrVz)9js)&#xT{0Os)n3~ zdm8R*c)*Z;X4}Hu>~YT%9z(X)@Lj?CPhD=C`#Y{<3eR&~27Ta)UCZRY^fKPm9k*lf z0XO%B(dBzxZg@eQWp|A3=+)R6N0b>-4Xa~wf9MK^C$mLYnzeGXRjv{xQL${xuQIIX zXJeELEr!^pvrFmc>Xt2@5BqK5HhH^8mifA4@?ML(mfVNNn7?mPtCh2KYT3SUZQk>Y ziSbdz*-7;K!5F`F4(^eOBrV(XNh<1MJzt1o4#rxNE}^PapUzFGaIUnr#?a440mibP z)NEl!Nf?xOl+>xzHtGlbEjN0c4$q7BfiV4YYl^LU+%2l!DGhIlJ=t@^8M>y}wgT-Z z6ni8KN#g?2Y0M$d@bV1j{DgtzCdhw=F2MzcjsFKTWd5BOLn(qceG+uGBVtZbJ6B>* z^YX9yaIQlWCE%$g&|i%ey5ACD(CC?$!qIw>u092q=(77r!_JW9GFe`cO`%mU9zrQT z{0jD&b}B_7@CB9-!)>Inj4Nczj=ZsetF)uIke;P?nMGO_aE(@mMO+WFY!f5+_8sc$ zB0EIv6M2lX$U1@5fc}>t>BpFvByQkl$TVn`k@4aeFo{!wqDQ#JBe*zj1;wi22$rZF U>Ag%%fnpRSX$8Sm8k-K~HxrJ!(EtDd diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter$1.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter$1.class deleted file mode 100644 index aff4288cf95310dce4443e24661e7578b965cdc2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 277 zcmbVHOHM*T5Udt-;7Q^OCt$&v2?;CW&Ol;}alr|ip@GL6X2{IQ;aqqC4`pn0<(BUB zr;_Tb^w;;tCx9954U9pbP_5#7a~w)yu`S+@OKZ4*=^u+R3N?08BTJcl6Z$#c$<3H8Ovde-B&^+8-A h%&#}7ML)_-!|zy!)AkJ4 z=2k;ns3w=<+>jy!&s`k#do-7ajMqa)h$E!@<%lyyHd^A6RA96&d|5RttRDJBFAE_ z(i*FHT(7<6^>(_W6?4PpUehJJXv$}puF9BY_!Ow}0TPwWVkkn==~ww zlz_7Wb$0qJ&9v+qYo^XUgL-EC40-6-)3fFExdm6X)ba7m* zQ|Xd_kx0&*Lh|H`Zvanb&Ck`NNF0W|wbS|lk%yOPQNKq#{h z>08StPHTj0fH9n>Rc)OWByfqOp)Sp3k{GUFn2-eKSA@w6Fq-*}&`*T@tuH(N3lRyQ zzhICwvJm?v2zOx!cZ01<8G^|oNk3EJcEks2yCq0m#kCfdQCz1}s72+_N>K5wZgi|} zlGQEJN?=I>1c|rFUkrE2MDVSWM(~Yfvio4i$M3s{bcjTVqMwrpk-FC=a-c~-5Ruy6 Z#*k`xP2(c%39^k+co%5DLo(w1{{h4XR5Abn diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFilter.class deleted file mode 100644 index bf165233db64b63977c3cd037e051686d7134cdc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1844 zcmbtU(M}Ul5IuKWSX!&ass$AV6}2tOQeu3d4L(3f42@tQP2}yeTwr0@ZFaXtAN(U9 zG?9>K;s^Lq#&fqVQY{taWoPc>&Y3f3=I)Q5U%vx*hH?snc%(m%(=bsmQA{I=l8Gr3 zPfR>DF)fgJ>3J$xaAg>(P$0j&76nQkEIDqZg4MvcRT%n#K&la_EytFT5}2;}!Hy*x z(%x0pzTDoIRv2eF-j223a;??Q{R)%UoE=X_&49_z&y`$VU#_gzEAtv;JLa^~j0D7n zz+lz!oM>KPws0=U%0{s+kX-P$$R=BJJoT!1u&IJ|x#=>JtNFHc>oRckxgAVKyAIW< zo~4lEQ8`b#q1B}&JIHrKzi)Kl7cYQ&LZP|y32~em3o_(*VI)1963iBg{RlNWTfrH( z_k+!)rLA{mvOF^*@ZwBNPE)GL=^F0{)4`n8nC`_e`*qZ7XZ`B(Ty0Ojm6j{H)+?KP z%8n}a9!ki2&_#+S7#$++;LQt0N=0r7y*|vd5%u(;FkUh^Z{a)PVFi zB;S<85e7anqs3y7cYw*bawhiE?aI2!?+B|550UA38`GlInFFOSFh0^T;-*?~qbJ}7 p*bp0G@;%(|kt{Tqe;HRLRyJPe1HRIR+W0Ida8|>xk)ykAZipbLwQ`t_2 diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap$KStreamFlatMapProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap$KStreamFlatMapProcessor.class deleted file mode 100644 index 0838cd1f3e95ae2eae6425417b94022bb92ce81f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2516 zcmbtWU2_vv7=F$s&9=#6V~ef$MU?>gXuC}XK~e&=Sgi@AkP;NHPQ%hJS#~p<4YXJO z4u65y&fs9ZaJ=>pIQ|vKQJ%B;fDD9*%|*}0+vmLR^L}pr`1jY}0Gz{jaZKXe1ef!1 zB=K1s7jQ9w&v8jdA%PfH61a?_TCV80s({xf5XY)o))KgmFLbQy_);Ji?AguKlECG% z=kJ=bA!~bZOm*DOw23NLA`NDRUf2C#a%Z1|NUWJUNIyv_E=#A zF8m*rbhYCml}DA|YNP7{$0~uW)mdp_&UJhxFj290T^Y1|nn|R)QOZ#j1 z8&m4^wT_a3GH&SDG;kAJO!$j@Z*Ux_PBd@}x0w|K6>J;00|p>B#60O1lYzVV#=s{y zWgvx9I=(e<4<`jGLns-Th<@jK;oz)0iq@Cjw9 zG0ekEI@3n<<%iVtJ=xr}_Jc6g_c&VI@N&G1e|c0AUGw;{Q|LF&u+G5se6v>{Ywd@$ zc>WE+82pg7A9(%~>EFj3zuZZF@VA|tJ$Z}enB)he~7UNen;#Oy1?eb zA;txMK#GS6L3#XAl+eV9=u;SKId+J*BKYyYzZ91!6qhOPKn6KvQRI3J^C7P+nbUR4u;+!X5<_|rYS;@`%f>EX{3pQbbc?H|nQ$ox&?Sqe1d7|&@e z5UQN5QWD`F%jTXzygqnyZA6b+42yI~1vds0OC4(M5XXp8gd!blc`Wm$;ZwZFeTt_^ O{#HA|eV%tD$Dacdu)RqD diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMap.class deleted file mode 100644 index 5acc280670797211d2dea1460baa87f22ff3e829..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1894 zcmb_dU2hUW6g{&*`4C%dm0Gp6T2!EkThRF6A}=*04b+-INPHaDDc!Oxo886OC*Mu{ z1^x+5G)*+|2l%6mclaz35X6w>&dkm|=iYPg?EL)o?FWFz*bO6qI}t45ZV)>W_>c(V zUJ&<#co0OAA$nk0!r3>u>k5})y;$}f!J7q>dxzY9>9lm=x-ExcsmX0y(BM(2<Iny*G!R{2abTU1eDeCogd|uGc3c)GdxMogVReaRb}w+w_a0LD<#7cFWSwT za4Iq^hSgF_=Vp~VhMe~n{oaW|LO+{@%&;g$i<_=GS{BkXK8k&2_wkPw0UpRaREgok zKL|4b`p@-}dWQ_l^|+;z!r5eMTDo?>ebqrDlXMZT%cf3b_VyT@nUb4Wh^h1w4})QB`^f(S(W7@dzr2Y1l4jWt?}cX#6|uF;JAdg$?yck6?9%LC@?G}^!o`j@-~LS@qbJAALw z>>U12H0$!p$bp>>pNw3xbonI9aD`Z6A>72R9;QlG?hd5CfJr!|lSUVA;{sd&o1O4| ZJyEtOIk|hA(ggV^2$7ZGJ`yql27_ThkbyL5~E(dZ9=LN zD5}WycKHJ^L_0x}phZZR)yd~FEvyi_vt{(chpFW_=i`d)NkV_mXC`zeF1J-E+;%Q; zSFIDIgmLA!hL4=@#FV@#*@UNos!aJ>P2r+=$2ORE?!5uw>&H3qW6L2(AY_j~V_7{G lzkA6eVf62u>>Gp+{ryojZRl0Ur*a()#imZG8d65qyaVJBRpW326g@p!#+Js35{yCF2%tDxY>yQj5+cPALGhoF4`6zV9`yPRH{F zj1||l{E8zx9gFI6e5{#6BQi5H2jM<#4-~>*p z)u%c>Gq8d?I#vzTaF_JhW5=M}2G($o)iQ7&pBwlBbpz}8Qb)tUS2!o|?buD{@P{XG z^C%I9i5lFh8`zE+-YQW2qjD6d2YO|OWQb%HIc|;ZizSpz)!H(}MG~!h9Nf>n3c#4x7j9h%S`d*AV)RNnpZOZ6vI~Ot!CapfW_EZ^_P@wG)Kt z{e}(34Tb4}{{B`+?WV_1n$o^@*oFHI)AP+CGb%kpM!b5+yl{m%4^b`gY>wE@<3gxa?&D)@yv03*zsww5AhJ`m zeH-r(n?k>5`R>qC-J@rj8CqT%TF%n)GS5#y`v++qSsl5*=`qhe4TTh#)p^>f3^ceC z;T+Ewo%HTh<8P0Q* Hk(&4)sb{p% diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValues.class deleted file mode 100644 index 64071b46f180db4cd15e9bdbe199f8f0d8871dfa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1812 zcmb_c>uwT36#iy`auHjsimghm7L`ll0{Y7(BnCs$fVV(M{AGr9O1CTv*H44G!zwFMtGOzuA8<1RNR!g*0>V8B&p8fjR4b>I!vX*j`9gvJnW8-0tr6I(F6S~0xZ>6W!(>xjSOs>z9B z-xW6RnSwSdT8(~`Th*`(x5n@=w<5Zh?=pnzqqmgVM$53o^U1I$?2aU0*ldk-Zg#nC z$oXtB>>e7VSA7{K!=hv@ZaS);x|DhPEW?NKoMkDQg5j&3W?7T!c|tLCR#oSG?WKG% z!*X11=_FVsmk*L%;3CLO(+|x=*`dQp?_D}3C`Lhy MtOO5{kO?S%0FRmpa{vGU diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamImpl.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamImpl.class deleted file mode 100644 index e65190974e329dd4b938f5ede5f50cb0ad2f8b12..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 11028 zcmcgyd3+ni75>(iy|SX%U`W6~IaBH*jz}Crh)lqC93p%L%Q$hM(2cCvie*ViGHGaO zq2cNcq(DncLm`w>dO!dhNJ&##deFPH^uF(w-t9mAgZ7)9-PP)Hb``h3*Y&WAJbSuZ}P;&~rFk1wdW&WC0AqKYs1@B+T9;(9N>qT;JQEXRH?z9x)* z-HUIC?{A8aZ+Y>eif;>;-%;_Bf<+rc{Ug1@R}FRz^eSlV->F@r1>#y_A`s4|V~L5N zg2g?_L?){xvZGpjCaS^YBnlZ#X<; zT3ysV+%?#<$xv~OZ1rpk^=~k`7H}%PEMBQ zE8d+lm3LRHwR5B|I5HYsPwllkVsYA4aK*ednIJc(qvNrNmW>8moLDzJp`E&vDXCzE z6VaTdBg8@xG@EyI;n+k%%g&@ZL2-8uT3XyVY6;O{N$k)}RzdZ&mQtWO0?-%Tt!IV- zEtO(CyRj~&la?r^pu-8r1-2BV-AfrRyAhGrgih>+PUn=f|~RJmch2(c*^ce8HfOtYrMyJ)l*wb1iwQb0Dm5dlhA?yA49F&Q}`vlB^0i7EC!5D?w95k~&9jmV-Vg z=GL@U1lUMY!IBaQL!9Avm*{thGZ%bLIk8Cy+j|zm_EF8HK%p8JC`US!=I&WGn>Q|Q zcP(5feoGC7k8m`#h6eI1D!AL-f^l?tlxL#ic=L8f;>KN5b+ceM)!A3-JX@k~y{)1d z$KK*Vl=nm8+RMD^D6Z7Lu%1ab#>7?BF~;xW0%?WF9DWSpK0o>~;Kv}=DmbAqpBUF? z3f9g;oN2z~O%-cMU)RRIu6B6^+U~~>@IyZ?L7yK#!jDDsCw}}CKV#yo#0FjC$xL|i zb9z{aU&Sx{_$7WNe&5WEmxRKr({`4JO3ZQ7BXs>*u&KLc-OWqhhNdwiZcc|}a6T}Po*!etGYs?BR-YOQDnPR*i|cTBopI2@02KgEw%(GcQ(0;{Dz5Wwkt6nN<`ApX}&bz zJ!*3c4`rt`hvpOGI|Qjo!ST(-16hw&IXcQ(2dUsTknhe0dRV^WP-o)D!?D8*3b~k$-I#6pVB@64a9jv#EZWO{tZRq^uhV z-4Zu!m9QTyW@$b>W_ioET@#5X6D+Fvv1Al$5lfEF*12F3%-PCsI=Umy+wMSDI<4`o z*P=vpS4e+97@^Ix?a|T`XGRF-g}#MCWt4?ui75py+C|K>LSeP6VDDN?T*}g@}p>`7Sf`JiF1@My=(dQT|qOB#Ys31 zYx#UWU&~iPWqX~i-N<&()?UJPr>%V=+ZWi{C$qiY*6v{YLRX)2PnEdjNH_@Xcb;Ec^qlPoQe~K`b6? ztDZx>=UFt2RkN=;Jd0zt97OGyK41>VK1wwbY`v~>fW8lLo-efl?XiINU>hzclqR%_ z1Azf_DZdHE04kog;R=p=EqkkQC7-UU2)f4|^ijgvO6c1b6#DC+8LaO&SnoDjZ!E+5 zabsXLspI8QYlR$ow3*=W5d4|Ym$|85hdE7!i&8I|p z&QC6b{1k!wRW6WUOXP1P@;8-5?u_%Kg*?TltC2RMr6{|cS_b)P0{Q7h$j^|;g}iw(LxyJV0hlp|%A3@6RZF4y1&?856MbY+xFw3drupSeWODg%AlSlcXK zeHdrY;WY&gG&Z*#dI~Lv(aQgW)oVOUJZ*>W$AGDlf_b@iU5&`4O*IFxoVmDNYFn-+ z+V&+LGuJlNFxiH0^e7Mg=Rmul#iLhddb&2v#Jih`w}uttQe2BS=oMz4l`Hl!GOp7pyqWPK5_E{Qzk={>^$PGx<#%cr)itFG5Nyp?f(vWfHTZkeFJX znA*1G1}R1X3a|jlvUN_OgsMxZO`fJ2$zvONTxk`oCN-~OEhP=W1KbTfh*fxqlswEN z_=rx{u*7B&lftc}Llo2$-io);2l4cFK8Zq_!aFz#qFAT!Ek=Dch~Zs`#x{QTZ_4w)$W!=}V)DiZ+oACcX`HS{AmuWOpIyztTiw z^uZtCk20QFph3|DLYTdC@7y!@oO|czukSwqJV7-F19wbhaCZeuxR*sSi~CtT5HN?* zD|Q3dSE0aGJqV9&*_Z8(vU~EVC+#Q>l{|6%W4kvp?WXhw>Q$1lE3i~^eK+0{cwAaE zx!e>m_JczuuvTXQZwDtW6*{uz(X(C;+R|&v(AE2~(}+86Bw)StS?<0kqeyXEo9ANJ zk5%YPFS1`Z5=xpuahnV>vfuYk1)i4bT{)DtCrS38)m80ych*y$bx{P0I8Zyf>KTYh zbX%bCC#Tc?8864#VI_v+=z?+$_t=;5AfybdB^}cKvh$9ygV_t_VL#Qg+60hzdqgxF z!60m_7p_)4KPAqlroz@d_uF0&aRT0`xDy;^@z6pEWeZs>Tgahe;XKv_p3N~k|LFoR z{#!SJ%{lOb}g${ebp3ZwB@3$zv=<^|h8<O@ z=`=oapJvVk<3ob6F*-NK(3lr+F~PNPiB<{)W)zZobnNn3>fgc5Pj>qb!!3*1;R(6EfR->1EvxOR9it+rAlccIK=9GlWx;(+;z2f5d17o zNF4Y9{3yiRxJZ=blw5W^Z)V<`+427R{o^NqcX%0M1=}HfYzKG|U`Jpr(MFeTft6JsPaE}day3vkm4l4PRxcmQ>{MFq{2S0OFLXhUV{Oiq6{vNLQT8a4MWKiv z_HsLlI&4*zxn;>lUyqC|Czf}! z+a%T8#AGTEU{~NnbMc1vW49ac%UgkD85Wi&-GkX7(>YjsHccSx=My_r$J+f8^?w%j z#$8+ut5^%L$6=RTJjwsxguubVr27Kz7cKVp_-1@QAJBSkLI?!(4rvAL>g(_DzD>jH zJY5*r!3NJ-m4Fa;>D{}9Y{IeDai2L`jCgoZCm9~mcl2FakJizP`Sgt{;fCH7H}IHc dGeYdVl6ZorQ>kZc4a7V~p5u-m;vs!t{SVU@8K(dM diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$KStreamJoinProcessor$1.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin$KStreamJoinProcessor$1.class deleted file mode 100644 index 44fb267b89630d81d7d76285152399ebe274cf52..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1689 zcmb_cZEq4m5PlY}Q0UoWt@T~8qViIEQfp$QsUJw2lv1lH!OsgUu;sWTcSmbK_@6Yf zG5WzD;EyuS9tCM^nv;|Zdoy#-JhL-9v%h}-_z7SQFJqX(W3@bqBZ8+hNFx(PHi~72 z_#rn_N78E8M+~{L?X(Quz!lQh3nc5gT8H9!YP{Eb%o2Y7ldD0o^UK~y2co1YD^ySt39ksOL`j&pVF7i z9fa<*6qzc6R21-L1ocU2@3x7b<=jKr+0WNm zq9|T4yqTDESh3xS8O91Snk4F9HAf7uFC3%ZaI%e&XVIo{{pb}k)hN~|u?zi&SIGJG z$`c9oXMn^8bI0&10QTHil<1_csXUT&l}6}l4J8t)VE}ql_b3bfXE6urJCdrThv}Wj z=1#z}%O?nB**Ap0kS&a>v_=RC6fsZhqVJu+H7t-uUOp$ob&_=-_xBrhCZSb~CNz2o zk^PELF8LjrhR>vr^Kbh6Nq;Z;h+oG|-12#K+{PW^Q;`O{IE%VVsC)D+g!>e0ASLn8 jw=Cfiy*0#jtKKMiQ z5BT6y*AfxCmipv_zrY{iU+}W`OaejDtgI*xnLX$1v-h`e-yDAX>&Kq}4CA;NJ(vyR z8s2AVj-~k)EMQSVx(Q1`G-8?GR}`!&_#lWDe8`#^{#oOlkAk?4^%gwD$3bl15lf%6 z;8Q&2-Om&}QSiAy*J8%8HMzfF>E^0!Wwk=VvIW$6)70!~Llz2JK_Gafn_Je=vfS4M zI@7!IP$mp%<`Q&RH*+b0#%4Eyl$&(WkNS~ zXHsA#Qoot22_0A$2uxdB6m)x9H?`%1{Y}kYlbeQH04pnvb!qGTUR(@tI0U0^6ydV; zpLYU9*pt5G?PeaX?+v?CASi!IArU0kJ^srqdh}A$SH49NkFF(~5K~n_}_P z z3GEUm@`T0NXimirbb)~yD9!5t8paxQuSCvFTyYk|Nyf3O;#J&Gk-(6OaqOuuuuoZ6 zVPac>r6P|nRM;pea8w-NP{k4M3M~DvU&vQKSE88O zn$P z6Nu<3e9^%Z_@B9ox9PXd-5y3WMo7Mk2(9(dAv%HqMCr*AyYwMOvPy=%y+L{kX4s;u zZ_(h{ipEY6h@GK9;A^x#N8?7Pa*C#(W6|&6KSQ$*zaY32eU6rm=-?^T6SOj(02og{ z-D`mq*_tFgQ|QHg3S=5@W6s4ED&j3+8)h1G5wszRJ4KXU1j7^oqr63yl4QY0_i9d| z=%oWA6s*t9(I4cDoz4=zwllN~JRqqxcJq65j5qW({EB#A!x>%@_zGdx;XuR>=p5wb zmj%wy<)@!+Kfd|<_kV|yMi5Kr#xjMrLX}y?3^Hyg!{nE|o~L;hqZlJ(y!H-BJ~}x; z1fL?v-4wtC?$HS@LX?u_o#gOX8l`lVPEOHDPyb!g=a7T&{fYj7g8R{8zdRtDA=>d# fYTt98Gw7pdKdtxBKMTFI(?ak3cb(o}3k3fGPpXTD diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoin.class deleted file mode 100644 index 101a28a07f4ef0f79b0cddba3caff16bfd36044b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2375 zcmbtW?M@Rx6g|_Hep$XMq9Q6PXer3H1r=$<9|#Fh!N8`5pVP9yLfMkt7W7ek3r!>> zVtfD}%6RW?i;HZUwlrzy?%lcPo;iDVroaFE_yu4FuOkR!B8mtmIeH$#R1^UyA*7-R zA{{~|gy|4w1O`?&tEMIoS1P*k!K{=t%c__Hv1P;1%!RVFEX@kxg+OpgH?}pp6+NM# zaZ)IvI|7BckB&NGjkZs^r{fyKmCnr&id#|r|1g~~SZ z>dh0`wc24(GgVnE+cH#2vRsg+&e!#1pt`490&}f&(v7NS8nSFDuIE8Z7|dS+{(RH` zU+EY%r>?R<4Xs$HjqB@kS$h^MtQ%HU8YPk^lNfJC_o(4J<-&nPs@v3#HPfnFL!g*w zhT-L=rcJ#HIhdFG@VKblw0EB?c8>W(DU|vqMl!Js~)NpymDTW7QDotzxq6yImyF~@3b8v z7~QPY%#yaG^BdpidQYc#4~(H7@fdF5WemOO6L{ak-sdUas0;eyg<#}Yw3&1anr4Jq zf!R(GXwCojddJrS=_Y92LJGWXQqy=Vy@hEULH-19FzeUwr^^!cV~Y`|TS9anSmOnZ z&*#kbr`O(u69A`67?k2@P4A9vNQh$)*+Z5Gk^wZdzJcBPeaSDHO@)Y75rTn-< zzj3=@jDDXI^fUAzfxDC+amp}+dvrwrb~|B5+|Y+%in$Hn8^V1WIqD}0zB-u)MC&0T z37g6?_2ByMa|E`M$?x!=q3bIp93nyc=A_*`=1i;soOwd!j(nqiVT_Ry4r7HzhVCM=f?<8OWDe6|7i<>^$1?fr eFbRy?vy;Rdvz`(SsL)wh%Wi7Od-e*+`;0au%=I?sv{D%Vsh;$p aP?PW8Ru_kK0UCkG zT{2x;=8HyVwfu5P7SePwZP`|7R$w~r*~>syATZ}_m;xP(mTfNY@2#8enq1FwC%Wk5 zWIiihOUdV(0R>&4vkqm&b#i90=(qy*}ECu@Gi#zf_8hObQy<6WgbEWj<9Yw5v*F07%UQ!om zIgt5%QmonL;fl0K-B2?rWX$8NG9(uPgZiD*s)YvapybY>F^z;1k(TENnpW~b*-gq{V_|bLaWZcn=|JvbvQce zeKM>HtfK=F9aqq3EJ88eZy{!z+Pg6DPe|UmdS8uj3ASblgRchBrFiqFW%- zgp-D>*3s5-6PWSh?HBtgFZR5=Vl1mauF5>&(F2PibL!TKrAF8&w)Z|AXr@7`bs_eG z>9Q44P2j$0#4}IhJTHXF<}Q#%*M^pJ_$@dp{pG-)-&Kxl)QUA%F(mZyKS2MzA+M@U z$32wp2JZ92rt|wJA`#W@T>0Ts-4B7Ql;$blb z;TmpGx_M#e7A1k(=%y#-^8;f9IL8tv@O?%gIe3N^KfWQj%pGm$1g(79&JgmW9<7~L zzyP91pclz<)I`P4c~p8-r>u-5rj8opMftnOEd6Cnb$x}YqW_?#|FEXtOZ^z%`|v0N zve8c+g*?k0g(8?po?kdTq#7-WK}tRhp^I~r Pt1$nCT<6?J8IjhX6YOWX diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMap.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMap.class deleted file mode 100644 index 1fc74a83fb8dc2486b0804f93f312aa6e759d7e6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1806 zcmbVM>yFYu6#k}M3Wy8tD!N|Q)kOt1v0F46h45$1CZ>2vASC`A>MYx(6w(%TpT(EZ zM6-z|K7bEpJi}GAlCo*qnK?b@`_B31%>4ZI?FWE|*i9maTPehGTg6Ta5#&|eQE^wr zJrxCp^mE4%?!L`EPk0O)M#Fan@4euI*KXewp4WF7;ypeX2%0>o_uY=h2i!aoT9+So zx#o@hEvKV($5YJ^CoOId1z}K7g(2CnIu7@Tu3+$2N2oRpz0z#yl~qD1s7yiSN{t?> zplUgmUt@SwSOsNLLb1gV-S5Ap{4;gS5if_mws4!gZIfla-Z#13;;tpv6JyjrvPjM+ zVaP1U7p}u?Pn(NRDd!e(PX<$L!3kAHpwvsL|J&5c2V(tLHT0+TWB!h7Hm4)-s(ma> zztRda8s#9*AY<)c#1++fCJv=QjsDOz#epT;G&66m`!eA)&LN#f21N!V#HkLFE<;H5#RNtn>v;!YQLN#&DZ|z{RL2nSNv<$`%zTy|<}MP>g~E MjS@UULZ+bn1{6&L8UO$Q diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$1.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$1.class deleted file mode 100644 index 43aff8e6ec498170eaa55bab47204136b1113a0b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 286 zcmbtPv5Epg41H10)%Cz_>@BPv1Y5BX4-OFv!9K20*BxhNX7%GN?gQ?l#HgLE3CT;~ zy*%=NyZ!+fq7@;E&?Lmm;_OrT$*d4Mvt{Xpk8?h)*q$Zy4}4)ld+JhKgv{;cQtpdQ zgqScc{Lb)+)4iCSw>g{e6f4W*uhC?#6z|vu^M1KEAbkHiCw^i%1PO%XVP-6=ez87C iS`tPdc_dvBD*Epus+!QPhA+h$>e?Gxv2sWeN#hRn;Za8b diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$KStreamMapProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues$KStreamMapProcessor.class deleted file mode 100644 index f7b6100cad887e566bb7331925fd97848fd3058e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2089 zcmbtWTT|0O6#h0XO)pU@2qNB4EA#>lMG&=61ZJ3FsHjt>?@L)Av?MiYxjg$X{1?vX zAU^m5{85f)Q%GT`4AVR`d(P%N=X~d~oBsLx>ot$YR3MWsl<_yzl6d0D0K7FOi48dhNhST=*UYMR1cu5rf^1S$rYSjr$8Xx3QMX zZorbo`>?quDvkWr9a-Q&RUFG|UV$_89LSzBgiUei zwwKx=X~>0`xXTnRk&>}v@`zoRHs9$RxG-mj~WcK*U>w>DJ)y}+O*s< zB$hMRu>8LvgSu_mhuq%62z_88^bun5xEx!O0#M`>gW)!5E2Js3M$?%eP|_y|d~-3q z^b8ROc!4-QHCK+}4(^gRaA{|dG=_T^B2UuiSBep&bvXSU%1;ExPtl`bowQJ9;uLBC z?HIQS3na-dh5JamvC=K;vsmPZ-VD3(hw*?U8AZcGk^(N1HeNDm;U9VU#~%I(;YaAb zf~S;{hUbKle9N>WS%lJ=6R!oDpAc1OlsvR+5+amjavp7r@{>_h T=%=-hq%i#fkn1zj5fA?Zb45XJ diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValues.class deleted file mode 100644 index 38692d6f8c4c0526825fdfd310ae8d4ef1236c45..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1710 zcmbtU%Wl&^6g}g-Luf-wLwS`_0wk$~lfbG;!=k7XO3FitEU}!JY3jzYgY6(#Bk?^T zfhvK-2k=pdJI<>u!AjJHJu}z$o^$5R_5A+x;}?J@*iImVyGcZ`6~nzGLdeE&KZXY} zJd7d7ka})e!r3#q>k5})wchj`!Fvb1|H`p-;kvfN5bg1PU(lr3u$_*|`&>T~YL_2& zx$64)hSgEK!>Ouqb07$SDk=<#rqQvuH*f^Q&VuFDmR72?v~s0JK+&pU8D5Ryac)7F zN?v0K@7Zrj!g9l~#EU_%Eu0o_n-p1T*g7{g?ig}C3WmKSgY0-Z50hbe!m+sNs#EiP z4_s26Ig9?WKV~?%7<9E&m)1<}*{8#_CD?7!X9Ek)jFzj9km?3@MONLSa zhpu?O;2;C*vq&qbVh8R67@(^jfOxLY~CAwhg zjzOB{S18h^QK2zeIDt~w{tot)Rw1f_p9|bV7`KtY239GS8D`@u)@Vg_BlOIfZ`UT@ zmL|;WG`o%)^p?E)gvw<7PlVnS*a^a4Xf@=O%7HDPPb!xzLq5qejSx#Zj+>-#h^f*j tvm=FXU=mL8w=slUKLr<|4N3W-ktrLv<7aQuHbFHC;xtO|5E+?-@)vi7<6-~+ diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough$KStreamPassThroughProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough$KStreamPassThroughProcessor.class deleted file mode 100644 index e58ca9119b999cd9eef54dcac4dbb17702d1e22c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1842 zcmbtUZBNrs6n^erS*bEn5kz?tWo$_C1yP0=oH0=rCE2ns#t(O;Y-OdLw=K%gGSLwI z;1BRe8P9F^f|)jkWy$G1r_Y}AoR{98zrOzf@DNWj7{RiF4D#|+7{v}nqcX5qjr|gJ^#@k$P8d{4Vw7BMa!r(`y-OyU$Q8R7N5H`15ZMPD<+~clW zZ3@S4G>b&GW!k2<&F~^OG~05GAyIPn4TiC@X&Z0+qq-rgylzosyzJ=Qs&QdTzY|Qz zQW&NOrR@nvH(b{d4C*V}HbluH-3Ec*536%AA{%vN;!mL`?XK9PVH0mvcZ=0paeFmF zKo>(o((SMZ3`@E4AwTAt#c8l_>xYK!6$c}-{qdG@B6~dmXGk7%%cnuaNrkjEvYCq6 zu({_8if3}6JV}$(EytDEob)nn-_|{!dlX6JUI$~ATf?v!xilU9j?jA3EIG7}pS+^v z_^`FRIViODm)Q3CI^SKPGd)!*jxTiMr72hP+Ta{kW#d#R7*(NSUZqcSJ&i{yHZZAT zPDU1ROT{8?r}3EK-H=~k)GjhSJr{eoY}z0_Y&XM3R0kvG#IPQr;opCx+y{>M$i+Ux zvj~R&xuq~qH)4jaNtVsZ$_zhS5ThTUCNGXLvMIsASJ-%=ZseIuS>2vXsqOkBgSPB}D P!)1CZlwkz-Fb?(`HGIm~ diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamPassThrough.class deleted file mode 100644 index e5b8f0d7f7d40e513b4d1fe8350cad04118bf435..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1069 zcmbtTTTdE66#ixxSV3CtMX^#Z4}E|>bkn9!LNG?!gg}%8vWe-_u#PaY&XU>1#D7(v znrPw=@JAWXf|AlDq=tt*w>k5jGvC>B|8R2$;5&9QNMggnhYSpCTF6`YXkm*XUvGt4 z@QWr7f_6{)!)~vseOCm5uNks6rG!58=uiXsqK_1RAfo2AmqxW_k1f9=K&9%39~2E(fxEwb`=kYngINA zeXnuOFS+e;dggKGT)1J$nGvhR(_UD*D$b~SrsWT{D}Kso9=SM+U!rcZNF&4WYr%

@Qe?VfA177{1N{oiP(4hM`)RN9TY2e?(B(b@qyy0(p1WM#)mc#ReER;MCw$fF&?JCb8abCjLFg>gc#=5I>Zk(ox?AwPk@l0vd=c;s5{u diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamProcessor.class deleted file mode 100644 index 8ff0ae3e677eb9a71fd79a76d8f18c5b4743a6d3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1205 zcmbu8&2G~`6otS2z)S2|(D9^<5^5%POguHQ%V4@;Aid9$ z;Z-|~1`Z!{_ri0A{Ns>2v5Y)E62ZV3N1^M*aTq!0lj*7yU1NZ>?wPk1g=cDB(`p!Dap#B>WUF*?2KW}^~0EI*=QXtJM68{n#N>?;)|T&K}U#cfomu4Ef<12<{5zMyOXI>Rk&(w^J2 zqTvozTd2~g@aLpx^fvYnzCoMutyHa%nVqypHQSZ>=*-^)rR6z1#3KS}c>D*9r5L6F diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$1.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$1.class deleted file mode 100644 index db5cdd0b74cd9701ece8851be0fb25043d37ec49..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 277 zcmbVHyKVwO3>@=1xFF@k7f?{MLZU)+5lASCC@4~!!@zONF4{fd-zfNid{h*ZRB5qg zk1fr3A(svvy)EZoLlhX=ke;dSP6>to?|A5!i8 zCP7M=`mi&6;(RZrSRu9D!`M)U9NU_|IHoRjcoIYxp&l_ZY5Ni77BkG#at%0Uu11;@styDRrh^+ksf(A|e diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$KStreamWindowProcessor.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow$KStreamWindowProcessor.class deleted file mode 100644 index baa50236e1bda0ae8eab8a8e0425829b05c5fd7b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2752 zcmbtW+jA3D82_EzwoMlri$)PIQ3K7bP0|*TCM`;_NE-^67)118n{L}B>279|(DLBl z<1;fd7$0y(-gxJq@j?Ba-CU@c7@JAvobNl|{hGY^^VeqpmheLmr;(fFkq=@TEAm(k z;tD>W!WXzI32RfhhE)lDIfZpxm*`g#x*?$r1)B;A0)a!r+_4S?k_F4&OKWwlvahGB z+HO@#H=4Gt9T?_by6S@IyAE?sV6tfJnOd`D>jI0TOXf?Pxl%c|N)3VLzR`$g1oESZ z8)j3tO|8~QZx$V|dY%OGhG{fc1+K)#Q4}u=_|~i)#xq?oOns|$u&vuAZM(*onSxc( zYGutfB;I-S$p8ex#~iwCTNS<0uxtVKhH2{dT1{&-bn+F)@GkoAHkifmanByryWZU& z!(6vY^8?NLWX=Vaho-#V%|*JGl{JfXJZk1-wv~#D2wWQG>kuu}(V+^=#tIL#hg!O( zv0ZO%KhP`9-0LsWuD7Z`k|lkOUBLfPtF>5fFWXd}WuB*sO|4SBsnwmbJ7aTG!4^j# z<~Fm`>95qRh7^p&%L;A@{5Ui--EkV!)x+}fAF|P`lU&acC(Cn`6<8flnZWfEDd?5F z+e}PluxPdHioR|*Gj*)F=4CZhgz>hDH}RH)7F2wMj}_cjaR*;3D5@yo8xTm6URsC8`te?`YzieHG<EJ=#SqmFH>n-lg*hq_N!`YC#mVKz%JBaXCe40P} z9)8&V;QDf~Gog^27Q%i~I6S1hLu}pQ1cHg=6L=C&;eE_kFV6A|kOxTOU7o4~2XPJ` z5c}}hON5w^6WGydcXUP)$*1ro5vzXoisxh-L4h`={E%2O)rRWD z-1DChT=va9KkNJLG39+Yj~HcRNIAF9$wb)EcbU{>TFB9hJknTke2g$#wAJ%5gIP+- z%w-TKo#*thRQoNlgKEq{V9;Wks-ZO+w^DpUpycdZhH1Rh{6w(L) diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindow.class deleted file mode 100644 index 04ee90712000265fa93bb5bf9be19a2484c0516d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1855 zcmbtUZEq4m5PtTK*Aok(qSlwTiV9RYFve&hB&JOh1Mwvgj31Zlm0qEDX6@+j%jC%@X+z%rYMmCIG z80!qN7pAG(n_ccWy2FsF)?HiY2X75?$2xp%Tbk}Tmdy}799XvWU50$kvfC=}acxgm zJAAjpRc8=om~FM=sj7!S{v`^o8*P)jeOqUEvQTijQ7tu^r3!(B%7$sU6^3GFA&648 z$q?AIb|`tgW|;cR{y|H(8@$yeOR{EZyxZiqA@)OKz}+*5jPg8r4Aa$ZlXo4}M;#TM z@3xbb7`k)yKka-LZd>|5IjwSaqqfiAbG6H9*k85wbfD{X!)`NQt%9gwjHR5<;PpG&I(#OR`NClHIu3P`UKd z3uhdEfp>aYol3Qg9j|-sg*VPP{UMI;+0BMDfw)j-yx86I@t)^>-j8$6yN~~V_z1u` zm?9)B1mMNh9xNguU`aqyL^qcC>zaTS5q_)+xGqBE-w*s)jSSFD^XX zeqyi5Ws7T@Cm05wHejdX7_s!s?3;r4#s3q#1_LXGo4TqoeAAJN6aJQ9xJsFfT&gKc zq=b?#k-B^2du-7G2w!?`0Ya}tG?_iXR0W;p8xq?s&3AJ14DaK*}%ev44^h@ZPJ zZ6$Whje@qs=AI$00gkh3n`^bNDMo>^bLLo}*X3$8T{QGsp-9-0!_hXPx1p-3Z>h8U z7u~3A$pky4Cat5%)%qq!PtlQ<&P>u1wbmbRi*Y~NETa63LDK(Ci@TEK>c5#$&yySV$?m^()l zM`pNi90;E6oKIeY0xCi-9S5W-F*#O&;Z*BYSg#h2$O3L5g|}(El`RiSNWfWnjU^65;ZneYfiKd-guc!F!I_`pkoeowld2%Gwj>9lN(mv`YaK(dbE=7X~N_!+Qo#+TU@7yWP!o@$S}sJXV0CXIh)TotqfzrX2br1Kze{D zqxaGE3p@-fW52=cbYk8;bZ3KJV(N=^^LBXrX9&FK+k-#*0D+jW2ay^*1HHTG`w{)S zII)Z8b}&IFv9ldUr%5qTl#4|aQh#bDM$y;G@O46R$kfuPSGCXUj51ovme zpwMkC2>Mf5Z!loZ>_N&}>qK(&J+jjFag@9T@})?A93l@6lMkchxj4q@?=}pbyz?c_ z;#-R9Lz?-W{+oFmDz>K@ad?6VN%1&5A;~CtRKy>6htNEP`^Q70jNBrPdWUMT?*x55D-7EKf(DuGf(WsxdK-A$Y!oyM^(+bLzq zhTj7c6%q^9h#!TxW4nSXA}rD@GS}C4zBy;+dVc=;_8q`eY-f=|C5JTbtLTA-s)iba z=37p%KN&NyHbcr5rvXE{$E}HA$hU1~_=s4cc;4U|hgfZf8N8IoOPw=s{ zhsG!h4QU6$v$^FPZ=^kN-gPIg>v%yWL^c?*T{*OQF!6~0YHncjpx0;-b)hM38MGMI ztL@G)KjDVON!089u`q+iUad#ocAWvOx!jSqAj5s(9q_(InPSH=xz*#IRL^lT6&y*j zoIZ(mqBspDD}fvfKj34Rc5jcNbP@F4J%=SpB}1m#rZLo#cu=VxRS>b^gI@zY1m?TKJTZ*@N5QRhHC25>hvCjZMidp_@D0^Wx7?; zbk!tKP$t8OY=dr*FkVEG#w9FM#R^3q;`)NVUH*!M&d$K}jWZ-a)5|15fFhLu7U3j; z#x%{Q9H58!0#+%je$j#~jMi|K7Erko)`@eC;!>Ph#UeeacWby#xk%WJl?*m;lY}Z+ z4Vt6oYx>q9Df&dAas(2g78r&CBA|v_u>cLXl~V%U!QB{S(5Pml^)LUz=lhRFlF(WRd+1RvL`) diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueList.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueList.class deleted file mode 100644 index fe731f4743b33af26378337acc749a218399dc8c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3042 zcmb_eZBrXn6n-wdn~)G_8f%fF2!a|A2_jle5NUa9Xxe^&NvDo8&LwPNOWtO8gVR6Y zPqAP8pffsj?2I3&<4^FnIF8TRO&}2Zp(exR+}*SHKF>MNIcJx@{{8(A0E>7KM*_JR z2CyXND>2Jati)mAmYCaOR-;&pBZiGAHlw(sU>?|dPr)kca>G_gQ{B%ThL;(q>f3Hrm%YqGr@B}Be!Eex*Ia)pfXpex zww?W|_8TtoH@gO0+9~FiiE8@VDKqdr$Mv5oM3$VY<1Z^1>FFY$uGaQQcctJ|?fZ?& zuI=vV-7-C?Lan6BMeRCr-*lS(fx~QLLBs{e^A$F_4peX$=k@c1PwOqxyFFoO$zkX{ zmT@?pFFeslI#bpx(1YD4cFE6ei6Bw0T=S`fg3431qp;km&fS#3cPh5$>q?!QxA;4v zs#INJaNF0V!+W|O=*H$!=uhYQD4?(&JFZB#mY(i9HIn6f0Mk-uz_ly2BYPtZFzR^g zmAZ5do8=pYp#$x$NgDTE4T>CYb6=h(7$PM2qXYx%yBmVBv2TNUx z5oRGxp9;D?2(!hjkX|ZgJL|aI=~9Sq*BWk#Is~I|Zd~Uir3Bu@^#lfyOyCVM*N}{Y z4YShidq9xK6WGGt1PZvP(6uTuUuPm*J)PZ_PZ;mB8@r7>d>T|~ zZ)$#iLE+DI*J|XtN9f7ap5-;K@Iw#s>Pb}Vy$Ga;34T)({Ms7)aYmqUl~;X8(Q;hQ z-E(rXweRH@3%aoxL%t%hMG~op-+q<#{7kT18>o$VA77#zRfT8I~c+kM(9a| zXK{mm(BJMEp7wipCL9gVo4Zc z4L&emHeEqHW)RO9qC}b?|9i}Kk{s{j1AK^Cr12%DgV+}NKL**FvB}j0cT28Wo|s}( zawX>s;>_`Mk|!jG@jz8MDLI7u)7&5KN^#dTy>S@-U_6Q$E7jEgBcjjZ<7U3McqeR< zW8crb2&Ie!&e=BktPc6aBh+C@=XfG>TCfJ;1$I(kPcSA6f5wmK4=X2*=jf+|Qom5I zuzvGY?Gx5wfjTU<1u)qWfOsST$wN diff --git a/out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueListIterator.class b/out/production/stream/org/apache/kafka/streaming/kstream/internals/WindowSupport$ValueListIterator.class deleted file mode 100644 index 70b218c7c8958b7f6c977441250012b1a77649eb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2126 zcmb_e+fvg|6kVrn8fXi&fOx+n^rECHUWRrU6wtw`pcdUKKrC=?X|j3+FyUZ{QxkDn=$kt*9#dJV~FFDjENW` z$i=9l$ha)yij1oaVbif>OfpCn-e{@}i3MF(t=R^*ZPjMDT{5gyg`2##rYd#*s?HVL zu~fdH>8ncJOBGFbR7>X#TX~}CE5_?`(=-js8Fe8G483J-Rp(CABK*670gKC(!Ze{q zi<+)E(+u59bT*7=Vr@K-QAo^X98;xj%|u@b|-t`01#V_L@jm0*rbsl4Wwz9px5~(R~np- z!%}r!tvN*grdXNku1|NDyh_v28cJ(=ih|l?VVzEWyutStr`0KR!ucUlE4|9m2rb1aJTJ~&XC-1z;lxhpIhqi zh+!t^TZzo!wVG<%qxpQE;cf6qFC56+PW`_y`c7ne>C@|$E{JcH;s9kKBq32uVUY46 z%BmDY;+gCYST^?=p^vl*W0>y!6oCiyWATuX9^nwJM!csf9-)=MKI)<$>)1QKx0l>w zK0(<@oT9tnU8X2{>&t#ZXcuzi9U|fPREW6LQFpWAQYO3yeM&*JUlBy7>5>qN&j>`C zMqk;suWU##lybX>XLDZ>CWW~Vdpw@R=k)m(WXj7ni9K8CZcOiI=+7W_#z9aH<2hvZ4%+ZM4*CA;j{-I_aQ3z~BQj9hR_!g|M61Y+BP+ zt5XUa29VQ5W_1G&gI`;0xDij;5^3KFph$X2^B>ZuL(pJv}!{aNR~`fx2LOGy{^|) zToCARS7wd6Q*v}$b1YlMq(D3f3q%WsX*flJ@NjlPK$*5y2z7kcF!i?@o6EXAuPs*@ zIWcRMwd#Uq8}iu?D$bfw7bwhHb|tUXwDOvs-_TY!w0zyMb#2oyEBOs?nm0^JFtutu z|BhjE-B0b%e{Vrt7xM<+vQaf++f>S|qg7`@^p zEV%R7oTHUD-qdQoq$t5kCO3Il#87JuA6}L4m$HmS8FP2Jp^^+K43Q}&_bg7yNZX2D z*O_nCxJCmm%Z1YHx^_j&S2ZR!=h)0|{BZ7NYje{w^ZREt$Kcb<0hT?UZGm}vd0j6{ z8=_U+tT<~-&#`JoSz3OGjF;RZ$`&|#6nb9va${AX`_VgX*7oo9{1H$K^Yi0HrV*br zDyHT%Y+9N!cVpM|2gTAW=SoOn6nyoS-)tw<@!jGot8!>kwQlj&*C{1Q%Ou? zM#bw%WN}eo?C1q7kUF>~FGGO~N5>oTyZP`H{Og|bAX?jAVCKInl5mgYI@hc_`X(=2 zvu&+ewrWj*-07nd|FPmPWTfoZM2xgT04?-*8jd1@Cqb)JfqGxIr2RPvNRdlp|#Xy^Qs8Mp*PA z!%@!q8ROfJdk6RlVUSS@{Sy&I{LT|`?s?jmwZ+r&d?q{*F7~VU(V0ob-p%#(FO7aM zu@$DG-!flwy;ae5V%{LidsK3RTsH$n3e%Q~!V)o1lnE5|Qc-WKB1KS97qOnfkgw=A zqHf+pDOLKkgFe@x!A^v3etdVHmR+Y+dk^-0O~oEpc8f?`)UZwNTg>vK7G?c`S3`la zAzvBDe&AIcp+6w4$`Fp7?+e@Zts00H`*)C*9%VYP_x{~^T6T?QQPV^9aY-_rPgzEv zQPyoD-U%dyXvMx+1A(N0K+-^qS+1mze|yhk*q3yfPD!tO=+VgT4^Wm0oP^~xqK@n? zNiklbw!8c8u?{|GZGGX281&8t)I9-p52-Ue8Rv^G$5D2`9dboErZYds#v-}CbR?W1 zO$?+Ix?HrFI{x>bB5xW2Y diff --git a/out/production/stream/org/apache/kafka/streaming/processor/Processor.class b/out/production/stream/org/apache/kafka/streaming/processor/Processor.class deleted file mode 100644 index 9f5410c4dd3bd86614e4632dff8aad250debec55..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 424 zcmb7=O-{ow5QU$MX-I)mxCCZXZ&)=I8zdIA5=gDGH{z;%qE90Ea?M!q0-R zVK?uanWuU4_Wtq;;2b9jCJBxixOFx%%*4`%LaIvX2P3!Yeye01gHeyx6|xFmo4WQv z-hNk?-bM2qv$LEb`RX&AiseQ49nq@!d@<`==6|q@{usk(<(r_*l`Y9}{1a!|p3N}*-xC-nzvB)W l7zXrXfOEPToC?Z-A@PLh0DGiUj5@xLgN}W64l(Z3(Fev4av%Tz diff --git a/out/production/stream/org/apache/kafka/streaming/processor/ProcessorContext.class b/out/production/stream/org/apache/kafka/streaming/processor/ProcessorContext.class deleted file mode 100644 index ffd5feea808d13307eb67265024eb61f9a2b58cc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1281 zcmbVMS#Q%o5S}e*oEAzCT8?s`by^8O5J2FnFO*6>5C~U!;BD)U%_g?jT5rnn(|F(q z@S_m3o6AZf2z?sO?0nyRvopKDe*gFh01eo3pzgpXgBfwk;DTGwRzO$UMSO4W?s?D<64k6MEMNn0ug6icN8*sc@B#0%^hM{JmX0R zhr|>idAryzYy`fTn!Su8@>k~qc7-ilVP!%FwL)`^yMi<=5p|27aZgBZPb5gyoD{VZ zVZmVQFLirK-n(ux;c6`@GJcZKJ<@FFYrH>cVW|C5)n3G4o^2n#9zSVLi0!z?hyHQX_T)~*zi$dO{=0yMRn0RI z+76CbX0^;k$Y8a<9Oc`i7n5_Pg?UK~sxpcS>>>~7NQ;yv!qc3=d>}h2M6*>4);{!S z{6=OPzv`!E>li#Ad$E&7F$$)=wM%Ptb?>*zfPM%1EC5z2)Cg1n$j(BQ;toY;pho`O zh(8Y(2j`dQd||}D42za$36`y0fh*Ro!kV>L;hMGA;fA&AaC5+Si}=>z_Q1bG{$03d Y^X|g~YajM>K7z;Ae_|{lHmqXz538hRlmGw# diff --git a/out/production/stream/org/apache/kafka/streaming/processor/ProcessorDef.class b/out/production/stream/org/apache/kafka/streaming/processor/ProcessorDef.class deleted file mode 100644 index aace23453971c6ee99d1d829031774839b22bfd9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 214 zcmb7;Jr2S!6odUrX$lkE0u~|=8!KW$0*MnmqSw+Nnkp$Aj)4PkC`3BI&SF`zKHIPN z;|XAmk%Cf432D*zjvFI{)oi}EF4x?$xrnYft62Na315pQ*R8WgythstzivYqpX|*U x*%ftYnEsNjWQ&sTOj!O`89>6MM|I6%^>8l5bP0hFh35dpgaP8-VHh+*@&TnWJ{tf4 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/RestoreFunc.class b/out/production/stream/org/apache/kafka/streaming/processor/RestoreFunc.class deleted file mode 100644 index 37214223ff9c8f8d4978464a5c4d2d3c0666c77d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 166 zcmYk0Jr06E6ols?uxjNov~evw0R^pzCN@;h%@g^N-Ry#iSF`W{9?A%v&EzZQo6P6? zdIQ*Fp`aAzgoK@Ip9HlovLf4@3Coimypgv-{gs-Ue)HC1FztS@xDkoLhb72={RXAOIAo!I`)6ej;P j+rybnXe%f*Bs>sBVv9%`$%OU?<1cp56@>(n+CcvW4*5AL diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TimestampExtractor.class b/out/production/stream/org/apache/kafka/streaming/processor/TimestampExtractor.class deleted file mode 100644 index 8936a8a33d0f0c9a3750f3147d100e78c2d67afd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 232 zcmZusI|{-;6r2~Mt5$YiK^wObM69(C!6IJZxvVkY4ZEv&HVY5np~Qp~3fc_A!2F)C z`vbrn0|6n>CkTD>mX#+gC%crdEORbn8F`zO@#;(2A5*RTA7#2F3?p-~N^g=}6UK+6 z(9UyFt=nv4XH7C;VQeh9Vs+Ls<5R}cd8@got7K)3(#{zxf9-_PXIeX?M<4|C7l1-S M2Vuj|tua9F4Z1BvdjJ3c diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$1.class b/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$1.class deleted file mode 100644 index f0e7db12b837c0a7b1c8322a889de241baf5c0de..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 267 zcmbVHF>V4u3>*XD$bpnActDEOtdI~700k*UL=iq<7IPeI&ZpfYQofcV56GhsrlpKW z_Q=wBtoghA`~sL^m?6vXNXS3*9KEcnAcXNo8x2~1FV0oiNt_({iwRG!wGW!Ax_p=R zpt}q?VXm=s+^|20JMsR+F11nk(^RgB>Ln$O?#}RKN*KM>wZvuMlq3;~+YcAm&eq$b i_?9sHk9F}3LdU}0s%wC4Jv3T&P_wcCK}w|n diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$NodeFactory.class b/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$NodeFactory.class deleted file mode 100644 index 4e7029f1c2e03e7064d822bcf8f8708fb93755f2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 346 zcmbV|!Ab)`42J*Ic2~C*JopG6^x|B4^CA`sf~cS`kWS;e?KoL7+d^N>gAd?CiL-(S zJ&Bk6Nl5-LDi51?YL$CEN;zDYj&wK;g=@Xt~K?N zlA3E`Cl)y*cmLhH)umhzst4Vemw6L}aQvvP2Ca8jy}#J`inm;1l1YEU{8~Th8FTiX z+Kw>(QBAI66q^N3NfP0xC%M4+>bu2Cd`Fo62U!4wLH<_OAz=^0oDDGgiUq2yRh}S} HI4G)5Qvqkp diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$ProcessorNodeFactory.class b/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$ProcessorNodeFactory.class deleted file mode 100644 index 966db047a7fc88bfe4d40fb74b903d7ef915da2a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1633 zcmbVMYfscb6g{&K%65w^3ohadRMe$B%1eBJh@fkT8!&;87(dOHVJVd9rY%N(hW|tp zSE44y&;BUmodS!jYf||#_ukIjbI+W6J3oJY{SIInk2A>PwhjY#ly_H05knd}RbZ-R zIExXCY8cazfu&(W!=#2O2Cc?@A;W;7`%QU=zvY(8Wz|{>eMeSj8B&t(35I;DJ$+Mb zInoIoPclrEJ-=%48n?HFwad45xfO)I;CqB))qKwuLE!n;s}t{;*dqA0aN7w6#~Ehg zrLTK6&-JPYi~EkdDSU-6rzp%byf2;);_sWa5F#G*nad4rka|nrCaKq1cBEL@->V3J zombpQPtWG=2KOBmAAd|KF&KIpFe{W@;&$ly2Mortl)_(fc@PLX^)eo%Ccy?ucV=Z;O(nWa??6Kc*(sKo2e& zDBzxf9J*-C4b0%afeXsZqg%rR0}t_tVJ_B8{^KsH#H@xnhUHi=Oz1SXCRXq5qYk-H}1arA?@laVMpLKsBEi z|6p?b#{BPysH;uK;@;E{4A4VpfT05^bg5~fw==IsD6%v-KrW|-i{b>GE1;LwOBB7M zI8AoI9Ik_zhe((sbtKKv&q)11x{fxpFI9*3G0N sphw3_WM4!D3rA_k1$7eO25u5)0=J@^L0lu7r#OpoTqWB<{9 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$SinkNodeFactory.class b/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder$SinkNodeFactory.class deleted file mode 100644 index 217c1f4d2cb9505db258d04627774efca0074c05..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2230 zcmds2TTc@~6#iyuyHJ+3fFkvNK`qN=1raYODky3~)CeIlCMLr&l%?CgSjF0r`G;tGqaI;OJd#5En$I&SE=DWF$msQfq*=zTiBE}u)=lYY@&h(p&e zP6?#rpyE0LJ*_^bd|6fkT_v@-ph8!A?n@O4jL!#Q(UukItSP%BS4+}%f^s?V?WlH2 z#x7ZPAG)aPx%BEA0-1QtjRwvOOg5uMaj0aO2-=k}a8wipp}iPX0xu|T&TP2eiV6ua zbItYLcv|3VVQ&!4A-70T^P%R;{uKS+dkiiKq-KK^%KpB2*H;fV%F8NTl*=CN^vnm2 z^p+%5gzVa8it|aQ7hJ#efE?$f69?g@fN|gVRXFR(C{lD_u9?g((ml`s%bl~#l@PdD z7~CDC>&Gher5D+cepHo-PU)Bt!ubDcxVx-0^!32l6xUvRK`%plF~(w?#RV1_;yR9Rs&<$3Tv?9`qVGfIb5U z(I;@JX(qI4t9lRoXeh>?({Wee_O1{b*fut%K(0RJ9xksdC#DwN6Y_7bsPi{#a!-I% z&9(wpH|=SH!=a&Y9NG@JK$DPvqQ zDv2h>Cm;M##53Dsp-W=Z#25Q;_MDkB-}%mEroWEAeg}}lq68VkF%qhT4vYwGRKgg> zqeuuNC&YvhlTqB0AYm$s`%%n9G0PAs+a*JD8M@xCZ1X*?n%pX=Yh}l<3Ns8Lix+i< zOI$14&H;n`#Ikf}!Q`&1lgqN7P!lE2_<*VC;k>9n!(6t%IZ(qY>yE`uSABlc zOF@-JlR9_&KXk6SIyB60W^_h$Jg`qUAh16w#F!A{Lga*)V3=wZnn9{IZ^;njlWGV) zES>=wJxI&Agl?fdlF^AIy(co}@K{D%XkAFjxQGD+*t7h$u9ayVTf#nbXWGwLlPv*KAA?!Z*RW>VmZ&^klSa05i#}vet=3** zf3N@DLz8PIJKUf@m<}*pK$wPwCWQW-l45M=8xX_deQ92RL_!P}*+JT&Iipj(WPKzn zLf<}Rpn`yMgrG85K}Z?;g76PSDu^oSPz7yDI$S~Qvxj$?WUqJX1!8zf(bouX9oO*+ zcd&sgw4WnJgo>-gQT+NmRsl~FJcFJl7=}Dea10Zto48eD_RzC|)Hg`4l|$lmNHh)+ zJ3{*>Iun545g~8jjTck)^i(w_m7)ik8kR$TQSy_Ofx)44C>=fmtLIY1cVK diff --git a/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder.class b/out/production/stream/org/apache/kafka/streaming/processor/TopologyBuilder.class deleted file mode 100644 index 6f2208c49788f175f2f5dc6e0b787c2442f6e343..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8232 zcmcIp34Bz=9sVYJyqDbv3C9u_MINUi2a!mlZYatjhz6uag?PVgUXmrtZrt4IcaEL;U*G(X z8J;-!&>(=h!sWpdY*uiS2PN2CiZk#v1zS9DB?mXlnOhXx>S24E7hlKi3hq#Frw603 zS;bb_enZ7wvi+t98tzuHO~pNO>Aec>^PmFTC2bDAC5e37gYV$GDt4&2U&Z$nJm7(j zoeFk&Fcp1rX}|n>P{D2wW@57(JmkdyzAwLipkUC8YV7f1FYfiA9uIr*L;R?en;((A zM-}Xo(~p&s^L_=7my-Mw3VtlxM-)7%;wh=~(;f`r84pgy-3or9;8_*Vd2pH>J@3In z{M3V=;pYl|q2QMaekGtSi$u)WqOcKSg*jC=4ekWXy(j~ zUm1%A&3HT-%bE7(j7CM$WI~d(uW603&Zz52gu-duRHN5j#M-i1A(aGM#Mo+V7< ze91@3v%*+wshP#6*_2XD%bl5*9EJ_DXimh)$)Zsnw~Rqnp`cC5JuegqCFTn_Dra3F z;9Nv5v}02!Vy@^}+iJ$nGg`x(8QT;M8sQ6!SV)fTNoS%nMAI~kNJFWU(bks4N8pmm ze3>uGpPDsNUd{+@C-tyk%(jexJ0UGXO~w`ADDY?j(elLzM%S*5M(W}wO%x7oFcKk- z7MrQD8EYKLo&o}m*aR+1O`_Xn_GSwvFn1)PkXd#2<;c_%p_jVgGGm@|m{c}a}FVZH>HsHkroVDX<>gZ1DC@_lnW_SqH zAGszZLTs~%Xhbg^HwA7eWWu5XU7E;Zv%T?X>hEbY+hs7L-wB+QuXCZOnXt5rr{ zfyI?YI$0=^Fk=xT9Is1LN(FMpH>Zr*0`rTLr2s9N1V~3|Po=Yk#gR(MP$-!Q2+Cxq zGb;|Bu|8;aOS6&H=5i8InBfGUkOHR=F zt>J{+WYXK6N+>I{NJ(Zz=FY=ePUcM`izJyB$c)9A^!<2TdenJjsunA8Uwrf#*o6|w zEBK9q-zs>4RUGRy7GA8YvML20f-&@m49LlnW5)2g0;ZJ!c#^s-3CS{I{IHWbTT<0< zK3IE<%Wy~+SQ?gN1^H)wI@5@EQUDDf!MPgF!?}!ad2#3e=3)&OqgBC+8h(eD1g2W_ zEoWF34x1fDctNbAXDu^E+BXC$GK-QmmF-8;!wi6%PDn4Wey(s@mayd+QQ zLh)^wxu<8&m0clAf&L(HVji`ECd?!R4Piu?+%^1BN;63h8WD=Bx0<@4Cliq#ZP!DI zxEXFIfGe&1n^A+hTQ-t=Ec@uaW@0uYZ=N7Usg+yyOQl{HST-VBX)I*`%(Bv-H2fKV zk);2ccOV&9G`xW~=|md-hQDk02mUE=`iKO`;kzFHqJL_*6bI;}8vc#{Xm|_snYZzd zz`UY3942;Y_%Ghoa5)TtQ;(2P<{)VZ;xd7gkCcE7nZV?XU)j#bA~IoSdEUgHNLM6! zWkkPx>FqSSyUhrXZgFTAz#`?OJX=C2qC^u8;bc*sCv_37z=WMK z=Q*uw%wU2Sw2@GnJ|~A(vvrMRZnwy%%K;XOnF%YokIjsiF^n0!hvzI}@D@;M+Jq5` zFt_-!E6Jq9jk7LfjI`{^VyrfMK~juHI~`?R+DPhN_a#YqruE=4TqR0&7 zU?dVaSXo>}kb*76Ihi4|ITfgG^GO#BwAr_+5#1Z}`NWHObePj99|7Fv6+J9zh0s1w zOJvTe4MD!l1#Oc_Ld9Cp z+V3W$9Kdqelt0o16xJOp%=L6dZl$+5Bi10-VQg^qF3yNAq9$@XQN ze3sv&+&_Pc5+|z)Mhr)X3?VZMkjf4VY2j6R^`cJeUbmn~1P`Z%WnFT&9vqo%Qr5Mc z-kPLo%Q6SsUZ-%2e0mU=V+);*pN&~{HN)rdp=9XG1k^=QR;N}vq8ZA1B5R{vhqWjeoCamVGP zz>JHTRs9fE11M?fhhrC?}C!)s`pzwR`YuT->`umIB_D%_#iwHCt(K1Eu7Vo z6mcOg;%E~q^h<1R%el+Jxw5Kiu@|NCQ_}~}4o;Ool66~KueHeN$)z?KZes0pne}^_ zEOod;l5S0s7OPRpf57&tolvTG!&`zam^J{dWdNh7$>_0VeJHQyXY3e$^k96&`&Rkhz)Q!Dy$OdtHKs-)TuOYJIn*x$@o{+m!sxtHQ*Vss1E;Wm7dU z>jS0!Qi6Zn;m*XQ94HlksfGN_$U**iiF~O73BoIwg9A7VucDdv@;Gn9*Wh)!+X39e3*iHJ6Z?1}l;LeIEp(O*f+8)G24R01 zg#C2gD=p{SZ=Y}Ja!G@a&NsNgdh{j@?&u`>QGmEc24fjH`Y^p8Gnxi4lVR+H<#xNm zZUfHbno1eSc4L;n7BO4)ss#TJuUdM;Zqx__lxjasS<7%#( z`Q3Xlbtg^?FzgLBlsXzbK98?-D|El=^VA2tHGc1I%qbz!db$6kjUHP2#cddr1S+;5 zAPH3L#N6Qoj<*Ta`n`T-FFurHYi)mN8?Khy^({DGZr69Bf!m#aRjxcaC7|`gDbj@p z8$6B%ug~lANWn^d-ui%6HS!+X8;RZs_C@-ShNEIm7-mQFrTg?v(q|E;`h7EMDG#HgP4p%T;ae{;lxVe zLW^*tL#T*~Qd}=QxSj1iLc5DUaH zA}A)3=Ohsl$BHg-oahmgty8ywA>(>ROX>PvJcH4en|np2JtyB0&ae~0N-+)_a1}jw zsW=WF#mC?lU5uz7$JIPsJX-dNEfX<(g6Eu1Vqy&w&9ywSlSL~wvge>?E%+3BPJEhL z%UGUV|C#jq>(c8#n_mApu9Ii|PK-W8mvUSA;$fo6`9Z;#6kKn4 z2lzBHt>hs5*61xvDLKR-sU3a~GZyohBs~r4;g+$=cpFpRLCHb|H+ZSWCVZJH$U3N= zBUuModH+Ey9+uF=2^I*+%0{)#m#nyMB!#bVX9>QFGuc+r#7o(tT!l_ZmhqPgwr8=O O%Jyuw)oh#Gp8o+Y9dVZc diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTracker.class deleted file mode 100644 index c443ff70e990bec917669f367cf402a0e91f89e7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2266 zcmb_c-*XdH6#j0KcJpIPOKnpsT2U#cNhK(iB56Z~rdUV|+6p5+ZIj!wB-xF-8x&vt z7kF`e*B5zkh95IBqdvh5uRi;qFyeRjZrWn|l8)KQ-FweH_nhxL-?{tzpWl85FoVz2 z7{#oCvuVUIo5Bb_HgK-*W%N^ql7YE2QkYdP^9IT(m^iN%3u^d84HwjKQ4Nb~s2Er> za7kdaF8!Kx>bA4B*4gmo*B$B9q(HWM+qz>FJAvISR&A#t>s8wiN&=}hd&{wcjz_02 z_H~-8y53gNYFo8ive>XT8&=T|JZZI7W0V90{wHgV+Un{cy8*tK;oiX=guIp$<&Tc0ZrQ%TVz0=1#k0Iqvhtsyh(%fwxkmX zlybdBj%r#8Yb6;+;Os$T2pH5(ff1NLV301Y0RpQvR;+fY40}~z;h^MnOvP}(lP&j- z+^$!`x4&aAuvL~hZb_Dqb9RvtN-66C&Rpf3Zr`TGzP!3|Th@Y7d9SHg!m4F=H%ym# zdd=;4HMwM~E)aSiCbmCX4#c#AO^caRtjJs#r0xs%BT!a1EcCDB_gBg&t$?Q!{~~ z9U59gfn;0CMwOos)-gA=S5bb%g04usXu5Z_f6Cq5l%AfC=Jp<)y2-;T{F3dd_!)gT zS{b%qX|;p9VYl7H+x#|k1P>1~(J`(}eUz(rxO(>$%WO9lEj0<>!}wt5x9<25%@} zi4fQX49<@e%n9;+k57(bPhlPfEMYnXdQ7K|fD%OLCFp;Oo;gBP;CH+fcepODeh!x5q3A81irqatfAS*in~ zA0)0Zd`P^fiEk$4kkwJQIpiY_3i-~SMy~c0a)md6*x2D+NO=*sqa<16TkxIK&|dq6 zb1im%XrB%9YLSzBL{=6p@&`n!BYsbeihHEq5QVK$Zg&IVe tjk~am=6IAY!@fIBfkY^frm6N+V!hX;nS^+O5voD}y5(o^5xrtK^Dlt2B!U0{ diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup$1.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup$1.class deleted file mode 100644 index 502579775c07a83044a464ab9a421d9567202e66..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1527 zcmbtUTT|0O6#lkN8XAgJKvcXGv`T4_2%^kb!~sSfuy}*veF+N$(xh%Sjx+uUUwm}@ z1-$qm&iLT??0?enY={n*7dveyXV0EJ`ObGP*`L3^eFrd&$vko>+puuMhJl+FMs2cG zER0#GTBtGP)Q%UGZ!gmT;AE$;3JXOC~~aVJtz@TM1RJ1rTyB8o!kc!3f!;C|%1 z;8JZ`VXD}ASO~KIH@B;BHZf*#<#y5Pj^ff}4`|DhK zdY>8%od!c;DF}pI^m!Bs+Rk^zqdZAjS#H>DaY=#SwVl>0;f8YaWh`P*Z#B@K##Z(Mn^Nzu}J0X$YQg>AP-w>69qV+`M=-h_#f#{0sR=DFQ`p~<9y51isH#)GYllA6NO`V z&`~Ugr)gsoVCwgOj^`SUOr&_T_2{jV+@&7Y^|zX>Xqndom7FG)Wen3@V}YRytXS0b zrb{VHPd5r=?ICRr8Ttcuwe|(9`W2baw9DWO$sG9r6X+#5c<6;9`e;Y~2^!6=u6&C1 zo^Cjc&PufdWEaLs%$OsC=7*0kGy7Wk1k>2h?vqP42`~tg^eK{eh|*mGx`!*6NditM zXQh7uOE^a)G@%mCCv;2<;Q}raRyVylM4E|9bl!!`FANw)S8<-mT~5i(ke^Pyr&{{~ rW1OA?n3~GzTy<^8F%4}HCGB;U0_$}cBeWX0hU+*>noZt0TIu%(!vLE{ diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/PartitionGroup.class deleted file mode 100644 index 2169a787f2aaa616ff37b35cda4ff3c05970fd30..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4427 zcmcInZF3XX6@IQQc`bQuu(4^Nyx1X4Y}rO3O=yC_P^%;{7~^0U;?Sn7rNu0kc9q=~ zpp-UA^IDR$rJ-#|TfX*7GLy^%7?+v0A3D>S{)5i+mt@+c&)Ho|D^NOeUp(WxckjzN z&pGEg=dAwv`S1S#;8FZKja~SzhUdHRJ)F^Cq>;c<8g0m@a6bw%m@=G|VL6SBI48T7 z%+AQ{ybLQEiXx#T%eKrMSuSgMA^Fl5;h(I!UGU&{p@IuCT$JONG*mnBVjA1=eYtd5 z4qn#q0}Ve^(5G$NbaO?+^Gr`+L)mbB%eNf+nTlBnR%Xr{7mRGhw~ESn(&Ex&w0LU8YRnK&X!## zZ+f2NW-Z${UE3&n*%M~oaSH*aiAiG6UkKn$RTr!hVQh)Pm~$=1wft&$SmD%{?KJ_r zJ`jCAm3zKXDj&Y&yGGu3T!klEGfi+eUZqvtzT+Fksmj^2rfU`y)U-m!e#^G}Nrj`s zF+(I0|FNQxlM0EPQ=sEE%viQLTPZD>?t-yYwBv|6Z)UxS^#8kLE+?uGmZ{n05S?}d} zVVzkBWzet5ARiQ(xUF?glWcB~X9|=5Eud~9%&(B90fRKWM0eTdB|peY${$|0$#H)% zRg6O6gmE!k6lyl{|CbOZsAolkm1dgdMOZ#D!bD5eN}oc@!W7bO3`KvKqK(28x=!gBMr%q2zvOuz3C{$av z;(x1LoSCjUldiM!XDS=+RANcm6IDz8HO@W>IC$|D+Hrl;*nsa`)v zTlRoprXY^Ve9`e38@@5c)ALTn&6`hJQW`fkmb2Yrla7b+oWfV?_BHe$4X^0<5uPPM z@9B*HJ~110J+MaW93+8gT{-j_D2Bjz73$7O8B|cLt>m)lZBsOIO(~PPwk* z1Bt_ddlat6%L*Io1|DBJZ{~fL)OCt%D9!Arb!G(RCiqC`t}n7gvKxCrJ(z03*Y)6^ z5Xx>Ho{nW(&MDEO2DJv8iLz5K<2_HYlG-D4XPnyYyRi#L*!d(1dt$erEteSOkCVa<=S!GRPHMYFNrLtcQ za-RltTri5kC-8l1(1VZQ96d;4khb(fe?5@(b%jn#I!h4h%?+2=D8Q@QIm4S}N2G_O zG2Z3KdEe!?a1b+AB-3y)#2*N|_@g3$KFNJ3JjAy)4D+1@q4qw?cb0`x*cB;JET!LyBJejlPn?*eh)z1cw+)msJ5JN~CdS9On!1I)y~%;Z2k0G0 zM&@!mGIxMGg)=mHAo(H1Bm&ht7*@cdy({QP5j)|K7G?UTcdr6Y79xu4-pb`p-li9ZuflFxvsxSj85F+3{6u zRk)5JPQI$dwfhn~w^RH}vPAn6io? n1t}V#pq{|U-;koer9*DS1olP(+i*HK{T-5xRG~<*l7IgKJBX&4 diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorContextImpl.class deleted file mode 100644 index b4343ea8cd03cf6d41218cc10491ba37137004ff..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 10324 zcmbVS3wRvWb^h;arCF`UwzOC_vM^r221%AJzYUU5uw@wwOSZ9O8$*DMRwHTc)vnlG z*%pw5Jo-w~Hhqv5N=logb(2C943bdP0S{yP1hG>lu2ie!QERAc zWX|dew5V&PXHA6)6NJs{*ale@Lg-Dbpdr_5FP3!Y1hf4UPNdx)cSyG3}Qb= z)CRRWpf<|8I-r{5y(yqJm-D?vCT|VmnA&Ej?Ln?!hctG|*UiG`7H)!SmRT+G)hdr$ zrP(Hrw;1ZSpxUK&8>-zS0Xvsl!^^(JJHmY7#y^dEegJ9zk=#i@TaZR zNre_Y$r`aDLw3_iYv82Slu9RUYd9JoY#K=>B6cd3NH#^|X*(IWVyUJcJ=#lRq8Y<> zIvI_m6zU6cB8lPQM7+tJ($wutB~^JO5sTT8bRwy6TOr-t{14laM6$mH`{Wkj|RMCpA6V@rKX*3;;HSJ0!t+74!1EaRAqbE8T zx6-3YTj6NYNv(^P=K%S)w2{e@llEATos3$s=z}(wxTO%%RdmXoWu>E3dya-6+ZRmK z+Mt=AvSK+Pg=G|GkDXy8;%u8IvX_bi?OX(zBhoU8`i0 zt96})jOMm4kA2TY>P(}Pv?t;N(ZL0~QN9@$^16)s)E}kqw$nPe8(q=3ePDFB&rbGQ zeKGdXZjlkOZIiE-=7B6`INjdKC!>xbT= z-E?nMX4nx zk|F%V6r(lS=&@RT?_%ecoz zG6ufZ!$1UREfigX*)5)Fl%q&&F0K=y-RGgDfL;!rbPh=xo*9=FLdR6vQ{*H;w0kmj z8)~niZa36ERwgVwn7-@knV`27Gl(mP`kjj)%SF~v+Z*n8%p1!akww*uB6P&4Vt^mD zy?~h%zbDGLP)TTQnYrMxehzMh)kh#{57I~^#tanki()}RM}QuQPxFlPhn;A)cQhVp zk$_#QgPX$ULO;9$Kc+V~GzwVJ^j{3YD*$VVK|7sW15-hrqe8E9&oe&eV7SfxxHZgZ zzC5!fUOYN(VqhR;r?oxb!FZI84pS}G@Q8SKB6-?M(z|c+igK{;1j|C#lQfvyBRVXP z3`Jx8QWq(-%}D|vImh>KN(Pi~^L#pPjt^1oj zx^sz+7U^rNBRz>l^tyR1vQwr8^(s5#{q`A#fCY2Y7=e+|)R3d7UGuc?%V|nP^2fE# zjgiEN1IqZsZq@G^=Sj7r&6G@~@3?XXb1WuZWVc0|*355nZ@Lc9B!yM+EYRHQP z8tq2TXNP9im#*ohAmr<*(Ef9*^BUO81hexTBq#N;Bby zoyeol6X8tJ6?UqyFg-ZzM4qsmB*XWZ>aglDRIhB!)6zJ?-M^Q7@sgpAn(COkOQET7 z?fFIBemR6@Dmzmhmx8{fkjyz;X7fOuQ#(_-sqRzvGpU>EZFtmFmg-Zuu|S5R6^Drr zi3TI8pLKz$Y&D>8RgQ;Vp_CY=8dO83imDT|=G?d*!f)dxQ=L=;ri!UyL&Z&%z|*wT z{B1YYh)j9_<8#QT_as|kGYqy{@o*v@8w*FoR%}{=^JI!lh5JXPTnuNAWqNZqitg?* zF^T6*mBb6CN+|}9H92`Yc_}c+^QJ&~c=gEfC7c>cjK=!I@kBa2WQ~m2@vxN&>&mdM zJ~OBV>a)W<0>$k9@M-1?J8aR4rCtvcWW+h`+tll*!m-N(XY13&fGhM~9@)u>i-9%q z=lF{`rOqbU#lfG)*Gzm_oif#F#pKgkV65|W7)cT)Ud2C{Y7Cz-@sIe580mwidPqG? z4l?j|)n}@A$m5+d`Ut+N(7srPp6p?$cbV$l>OHdz-O19t1(_Fv{G^oeJ=U9@2R(MV zBsTQ{!_LfUb0J`E60(x~36fcnroQY!^sM9&rFn6aD&bUH#U1w3U zStVSdlx~kM8MNoJ>f10ck=%^b!t=zub}kpZ$&<-mPR0t?6calKT`ADsOHx)b(f9zX ziRxlPvbqU5yQQxnoL%D~w<=Qu>uW+90HvOE0kU0%RqJK1G!EN+7Iz|Iw=~R&QU`d~Bvq$= z%|2UXi*!R*LI;@`SIpjC*Ki3{5n)kjMdPXcQtHpzOczFONlHdW$?RtDR=cAfN6XAF z>>>cIA#gk}Nb6FoWml7bQV921_oX1Oa_2O4_{23eD1()H)nD z+{|7Oq};8b+T=`h88e{4m+){SlWdz2BN(@B{=(p#ZmJ@q$)t$U{Q0XN3N}ovlG>aO+YfV`rw32lCsi%6YS{M(-bKuJ1I*$E0_^>4 z!z5J0GRE%Q@n@e-eWo%Z4 zs`O~pI6`b*m1{1aM)mOxp%oKYS^GS$o$Xa$cm^WfR!YHI0qjucmHa+xjVNQfT(q_@}U0;W6AVUt1Jj z!q(=}#+uRzYb+&qO_N^mUOJB4N+DO-fBbqcroaBTGxDAy=$ zXJa$7ubrJa>wk)?_v^hnOj84IEdn%xN**Xf^yf{y+1N#Y zK1NSY(Qi-V4E=MAp7;pVU5Gwk^mz4Hg$&mR))C)E2JG`frtZlckCh-O;K zSxb2ke2iSq(TpF*@6bl5f7dGbt2_<)!`z)pd>6lqCuv6^d>g;V*i%A|C-M8F@Z%$- zk-c*;K1vLS@?%-bKOm(u|D&Auhn(Mr;0(iAse!Ww{>Z?`m4QzHm(Xh9k1t}~MfiS- zS_4n9`x5H-aR~u_-|!}U93wJs5%|ov*iuoU*WBM<)aXv(PdF=|cuBs+ey|9!6H7ly z-k&N8vMe9u(K$gL#4|-C3FU)4ep!%nl;qi>Aj|VXo}3fpIgGms-NLuTkATFF3GB(o z`IrYMkPRU*Ot|#k_+#V~f80Y{nkDA3-zARIwj==7l6m7}STc={lh0qy&4*VC zSKxE!EIx;y&g1iW^7#V!{LS2a&R>yFkB855=*{QzCGvTde7-z4pO+aNoub7@b(B%q z4recpOyH=F*9SJZ(Rw34bg(|w>}TaNiMwk2VtvP1<=lN1W^<{0)Rew>cIJ!X!VTEW zA_4s0tlv%?zg{Q(*RU2}H zAU>;2sRN(mmruuA#{^~U^VzX4a7=9ECipL*lEHF@pH77RYXkiE34{BNju2uKih zLfPLE;AQ+B+Y%4IP>btPPndeF*KbD)*U~`+x|(^U7T@EX z*ZJ33-{*|iv4wd`G_Z~N2-uok_txyXx4P?=(ZAQZryAF~n)(Q799_tKM;9_bql;DW z&5%t_3tYaVgMhpIFHvedZ_h_4XRC%0QJbQfC^5mpKG1nxT~(9ZY$RBjO-Z;;2EtT!)ujCz&JL*bvY$UBIe ziPwxPZ_CLlf1C&Yshois4@NF#|1-YoO1F#qE@4+fUl~hiD-T4K@GtHBp2e1D7h8_k lXx%)8;a{>MU8e;iT&VAPM4rVefm5C`bTeCDXP*!M`v3L62p#|c diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorNode.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorNode.class deleted file mode 100644 index 075c28ad35f95d7f1256ab3774d9d5f98a6388dc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2779 zcmb_eZBrXn6n<`A2qar-X-gGbrLAp4f`ry;DM{0&6x5QU4j4y2xr7zAgoN2F9r^AL z(is^Bo$&|wqa2@e?`AhmU}hcsa$nB9`#k44=iHP0{m&nN0eFbDEG$&AxPT{PTvcUw zn!&Ry(#BHD;W?Iac!6aZR%G}*gD*1pQem{YXur>*c#{k8qZE*K$*pWbfO}9jCNw|FmnDy1wVwdv1HXwC{DAPPg0fN^aYCytduy zmcH)U);e2GrBJArtE5P^?LDGT81$CSkJo*V7(ySyD9p#B3rYC-!higvvawp;XlSoS zDsJ2Ls|r{1!;Z}~==KstC|s<&ZD;LZZ`1KM?9G-I(rMbQhV8ks4~!{)$L%Uq<2iIj z$f#(t{ucP*eve?~T*!UGmrv0f4?Bvy#OKxdaYC;I?@E4VI1vgDXpU8ek$0@M9#(&; zH-?P=Q-P1#TU$%=$P{Y%c+Q?x01}9_ey|G9&jqEQp8~TTD|%-*q*3&1kk_KH7!xi` z#!`pf;b*@r7@4NoC_IYi)DQ|W_DzpbxYu8QUv0j2ntr*jfC7u~CU>2~xBE|U$XLIz zTL*m8Nj3%<8`6{Vrc0Pyv^rfm&&#e-SzR2aXq;lJ4qsEqu6GW+rnBPeE)t59W;rK~ z*}^qU%dlW!0+%gZ!BvIJQO~Y za`MEO(d^R(C1jrp*omy=CA1&L$Em;%b0#(ulAAgxw*uic&f;~t@H>*fa$#hTXydFV zHL}tHX(UuEBJ>ieZ=V8}8ibpV2{(?9g1Gg2ml=e67z@sbzC-`+GMFYEKXR6ATNIrs zo?tYA@1g{llrJEM$C|-}(+Y!bF(EgICy|KGBoa~rd8VpYeFd)gvN%c+d>gRl$eJSS zSOSBj71rC6KGIxBn#0GK3rI~uw4o>5p$bC_!C7j_>H z$RRR5;mTA~bgL)vKL1qqZhgvICd8l7#^%9M#J=P#ce3Ki336)4j~A@&<-QbV1Fcbk PiA6Yj43Dv>cjbQpSWa~D diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorStateManager.class deleted file mode 100644 index 028361c9e961a4d80ab65ac793d33cb1c84c4b8a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7712 zcmb_h33yc3b^g!HXx?b_#3F-0VC*rQC4m^g7>E#ZBq14LA+SZj7~A-1^dt?8X5?7_ z)p5GSZQ7=5N}WJbmpHMNq-iz>;fd-cUD7s9(k+dfrhAjRP205TmV{!{|J?URGeW|# zzZSml&As>Cd+s^^Ip@FUK7Ibx&pZ!co1~*~F;xv4m-O+hg<=#TZ{#yNGOdqa)5mib zp11IVg%>0EY!o&4^(dnF4SoD(4Soxs)7Ed-pa;Jb#c}+uh0jOudr>UK7xeWPEqp18 z224fp`}+F_5&WV4{!tD77+==MpVZ(hc&P?|im&S9&n$c`inaK23x5&8U+Ubyief#! z5W!zZ@HZ;+Z!P>?6fO9=F8@Xi{vIz!@DJMgM}2%#AOEBi|JlO7M6n&;(vg4F$G=t4 ze*HJ3y>CbH?|8|=e?-xa|FrO5`upEC_zu1s!S{6KWu5jv7Otq}uju1d1?6f)XmyQ9 zjRYj9t&r~bazv_hB+QOjH4;&~dZaocQ5~sKRbR5ij!10{j>rO?yHHz;blPINKT# zHR$BrJ;|(KfwwN1iT5N^F4u%}`ApW$2^M;93i)IzzTX+=tLlN|NZQF4vJ`b__S?3( z+Z*MbaVIhA#!ouKC!KgMpLLxvsumy5W)f~Lm&sCBC+`mEmb==y#X@ea9aX$A=2F3p zbLAvbNjII(#S{Ldcxg5TY<&aM+m3Y}>#W>$=)mxB&dt;K-fPWMPs}nVkr^Ayq~nJ& z)z8**V!WhlwEIlL9ams;maG))`9DDA zJ6i!;)Y#|$S$@gRmX-(_BvlO+RrQkv5)<UOl=x(e?_yLc0o96B}pYgNc3#FYgCajf6xlXz;&I6#)WcDyQ z*^*WC&2Xxa8>NTc^a#_O1Bp~7r#bFxFhkS2u8Q=So#1b`_iOj2({8pa<>Ye8>`P~p z+gdY=0T9x#B6!9T{YIn3FE0~HB)p`*_Sj&Zq`T4mRn>sakga* zp0;JJTKR;mvt_;9Dp*}EDtc2Xcf?6`WJh=|r}Hz+EV$W!6*F-Y(@<`WU9(xEElnzL zSy`Z$E}eyBYDj4}Y*LE)G|L9Ts@SNLi#aJ43_}yKLDx;kN(}7|ZL(#fth1!WmQAwS zmbh%TWQ#3ZN1E!*UFTeiy%!MZun&Bv&=?36drj?*I#Yu#X`c&M>VD9dxt z@GOc=xOrQ4NtwT|6#hj$%G=(20-0mzJ|s*AM+PlP*!Tc0uyX4n zm(Q9rBzG_%47Md=_gGrcP&t`37_rb3!R+tK4wcW%huOkYn#>cL)*~u!QQ22s?%6V` znUa*fww%DzmYlRDC1bN@9vD19153(~lX?$SBt%_V#a)6CRIi?5kwPDCXHvTvz^E1|>FD?DDvccfBm45^RE=SxbnX5Wg!?!yIE zv^)p>4a{rL&|OU|qh^g)C{*L;dG0QkhiljwA7>m2S}Mhqrq40SCpsz`$!EM-g628# zc-_*3H_{YY!DKvjFBgxw9tC~v^_4If^>Hot_VRURgvumm{8nFr%e*f6lZz~Jald

r8TgrE#x?LUoD+>|gZhCkhCu7OMN}Rehi8zR=UXlNX|N znyjRCVMg<6d*i-&YTN4utjFUyLOizy`hJUO{7KERs#D&Fi@C=~Zk}%m7551)WqPJ4 z!fMo8z8Yp3VF!juW@<^94NYmt0;#|g(mxC7Ld`a<3z=-Bn>n0j(K}v^ypXMq!CWR) z$h!xfJgf9Yn%0%yZHQsgyW>@Q(+|}OO-lZywpBy!V1eL=jn-6R`Kny}bBA5+=mvWpv!l=4dUabHA)RmN}(2jMG_i1537Rjp=dB ziARVi%lj_HN~_W^bJN)61xqCiyKoD?uDSd^7r_cWP}y36Q6wQa!FB+(sM8{w?K-Wq z%`eMpzJJ)f&vKj<_DsLPK8x*{{%Q7EcUSZyY){|^5X2*t@J)oF-v}RUz65Drc@cr; zjTaGYo<`{CB~)ERxVhybta3k6ZdOktdUP5!N1H#5U|sDcEVzV)ts(ZR8$wf9RK(%{ zv{m;>?hwRV`P{@$ia?wnUpJ!~Td)9I`Ni`#+=Ojt!tH$Sz&`B6z1W2Z(1u4jdz+^r z-i9A!kGdK}#IzKXfN3cTL9ShbAHok){*Q3frmZ?ZhXUWdo$U}u9`mvLQMSkMV|*$S zasKM;Dz)YlYy^zLVKaD#QN$Bh~0M=gni^1wPlcNZ?!fySW~#;MZ8&mJNR2fXAqNDzif6h zYSf3$o+qMMHEZaJs~3z_T_vl0ox_|JX{irCi#;AZvo-4uKvN}TBJW-Ns{qi&6xoC2 z{Ch()?!*D~ktX(&BKmQby$d*qCvi6`+W~wCZ^1`#n155a7hl6s#`68L7>=yLplm}z z+TqFpB;_HTkQA@%6UfNpcvvQJTE2uc@^#*pUdAKx9lTX8Gi;-?U#jC%Yueg>B2K`B|;`dR#(<`7#yk9Q#;YvgA9 z0wn~g#aHl)cpM?_u^R7YD_lxjU)p;p?Hu=P#=_;>U%@7^aDM$|1TG^USbGJ#Exh*% zj)WFn!6Lr7j7@b7-0OY#B`+`0i2(lt3y8DkpNWi~YqR^E#;$sAW=V76{)8c7!7mf% zfFX$sq!FddJ(L%m!kv8^p5Gm8Uv=Y?*tTKSR>OT^WxTy6EYD;4r_fusPe<-5qAzIJ zaQ~6Vf--sai)B(=LAybemJy*wVzh(!>}J1x{crQoX0} zG*u%52bOXHfsmzTB_*Ir9;#51H{K#x4yZ;($u>5Ol1KV=?!C-3#(oj^1;OSq<-i`J zd`0hmHRZE<#&N62B|Na-+wwtQct7VF;Yl`(@Z+r^WwHh(9c8grJtpYtQb-l?kn(QJ zhK4Gikv42>s4~47Hi|kkX+H^@uZwU3I5K(l?0g4lf6K-x400EeWkM==p&vI8#rG5U zCyDz9i2DbzmoOeCkV9m<5qy{j-bWZazrxXvVHzLD=eYhGcn06XMSL62N*G0{!E>?% z&l`X3WuEQCMy~z{o$(ZUx%Mpm`#8Deqs)R3CB6s0%3SAP`#7#gEaOR*ma>JdkMXUT z{EhLQC+*{Gc`HwA9GF?BIZNYUJ*-k3bi9HU7Cyn4cm;zNKFPHH9wH{s8!96DVXg|` yQ#@&(nThOMC{6!W*V)fv=o$2CfJ(*Dy^c#h&9MMY|1f`}e0zj{y!=qO`hNj;Rx#iJ diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorTopology.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/ProcessorTopology.class deleted file mode 100644 index cfae1520021613baf241e9e2e2edb5a2725b94b0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3668 zcmb_eT~`xV6x}x;jD#U7MXDABTa6(BQYqC6lv=2^Xas5mzYrbb2m>LLl?*O<>|f|x ze}S)ctzs8_X#3QM{)@i#H?Z2;eeX;rlK`!0dEj!-{n-1QbN4x!U;p~yM*zdHW9Y<< zR=kC*9B#^Cx(zd!mBW`BZfVHL>FqYmVZIG_a90lZG~AD&9S`KNpy8q1UX%wHH9XSr zSRh_1+j+B6vCGr;6SE?4B)e)nGcp^FRmfzmiZd$EnzNRRhO<#N1!fzz$Fg>LIb)QJ z{EC@bGoG#)nTk_3jdiQIobjJ#|a;=*c;`%Ik9=$E?IfHa5%tm z(Pb>$Q3uA-}YKAP+cFy>5IQ|B7j_J~my zM@$N{3e@7_3e0#34^}{5vril-0fDF^i%*J3m46VJl9Nsd+&mE0YN`Y@Hyq0IcwIIo z?LxuKJC2oIw{|Ty9g$z7eq)Kq zz9KM^d^0Wh0FyTR^K)lZ^_QaRFXL|pv0r27>*ifhflF@&waXiU_MBtn*VqVEH`8G7 z-SP#y!fqE%N@oRU-2uto+9*2Kx_R|^-Yj_nYBj3{e*o3b-hGyptA=ZqY@A&_bG6wG z$|BHl8WTE>p-aaJoYc{clNy$EjGmyap(%QQ(l} zlXcbAWz`8b3~j}zOtVOo^eCZ7Q5l`p^YvVmscDpPhF@x({Duo7B!vWlw`qlNoPX4m zZ@)vE^78Fdh;e=zJzVeQjj#DX!vCGA?;%p@Ere42TL`BHwh(#A^)Q)0{3yrwc=9Y- zaSm}L$bTO1ay8}N=k8frMDqNRGwDf4tw|rkya}>~y;H~OiF-G?TKOP-FUzC(B$QGsu1g_L(KYA2<<(cM#?H!e~i*HU{TMfOA*{70lT`ia*|_6hP! zzvI%U^fp>T_!(NGUqcG5%RT95DUJaQ5=S6|_xHeyRPhcq;$8d~ybBm2m-Me8XRW() zHnMM|AJC`aJ&~BRJ&?xtMf#9P7jen+DML%XW0W}yHHJ130EyN;;YYm0Cc;q$_7%aT zXcH1-8!>)C^lzh0;2Rv{L>GLA_UVE2E5u*C{=FvS4s~maBD+qe8pDBTlF;3P`$kv9?ABbtVvO_H&1>2eD&JO>}`j{2j-41mJ9D0SrFK}oiaw3v_ z@%lIS>``h{NxDN#m|afNzaX0a=_PS{=t=@d=++rB3_<5O!|-$5nd9DF?#@%&caX+C zD(V3isFwETL-u?Z~Ilr&F`Td`te*tg-nGl*W zsA4Dt4+ev1LP9J_u^kr6M`9UKF&ab=V`4KN!evaTn5@B-5Na_M;+qL^cr}ENaqR@A zab29HRLrQDRp7~gtss)x(C_N;qHAX3bB<{{rn_~sXcUd4g5bQlX6bIxF%;Z>lihI2 zcGlv0UeBx>@lE~CrXDZ2j-lsFYb~C4?2J(;*iPKET*I;SY#}~Zv@&i{cTL-xr_X9J zYa~a==5zJJrh;U(c=M7iO=p4xhE2vtvZMUc$MUeX#5gX!Q?RnRsa>$ z&bxYMb4Jh0@MS}dvZqwsAXO%77nmn+ry$mH`qsWE9{M_`#c6Oq##L6TYDxP|HOmUI z=3dv%D+rakE^X#WqfGN)-Yz;BW6BgoYueYqu9Jz5iMH2p8ZiZ@-Y^0hPN79Z8{XD1 zhnpHcK}^Fu(h4pgzv(sH!bJ@WNUK=X@F|uQoI3=Yv2!`wieDFlb>&{h$cwpA(41z! zbrja^Vs^E|vfYk#J)bu$5zw;0`b@>=8ot1nDsF4|3Vj-MtSA^gL~!I`m1$E@FO!wk zIo&r_HjIp`ApGhmz3$lGl;*^%ud=#Lou!N8fJ3rtS$2t`J8X!&VQ}gjJG&0su}@r< zv*7AE7TA}!vsnd0RYvgYdLV=)XL^ieRie!Or{`V^ixj-=U2i+q0-AWGl`cAa1J=xAFWu2|MZ20?$iAcvW<%h^pv@qN3+d8F?>aWm(^n z(@X1!l$DaQcA$0(Q8V3pJ!Mh#2-7IcNX7G@s4j|%;w0f)&?7}@geG8Fl#tRSq=ey- zQV!~fqpzex8KSdRD-eqptP6N`NMcc8sM z?K9LaHJsc<-SY@f>JtI+DiYYiTViQw{|R9)gqwKXC=y@7C{a9m`LLHEhZ1V#Mr_Y?Mm7mtV#OX(Xcz1|&Ube}0rKa2bVRx}y(>D13Mwqo^&i_J;>mL}H Olw}RjWPs8 zqx<{%%fXTG&2&@dXanP9F=qS>WkW!beK)Mr;+AQ}DX$$RNqLf~1ETY!^~ql#UTtlD z0;}_n$hGPpkbg(E92SVqQv|R=(W|7d5e`sKY*j2`i8PAFGAcE$_p zjIYl4qa4qcARW^ghq0M#cTdiK=X~Fuoc#Rt^*exRtaYImITL*tGmtkijtM^VcGBrAfU zuk89_xl!K`YMpO*WEm(+hkMVta`iekW;787Yr*%0DtSB%1ugIYXCm1t;$=y{B~WOCsVG9`Tv-7JtQUkCT@3ZyXV5q!~>z6$K3K0A;-{ zeS2L7uCvNj^$H8VP;dEfD5NX0^&H)X}ZZ3~AnU||q<7)BGt<+LMcVAjH2%vrdH`+951!UN1( zc!);^9$Q$zqJbwC%2=|nj1`9Ad8MQp358#w;3KM8mA6I=tXg=AXH?jU&=Z-) z{^_l3Y*D#0Fyz`)liVM zM0Sy(cVF|e^lJ>pEABPKN|j+Sx7f-?Kc@XX%?*2i1QRp}tk}wD_wZU@#@Nyd7n}AO zPie+lKrMghRj0S?vaHpFZh8HG+emVu3z90NHFB(Yi?-{VO1{?pi`oE2sGIkJ;K2I( z^=m^-K1KH+hRAw^X0x=`7BfHo5iI{1srOBvVH!Kg2N=as8i#kia11ACMgC1RMjj_| zD#m(?3|&BR;s?gRA^mdv6Ebuk`izd{{0Fcd7?)|?xr44RFw=PP4!t|*95!x7Knx~OIM0DN2WHXyQJsG0>PvZ>jpoo9+D&j1$)_E6k4(BQB3pCFl bMc=VBvNTWOB5~5TOEey*+a&*9`lsJ-gh1{& diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/RecordCollector.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/RecordCollector.class deleted file mode 100644 index 475bcf630be429d8d5b525e614f1204aa11d97f1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3402 zcmbtW>sHiO6#g>z8KWZJs*-a-gNASA3LV{6yygY+%h zT1u^J^#Qv2x3AOHZzo|GoM0WDS##!`ea_zd+t+jc`uC4N0Sv<~;xNW#__&0ln8;(Y zgbY3r%9IS#dE73cfQb?g;EoJ;_u(Gy%j`iOGx9&X51(RAhKDkImdAVvpJO49FBGVg zj$?Xbw(k3;uOMx^bp_qiu2bY24kOegTQ?L*qV_%u0-GgjTvgNA1+xYz+}bzyX2v_=7WcX`=21L8Y4y{bRbwMJms z+HHMn)ABwrv`LEAc$GTm0jVZ55Y0F(Hmo zXozn*OA3DMggt^)X}89Rg65mF-nPEiC1%=8ykb6f&LI$UUg(C-(lExTVE>x=cn3nV zkpF?rI6upV@Gw!NZZufQf&N#bOe?S{X?{@5R`4BPctp@;eh zc6T58Zt8j3YjvcOZb&Jg9vT`_a6iFiomG81+|+jzyshPeXgNXylG=ZjEk5dTr5U&3 z8RobpUFt}bcg@anUK+HD<9I{GTXy49w0nd(HEkagiqa>BaqmSNnE@ zW+wFaw^V+tv4s+=8ITs24V$2XYcIu;AcU5d+hVb2R2BS?;M_fr#|=NBT|D1=n=gRY z!ZKXPXm}ocZ8fGktbu)A$20%vLRo~NysD5;0;9Of(w zv8IQ6BIz707rTV>gC^${?nZ&Q%3U@Oj^b`v$?1|<@Dzs>2-;NW*HX+~#ElSTx(5x2D-@1s#4k(R^-TydAt>_zPh%j!1J<8biaeFsgA_1_=#=VMOu+GlC%uM{pdLG$%BSgfRz6X;K=} zVc5vX;H31N2(oZBuZpKE8p%EjNbsf`4S?S?e#<3G-HfuXEEA5(2+DK+&J)U5f zJ!r-iLK(wxE!VQs3Qgsj5_T$Or(?Z##!4KR+$my1){uQLXXXf?zVn1JYQ%D`m5gn1 z9OKM3^JLD`-D3@>4L9d-Q=jjqwsMP4!0sBUj5*kCCTwRg-cBSD*Gieaj*%EK9fd73 zRoF}EtSBw0pvLP@tHZ~uMA9NYmV>5ZFrkD9rA((q>mKbm6az!QiqMN*D{LsY=Ene^fJvEoh*@Vclk8ew zuJj2mv<-9&bV%{&^1<%I(&?!ag*81cE73k9<1t^uM+z7HnDZ1y%DtUt2!(TjjGas> zZ1y_}erif?jVUAwm`RgYT#h=k)>(43vPleRR(5+TEW#B79U8vW@f9xX=)grCU*ocdD>}Zxw=AP& zx@V9%iO(B4uHukH{X1#C*YN{>)Nu_z>FATy>tgd~8Qze6Z=zqJb%s_;lFXD(t#Pam z50c|m4OT^uExcUvOI}5sEGP=qmK0rYS3TKVF3r=Y9XGOF=4qFlOnaZY$(>Wmt1Isr z66B%f9@pWMiE1{?(A5Wbqr$8~Gi&nlOY%(YmHOUPdi)C}20983JPfP0M7A$~0j!%A zlp-@rO0C8K$Yn!!d=Qk>y+^(`n&o9-4-I8aDvA3j*}Z34YM9Zre(#dUKZ)XdiWsjOg&6UHZi-mld zFX2yyAm&PiVQ;RK8}G}i0)^%5S0KiD)`sGE73Wx+xK_=%HCW4b9Y^O;$$wyJ^fpwq z4wYjFMq9>E62&jg@IOa<8 z%w5!tV_vHVwK!Yv%|6>&wKP8!Y^a*Ra{;W=?x3On4(9Y1w>~eu`3>B08!yyrxA9_Y zh!(;Pp>Zr=r=}q^ftM=q%l*;lI2wN?xPTXp#R###2lLU%6z{`Mbg@Dnz&UjDBIv;t z^s>D6c@B3o;VW^JWqk`tvy)OC>{W6925coEG)mQB8@5yDe2iiT_XH#i#e`kMF6{PV zevQ(58Lm#Q*Ylg-zpt==7f~qe=Nf-6-h-y$z`wXu;Nw9qbrV`XQ8y?eM%IYC<5+kX z4V^82BdBhP+f2aA6_DNpUf~a_Ms|Z&Z=yQdNOrbJVk^BkW_6+`rO%Ts87Qv#) z6C``m^FL3&H3e@XWJ@o7RoqwW`Cs87)&=n#4X@wB!T?u!2Nj^M;#UY+orn6WWTBLk uxe{7c7dgrM%3prbRQdOO<+tKcK|V%nlLsRB+Z5v$N`!+&;{)EVxBdt0%Yg>~ diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/SinkNode.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/SinkNode.class deleted file mode 100644 index 05c27f9babc3a2319a636fdc41afad87541a7a4c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3306 zcmcIm-%}e^7(JUG1VU0;6e?O3T4}=q4Fa`=L|U3stx%{9jZ&YM%?evKyP4fhZGHC5 zXCEAA{2!btWwbNun=}4Vj_2OZFWd}}GG-<>_kQ=@^PPL{_kA~i|MT1L0Os*k66bKU z4`*>pG|Qs7oj{=%can$)Gb{bLi%&%JX#)3>=*MRXtR|7hS`zoMp1?)|4-~|Gr)ug7 z2G^cwPqnP2**n>y@0#{bPCpd;2JC%xK zXTAC=%{Mt}?IB2C(cG~$zvdbWu6GTP-(1Zt2`W!Ds|EXdNF|As&>h=1p7|XyJl{36 z3QwP{x{hvmp5ta8G(;^2wQODMl$$6ih!vbNRUTL~ZDYMw z**4rwZQG({aLv&*tEBNNIj$RH{;ugMSP1*-q=HG&TwAleY|*s$*6D~k#B8lX_~FCw z&#Mg=%dd+qu6;AOuOI#&iG#oj^E)P>^OBViyH_ih3%jOOR9%sZ6G;>qks?nNES?aqDN7^O1V*~(Yx>@Oty+)UL_IYXTxk!D zjqN9f?&sRFuWk_-QC70UMXo+c;aOQ~4B|fBu`D7uEOfWJ)$pOAJ8rqqK%jzAMzCm0 zZ2K}nI0yG^IUe0bGh$TDuP%lempa%^7Z+{$lq@^2ZPtd-vSWDc`J;yL^PW(jzU#75y$9xe&`;Vs5P)|X=%bm?g7X?7UP%IDqcfMMlVY6!g z*r6yL6E3;#MeTmaLf`|wb!s0Gb$TCBbtJZr*e{%k;(akMSB|yt+$84$e^cyD<0@OB(RaiV{xa(1 zK6=#`i2jANI{7R56bPwaAXYadg+Zzold28!1i%dS`H;x7Ji}ZYvrLm&M&3=pEJd&M z#W}$&%Wn*k;j^^P5GA;^pU@+b$LYKGPxQY;YJK_@(pwxyCZ`T?Mz9pbUSgniiD#s= zH~3e7fg&tQDrx}Kq%uyLbCSY1FLsU&`$_H^KH}cim5tZwfh1>fy}AAd*9Q_V(6)#i mTOr}c{EA~KEi%DP-IOGPGxK?n5hUESEJB3_XkG$Eu=FoFVwBzh diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/SourceNode.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/SourceNode.class deleted file mode 100644 index 16d8c1754659b4a36d1888524655a96decfde333..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2590 zcmb_dU2hvz5Ixsx$HdvzX-Y%OM@b8%YdftWq#sFK15Igxgo5PAFXiQCvrV_Lz1DhD z2oi5dNc=?|6Inynz?`y5oQEg$j%7LAYJleeFK+>O1=J zj;^~ww;S~82d8x8(@ICJC``8e?VgTyLr-Dp1SuLDYb#CWDL>Plbn264YscA&ZqSRo z?;?d;Sxkc{^mNznZP)u@;Ch2W5Z1pw*sYpCBARIUJwIwH{NSuV)z5UjqZ!hQLPoA+ zvmQ^@Nn$E(h5Tx;MRv>Ue$RWj+uiiS4ZYc+XJ$Qcb*HT)R(y zVNZ7k^;WPOy52*UV(DqFdMo)r*1SChXAHDQcPpvs_-ga1=SC#35CwhTC6QY?(h9jp{N^r<4`=HN)2Ui* z-PtAiM;&jAJ~EmL%TBhzag=AKTiQF-inP1)rS1<0r#So?6t1R$4qM7LDLE`&BQl4{ zm?RAFgO)a|xIc@m^0IJ@&JtR1JHbHSJ7V=4Ys=Xz?J>UQ%e9;g_NaN8?bM)^%*5aK z&q&;@_6{b>isBj~bVq8}vJj1(F;igl`s&s614#=5!Xb%4m z6dwoSw>sRyCH|k4_;+U^C&GXuOe=@e{86KkJx!Z3@!bS{XYnfSbF_ZuGtcLAwe}lS zb$%bY>a~4XztU%6mg`LX{vryPW2H;XdKs_LUo~+!j|;Sz0k7i?=9Zlx$;CUtVBt;f z=lGO&=kXSw=9Hk6U70^?i2NdRyv@@F-#7G2oLzm6Ty6dZ@)mxfZLmo%z?B$}$#0MF z*u-k^*hKse^9c%JL{@{saP=;ISH@Pfj;L7qj}_lz#j9|*iUvOtoABUNYx^ijqkrO5 zwe|uN3ZlnzSVKoq9I-a}ls0v0vq1Yp=3I=C&6<28E}U3YuW6g_VkoPyI*`+@euI{nZ9O`MG}Mv=rcF|k2iK-fQ~PnZHTWEh(G6WqHr z{sJ~8HYP5NYk!k5o;M(%xawl=yYIaFaqgM>*l%g;wJ)`n%@d!Jd7BSsy8$e`#m*?Wv?$FP6e)&{lJea z0{OyXfzx6`z&r`hn4YWpfqFIUHC0rX&2G{$^klaoBVXSqpc%LQfxx3`7`0s4mtI@B z9r>ms-9a2F+4D)SABCP848q9ugIGm@><-+TrliiuFkk(v?A!jI(oe5gv&_LOENm6% zBqL?Ku-BK7s)qub|Czh^g1UxrTSWpZ$yuMNwoIyC=cVjVCa#nlj~At@EJAb0Nzz}- zMqb%kIE*~?+}EPo6Cv*FziDF`cWqc$vT+@@z{9JlDzN(RatLh9pcx&mNuk)lK9?*F zZU+q9(EaIWxQPa`ypu}e3({L5mg8-R6mx)F=?g^Z_!36RxkTy{p%hkW*OKQg+HIEF zVd5@J0S@1p#`kre+8ejio^E7dgJ_=TIBHV|B~jTasXV=#xJ9d}ykied#X0^C^D|P< zN;J;<4@euMvHKBebCen}#+>8kaLfa;IK&=~=D0aiZVt8*DVEr4d&P1#&<>X>jnw1#xF+667=5#}M^9;-Bf@$xm+~V54P!IX; zAy>7sE)3UEHPdja6=AFEzVk|Grd`TzX0{n3>t;zX3>0)jY*i~e!hX$n$`nZ!OpTYf zxvk5!AB?#By2FqU7IR!%@QQBisg`YO!f{Mn)eTqJ1}{5m(d89Ol>EZfx9zpSunTyj=*BA!VnTB49u1##qPAqP_x~rQ8LpYszLH1A_H*fb3!_afY%T;PR zYVPhj!u4{9!(P{{BQkslvewabbHTiGX4xSNlJ7*(tlFB`&^>LQ(bJq9nS#rBq9Bc@ z4D;Py1qHWoTfqb-72LsH2}34>DTd*HdZ_cWR@KWTVas}J3Z`Y=I-W7~&!kRenOaRL z7{!=;^_(FW=%*t->$ipO_Wz~Ry+}9c#TS65sAMqi>43fmQEpknC^5`7CfARi4eHNl zg9bF3-faIGqDY8JGdW(evfvUuWmezRSA>2dTtnouw@ns} zdauU3f_11qzaW$1XbaKvu?b?lB?L*UEkv}ri^-M{B+RxDv1W)&O9+znDMX08MlO0k zH63KOB?PJ7huEUANQjB=h`&8S;*{0C8kFP(G8E~rVW5VK&Fw8TU{Clwhy0<$Jh1o z_ZGepLNoqBU%qMKTLJuI2I5|;|2UMfEO+NNb&rqh5rg+F8TX;EyWq3tjeq!N&Ec|Z> zEAUJR9r&p({-1^axA1BJKMSD?uPN@=oACyowQwec%LP10gjl6cAN9@BAdNa}(uWno zJ_!(y1T6`Lunx}zq?x1G`j{1vb99{DEOTUTNLpl`K3Wy={D3S_=!Lp+Zb;hXJU!}s zMf)~ASsap!WJyq#$}-(@F?o|EmMjlpk96qP6}mM{xyVYgB4JB9 z^^jGTbcLi_F0tfNL47E44I+;re)>p%4vS}wW9!rdMr_#x&lgT90-LXX0Nhc!lO!tl{2#9h{Dwl|6a}h;) zL(a*OzN>3ST7kl#EgVkrv(AyMV9nW~-BAHZWPxllm87VT3O1e{k}`%1=yrKHh;Ozh z+mY>~qZucg;mX!qBDY1lbJcB0AjNPjXOccR%OZD2z+x|wyjb1DXxq4DupcBgY|&dFpQ zwY+!IE|scS)@5?gaYppRtVD9e8KSw3}?SFrA~78S(NCU1$AS2OgISla~ir z7->{B5!gzb%t~CXep?vkOtf^XxIv#Se#B2#&bg-M8G+i`(YHeXTR2&BkL5)XvRDj2 ztZ-TMT5=`hn=2UBDpZ#iyHhIGnO&-~Jh-NI6x9a!rYI?*@p9kQJ_4Vam#G+3aE$y~ zW3d(J(AQ9cWmdnpG`CO)m6cl!9oOr=fC)LbMdF-hykd;S?OPdFMF@}Q-3^iKpj*51MC&^Vd4uRSV&w$G%ke!!X)Cn#; z;Pri7uK#4ZQck+7nDcGvkzPyI+OiIxwK1v$C&8|6ZmAjB9LXGH`eDMcrBBu~+1Ro{ zuC`^P^b0PoqQ+O7wp=5dn6PZTU#_+BWqgHs%a-e8vtVA)-@DkQ6)KK3Rfzo@qq^X* z+3~WWSa(+7rL?{4#-eMz$TWs74c1V3SqH_(Z*kUQZtu}JFP0(K>s-nkJr`pC2 zjHhS?%d4<>avyiH=~y(=z0Do<$swIB5UdO(6HdG9EbV)3*(W#YV?R5EeYV`JMjWx_fDBp^wPi&5ZFUg- zHjc}vU@_xvD(RM%33Mz$kK&_|{zbHp-d`Zc;9j636H6kvHo~q_|3$CgW zgB!QBRkGv`Tke$i@LH&(v+W0N;Z22K=qnVeG&6IbRlKU2X)~>JY6H${s}%1_1uav2 zjeHb+wNR(BelKBW6lHr=dN-Ayxqxv!)7a;(gX!d9x9jfn$(1v^NlA{Tnrs?#vI7YU zZh1#pspxt@uU84XH7K3by)<+KfBB+YD(sHLwMh@ev|2SZFPKs{7kFX%v<^lxgY(m*EMDe7ev0dxa5y_yykLaCSu)f#e$^f#;+oe`oR0AJ;T;~ABC6_xtN~x zx}#usM)+G}8SM$rDLJXG(1x`hWMuRat)HBValp&jhMZa2Dn<^wEWPz^k?m|_Dw5Me zyR&0&wGw>mFhfXC9nxI)u&^QfHF{H7xTmOs^agiDaK~ASrD0`X@0*jp`KN@>o+FA>(33$1?Kqe ztc~We?JPUc{2C!>PUOZ5KTQc%`q+>E$$FW?D*?^<0((4iw9ql>D+6-Y+2&TGg3W&X z_&IIKDvRz*krpvg&)3xGDZY59lhJrC!>+Ki3SA*gmz6Wx+wbC?DPz#k@E|*`u4;ds z0gO~{_{G^}CIlKWuk%sNFA!mMv1wut?*^O*>+Vq>jQ*;wW_RY3%UoJXUs&m2*P1JB zx`swB_s3GfvT~0vWa>)DUj81v9})gwJ{#w0W5HRAcKK#x^K3>o{AOhHXht^eW@HOy zM)uccWM2tEhVwdPiHFUhne#fqjJ#1WBQF)q$a@7d@@j$O9KyaAzeL#g5y4Lpgbhev zcoGt>pFo{H>cg#Qm_SolCSd6!5I%nb!EomULgABW9_DU)0<*$xO)azYn3Koc3ABXU z8YeLCBwB|jF~7%ZYdC=g%zS~izzHlA+>g~dJ6G@o+IoU*jcvg^&dcNco=~_g*cO_^ z+j^SYEKZwG;DUPWIdwO z0Pn}I@Q)bop|0=2ukx=iZpB6T06s|CUf}wN@M{PX&oX?NHWE^MAmpzTUT12jVrFWg zb!PS(_)XNC*}bH*!OYY?8_n!v_$@S%pC6I_G%RxPyZCMV4gxd+AnffQ<7xrh$BWp0 zk6&HL-zVfBaQ_gjGZatI!XJu-2P}NT!XH`qB=K7KV+(%*yo#PP*fITh22XZfTvpqnEZMI+T6e2|kmqsu0q zl04r2`3m^E%iseZyi3G`g#1(dnTO{-%8>G;O%u;b8sD0}3pj206ix3qLJg0hZMc5r z&<-qeu3&tj}WucUXB`-LjGMJo-=InmTYzMacC{?r?1$*OkF-o*r%~PZ@B5 zu74C;+$a)eAgkwM-TfpZkFA_4RRei!-^Nq;E}kaVFBxsN^HiIE52Zd_$FD7z zV2r09;$=_6s$k^YYBYb%YXBkGPad?tB6rdF;$%h%fc`<6I7!1eQLGT_2xj%#z1$ zJ=RtBdBzRCgIQEpE51kP`aZ+^4{!;?tGc1$y2lV% z_6)CVpTs-z7_LP7It}{`gI0}`07X&H^+c^}dVsRWt3vO9L5|9<3YebtPL=Tq>}3~m z9P{#cSLqxR+Jqf})M1X)vy^Bk6U4dRd3E~JfQNfCNmp7zC$VpMW#=SrsxU8lfojF2 vI8>xKR1A(GlQ8R;u>Pt@@nSS!$$ diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$1.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$1.class deleted file mode 100644 index a0d5fba6f52490b92e824c1791579ebdb764d3a9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1903 zcmbtVYi|-k6g>l23f;E!VSV7ER@9c(F0{VE+J@A`2CKFN{W>foOxXw7U7Gm8A7!Gk z`oSOIk22m_T8aTo8z9W?%)N8YnLGDhe*gLT3%~>9V;IGqIAXXPM+*1UusDV#q}AK9 z8djpXAH`}EI>U%xmtHEv@Vw}_HJvxPQ5SlHe{68w^IgF$Y1j0o>lniG99NgNFI=0O zp1vPgOLg+8<_IV+ZRu|?Y%UIkv{Yt@6r8GHm@GjL@k_3Fzz@0J@})^@rYQ`CmQ#luJ#l7O>D4+Iks(e;Brz6IGR*a7P=E~+h}b?!=6=hiYxK?&!a%~X{3_rx-S@dC z4xNU&&iKC5at-lPs{ESjPU=;4b(DqMwqZIRmBDM_*PUt<84Y95G>jvwVGZjVmEFTA z9%*=tCmNpOnT8xDsHTP}sZ^FB5hRbvsRG|r4k&jRay^g(Gi6UyIPEn=ftzN98;yRu z&tBsU;YAh3I5$);NM$nAHOhT%^6ZfRsg4tttYR^SV_6!Qr>ZVJy(BF`8M@FL+IBXd zdQq3Lq-jb|Q1Mo&%TP#mPD8S$#BL2IcBebW>zNGZsh`I{Bw&ezN<3=wVfu|@f>tN# zZ37|t0`c_nH?Z_~guc)$ganNvOV diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$KafkaStreamingMetrics.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread$KafkaStreamingMetrics.class deleted file mode 100644 index e687001a2dc6d12de058c8f291efeacff4222a8a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2462 zcmbtWZBx`%6n+liCWKgF1<}4(i^$938f&e!1r>|6wE|Xlg`!ryWG_o#lQ2mZ`PiBE zNAzd3Geu^~On*SX^bhoB^mC`@CSh6jWuVj9ncUpxp7Z3Kljof5AOHOJJAmtOyD@@0 z21YTh-kI=(Vs;DLs(4V=WZfrpsYF{fePK%eq|q~RM4 zj|~i{(|#=IC@Alujwd=6bu8)lRsqX8p6Ym}!1QnG~+$m93hu<#>?{ zJW&a)LVQ|WW1A9(lbLcnC%R3TZjr-(U>aW}bj|oBF2<=@$CL9_cSQz8u~K0qJL}t` zvM2&a?NhKzr9c>XsOp?V#WAM^CYTRAPX;p;5r&e@3jZh1;g-+f>`#;fzq+XcTo+ry zst7I`hr)5FH(c% zSWYG+R!!tEK^SUBlhkQmDD`FQobzIx*kjG z(unTocXr>Fby^AY^`ZA9Et)dt6?3)@su3Ro_jw0X%VyeatvT~{+U+K;DE5JalhQ%Y zo3JpUp+b1vRGymdj_ml>oj~Y4(Ep>Ab!kV0&N0=~nJ^w)gsHLS9yxkm*2dmw3-=a; z{KS!_^pu|JW^lsE0g4hUYpKQDN*xY*WU_gBG?30o5mo~=o(kL&lPdKa?U@g^PeHhw z6n}UCab-HzJqSQ@7{y4d^y{*3v&-A z+32k|ICX&c#&{p&*+cm081G-+!vLS1*~Oqbj=9gq+~@fIT*Cc!BX^3w)5x9T@3!ZD zFXlea+~*VS_ZzuW{6Zsliht0adno3<$lMnb?xFU!!?Eopw!M_t4!5%Xh2KsGKIZEJ z|78Ic(M?Za(h>$~8N>7xQ}hf)dXA7@;Cp(BpXe2SCV@Z6#$Qyz-z3STRhpnRx=9W# v(FT>MLO+m;8_au#`T1Sr6MV)yleRI5DaJZ*D}MHEe9EiA+8BYrO?3PVHO0cu diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/StreamThread.class deleted file mode 100644 index 158d301c541f465c96951cdd2651ac1abd636916..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12434 zcmbta34B!L5ue#?@-~~t6(JZASRgoyZW<;Zhw>!(I9cPP+%%F0%EJ_mr)oUS#nXkj+)c$iMBZ1(-3%8WEw5*~`52xh zuPa?VTgECE&yjJii|5IBtXz(hOSN3)YrMcsGkBoJHEx<|?q;FJi`-Pn#|y<`7oQ+| zSt6HOjhDK3nTwapwoVkHlU%$ak5}?4jZe6l~ zk>D&BpDp5?~DbRJ(Lmj*Yj;cJEVI(Zx@Q?Hl18)UmT%2GGE z_?O}FB&!Qm=z4KWy-G!N7j0y!G_7Fd27OJ*BX%uroJI# zb8P84FsNS?Z>%#SOk-ko(g@kqo7#k{^)xG~`OR;r;o6f;Z)S|lC{ zVeNyPH#-~-8om$~*6QMcV7;t2MakuB^7+>p-u1E}Z#1zZZ&M`fH=@yS#Je=y6vjT9 zXPfiD!iA6Zalh~wm(If<3_$Fta8N|v#nugG=GF$*hJ3Mj#9*3u2vcXQn!RfFOj#O= ziN+hhXXylyx5}c%%mXZ!MxIj z1F&c$jfK7UvvIN53 zd4ZsObHji^p<9Tp@O-8`zc@qUjV}BGvBcwmSUBbjE;0P!NPTo^7}gUeSj60ewF9G` zsb9pX^T9}dqskWy*7^MFnT}51O~QrK9AioM=_2=xKqwHK$z&9#b4m%0aOfeR4m7L# zcE>D-V^oFf;TU~t0wKf1j-|f3Al?9I{J!9FUnC&U)=Ot>T>#e0u**!crFo;WnfTD$ z&3>aP7C=A&*ML2^7K=1UA9SS6`lI3h9U`+aC-^G1(Pe z7mwA4H-X~-8y{d}c$4e{3-NiiMj3e%Y$RN)>O=5&QvUxlO)EEFdy6Ww3=6 zHo&uhe{q5m)0`6IOce*RL&|z-8j^Sw5lv@fnv~Ax0RE(+4Dyx$vX<@NRs*+|EV9xN1{VP7mf(LCM8j`j`Ky=)fzF_ zz>Tzx{M3l_#Z-`)F{o1fB&23zR+@Z$m|<@dHd% znbmO!{6uj8&c^UY!>(ty&oAgzG=BVersWxe>kz;s z>y7ErAY%Vy22CtEQI@RyVN03;VC`X&Fr4+n7>Ze3Gk_yzs&Utb;}O3h`5g{!CJD2c zEE9j!*-208^bvim(|zyO*W-EZ|5C4-9*3Cxs}`G zv0dk#+yQES5RNZ3?f2G>X<%uV0272sZ=D{d-8yy9gF4+px9W5Q-H7a81u0?xbPq6B z6_6eiK0OWLh{tp@PfYRgtkQWGcK|6@Yy4ZC9}>a5X@VutH^I-VDv-q%cby;R-AvQd zrl`Do2~*lfb-I&&4aez3hj~ECJ+Y z0Ck;z&yVSJKW)?b5A*?Yq|~8hjZvK+=O;A&qs~wAQ#w6B+mX=C1X89vW}{D$QLB6@ zP?@1jm8|imvU(%vYZfN7L%tra@@AQpeam9uGAmW<)JEHN+9j9Y%H<&;FHYNvRTY>} zP}>}h8I4j2Go6q|n%cFs22MvkczDJlSEqK`$uuUVc9=QiG0!@#iEBNr)6MiNY!pQl zsvE>b_ZXnRNU%RIF?>WS#JksFaL#2Rzc0RaUCd)dB8a8>xYSJ&W&-VU8we#M3we2U zxku+`_)j`N%TF^6u0@@u1dw8_yA&Efr}Ojt6mn}yhGY$=@t<{iP9!Xv=L2ZfdtzZk z2I{|5l-m|Z9g=}E5ffd-9A49CFf=0~np!x9^onPP=HQuB1OZ-<9Iy%2h zAL#rF{|(tLB`;V(4LZNduj%x@P`u86*Z2*c-{iNDW;4OWK{*AA4xRS#i%g@^yvlOG zuEharFr8lEw{?Dp-_`j&ATWI*X8wmL(Wc^LQFs8>${CY{;#BYJ{DDN|kghiM#XNC9 zg(qTch#S#Zna=+ZNL|v6D(cx}jNWK~Oj-x9x10@}Kje>u{l`qlR;f}lf%)bs2}VZTIx+t0{@GJ#P9v~fx&Tynzw(4uZY2KqS3QE9>#wg_4i?oUC z^qt20bl%VZWg41tI`asJ(zMYROByK+oKc~|aP+lKdn6lwz@55-aOYtDR(CiYS@`O3 zq9C?yToS3_@p-JR^L9tJ?#OXy@cAWD;mKyyw=>QX@nDL>ZL?K$CMP>w{23Gk7j)!` zg41)*K?Q=!+nU3zJMtX*K|x`)+3}L?_39F=W}$-2p;W0#`D!R1?J^uE(o5_2;uxwYE|d^7vxF5QQ&f=9KSGKAm5Na71+eAv%0P2L76*X|S9-ba>X0e?* zkO89`(Rv6(5pbb;xLD5-jhxc8(abbCZGlYagp_OVY)iXC9jjC-PV;KBN6;%3_jrLr zSs;ly1zmz5fkn`%hAg!R?S80Ey8hk6^7y*bkU&6}8;#dR{gFT&R9#SPH9>61V0AT6 zM5B3+CQF8Z>e+=8jV9X$5R^bC6x&Gl!2T}cr$7yY&CGCk%~-Zn*x^QLz--{T(Y8rK z=P?J(&E`f)X5xC9RYr)=Hkxj(2G@s|)*E$kxWA^hd3*zkC(&G2FUJ-~+ZmbRR$-&E zY^3}?7V-IEA@r-I7D2DVyfIG=J@3tGpL=VK5Pnv{l0BA!T`94WSwKnT%~6Peu!zao z-}KRL-7@$jI#f{uiL(c!M+zzEnnJ833!(B5ZZRK}$b zD7Fse(Lw1ivqndxMXTjpCLAo2E)DuZkw!^0CW}OCP}I={6ca+GuOzV=iY1T?4hiQ_ zzlAZ)f$OKqkO?us!1}QUDbOrb6CeWJy;)qH&Zft@BU3`?@uxF$J1V={A~SupLlyc! z91sQO&zoivRp`^Dgvp?5GjdWTOjc7IUO3I}9M-IK1KFd}A_7!ep&pJPnZ_b<)!v2W zq#b}dZ=jgXUZ>3Ssao7jo93^rOe5{;FGG#! zc&HJ*3pJu)phh&wFgm22p+w~ZRBpFyf)I?sTaQWUQyCceOjq+$+%YPS2DJh@*kkX5_9@vPDvXLz>r^n z3#N6@KnG#s;VZ_qQQ z51q|p>0F*l=W`WZK#zg5$Fv#!fgZ;btS7xu*j4Vt3mo(WUOVxo0ssC;PvQz|)88-; z)~3}q-OuSs_w6(no1E7|6KZVa$JxlMO;eBUp!r!ecK1RdAGdoOmKZ^!sf>lJ_9B(%6Xq(x>j z;WHtv&^~J0eT{ART^t9)ZYM$mYg1b3WFN7F2f28G~&i|8uo;zo?7W#&;o?D!^7-vavE zG2>RuXu+JjFzZgL1aujIs}LY+1KkT}z7HI>fyWMTc>pup;MDC&C#|F+SnkizXe4V8exiL8M}D6^L?}CL;Ia!EdWePt z)F%Pfr$FmR<7+K6yAhyp4FKSF1pb}S;obBMEcGlr`#E|U*H@w0H|Pa=hh7BVzk>4~ zfX_?xCB6!{EJ17xhabFv?^)o~3*TR&9Pph8>X#wK^Pn*ABwAkqtT^!hyWsp9r2Pod z{S+WgcbknxtB&Q)99^EymV;vHVQdBIrme4jo!iM z{WMggcX9JAjoeF)ebiUY8S*W;XKVCcF8HAn3;vIS!9@!A2-u9^Z==9<)CVz=e_98v zcTkqN!L;11*VAy^RLBGF3xpsj4?C%`gF-;bdz|-xml+j9l=r=g6#p91=<5LYH-TAi z(F}T zH6eu#1)>OinW7XR6bL4B)5oYmK7kmYB2Yh5A_!E1xg~-d(E1@VKPy!OANSFCh$KJw znd0zZ@XEoA^RR9vPhD*DRB_Y|EmU5fZNr-g@J2pFQ7PodY~W)dn}i55Lt*yrmd<;z z$gCX{ABxf3yK!L~Z7dmz1S>v10*P`geB9h-2{uUdHQs)M_xteWTO`rFNTuIV9qk7O zbs|MF(xO9cy#$&qrK6PJWn-iRD@$%Bsyfsd*=zUhcw-(S@!KI>Q0r93sAVrhGpO@24mSJljcy+DBr@oFerD4K2&!;ruZx?o{clYZAnO9gmVP3hjMuY4d)^n z$0PAKy`%8AvBfl>$4~>8z;a{Z@+GjG7eDn)qT6^p-N6&k>Yqq2^JIFTr_hHyjlSdQ z%CaZJVH;=wYRZ>zcL%^*K-op}abEcb;iSwa}DBHbegch8B_@Kn)%WOP7kFfoGSu(G@M7*O)7BWyB8t zxQnJzc@oOXP^J>kgk6uJNnA-&c`lXnY^p#tzaI5^BhRBvd@P;L$I%5`t!yWnt*|VX z1)D5W))92yV%bDAJ2VAiC6pz)N@3WCSpi&17sP-fka8}r0tXJeMs*(MuI&I2YTgfPz^ImxV+ zK0A{T$YS8ri+iUDfg(8bUb+ZDF2G?T3y+6#<-h*%wo8OxEl0R(wo|U~VpV0C f=Y**W6deFkY!py>J;r=&c_xl<*I@+s%76a@bI0(; diff --git a/out/production/stream/org/apache/kafka/streaming/processor/internals/TimestampTracker.class b/out/production/stream/org/apache/kafka/streaming/processor/internals/TimestampTracker.class deleted file mode 100644 index ccc05954088b3b819bd3fec17f73fe2579dda729..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 511 zcmbVJ%TB^j5Iq+P2#VqZ6aN4rrH<9O zz?IHsa!zK>oSAujdwv0Mg2NgPYG^CeM`Qo)W;DJXDX43O=0&P=OWIB;UUKFAI}{o; zn+;-4SX%|JGfYkBQbxfey9;_)kege>ltemoGm{3K=c#ccZP{pwb9Z7X$#~{lboO!%4d0A9Ll9#bUYBm35pvdXgyut`s`G!Rc^LD{^BJo8$XA z2iEKVrexbmx-WBikh#=UF;QT{{z`sj+6iZ diff --git a/out/production/stream/org/apache/kafka/streaming/state/Entry.class b/out/production/stream/org/apache/kafka/streaming/state/Entry.class deleted file mode 100644 index 1b0676b4a72d786a177b36b4ce41da0f44b4a5d7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1283 zcmb7DTTc^V5ItX8dULf<3V1;&qOdKOMa3I5Bx>}9MjvRJ`1rM5VQFcbWlO@7e@PQ5 zi6;I4f0XgeZlx{71YdS$PR`7nnQ!*j@9#eWY+y5q6wCx}ASa)E49iJGV8*c`b~QJ)}R3TiqPZZL{1AVNmhHmazA1ViS)!R}?>^gf*?R8w6w9*cd zv18rn_)(crwBR%yZ(Cu>98DiZa%F|cn^ukasiM=g-**lxwp-Gb29b2JRn?8Mb{&a( zV8q*Zjugyd%iXhdTUYmOtFAxQwRPmVwmx*I_Q=zoZM|)J?g=GL50^P<|1GtC#To_9 zSt@UB%lKyCi|&k>8y$%IFmu6PXk{aDueIyBlyxy9K*=Z+b~Z;Vl`p}zM`;Sl-B!n~ z+V33MS<;_+OA7LWyAu#w&F>{5bh4K?wzwHDIj1?`Eo_{95KEGJrVVak#NTH-w^sj zCy)~L!8yOZ5fFOqrBxr1R>Ac1Wbs8WDI306cHuwD-e+0f2fw1W#`2=p{DH{FGmNF9 pUBvQT#JfnOlYxIte1m=$*DrQn8t80}3?zsQBo8a!hgkIO(jSn-_3r=x diff --git a/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore$MemoryStoreIterator.class b/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore$MemoryStoreIterator.class deleted file mode 100644 index 82140c4e18fad32b73d975f1b791503f2a1fa846..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2150 zcmcIkTTc@~6#k~rE-i~9Hx+N7C{TndC@NGGP!y^90BO`GhdRPS*^=#6?Sn7+q7VKP zA2jh2H8DQ>qm19|w%t{ll$e;NGv_jM&Ue0Zc7Fc)`W?VIEGE!~vvHUhZ^Z=8>E*nE z3kEK>B8p3TnMxprY3)fTc<{1;83R`gToq`xT;&LKWj5p!nJT$fKD9`TbPLY3Kz!L+ zv!z>dltA>l?K))vbJ4byGoP2mqAChZ)VY}s*>2120qnOQp*=v-ft5S=$e{)Hdvoc>&Uc7-& zw6I)_%*uNHL|5LfB7FFta@Kein5;);qh@HXjdfYPqn^0}O+&*E$i3|;$)VxKpuOg@ z@Tl_VqdU~lTJz~YQqLX&Kfdc^p-VA`&3?i@6F2LK=JJIiwJyf;20Aa<(7US}D(4cy z)AZ8RPJ6bFZD!}O#!OU4r8*5EoQdT^$;qh&OAl~w-Psz`fK7B`#Kc~-8<;n794AZ+ zVAQ}hj&ksc)_4^-)o_x%2AQ~yQ4BbEcNu6f^Ck3(%Bh`ENMU z$5*)x0(3cklUL57FB|rKx%7L#>yF6c4EptC5&L3BucY(XmXObTf`GO{ax77T+8X@r9 z%V+whD*1Glz|u!e=$Wh3QX*1~uHTQ@Ebs%-%s0gDk9>h4z_mr-Gve=f$WN8pp$zq?NLmkBNoRY%;GQy!K8EqiQjKP0TuuN diff --git a/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore.class b/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore$MemoryStore.class deleted file mode 100644 index 0e0b09beaed6d2a582a1528bdf638a6ca870832e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4032 zcmb_fYj+bx7=9*~Zkuem6etzZRz+yjLO`WlS}IVA)LuYR3oW8<(`^&F-Hn?~rFcQb z+h6b(@Wl^$j?hy%NB!)-@NalL>NB&OWSayE=!eYCzVpuWKJ(sY^ZVaF{{rAOe4D^C zm~BA^u4#BDfvEp-T}s|f;yt`C2P26&WRsYODTf6OiwWq+$swPh>J1G`QqhvbauPPO z32cL-q0o#E64(V-gQuaSVMQQ0VJ>Oid3AQC-VAwrAe-1dfe6?qb>~7}=beUNRPzjC9d+O=Fpc(*@Vbn#H2y zrZ1Ie!~RBMG!5qMyf-8;-d#`mDN%c->4#xwo~PJ4p0~}(((;_?PRVmh&p27bnl{|L zd=G>%FPEoF&ppau(K9?VJ!<=IA2U~{4Xb3T*z601f@f;qXkxWcHZ%S|;D`Lbv%9C> zX99_W=@#=v&$JntXm`&n$3>I#{b3hhoV#IWy@4=7Afoz`xw8HyH(&oq>uo4HO&1ZS!-M>!I9nsaIJV#(y#hE$tUL1|HRYPJ`|Ie!3p)1UWeqoV zyn zj6%V*<^HcVd?Rr7zq4iMsq*lN%9o?!e+k>avVCbjsjMNy_^-jZwo)WfgVsLy?8ZFT^%k zZ2Wp`leIbB#{rJre7nURQ9k#i)*w>7_Yg@vKy+pev3rQ8`tG6W zN3KTE!?7K4j{S(?7+P3bJ8^`_|DM#g%Fp63Wa3ojDnDeQU*RYhacD49-z%O9Fm?q?S)WYDogV)&RA9Dua}r z!caivE{Vjblj6Bs1SUu(A@Cjca*`DI4fqf$up+XWs5q*WrjP$ z_o)ikJ!JkmPOH^ag>?#NaF({qTz#$Kik8n>YMrCj2(2U$`3stc^M5kWee!Q`=>kRr z^4GXILPagrD@d-^gICdnnNZ-g3Ky*|aB|IOAlIsady@voFdi7Z15NSme1L5;DtNKJ z=sLE?v5q#uVTZsPcGhOk^@?l2$}Lw=Cni+(NZDjf7^Rj^M{4vxk=luJ9J>TAD(b0J7k2Pq{I zQsC_h)cJsN;sJKew6(8c_pj6lcLghinW)2*5+8HAg85K-ZV2k(^rFF;uiz@FM3_8# T`E28?14TRw`JcK2&mr+IuK%NV diff --git a/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore.class b/out/production/stream/org/apache/kafka/streaming/state/InMemoryKeyValueStore.class deleted file mode 100644 index f6d7c07b0255d96c09217c5d5d04a8c7d788e5de..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1518 zcmb_cZEq4m5PlXYlq+p5XtnjFt)itsy)V8+2r-hFG*C4;Leh`Rb*0zd3*Ie8ewKbA zv57yxA7z~7XbS~V8^6rX&d$s;Z#zGJe)|sK2?_~pYGByXFpv8QJkStBb{-EkB=IPL z?F4cR3;V+H{GK5_UofQhTvzy2n+Jgi7~*BiwPb~%o~s}8Ph7XT+tv-~TW-5l_x!fb zJKQ`H`U(GV!u3G=4Y-*y{dn9oT z*}u_eNE%k#<+7{h$12xej!eqC*0C^UsX0YfnRI?JK4E3{g%rLx9BbxCVKl?*`lfhe zsli#F+WN<;-we5lNgd1E=mkk<@lyTW=H;$^0dgXIQSjAp?RQ$ zd4^|KbI>&tK+u|@kUNh=MA&^n`% z5?!KO3v-Clxs)#yPr(lQi1ZQtLXs$!iE5+|Y?FQt*NEE;wXuRU-N+6ZTqj?OyMa~G zzDay)C|SWRx+%Ohq6+UL;J#$6PTT-|ZuD8MHtU#Q|tOG&u*&oP%bD lpyUwixJ|)bz-LHb!AEh2Xask0Z*Xp4fljs6V^kCk+Am{_vikr4 diff --git a/out/production/stream/org/apache/kafka/streaming/state/KeyValueIterator.class b/out/production/stream/org/apache/kafka/streaming/state/KeyValueIterator.class deleted file mode 100644 index c6e62074c2fba7ce6e2848b53dd1fceeb729e007..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 391 zcmaiw%}T>S5Xb-1*2dOv!FSM$i{QzGUZjGg&_he^(=f(u*Q{(dLLbDJ@!$jaP~tX4 zs8x`Gnc>HL{AWHs-`)XS<0!&jg#8Ey5e^AExv`ZH+UFM;p*PV*$-Z`hye8?*XwEOJ z43~u(&*qZ*IQxSQewVFD?X^+&UL1Stl96?V;%CkuMJ@PY!K(63_(_N9m1i$%SbDci zrfEFQ;z6+W6m8Y*N+i#W#DxCD)-IPjZGyCw8n~&dOCYp@wb2PrrcU@Px*=;JYWyyGP3(UG9;9=V diff --git a/out/production/stream/org/apache/kafka/streaming/state/KeyValueStore.class b/out/production/stream/org/apache/kafka/streaming/state/KeyValueStore.class deleted file mode 100644 index 57365a0444dd5d6d8ab2d2e23748923089f4b341..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 982 zcmbVL%TB^T6urZv1$-fhZ?_5?W}_R6B*tK3AaPOB#N}AWS{NvqPEGhV7k+>rWxOrg zC~Y9JXfo%{oO91TGyV8{dk27XIMLxyha(-131l6v2%Oe?1NO?OU}J~gKMuI9j7G9V zplG_zn5jT=0)=|NYc%?nfhv;#RnAgWk|GTYWkM=4!?lLoYas|!zv+Q;1?{;$TEOkj ze6`i{q(j++*)N<9+4GQ5UrEkJE_(MBQ=E3jN={p|7Hy+FYxBefS7^JI;xh6Pje{NY z5>vo~4=D#z##wHs%cm9-0l!mRGUZ7O>?R5TJ=(l8gfwcLpEAVnEKcPmu&G7XSXfAyje^>;{ z^RJ!&U7(R>Od~)bgI^Z^EiFeN2YEyqq73MW3s8)43Go`N$9MzrCTzvHjCdO=F|H!s Rfm)1rQFEre2m4WngD(M$9Wwv` diff --git a/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$1.class b/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$1.class deleted file mode 100644 index 88c9f6d94a0a9c7ce5e5453d48b31dbca10bc456..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1824 zcmbVN+fEZv6kVqmYRf2(8OmsZ>r>V{~P zxKS6Iu4Rh?LqfVAh{ki_2wUfd{z(R!_XpCfYSl8;wiS1E+I4%igHgclV}=2DPj`}e zhDCqWv-DmGSJID)oIe(?XvZE}{%HgX!KQENkmd97wN z7$T|d)$LVDHAt<(5ZGR22vec!L^8H%)oo3z>yq(kuL92O^FvO*?}e!umP5TrMtfEz zf=3Dxm{)KW!wM8!R&WJF3gS{Eu%O^E79)6~U*3d`%TQ_?xaxi7521TEDjJ zRJlX{TlgQ~R)5!B-Eh<`y($V`!BbMkxF$1^%;#y0H!eDZX+I5yIbSyW z4jRK?rSn`dqCT<5z2f+_zT?z zF+y^He1P}3L2|^4GlrWOqaFFnX`=gEv{pR3_8r4*+-aeGCafS?YmxAcv4RxQlU*Lyj^tPE0$jaxJSM Z?oq4&?&AUJ5dF_R#3-#~7Nex;^B0bF1Iqva diff --git a/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$MeteredKeyValueIterator.class b/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore$MeteredKeyValueIterator.class deleted file mode 100644 index 9c4cf4f9e4b9f20ab05b7edd7ca8f2cc9e55728f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2680 zcmbVOOH&+G6#niE&@wb(BndH^s6-4fBn*>>(F{QmL3yiCv#`p=rMW_fnQp4PiBkRq z*ZziVT$n1*#D#I~KXK)2;l{H3&Pcyl(f;|NQVHfE)O@8+~{q zgDh^z>P;Em%HTGNJ($ED*-rJ~dAu#dI~Jy8epiMW(RwdKg;@)87T&k;fkKBHnLy$C zk{@j4^{#fdO@2pzwxjc56c}A`y)CYEWb%tPo{lgdaE$nh%`-^*7_6u3^9U#n{Jj74Tav=S<)1tLV-Za6fd za4U(>?98&E7iP2W(;EFTLSoYO+-OSS>lQ3dgwft~-R6IBOde_N?`VnXn(;Ro?7k(} zGb?+Q4HK;EjWQ+uOTMGaB^|i3kA)o)5ruP2-qtNE*z=xef|;@oLqo;YB$bmmJ#-8{ z13K^|t^M6PHT`U+STd}b;?(#tCZ8y&?Gq%iIU5ZCNruPg#oE^4iuoc^NRNzu!aRA^ zO>Sf~iP9X>SuYCq8LJy7R&F54nhi|FzsC}wWmZV1Q}#pdde@4fuYAslpS2#lh zclAA;FKc%7>c(B;M6~8f{jHiSy;BQf>j$Y;Jl3yjM~WIn!@F_ms&OUHm(Y6UWO0x zksKaVSZ;f`71`r@K_S~5dWAw;6V0JhIDG`Oz8(0V*V_MPTclQG8GY?ICJcuP6TCM+ zJwd;;f+@Br^0-y@BDWmo*WHTTu9p%d6Iu#Ap7vP4j_+;qMq*9#X0NlnCM-9fm1VL} zTle$A!e|Mv@;#Y_;(N1$^~Q?h`V8N|DV*hw#m9BO)Ekh^N^NrAMV$+{NS&9t`wD5E z1qO3thfulik;+{;L^^l%5FNR(hvu_52%#JD;x*V$?;co<2Rhb?=<=ciTue2 ze9{~T`5WRaDC?=M=u- zuEs=M0sg95wrg6kWwCN35i3giC3bqAJ2}}~!|Yc037KyxXoUT{0k$^*D-waNQNBVr zb|*)nyb#03lzSA8f|n836+cQ;ssTTfz!yo4HXA<1z0cf9DCcUlZMt;8oE8hg~G+` xv|Ma4>FGEaow@IDsyW8LG+=ioutg%+={U;Q`DH1!eG{*7&JehdAB&fe`WMK|sKWpN diff --git a/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore.class b/out/production/stream/org/apache/kafka/streaming/state/MeteredKeyValueStore.class deleted file mode 100644 index 0a93ae523ba4cc05b331683ac436f79afd6b593f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 11596 zcmcIq33y!9bv|bnz0v3y$+9h3#$Xv7ER8gl7h#NTjAhvd*|KaTVJsVhpJ$$>u}3pv zW+dALLRg#-vXIb(G$d`)l*Zj?<6v0`7o4_~CbXmpWNDHTlBP}fHKerJ-2dGBW`9~x zTR!yeo%_x`_niMM_uTvZ($AlLo`~A{eL)(eH!AV?kv#r0NJI3d^!`g8|6N5trk_af zrvbVkq(74%KM&A3X)gunCA3TdW@$YEdPQ1qfL@l?7ocxQ>kn{+v5SL!3{wQ(VKYik^o;W?NWoU z5TYvsyv*Rn051>lRY6+IR~x({Nb7}qklz;IYXZE|;HDsLqBjHF9N>__Ee5v+X{*rO z$YEi#s*2lqb&%S4O%=s>tvuGrqg@`?R`GhiP97VC>-93Wv5GhG<|@8{x5(p0gSQ6h z7VeOD+YIiMc6*Ted583N%Ht+^+-z`HkPh-L(Q~&vx`pv(gZBivhxZ2QPTm*fTew$- z`~0vRQS~-3Z{vQ0Z#8(o!M8Eh@3Av>%8u@~Pwltjqjnd5S(#*t$?Qrb>{MsmN~i5K zlRqXunO1ivQ-f{Rh!q*K+m2gDk6Uf&Ov<)~V~IiZtc=~3<<$p1_9mvvzSv;G%8aIL zrVVd#`sV)KoBH=}+5$GdOl%k&R+MvyB!`ESiMG*9ES_%b2NkIOgQ?``2vcqMF>B0f zi(3${FO!0lcxjA`W`vHZX-aOxHmD=%wmv(NPNrnipzTZwM(sFY(;R}})e(>DABL6f zRi?0_LsEG(KAIlV6jdoZEx0)J&MHjJGYSGri9n)TCOHy|0N4>Lm5F6y$pjO3VLTd3 zWlk|wYn=rTh@ZiXg~cO+sz4eTwoY!>bBstbk+Dx^n64|AByZu7R5D_x0Z7}vY_C&K z1iOmOu|zDhh3Rum<)}}`)y;3Fqn}2K*(_Y{FU|YmU!BP)JfgZgmauz9hmY8)e(Oja zBemVhh!x*&rDF2i8LU2TpW1GxZ4Aa^cS~fnmrLv{l%=x7RBeT`09y5!6)zkTM<0qI z;-Uh4feBHdCO|4%+ zMwr)^u_DL!SR=|NDy!8as~LR2-~odVGCljhcf{O!(~0~4;*e$pc{O9}1jN+irb}6J zn>ml<5?ZB7L_Nw^Dw89U_*E*YKw7aA$OIK>mHU|HThVAYL`g(WIa!El->g$-c}N~A zACd^HNRDV}yvN7jwF_Lxm@XT(P93p3qERgg(|xlSq-^VQ0#vh-wKtVp*`XPB_JcjP zl^#t=CV=;4Hi_I*Z%GA=5!d3>t~==TOCWtWHJ2mC6ok#vu{7)gw%(>@IJ_5|6w@_@ zuCn*YF*^c53phAKAV(G}-Xrs`ut%Uuz$VdzUNbgfkYlf7TIs^QoVftwM{XNMCi1G; zmBI);Woo@^!pIIYwOMsT8THx^l~Rz*#k8e4Vuw&dHbbVwq*STV!sI!+V(!W(7wk;i zO1Nw;_l)UTYyo_Sn0AyZI0r@wFpEpmZh5hyze>Z=v-igso>0LXcWz`lYi zVr@A~*D4}E2TY`f7{x}BFGgI7hZ)!i7*Z3?Wu|QC2H$S*9Z)Exbab{0rm#y4m*hKV zN7}VZVQ(j*QnLiDh*X*@R(5ODrg^@x`Iw@nP96fE)IT|m$L})fHAc30 zo$p62S;jOHkD>TXw}EAJG-9i}5!KP#y}=Kd{2)IhGM>gZY4W@IJtj+rQpGVZk||F)rYuft4df2l#43UsWIHIte=Kiu?U5? zJ>WZE<;TK7rdR&&N`z8ZDzg97$K7c}E-wZg%%+(mm0ciZ56Wp3h4ayA^~>8^j@US7 z%RO@Ws(H=w6MFUS>dw(bL>(kombsO?eVH~Fo|gJ_spxVRDVd(TOsTL&Mo_t`dN=R& zIXhiO#o1S`J#azAblcQ|=rEr14Nrkos1D^S z2Y0@>i^>ZtR~Im5EcgASu1L9TqJxIUkd^LHH|@TJYKm`nau+Zkbpz?QTpQH!4mneYtZkrs>*UectK9 zF?&$5i=i^=-lMqmZt5yL0Xh_zHbV$x2@7Jtu}k@ggn$n>rEtKHyO?Z-;=m=Z&7|~{ z#+ni)b*_%YZ42O3X!GHksA)S6+_W5DWrv6SpUmuhkLQY{W$s>QiWwK#gI7N;-O;@G8HoZnQ7GoEU32vaSNTe)=~!g@G)sd1do z&@RBchv~hL=@AIFfdW*8(KkX9DH?50_iSP-9^%^Q{8&$E>_)!P|G-ZWtkr z)KxfhCWy~a!^U9QXhZPgSzz$%^hJ6N`oBi6tJX*F!>?ns1kbJHqYkR1PO7Huw1{@l zGTMo+-Z#;;bTi&|(QevB`|-4>8`j=~)AL>&iud6p+zUSa^f=v0pQHm==@lN&t?;qj3cm;nSs@NT_0Y%BZzRtph-et} zgh8J$=#vKhlIr}jkJzAJS>*Wjr$CP@1jq$%q8G`)%0#`eL7??Kt?UVfTfNUw(*!lk z&;<&?uq`d;skJtY2T1V$D&KR|HsB5Sou}0seek?iKc)-1M>N(7M>DL_0dg)2@Jebu5WJbYj6V=ma(DNyLOoePO zW(pP$$i0E1(~p3c;QDB&>yAt0}K)gj0--xBRVnW9RZNr?iIY4Tacqs})!GpNU`5=@MrxKGx>>L=6Ds{zy z9;M$@U`X$`=(hoexQK>8=mi3g+z>brJOB#eRWCL?Po1rkwB18bP-ROithhr&(Q$Jb z2nm-)HMldX!SV(%mN1rVcL+C{e@kK9j9VEfw+L31`T44}` zJV*^Y8I!Z_QK8dA>q zekEh0GQ+kUxfjek6sFdJQ_O4UV3zc92DM9rNs04Cs5|^-uSB3wi4tXcyp9Kjn zHnfUdze?6YYDf5zO1#o)1g=qO>@jHf5hUryk-9!ctD#dnJai|00@P2?VWi=_Gg>)h zEC(4v3V#k53Cg!AjC4$BB!ymMB=mxjrhgvvf>srIe@gAXe&oeE2c?T3!aa|JQYTO{ zKzKT*Yt|9=cnARXU^DYI_$Fzuz`tC2`(*wt00X!+_|H?XhVjg$Z_9`qlJkpP=?pE| zSRnux4>wdeNFG4iB>W{+tC3u#N`X%SgHK}%dXm;6Rdyh2Y)3NeN6t8aTyYF>F^u*h zWQQ!GYq3H8A^nj`zNd58o~9S*j{!pv*T%9DX|EGRHT{b@dX25n{|V>?(OUBVjF$Qg z`ofQC<$^+BzX%Fkd}(0mCjeG9oWo_n_IYOGebr@{si-U-Ztyvnt;4oY_}iu0Lhm~Z zEuRKvC|TekZSbUZG>L-YSx`>Wod_7gN_*4c972a3?+c=c$RniVQm8a8LY0k6NT6%ouRtPP9y${B!ax!*bvcP93IA9QqAhTd=LIE?PTX~lJo&^e zBeJ$$=d0k0`$;?{+3rFs>jXvVNjkvTE(lmiJr{&5gvzTDt|Nw+d1Jtpr^ELm5S7g@ zpTxK76$YS$lf>$mVW=R9E$$KMi$}w7A_<0iY6Um#JXfl$@Fk3X87cQGc=swb(^o<9 zDgy6oO4fXumO3z-!nt6}&Zck)ocU6>wUP&kTW9TsP#w32_Qt>mI&4_O!f6`|%5T=IuqPX@irRR9^B+jV(J_1uCOE2SD$|-{sHH;Rb(0 z#ROSW9UM7x=>^RRAA9nZLI|1=1goNNB58jMVe^-CH44rS`VRG=7C%PcMVk2@K1zLG zp|Ay3=s|`2O?YDksG2cnLWTYb$4jbE3w|41aIry0!09_sd>K;K>*(oZ-VS=rS9i;hg{(0_}0I2c3&GD85_{o-E(^BVa zv=>rz_>nr_c{*C^i$6fh{9C1&F4JlpEif}zrqwt}LB@W71X@_B3OyvQ34qJrIcv9} zl?aeM!z+RAQO>U`qUHj}_tW3Yp#qctf&S5DvS)Ua|B3$DnS2X$7Og8oEsb?P?Bk+t z`RZELO1-*})4!lr6_G3HU)2sFy?;}xdYvHuclr;->xU@!Wnb`MCmMu%cxVNF%rNt& zX$r_9vWtj&9UAB{I}Rw!}aN5iqU zn);?qRZv6m=}ugE_(X9Eu|i?@*B*t$r}TAS`@vXbk=O(LSCF5VoDZU`qM z@|=0;w%a2qg#~r-WLr%*5pHg`YBq;kH-~FdcG3!WL@1X!+g43|Yir7~m$h5Xn-lR! zO!V-t*xGC*>X?x|a=hLpPOz2e)gW}kt-GEsG2@6^fpx`Dqb6gEWKVqrU-WVGsziq;2E zK;mp_&)3FmL7&2uVSFAHfLht@jcI%mNcva?8_VDG(ED}tDxC)jI$ojT{T`w^4kVbY zs|Y=!zeO@^PJVRXA!izibjDlZ*pXq&$e+X7=h_mEQr8wMYDtlBRY-_@cHA*p;h3yeIz5IQ;ctki zlg*aYL{^ETOVB*2nI<-3JKdC9KNTkG#~fx$(~&6MWa965lU|%>cO+!}4-KGF@iSc)-(osU- zK#}Q|4S|SsUefETOkrv+Wkf<7YNDwo;TjWn;Bzbs!+ty`oN8yyHB~^c;kqzYWva>Q zSOs2w(bjmf!)ggi&ITi~pi^Liyks+%Xt0%Pgo3lnQ?uupYKod_s%dJvXcRQn3^mhK z$Eo8L%2#A+K(M?eC|7C3N(GzJcCb~dPB0#ec99|{$`xh~ho&|fwc5hbvyyG;4j#*K z`ElpPHpk+fqp6gsETJItL(O7pW*xJleq|1ai8^RjtcCU^l2(VlP@-M)*yjv8hmOhm zT!XHbmv7jPvsyVw)BsxFo{V=oJxWgAWh;X>oJdfL!rV-O(PuKfP^LJK%yU-Qy#1Mq z^1t?%=JL<(XO&{DI-gxan`NKXl!`~wwzV#7w<`pO3tnzM>wag#FfG$@=qo9uOE?*_ zBKjnbT@4mXXZ|I_SJXboiF%Ju`WDA^W!YMp=Te?A+QX@})>d+K$Mlq6RfqdW+OpbW zJLcm;H=}F{qHIYrYbwYa(9;@Cr)Z~#n^rDo^~#FPTzbo`RC6+tU^FRusOAYOhAGq} z)fMA4=V^|{Q>@`WC(GBzM`>Q^Wv?Vmmc*UiX0J8bJq-0_aVA@HrWy9JZ?(HL`e|{v#S!62C87}; zH+vN0=Hyy8bpQF-2&621zm5t|;z)XhlqC(NC_l2*kWiM5@?KkO@LEn=^{vdue7kjn zO6;@cR#=#OJmdKsL8HWC2x?{de$T{%j-QfGa4gPT(u**IA7&=Mk6f6HdaUDj(uep| zfOGkA$qSn661k8$`$*%hOIjnJI^pIDSWwvu)sKS4ez+QY;jZd|r^MR>qX)iT6!yTs zv9f9}iYohIlCJimcn^;i;Czl{@NhhhtU{QHMVO7nn2*zOfu?EpxDX%V69s8?6fWZ0 zO;sxRe=$Bv#i>p~1q}H4mFKN|1jUVOp3KpRZ;E;_hAQRgwYDE)8|U_7T*%#z@l?K~ zG2rgSgkBsI@>B*qu3nUJs@jK%il5SDTuf5@k`5?@yaDe%Ojc}SYGie+!V{PhGPpfp z?88)foF=R3vI_QKMlWV|4?Y|4^x(LFy9dYjpj;kib)z`s)2o1QV6LO*?Cn_Sto$2? zUzPWxqOoL-=P^3bHJ8>jG`cf(HS~cYs2Zr=-wl7LuzG)>a9|HTv>e^oMPKd09z7du zR*vxbh$_&tTSwr0#sD~z-djqKE~ocba6B8U`Gvia<2HU?ZznKz5E|VC#ZE?H7w3J1 z#8U*rGxYw8;d2?F1Nu!rDN>z$FHf22uuW18DykWqO(ZnX^LNl|b zMVlCO4rEnFu{MXTAXk^sZ3Q_Z3#4(&h`O2Q1pd`klv5+3#t|$h$kT}#+0Kg@eV9K& zlWF1#I?zXHW9YsBoid#+n@yKhp+jp{N%xZfkY?3GnpNWpTuD8IdKIpwCc|z~_&BcN z8@j{grIayrc{RaS+*tRdTGNjO%$AcH1%16ZxfiFbA@0`pVPVz&-Idk7IBm()hsYOr zj{o8X`J13Ar!dz-ECP!{9vZn=66pY%0^X9-d$1&A1dPYMI}p%NKf|q|9`I)29WY$; znJk8Tejm;}r7&Q4xH#)nf56j^X^jD2X<;vxmihazY#)}pu+b6FJ-8#_(K%e|-hnxq z^$I78O5J@}$u!E)R+&DzkxdRnB=94(a(_HAl}*GG5DS3?#Vg5^_k9 z#};z25k#6Nx)`f%Ox4Sno|h9>A7ct$!8l(@9A3r9U(N9oxDMCiR*rY#dM5r2tX((a zQTFQxa5G-OC-E|F!K+-qhEL%Q+>W={X1|NusNL;q8t$MrpV!z)pont4kg}#}6#1!H4|%U6?$+U9ox&b!*-g1Ol2T&C z52J{$2C0C1=mbGH$lEQSMdu(DFCN-M+0GxUglsQBfkP8&=nchkBrP=R|efu(g4?#;>9$wT=% zS^J1B9q(xIU;%d)Fh1fzDSo_>4`TxD&(Tk=rR^ELPNg?#JPNZH%5Wi*-*06y44hc&(`TXY+M5T@E(OKAtpzDkQ1BkNsUL5BE4!=UU-Qr(&>tm@cTXlH zlfcY`{9Vq!0QzWW$TGAD`P?6Zm5+@R!pqFtGQN*K<+NaTka_Y!dRsm+;dxwH=i{bY zO(m`UcZQrW^kHRq&vq@QB|J^4O2V7t7$p*l>k;EvbzjS^ i*hCoI6 diff --git a/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$LexicographicComparator.class b/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$LexicographicComparator.class deleted file mode 100644 index 9968402cb95a619834122214e19a12295f9fa9ea..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1441 zcmcIjOHUI~6#ni!I;FHykb;OHDo9I521FsGgoI*@#HxukEUcT$FtkI5PBWbv*tp_< zn7DDn#*HQ}8rLQ!{u3AYh$f2XPU|B)HZ?Bhp6{GF_nhy1XXfpX*9`#IFcpD{@d!c~ z*DxrYq=uA+2@Pq6(1KmD{Y3^jnObE~mfdxWp(|%sth@ECHOqUz*BpxUcbJ^Xha*tCc z!=s$*6%AhHX2~+je51^bn(tYB%dQj&ao;jpm2X=+tK6wuL5I=5+l`)3J;R`xK`&yY zB@5X?HnqYKcB~CQyW>-kk=xqmk5#hDnBnWSetNXT(;d?J0)&woA~EdAnHQeqHQFKgzNhsr$BL z{8`RIA-hNtM)GdmGp(Dp_zDJ&TgIf&A&M|MqBxCShKK)erwsAKsx$Nkp+odF7!28e zXTmVoUXf{r>uue{2>qiW&^Jr}uL4OZOJ0OLgDfMfPP~Hjf`EihIz=+@0Z}@m%>=~I zLmm?wp;KIbPuZg6lDsI-DnseUGt49uos6Cyn(4;DlUY?yC`}YfsEN>1bO^J)w=1(< zg*`ubmu~2d^*t&g2aSh=g8S6!BL?vaWB3dmUvL%s$lzN*=oaM+q8|zL(OhFhte;Sb z$jsvmA(^7H7$BqwaxLNlouzwXuqB)$Bw>hfC*?)k|@^f6bI=$(ke1fR-!vXUv|8{ea(d5G*w){MY<;AQqXLaMifmigDTSq UGER{sU9Rh@JAWvYzs)NBm!wNXKp*+oH;YopTEBS0B{fK2s&|7!>tGu#KMRm9zg=r zlADpXSxM$PVc@ogc@1|MBBoufbBBAj!!VH9eZWoLVd9HcFhfL;E`BXf|pr-9@irt;D zYGS=nEePioFH}g=m$6M=$#KV$a!aXtCCk;Yz@Yo#R#REZpS<<8T1`0V3U^)MGW55s z8@yH&hnNhTenPy?&5|(6e5cF}*K-7~TD2l^+!M_#S5wc$+Z?YngdcwLPpm`XL})62 zYSGmgqNm)MoOT?$%Al3FyDs)ThCnR7#n6@YxLJO|>#b`>vUbBU#WPD5Vz`BVBBeR` zUOM`4pJ7W@`J9)^h&{`+iw>`sEc3Vn7zY1gk=PG(=;+e$P{$)Ab@ZTD$90VBSi)ls z%Q~K5h2ixD`5?O@}+K~)RsO^I|Y}?(od2Hm43izV)_6q@g2%0 zJ3wF`>h#PP1QWCS2z~Ouxv zM#zfK$s7WtrRiv8kn?3=JaGW+9ioZPP>JoJGxQbV0Nz7WKR^w9qyPcPG(s{g>+)fB zA7Z=(J3-bUl(7}s!MH(!f=P_fnIQEL{X+~>eImp~i7IqX`SlvYT{;)Z4?P{f0d%pk AWdHyG diff --git a/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$RocksDbIterator.class b/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore$RocksDBStore$RocksDbIterator.class deleted file mode 100644 index 711e12b1ef71894cf50c3cb60b6d71416c9c2946..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2160 zcmbVNU2hvj6g}gwbvD~3xWOcp6jMTKI}o=afkHxPosSaYCJ?8QLY~I7;tE-wnLBgunRD))nSK2Cx8DJn z#+HsD-0Z_8yrWa|u8ss|HOy&vPscpo=jIkq-sVw}eHJt<>M(GVXO=W9Cviu|06yTw z6%8M1Sk+Jxh?$;p1TL3sXSd+k)w)}&6gIeDp#|yL&a6OU&a_N#Uf@dh9KYNa!7bP| zB`{DjEw$F#uPA3zRvI*uD%n-p*piOPb~qXH_Dom9nt>iZH~z0qxS6s(nl1j z^6_fJcF7tdwfE(LEHosQV!d)-RXtL)`rpak1lqN0*S*6T*f3B=-arm{fs8M-<(ZAb znq6*H_m&!JpK7%9psJdlXmb2^4#V^`u9K ztj@P`grS_XUMV_&2Yk&b+1WFW+G#Pfd}KA#rKiEH2|V8&Q}!HsMSPE$|3w`cdSr({ zu=G#y@uO!vN#kLRQ2QcT%ZShq(DO$S`H623`HHLvE>n9EQEGog3_qa{kMR;&>A)MK zxJnkq`Ka_T;8l!;m^0K1TIi02v~E zjKp^SYeWvA1zqw8eW%fW=|W3~XxIE5vxGu7gjz}No8TFj3Q4-ewIr@)bY#-smBi308FfUFpxtO27G~zV=4H|x@FWw(ii9c!nudY0f zhMNpgeqYBML~fEII4!|UfkxM9CJHft6GbF2$m-7YlP*nq zN7I`fgbDIz1aVULAQ?})0!0uwc$<j`|zUx_Nw6sZo?Bb2;(!g_$;23)?Z2SIRl@Mpc+pY_?s${qWXG3 z2L4v}fI8j#I|F|o!7`7<7o_z?Js;u<>3vCfeOX#xk=8#*>mQ}~Cn^3}imw{@S_B(B z7GIavzZm$}2sV2o-;mb78Tj`IF2fU2d^3!1N&O#EJSA-ZQ*{5fbiY%BEAU@o{I?AM zkAd$-*#F-MJKvL$?@RZo2!4S7Gw}am{4k6kMX(<~Ht>@$ej3Klr1x_n_(d4M6mXsj z<5$xEwG_Vzq1G8{y@ENIP8`d2w8hT$1W1n;q>B)>WGvy@O)6+ASo>W-otr^Rclj+b< z&uCx&(CFcKXNQ97R#?)WPGwyy=WG6KQ)SWI@2v9nnxof^7x#%}SKw6+h)iulk-PuV3_!sUv%duLLX z)ONIuc6Il3j&^hpD%AJfVI8+(N#YlayBQ}nNs=lzJ>w)4YKWQZ$YKQ}kxsexNmpUN z*2$W&5>s~Um^E?Cie=r5ZB27=Y=*j0!*nJ#kZ%!5SxCM{dPbPhZmWv+)$a|HB7DMc zE37Xf8O}Ja9iW?7fUYY!o1L1Ys~#Wgv!>}c#<-L5TA}f=Qu&@kv{8Qb0Vn0S2Nil6 z<^%H__r2qYLS=h;oLVgFaZ+~Q?DUwO8M4Nb9I5X~C#>WVE8|GtAFOnz9D>xbm_RzG zYsY-e57~EWKh=k1n?I;K+*4&rBT(y#1R7()nN-+Xgl=1s9@b`M?QyMvM0NH=F@ zIb(2!b2L-caXIaz<#=n7a##5Ko^_pMtcPq{G#Qy^$fH5;X@QgA8K_ezxjRFaI1;kz zJgw(ADy%Mc0v>JO5q3R80S9CQ2Px+I66Ob<9fN}{hPssJKJHAWEO%A{ZhI*L&zhwm zRy}SfZI|q-#0r9RMk2Pt)gJa2S2OS8`KZm!R!(Hn)3l_UR=DoD@+m-9VTr-_(W@ln zE^5>=y3{ywFNtyrmuLM~PZuC|+)!3#~*pmvzZAZw~d)nuw>wbN9) zRLoSnRZL-RkWcU`HSrR>)KpifD@%xi_o#`N;pL{FMcoDt1WHkHRfN!uBbNE74?nGU{Zbsk9r-&d$uFGp;=zok(Y*SxF;l zs5Q~7lSS9JEkojCp)vh{B+^sqnT*b>NYN#$#Jzw3x zcx|d~yvt-BdZ-*wbE#S2LNf7Qb;!g$QrwOAD-6#AxOCk`{WkB!MX9){dQ`6f(PyZB zQw`u@g~8vOtU?Me5^geaUNNKHULI;4H;aJAc{%SFr1%(4oA@aH(ok_z4dI1`I&7*V zklT9B-@P!Uu2|oU`$*?tytM~Weo(s&3b&&k>tSPp?MZ%V3So+U z2X_?uE7(64=-05H4D?O*r%TqS0{t}SnT3+~ddWd`?P9jTKSm9hS982xQfgf@tJXD> zSzR*^)iqO1T{CyoHM2z4A;||>FDK7e<25W_3#j|~MYfW4ZR0tp#-_6fH8!7@FKK5{ z+1Pv*RToe_at=n{&XuUip>~+HnZwfI#tT@+iMq2`-uQ7W-n`SEVgoNE629U*kv5M zoCBd8w%0eXV4HJ0a%en@rT{_Jc{D5Ju=5Od32lspCfeOdsw+mgZa4p%&f!W{%R}d| z=PdR`vDQ;nTE1l#vq`2|~ zMwz@@coW9(M;OQZIrCvU&*$hm-^U#)LU(yRxi@<7;|+Ktdo%=Bh}oqj1`bi$ckyPt z1r=;FulEdxZ*ZM_DIeVsx(8-+Qx5wc!d0g)VE>4ZQ2p~fXf#Y$ds{z;YaY9%GDvhF zKy)qJ!pNgtS9uNxAEQuU3BP)`U@1$fDAL%yI0Y%Cf1w9NmLv`~m(@ zwnr%Z?XmTCyhFCfmiVo3A0%Qc@y60?={9+mQ>M7q0?-HvCc~@X%sIZcc4S(g^sJnbu-bb-}iOY4TdxOYb z@1fPos=kedDCZ&9tbr4a!}OaxU`sW#H*n;Q{AuIOT6mOW5j1I_1d4qHp!VUtcpnuJ zP`J@<4->vLeC3{`asur@^OL9-1^K^;O17UuU7_2YL&qt^`hseAE~xfSQTqY%2cp95O? zt<*l@fqW06=m!uZUr|m#UeCDsA^e#(!R~@`yKyh>qXwr)wVFD3R3G-K?!}*TP7F|s z(63<__={g+JMQN|2p{1TpF1o?@$0xm>}NDKDXKIVLLz$jKAP~9)xe;lrck z0mV6C*3Z);ATGavK8aC#bLelPPYsm1{{4mRNm$RbT#g5{=g8O@jku@rgVgoojO(6| zn|zH|`2u?AJ2zdx;0U8(d`^&u=V?0f12D->@bvbu$`=@BbjX3XhY!p{Z1Gv>1M_;? zMRpwXeR69KL#O!ZJm`^!iw+Ov=e@9&fhB)>ym0dnP4Ngv&+~V`3)qEE6ck#9P5Kmx z_r`qH{D9I&Kifoo_)hh;I>=fG&Bh##cmNE~+05JXC#m_TDDKl*gcn~-k8xcqt@Vh{ zYd?V$Yb|MPqCYlkpS*bvKrcvL8DO%cz~rOU-}9e61W913(M)m}nbT%IgPH<^yYc8E zS_Is1K9e~u-i^l=(BkM~+wirZXFOiObq5*A-;y)9b%7 diff --git a/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore.class b/out/production/stream/org/apache/kafka/streaming/state/RocksDBKeyValueStore.class deleted file mode 100644 index a863a1a687cd9e0b9e7b79ab99a7848aa9fb2f70..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1413 zcmb_cTW=CU6#k}A7FK!@ytH0hMWsL;O*B4G`vAtoh^CQ-r0Ij1Zb!PZ7qT-L`?K_g z#3ud#f0Xg;#Tp8!NPL+&bLO1yJGYr1Kfiqk@B~!@w@omtnaE<@z#S85+|8n7B8Phh z?i+Yupv;he<#{q_IzlO_==jdJdg?`^t^GhUWVUS2*4qrNa_gh`B)B8IE^q6=_PX_! zA9T6s3+qVop7_ucT>4$p})t&ePq?b&&RN((J{68&p=+!4+|Mp@K}&J%6~K9&Q6Va0L% z1^=9c82bOOI^#K7zU%rPA86ZAeAjkmTtQ6|Lund&hFsh3dO{Dvo<1LU`O~+!jBlit zfjk^3IJP!blSl@QnW8tsj?NPUUymT5J~9;lK> z4HrpTi>*;Wkxt}?OSnu}p>zdT6ZDk?{XH`D#R0`i^#tj?YHf)5S|PJJgmIRF6Xc_a uq7u8DQWQ@qiU|b)VingYloYNL{2XrJX0+eJ1=aIuSS8f=;Yv2{GFz;>>U?#`xjVUhyilq%Dhn-9OqV z&64d0>_;s-EvVpJVw(2!yglbR=iK)9pC7*ftmADQBUp%HF@~3MB(N02atyCxScxGM z!)hE^ti|yfZ=%RWp)d^YNK1O}7($uVDnoeJHU-0ISz2PR-K+`cfY%yiNtJDbH>%u` zzTP#4y;JEjWXrZwS9psXr$RaBC+A!-8d6xEtLz5yvnLzwfpEQThB=SqK6hMEvdk)P zNRxY_WSWj}T|rY5EB&k8*-d`A^<)ag`2+=pr8Pb`f@)`fil?J!!O2u=eRp0i9 z6KXpRhKUEf4DpKHb`0@B`oo&M*LKc_OCX6+>ft^#LsH9c=2W^gy{M{c0?#nZFy6d0 z?IxF2&aj*0D5_NDNY}OFAc|q_D6guTZq|x=af281qFKd0yy+O;z74tQU`U zEnirt1@^&dREO|}ZYxJYbv5LpC?x10Z7^gWyVSm(KGx1?12?8NEnX=(4!@-B4RW_E zUGgpW4Y-rBz1o?1kl}~XLY5xGfKE*7|EFm&b07AfI#{3&WrRK-APiatM(H_5nnIG1 zjAXCCzSGM9#_2g27^jh<_e3WE6PP58;Rz)V1j!yEfltqde`4S;lnPa@5WZ0&y~<#q zL^@&7%3(Ma^DTE3(}V@g63IC{#dECS1-3BX!$9vcNO$>71#G5o+3a-LByR`|^;d`! zYhZ~gEcc)$dr*g|n<=_y45IMOKcS@fQ!0`4lZV!>5x>Upmpk4a)C}my6QHI$fn8Ke Pp?;oII-`F4>CXQJp~nsX diff --git a/out/test/clients/org/apache/kafka/clients/MetadataTest$1.class b/out/test/clients/org/apache/kafka/clients/MetadataTest$1.class deleted file mode 100644 index 8438c6ae95a0c63de7623a740e90f7e4cdbad296..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1619 zcma)7+foxj5IqBdg=Hatc)tJ=APMSn^9rIEgMtB6koRF2$jY)Cb|SDZ)b*~RTZ5u zHO??v@Pi$jm$)cOdzWwTa$C5L^i*gsOU1Xi;_EV0+GW=995u%osFS2g+bAdY%k&E)PRVY9?L;sR@d^IVgKH zruEs3jyc&7R{ph+CFS@Y?dx1qTX*(y?w6@`UX-YbsD7vdDal6+O=yO)u^C!o z=-{uoQ%7q_FD~hdKEj*ScpI_ts>zGO6}~6RK|n2UbLHkSl>LC_x6#JaG`7-4z{eRHPesm2oW*(4$SH!P7XcS=u?BdE zM6@fPeFydti3(asYO0|1E2e6;`jSzV+A3&2{Q135)jGy1uqx<0{Q2cAf%d`1Bt6c} z(BWKyLE}DIL$V*xioz7im?!xmUf08%qWfP$e-wPa7VrRhwh_+0fx#Y{FO%McA*5(d R&^KTN=SVWir;BF#{Q~fxsfPdn diff --git a/out/test/clients/org/apache/kafka/clients/MetadataTest.class b/out/test/clients/org/apache/kafka/clients/MetadataTest.class deleted file mode 100644 index edb8d7e54e2971656dab9cd954805fe7bfaaf26b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4420 zcmb7H>wgqy6@F&3o7pTANK*>Z3oIp&3xrK5A|;3kG$0sKB_S9J=w$azGGsEd%+3&K z>s{Nbc%@cLy;sE>S`;?1C|3K$pMLQ_@S8vX2nC-rGrL(x3Tfas^S)%>Mn> zn>PV$$J-5fFD!YSmd8{EJ7A}HB!7i?QTHD?o7UlEkb$eCn1KOX$C(V4!%oAKcS(r8 zjzBhNGnm1-49??O;dw5N=jHtgA%9Xku!^C zep2P8%<(BRm$xnD_@&(8n6F2}iZRNZ@Z7*D^n0G`Y3Lj{ZJsrAfp6KlyzArx&m&XL z^xbJIpX&=>N7T3?HLob4VaS?rOh53HhQ4>hyF0pUo6baT$oDK~qGu1~)$g_(%ip7+ zuC0AULvo*6pyU+;mZJuP=`rOEn`1U9%>!=Uv`0+Ol6_1}`jZyJ(0YI0hgHd^wzRKI z?zlsiIu!UoDDZSZ%W{3=zVgUw_!LnIB@Go`4JqF( zT6r-q3}dZ#8WMi@ohQSSo-zv*mhiV}XdLpHjl*U!^dafz9) zF(-E9;aY2s76}V=%(VRB(5Oa##vM}3WgQ zGjeyym5>>7123=kGp0mL1sRf`t%3yu@5dekJMpA}9_%*IhrI@N;Ykha?^~*J`!x-l z-$iX|SbaAH_Xd_-P%QHGRX}AeCp+H*vQaX#yRrtpi|-luK7K%~+%3nz5Ah=%KQ{0> zexl>227ZRi27ZoT(D6Yx8-)1FI!YB(VXJ{(;#UTKjVlIzgWnQcLu@W~)~Q@9#5c2r zKxKV5w1<6;?ncn896Rm|4K)4Ag{+iZR)yWKVBig0kq&siGb}*^Cvl36qi=?gXLE^- zVm%qSir?w@y@6}EZlH`=19RA`q2*3>u{9;%v{km!Sn16l@TP`6Qb}3YsUB3Eo-s>e zlFIw4AkK)5Zs2dpjs<>}ECs9JwE6~a;Ex(|3lr~7PrFV|Ni#0xqP}H}wt+wYxR`4+GyV3o-Z^UzihKi4IzvdA2D2we) z7K4FpGj5BRQ`6mIP2$12h0%&qw9gJooCNe$O&9(1oVYahK9^~GeN-Gw!(x5t4$(bf z2%7{!!|tFu=STSDtZ6&c&bGEJ@@T(lm)LPuE#lH~@u1ODX9A+2!p*^Yuo{kGFCedO zmHKS!ELS`{zL?W8E^>mUqpV#{#HJE!%2vMXRO7H?5d#h`B_yemtyD2`OGf@_*Wx^0 zW=iE&j83&345N`8=giXZ&?36TS0f5Z|CxY6;Z|x*qVk7=a88W}c11RQW;Ay% zmXAe>siy`fG2TRTihaWKv7XeSC=6p@HjSMLyd~oSj9n})l=}HP~C^9u1?e?cG1Xs(rz{MQfS>AR;?>z zb@O{6|F+bB`gc~uVQnpq}8(NOwM70m>4U|%%SD-?SBa0Mq%3c4yMcX2HLi% zR$|4mwGzcTNYYMWHJ`1FWg8mNjuv!a3pO)WTNt}enAnN}a>Pd$JaprE{%ClCSbK$N zdmZoN@Y{)5BJCzI_BOvX{zi=bix~SidhtKvOOn4m1jem6GYR|`14O1qxtDMlgS2D{ z7x6ScOz3#HfRAvM!Vz*_!1CM3BB5g_2~EfFZ4#p$pTpY8g`^V;Nw?5SqK>2dzam>- zxsej7)dLYbtljGjBUDZ^a10f6eUuapqZM>%teX&BCzgsX$qo_LP)2KW+Z@^nyup>& zyjSS)8MpJNsa>h7%vB4c-os4w(xy?qqg?J_2Eo`S7}Z9`wS|#_oK zD5=NEcap1T`0rGhqby|rC#pF*9_Of+g?yaZstfb69rgFj$EZ$$x5-i`u?jIQblH*ym1%jo82zDcZQTLZ?Jwmeoq zVI9T^@<}>m(bcKY^;KabIS{*^B=jSA)+RY2Uv*5iw8{}m6JwihVB2WN=2<*(cuN`E zx4c1GA}p%q;knD-+0XK){&O_}Zi=;#JA^*xAp!F-0zT1{#>ZK1k&YbS(iKDZ=LJhg zlNLMQQd4w}EtSl9)>DN!YN^7UxKg$1Ud)PdsW8uL)|Q}|E>DCiCfN-@cQkg0@bCZa Cxhfg} diff --git a/out/test/clients/org/apache/kafka/clients/MockClient$FutureResponse.class b/out/test/clients/org/apache/kafka/clients/MockClient$FutureResponse.class deleted file mode 100644 index 915621b32141b49b6e03fda047e010bfc0ec8ec0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 749 zcma)4-)j>=5dQWqO-y>tk6LThMkA(BFR3e~Z;B5M6ojbINGW{~Hk)mG$>lce?Fsp_ zdEs?l&_#-+VK(|E~W01#p1ZMXX?}ggl-(^xPdU3aA(Gk|8iE zo#-T0hk7t&;9YGlxR?lesrX2oj|7)`Jk|-H7;UxGF}Kr+N_o$kOxh1bFo;sAlSE0Y z1`OPp`7Zh~dDgpHiugpB$k{i}oNIxhe3aSDsNnktGPEKqW% z!RBoe>bKO>nUU%!a`h|!NAYebeu@CA2m-9(Re%B>1t_9aK!aib?rI*Y#ZYyWJQm3X z|JEPUk{JA5H;*?J2T%XC)A`T-9)x;Mhs3@rQ& zFKo}?hxHk9zsT>QO6U6gO-M{F%538?_J|i$&3CyJe?qrIuBJCfuBuOXlPnKUDF*T| J(zi|d-ZdfZy=VXc diff --git a/out/test/clients/org/apache/kafka/clients/MockClient.class b/out/test/clients/org/apache/kafka/clients/MockClient.class deleted file mode 100644 index 9e9f437c82c37e7526d20d4c0cef78fc148d8241..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6787 zcmb7Id0-rM75-+E?lhZ8Ly{f@+oqJZB)e@F+Hy4|w4_(sB!vcA5>T0LrpYv$on?2n z2cV#+1r-4Wr6{PV6crS-O$b%MTMtJxmzyxMDT9BCxZ9l z-b%a=@0ZI5DsdkUSK@wrP%aXwTyexh} z7QZNWUy2avNgZFVz*G3jbbK{}75JKtuj}}Rj&Ew1u`%c7GS+r0n|4xJOM}stN?DoK zq?yfHSqEX?M|jIZl=vdzZD-acMq8Hgflqkq~bZ(PG;kma96X2%7l~2SV_~h zos0mIX72$F)$M!Cy{1BqcUdmqRCL*WDN{%ajZtscc#lal z)faC|xmKT*X>Os&^o(^?&dR#vUsEJ;an8z#uI908Yf7+7+E&WV##`0zb`MX1S7dx& z5#1*sK8P#yNg#QSR#4TuM>(H>o2Edm!$}Ny^cIaiM1>t*W2bDlMZ@WhC9E5Fc1=5Y zR;}u+s@v_9)tMXIWo0fgcO~ie+IA;lCU=+_Tb}*Ja<|_mRn z!ifo~VA#^sqoFM2>|+FNVMHVxMr1-lYCS`XF>+!6ik(yt+i2QJt5?IRCr~VUBfWOk zm((z;u_(a0+>D*-YZgc8etc;tg;+9j#EB|)Rj~S4bh+(X84WXwj=ha5rt4(rv)QBl zejuIqeM^b`lL$SP6e>EZx?D3c&|#+4Vd{9ARI*lzPKr(1B=JO~5WyN&O(qn^#wry{ zLTPB4SUR7Ce(F#p?Ie?QTjQ23O+~I~x3f|)nEv@{Rw3of$+;bj)bfOrX7rpiUZA)j z2*xB_8Dz1JZ)v!o1d*wLsLDB*PFk+DP7GUIBJfICdeLO|O^rxhV<|-~<5#B&TYOOY zN_>9fxYbF(7Q&c`CyczJ3bjSbk;ZaUoz7Tkv$+52S<96&IEiva$eDI(W76*Hcl{34>f}-`t7l^q zdvD3|#r3fCNm={zlABqar4(H;3Def4?XA{2F2&hEJ8m&>K3WW1fVFf`VcghkX8Spi7+8l3>G#5mb(xHLV7&|o1~y=$fz5K+ zB9~^Y(eZ5q-@!p0-!WbK#(y^O7g6+A{7uK-4g3TDG?2r7Q9WqjUpT0tK>>JUT{3C) znMt-Q*V?c@VWqv%a-knr*#-M-x1R-mLB8f^c{gaa3Io^Tda>_W12@QBJN~U< z`FQ(|7q&Y7BWL&mTgV`1E&+K_!)fC|#?3wKHU(3*?b^dJhC}Rxf?H*<_L?2%g@PCJ zBLTWWK#-ZEAOz2D4svL2~CXK5^#;v_x3 zn4Pan!x>zj$n>$rr;CFP6)c^~es<(uenHs$@=)J0qiF>5+ro)33!g=-`-65eX=g3^ zub0ka8RL9fvd)%D^X4r*6IwcPYYQ_AoV~1ZoVzy+rd<|YBcCD{s;*b>&kpM2rIquP zH)E^D7%l1DlI-Xx9$c%LOzz@DDNz#Ko^?DFKY?2vHTKZQo6S^jk{@sy&K&PlU-d*n ztm{t7hoa3(OZBiZ?EL{wI!bxgzM4{(9Y?BA_nN187^?F(j<;ycfZ?}0P6b^W<}`MB zLu&EILR-@goXSu7bD-f|{#&9}+6(`Q5t4qTe%Z44TPq!kds@}ndOou&1<#xK%mx)a zxAB?1DtO+C*}QK@2d|w(xE*EG0IZD;LyMLjMJTErMOpM1%Daa#?I^;$)pBhy_ z&L^@^Hq%My!X-Y#ZQN7NcQw%?2*-v{^$2E+pgM#HxEO+{409{!B!OH&gSa_>tdUNy z-q+w#?4X232j;;q^}*{#fu9t@&hi3$gquKr*I<2-in>5WU7(^aP_Yx2QIU9ft*;_f z@gSdy3AYuYg#xrtfEEhSy3ymKoy}ii6Dlk4^UUuo^5}Rp4VU8zALBVpS03%5(G?@8 z(a^DU1hpCtp@lbf8lJ$+&SgWGHH6u*`>=%jBF8YNdl_$x5uD7F(uPq#f>Y=yO1wJt2Y(+Oc)WhrLihGnY^RS#l#4FSx z_5?oZp-)~(_7U#85+*&ciyDkzCBcvp+gLe1qLoXf(86Lm1cHwF|y1v_AqAfsUqe*iJ4`}%=v@|DH^0#Pl78+ zumuT5S}&7rcVJ#U7NZ|F>8?a2_FzC6+7TGqp^Vp*F=2#Wz`6(x;N(OJ38tsHUo>qx zhK1e2YZ#}ko+dZGP0PJ|Wz(G1;rj3p77bzXA=LV}OX|ZTI6cGzd#ZlgLxl3q$unRN zs;I1<>P|xm)f6^@g4__TXc$71oGz0fs^K0~9Yd_UcIhydJ-)2qnp(B}TKfG}4B=NZM6WAS z-w>#8P)ga6aWYHrRWu-Py0?&Z@&CT%52XTL zii-+~3m$-nLj0RptwdtP&Cbqv=9_`( zftos1UY4q%fcQW+R>?RD6D82@Me%_n$I?4g&PeW$q~is?3e&`KCw4ba-4dw2@Iyb{ z7AUW``vMg=8j`To^Fy_h9S>BzCkFu|&0gfmpf6)z`+2aE9{P#E@}5dkR|dgAdZRZo z90n>Du-}HEirqjaiDJ^rVo^IP{S?KcsW#18n**d)D+=A{cpRvdf;QHRfqzZe?weQ> z=>DT(Rzi16@1u13L}0VH)4VzgeII49r(XGbbFG=2%_DgtZP>VEqlN_=7V0J*3Tzj? z+YDpgiUb;(>;y7AaNZ4$l$UbqLgudenCXX`uGEjh^QYS=teMr(73lnbI_~i=u0p^> zi6^J$zs`lBPFrS-OQF9=@7W{+usYvS>Xgq={>EqtOFS)FAVUQwXz*sUMn^B>3a!9Z zv~tR8X6oE}r$R#K@fj)?m_@b(K2y>c7Jr?fTGR9##lnd=!7etg=g{Z08fs9QxWMVT lv?{G=iZRYH25#Wyr0f>ImT()(v`yX&?xe{k*XW~B`vU?)`WpZM diff --git a/out/test/clients/org/apache/kafka/clients/NetworkClientTest.class b/out/test/clients/org/apache/kafka/clients/NetworkClientTest.class deleted file mode 100644 index dd738c6567c47aabd1e5d3b5cbd252df79b8c3bc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7304 zcma)B2YegV8UH_9_Q}U($r&hwpg;h}P9hQ@1v`W|jzer5CwL?{P~a?`?Mq~xkxnFO zOIu3m-le6bbPwpDEh&}_E|l(;?!7lHbeAsZp!EOUoxF4+`1jL|ckkQ(@B7|+e(-hM*3O(?#u z;TsyhspDJtwvKc09l3m0M-t!D@O>Q@;s-MAhZ=sQbBe?PVR7< z_bN17?X2x?Q>boOx1XZgJW-3f?5x#Y7#*~7edb_>!ed=d%FOIHbGEz(gAsScW)#+k zC>AD8f<=)vdCM(~Da3mUS&7tsJ8zR`Yc}h+rfWM{vZOf1?offcC3jduPEPDwE`jK= z%=EF=YoWd(%rQ?df}U_uSUihIAbOc_=(V!xKBqsM6Hekfnw{sE z!p0Co{9xp)YYQaJCwu${;X#eGVEidv?)y3*BvI5lI!3iE3S855$h^)>X3XqxvLlnR zhRqC3ajo{FDQiqz!Rc{NXM1!kV+A~Vto#_gM7BJW7n?phYbu?l( zeIal>=fw1Z5yN%z>8l~4A%xk5@%8OhtvqDN~73qwQViDiChvbK}lam=-Lc*DN8eD4#bi~TuL zPLf@=2kzx_^Mo>jTxpp%cWa1-fwM}0@#VB^6Tg^8Oxwr4LPuC-3v+Z!hz#EgUd51v z*L$5pE@kbqCCQi1sSXmMje<`Dc`(b)dV1y5fQ2E2)5~THj^~t7@>SIahB3lSD4XsV zvzZ!7w9h0h9MLQnyN8Wtbsgn$4s09kE%H_;> z#DD{WC&bV&?^56>TZqDv8NE8Pt~G4sM74zmu0_sJi_~I6Em1L+7eBCxL2KB~<`V_K z0wv6R!gdoQyk(t4P}7+D8x2*jmTGF5p_Z!^3hjL()~wG7JD;Grtj2bJ#7YYnGm*E` z)&`3JPPqwZDB+SqsvL`XKA&J4OdU4V$x1hHvx*yPr8>n>tJJB6S}pNi(d``ZNPN$W z0Ix(E>NK^&PzklhP^YUi6t=exv3DhoSP2&96lIH?^mIL@?(^8tt_%Z48js;dhN09M z6ZsLRkVz+oEjLjZV?(vlG<>Z(lWnC^s93MciZT1@`Ia=9XqnooA@? z)fVCL3`1R@8Wgsb^9X~cq;6P+6;>A-cnfYbRI?9($kS15Zkg3oi=noXoZK!_*if$1 zQ9Ub*f7Z)ur|@k17ew z`iq-nP{w+!lx1IMiI8qX?N$3&W+aJ%hE;;zh*Lb0<;!ujWVsX1kdoivSTfsmmqz1{500 z4XG5(&~&IYc`X5Yc36|}xR=#!SlbL-U8)#CH}@89C969-I>e2}sGVU=wphv2)Sm0=PA)6_ z+NN-3WkpK9@q$s~RZ6ze%JaP*-ns4|<06ss7t5{O80F}QlVpdKb9b2;Stl+JiPAw) zwl3&5vKXbN+?p6ZVf(cOlCmj;Q2KC*j5!%L-KBz_^7z!GGM33@l>B!d9Whcd@JNw((B{G%OFI%v$-Q-a(wX?f2 z_V7pvscjDT+E39r=Xw~O*2nSVomIHo$p!^XWWCe3O7ySOqnNxz6NI< z@V}oOYmg88&0(-kzRF+`jYTxclrY-$@MxJvwV+XHoxEwnPqIS;Nh3jz%ZpLoh#gpG;kmP zjc^%F>p>s!BgQ0rMFL|Q(S#fELMpozaomI#5g1~G(!IEW(ItNIGMgZ!0f&pD=|PY9>fK> z0$sS0BhO?-yNc&kR}+BO;AMQf4THD?DU2hHkF&~snh<@ERqpGoa}V>-?QtAZ5gb+v zdGdA&MpYy0oWyv{4^a@~MQRnfy_{iOsg~dscqL=J5l?t=ucrLJ;Z=Awt40@o<{_Yl zv)_r=;I%9QxADvC@OlP)oT+g$>uHplICulz$ZGXCtJ5ub6T$svN|l=AkKD?U*{M(^ zrycw%b#{zJL{cHqBzYX<@2#kBn&ssKw?FyNjMo-=^RK1XayX3@dj`dcz(R79b3RIt lC#mvPkTi|(zM8+c^0^9cqbI?VS4FS#Z*i1~Z_=~+{{d3#N%Q~! diff --git a/out/test/clients/org/apache/kafka/clients/consumer/KafkaConsumerTest.class b/out/test/clients/org/apache/kafka/clients/consumer/KafkaConsumerTest.class deleted file mode 100644 index c3aa8b02a35735c899a967a43ddd5fcddd257733..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2103 zcmbVNTUQfT6#fnbCx%!cURrN8wO$gzjIDQ2EJkdB02UBSZ+bF0gwe^IbmoLgAKE8> zOP}=tS#>RcKz~%*?@WTS46SQ5Yi0J?d-mRE?{8n`&%ge74d4=1GdPAT4z6aqFq8}3(+(y}+?l_oqa5sY?T*+V%Upn|IgDFg>F_Xao zR4g-_#+;?|4(>VlT42u&KlIJGKyRVAERZVeni3eO_@SC>HL5CFlGT8cTt#~_SeB7* z`HnDU*8EuDaz#gVS2m@$rralT^@()7z*nJ(T~CK`tDz!y#)``wZb`*vOyGbawK5a6 zJfov>pkrPo_%H0@Qa2hpY&)8K>Zzvjbx1FL%}6&H#i7b$xh~z7@dNiBMJh7Bim8y* zL2ZgjmUSyMbU!7qkD}zZj%?qek=~P8*-R2ku(?Y31Tt@KiXAKn9A2OgzoC}>*e4DX zVW^F?fiQr6 zWg_1zMXISIlh_rz;;}_^XEotVl;eetvV;3sEaO2I-(bbTw^@9L@3Z&;D*|qpm;Y`U zfw69jc0Vv9pE>-JRKdNyVbnwv$&EW|`-O@u!aoX}`8O9yUEUPh!NV*bK@y|8l7*|~ zjm~@9U7An&JK=3#XHm5oK7uDO60hl2P|L5$b(QyItG;IPGR&*(UL{k@N6NIKP_Z;M zuqQ5d$MoLGVO^4531nDz=c|vE$FAQ0R`ezlf&N&T&Ma&QTr0FkBr#lMWyAVdx1_j3 z)b`F98hGoz9LSgr@ps-&C%W`6EWMFd1i?{50gK&WWXZy6{e@HW`-cl zsyLQ)*5mBXGIuK2UDIUSud=}f4(>GQ)UUXVPjHkUBm3)tuuF>16XaY{LE3ljCB*aO zb)L^XiSQ`RfNWauu4*Dqj?G<`fUZS@$oa@W&*@W|8IG4`tZ67kZ z{!L`<`33fE;)9znFhDZ?3WF=d;uVHynd^|3*gu;39V4TuW2r~a+WOD7aPWA-Su@M! z4joQy;=?U`^gMTX3rC(as7si|J$@K_k!L{T7~t~+(&UDi#1RI6hJl~QX=25sA|86#j+;CJ7^8N>OT6tXjgQq)!08M1kjI1F}#L%RE$LNF5Ziw z2BT55VNAt%1n?}~a;B@`YT9wvlX_k^Hq7Lf zzP6<&jjUzbzL(r*N=`e*)_7TBNI^7Xt=qa^a7_gZRf-R%H}!2jnbqy}WX5+bdwu96 zWrRlRhArFjM-+q-9SaJ=@QSU}8nBF{GjQdLp{RN#Kz@q*G( z&~+LDr&ZZdM@m6=C8wOrS^hk)4f>H;oEc}|swkuO#Ry%UK@TkA)O(tD$IEv-U>1wP z*+fgizPZX*QX7AO!p92RALlo?tHGQ6vAv+3V}g`Xgt-FFb?!px+(XUcV}y1QeuCOx z0)Y;W^5OwsLSbA)Cs$fY1YPLnPQewbs}Xfd9udd`tl0T0YQuk^ZZQ;BGetzALTE;` zi25S5M^EwG9vT+oYA70Sj5qD#`E+?7i^t>VxStk*t|FRc?k>+UiqBjr;{0H^t2O)^ zS`<9R3*G-MW%Z&ETVo=(hYMn37cUiYkv4_)7%vy`%8Adf^0P!7f0HyvC5@nS7*pqvZBwAzeah0nY xYFS1vb%&_y7OwGq8^(3o?c`kIA2{G2IN-l=!2cHc-X?z?xdu72KX9zU@V`a3Fw+14 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$1.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$1.class deleted file mode 100644 index 46d9a7c75bf2ec5ae3f64c9a39089a6a175af199..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 311 zcmb`D!A=4}42J&-y31nXK7rxj*$D}6umR!$J?IORu`bIF9WpcEyLs>dcqrpwyn3@u z`=?3&eo0SXAD;kb7#AoCj0n}UKLjg_&I#eR@fr_136GNAedrt4iKZ8c>3+*cCfv-` zIv+c2SCTgVwLnE!`p_EQv)zd4`K@PTe|*u{-*cwZBth9l^C#hMO1OSdmC!hjQ6ix} rLyco?&C_}-HWB9k!m1}AT;xZbV~`Vh8sakR5LdaExvIQFSy7kYcQ#yV diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$TestDelayedTask.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest$TestDelayedTask.class deleted file mode 100644 index 4c771d85458783888cec97ff16ecf46e33cb8eb7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1076 zcmcJO%Wl&^6o&t?W86B1gj>0lQrav+7MMy%fQs0di&8~pK`3I!#2wmc+%fW{7v2g9 zgv5de;Gq!zq!u^PMWwK?XU_3C-}%q+%(w4fz5=+5PJjmH19+I9CI?b_KW zw>3Gfp(@J9DwT(7b0~|kK-f1%<#8&DLNVagAj`KR`A#O=DjLhTV;Lo>R;DbXgi5Mi zl}Fl?DmOAMqPTXgt8zcf$MGc6mgxLdXO*h+%20K8&5CzTw$HBCd@KnJx>G8;vTE7R z!5xAA)R6oWFS066)KhJzyKrpJw|C^83^9u!gpU~?%K|U|%QOYrwpNr%vlTrb?Wm*_ zxOb9Le;k2lC&@gv{mzQOlc|uf$gALiC!P=NEctff*wNC{q;;Q`{TB2-!|A!7;C`Ux zAhbTNKt;wLPIDI?MdA$3(&GKV`I-`w>eAQjz5WQd@#Y_xL*m{~Fmn@Zv&Jw97jO}x z4a&My10Qg;#XZUrzZ4zyQT>MTequaa#+6ChRVF&P1`?^!q5cK5SbmN++A}`@-yRud diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.class deleted file mode 100644 index 160f7ff93686bc102cf8fb9ef2a178dbf88fd9ba..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5664 zcmb_g2Y3_b8GcWeeb#X-V`xJtAz(nZ2?`FQ#DvD!X4-~gGfdm0XZdU&mUZ&!#7NU7 zq@{cBz1ws*P%Js2O=&vPrhCwl?%k$)ujgs{es?F^vZP?z!lQrP|9|8C#{d0(;NSP$ z3t&C|rJ)_SR^zD=wBczgZqpFPtunb?#VHLc?vTmTH9P~SW%Nu9&%(23;W>D&is#A7 z^Hsb+#S1HMd)WV{VIE#2A6~5DC6(kN|N1p7!b@fKWf8m_uh7`MQseYh!sFEuyaun8 z>FZ?Q>!o>vhBx9(GJ3Oyx8SWZeVa_*E~9tI+B>DWQ^mVfyjw#f-lJg+-Yd=f@_a*b z;{7T$L z!HN=l^MVAMcwkQ4PNp*>0+%m;Yx!gOtuZS}!IK6NXbnzxneK7h8J^ItYKxV$+^q^K zqV)$Ueyb<`+zu;gc4bETOsB`_OL*wpaU*fia4eYzi(z-jqTP1$$1Y!aCuXC$H;8{6jFI# zX^M82Czf(-*N)qXm`kgsB^_nr;=h%q7+s-&~bGs#YEiJ42TFGiU4&kt% zN-%Wn!fpktW|q-(>_G=}Nb&A`3SsK=T|=0BTt_FmRD43mC-EsAd$CW&{W?C4&!~7n z$7k_51xt#F`4~#>((!ogXD%$>lV>=Q*yc<&9QDEvRKqZ>U6TFM@usow~-FX%Xc zgDSqL<4gFmj<4XWI*#GEj<4aOim&VV2EM7oMMlR>_?C_b@og2~(eYgof~Mk~9U*?*O~{MdH6p^A8>W#6t?&i|PIsz%X@8I_BQK zqh_4dz>fb5={n}i8K5_LEf*wrI@_fR8Lx6-zDN4|7wFnKB$RUokL&l+t$^t%3_jK#CSQLSUK+fm>469`ht6 zuI_B_X=!WeX;IKn7Le)3D0Z&ATwXOhb6RD0P83oteQDP*;;f_m%ejZ|2Elr(&3ecyb$2m71USF`Y2o|B;RjtFB97ZWnqkSzJMI#C*ON+>m2R_>yy0oo zBMN5u6t?rcR7uezUQwO06A1-dOOczYE<#81iT5}e(vGAv@wjRBo2-{Bj~NMBtIY?W z_ekFuDR92mm*Mi5Y=Lsw-DR;GN`z0f;=8L_ZuX`0CDf>7INkQlJEY!T7 zHJaD6?(%vTP+rf{hrJ4gH{@Zqy_}7snj;|Dkb}}t6BeO_;*7BxB4j$t9Ei{x~KzlHo&)#6b( zTrq~lXFPe1uw8`++ZFtaz)EPGEyQZnA&P5JPfZ(G{54_(89afzbYD_D5l`YwVTE>> z+EWWf<@qqzPI=tU;>s+RnLK2@p1 zI<`%?k$?TzfekF8H_{ZFsK#cPxE`Zu#_ibRDcRsF5-3^6JvZaYRMx~280e$Iqx@@6 z9R1{Dl7oQa?;7A}kSkT}huAByCIAAdR&ebOY8Gjj22jz$bi@P>0cjG%&RSe6s{xF` zO!fpo=8t626re}M1jI>%8fb73llf@Da%>|mt=LJUAEbj15vMd~x6sK?(apvD%X|$2 zWH#_!a@0d+ClMUxDCFU>h`s+!@bJG4aIFxTL>?L3PrO2;K71b6^(r~6?-0=&*;Hk* zA&ZSNjbyP&YdnX?ijAj!r$-xQ_K3GdjDF?BI_>3z+OC>QM-(9rF#1O z7Dj9vpDLYE%c1Rsw@4NO1!gB6NunB&lklf-KItPa^zcVEWx634c?p=&72; z^?4b`(R?Suotzf9#0~PDR?LUP8FE|9c#Trdu;=W)DR068O1hYCx|~w2q>D!BsP!12 zfd*;dAsW+~qQmY$hXo9Uyp#J5?XQRdQ`BGqLJyNvNX4iMN1{EA0(C?3V&i5}cM~c( z0%o7b*4~=Og==oiq2(?vggi+lI6%ZE^U3T$ijX`610wtAY7#DUB-F3WVOuA?w{?<= x=GkDBc5|k<=A3{M&*214l2(MmNT`PR{}xY%r_iy18qT+~$GmE$gf_aV_8;?OPYVD5 diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$1.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$1.class deleted file mode 100644 index 31b943f8317aeab69cc659f700a090850881f436..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1521 zcmbtU-%k@k5dQXhP+G3GBH(XDt9u_cxYWlfb#3b$&v4~n8^*kDR zEHO+U@t_(fzAxgKVX1YMFyN zmX10u?{WV?xLv;A<*pw{@{CiII0;4MN@YZ(co4gu*3q6+1RCv#*rY(~Qc1JH(5;Py z^dH&RcNuI??+J!-ODgdy3ELvt;q8ESW?I_k!7h(vs`oeTlr4j^sg#JkfXA^Q_31bX zSJ#MqQ7cb}y+9a>yjUCHu*G}L-)g{B$9?h67d<1jYNpBO_5OY=45d&Wa$zWl6H|t{ zkq8X)r#Zgwo$%Z|4YxjmpbkT=+#TJM{u>?{nI$z4r=?Y=xk21Y+lKpH;?Y0D!nW*C z;gg7{@*%2Y1R4l^`jCp-I*EMoQl?d%9csLlwAbnQUMN4%u~eNcVGi_O9?K5yW5q!c z(+;LkaNxkrW7WYL9#FTB(d^YVhDtVLSxH~F4~1_S9-j^A^#_T1C6vbFLC~h7WoVx5 z`q$B8n4&)K19b-Lan|tHM|8#T#E;IHx*}Ig8@T?@R}Fbkr_;RF-qv@qKzG?BXaHbU zDrtEDTB%la-&yoJDAC_@BzdIgXe~FEj=>ts$FLfok^7Jtmg$?qfc-#)el)dwssf5UX+BP{z9>=e!>bC3x|qZDk5t_jP^U>C9sN`GKW6zF0a yma@Er%Vd6qWT(F~kEgX<(%xIxOzOj^J#T5a6M9n6qaB;yHM>G=iJ&8;Q? diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockCommitCallback.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockCommitCallback.class deleted file mode 100644 index a65f3ac51cdcac3d973342d8c51c56b5fb4df892..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1394 zcmb_cTTc@~6#iyyLq=f^045cWSOd&P(<4zR!$!p)3&7&I$JkkyJ|WeWve2J8X{~n_-k6rxEhHhk>s;k zGjTHzT@kiq&=$LG5rmPF+9pA0bkgg{I8fTkSc@nLs>Z}SN(*b^x=if+-vtPI-e@Y? zjie=f3#9?Uo>ft>F1nR7`8tZ8+1*W~B}|2p5Qdr4EI)UTn6906T{(jd@i#iCo308s zMQoMp`$-fvqnnjg!kybvO`_I|3APx4eTl^95cW`Jcw0IH@V~czx#Akf51ui+zGNc0 zwk^|(L-}60v6>#a>8EZ=efXI0F@~IvJjOjdVc7V;M5@dV zk^VwQ90gh@DF8f+H209>V rVaF6)CmaO_-2o+sKczCbPu+5156I5(5LZb~(Jq6>B=fjSI(@$Zt6*;Q diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockRebalanceCallback.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest$MockRebalanceCallback.class deleted file mode 100644 index 313d381220fb1c4b3c1b24e5a07dea381f48aba8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1662 zcmbtUT~8B16g{)M?b5PPK1!7jwIE6<%8Fu)0a0vXLQ=ngkoaPpE+Y)x?likwqCd+- zV&a28z#nD2)0PUlFSh$|XYQSud*+;bcYgl*{sX`|9%f-H9uH$04GJJ}Yc8{$z!2{okC1+S=95HqkHhLQ8tZY;ei0~v2I zn5E?d2CJeD=^DAJ48*Heqb95fhQ{5 z_iNm%6Z}pPh_K=lq@Znk2`qdmf;b{JLDXu9(3L?f!hrh`NutF(t4ssZCFs{ugh8<)vY~umL+ke}0z``<& zwf`3Cs3Z4v?Sy_78|w@^eIhL;wC8w2>`9;b+bz1rura`ShDUv#Mv-2fA<$C-i!7Ce ztV1uAff1TftuFagMOtIJkThDfi;7Erp1f`H7>&8|HyCB}45l{L7YZ4eB0CC`rsJ@T zJTk@<3KVlXM{or*f;irA6wHO7%#ka1)E|W4CD4z->&^I73&oaR*naiu2^_F9R2A>CA5md7h~N diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/CoordinatorTest.class deleted file mode 100644 index 3703ea531e87e1ddd0ae92c446deeed6c52b17cc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 17373 zcmc&+34B!5)jwxQGRg4RLI4pIP>>LoxSB!AIVeXZmN%`@lnJ1XrAun8Pal$muLFuFdichm0q6Zqe;d? zm6vDx=t!O;Z{~V=UKv;Ou|Ar{$4SfaUOvIgCl>Gg*!bt889ZNJoaE(`eKeaF$ipdK zUg)EF29ZTxUhJde_*8kbL};Jp<)vP(@zF^l>N2_3%GYuq&F4DlwxWzz@+u#`{60Lc zmQicuZM`tsAa?;g#82(flpU)Ra^G+dtp}g4T<%@ij5a1Ggv3y-3 zy)G5zFB5{F_VR9F;Bxu;OetR>!ozLC{_Q@xj_>gD=Y4deytz@{-07p6_zPaX z%gcM1#x=ylolW|tP&5<^>$Rb1bWG%c&xfv2(1kzQjt_79>d7dm^F#p=17+wPp2Bd0{fXpCiGNd$1-pjZ*NzPtI;>1 zZ%ymTl%wyMge4Jp38nOo9ZW?j&>Dz|bSe@}R*RrQ2~3oPqY*upVw&N?xO!!DuT}RV zZ#bGxrt}2UG?x*EsI^8H^e&3U+rZF~1-l0D9{oln)0>jvM5J4=0T8FT3^2I~qeHq& zPgF-@U?UcSJVD3cU8?$BqB*oFsw_1g4n;SF5)pYe8;eq#BUq$U4|aZ$5j+$p z^;8;`KCvks6LZ@TNk%Y9O)M6N>KW==uIAX12D8=6^!9i{G;(YT1Oo9yS17tlhacFa zhhTRl?P)_j77dB7Cc_`BP=i5ye0`S0JGw2Y8xEeVQ$shzBW;UB8%H}tGZC32g4Jf$ zI1o?OYzsxA=2{qHJ3GK7OQSy;)jLAbl{&nzLe6YoAM1?8x5q5S72i5BXNu*KB&;H) zhk>92*VXYzY^C^P*rJSR(j6_wNQ+6^_QN%B!5y(KxWJ4TE!EHI24x-d&UHc0a0PlI z5l5T!J;p9kFo1h46$oF|BnGXdvHV8*VeeQx={b zA;HGcu53UF9GoCpxc17dqo93MZl56J$t5(~t&?M_Z0$Qx?`#ID-Do#&PQZ(wzEvgq9I!1;z>Fo(Ux!F(%rYU)C_3==fKejdz??6AONL5~s zY=Bn86PDpH&CKh3dL*?u5MQCE!ka}BV_uLw4CpMDkQl>$OX?ssgB8a&#htlGLXys(Q5zUhoFb_-HVq?e>oJyd`Cykb zOV;UGRIs)u9TXW1EJZ?V?pr&YOA5;$FimhnCpX|*PBkyI`-1UwBCM~7NK#$l2(rxx zUGro&tkGlixNJPppwZ{)PB> zJ)jwjr{;ErQrPinbRXU6n44zuB zzvamX+TMEkYZ`x@9}tQUYV;uOMSfL~q7^E!lg!jF>f^Bv8NOGeZ_qxbnu5b!7_#&w zYfCBAzM;{6`WEtzd}^*6XpMTPmuX#o_godtZ1!`s(0$h_%x?WALJC4 zo;Zp=%hHi(n-oAwEoe0E<9?05$q&iA4{P*odeqC9`rG`d#*gvi8of(@(D(^{QsbxS zagD#j-_`hg{Itf;@Uve2zQ)h-UN1kd@eBN-#xL>98vU96qVX%z@dtS1AIgg#3Bgz8 z?lpd0zJ4qtf1>dl()Lq%`!jyi%WrA?bN+?KZ}U4E|5BQNB^`b(cTezdn2yiKBWqui z$&2k|SL6Q%>-1Opo5sHtA%4fd*XSSouEu|Wwelb3?oa%l#_!AHpXKf^e80vY$gF>5 zK!TAU^4~T72mg;oA0z*xSs5z?nge+%FGl=+r zpJzCh#G`peXr7UtQJQD85I&_4VLNHCjL$M_E35(^<^9?ioA4>e^KD5?C2tF_@8g zIiZXkhw|0|Id@BUWbk;VWrG|%KycMO6U5LS^BjiM_F%{j(JSZrq2pxkiA;;~=N_~Y zB2Uk_=*CT3RDEN-8SWh}XpvvaUsLjClE_;vv3b4Jc|k3SB6Uq<2ZEyIYS* zQO!jKol0Pwt(aj}Qr}eEXhIHZsg~0iyq?3c`_4b$Vcm_Z+|_Yr7DfpqE3H}N+$$J6 z4!13Ro+*Y4&y#;#;j*#@cwNRpgMHwT&y2&;WZPPhl2ZXzyYwo|u4C<|=Yg#BpkTUZ z*MI6%wG{h=Kx3e8pc9sy5m^1AuqyJS;vYm+w>zPChZ4FK^boMz*4O~grj=EF5>*_p zW2FkQjvthIzpv7gF~Ev77k^>n{tZvW5o{Ona%dvI8GAO znFfykMhju%~c*J3NK#P)Ud%wa(O%kg9V%$aWZ32UIDsj zE2`Wm;1qSoqfuS@Q{y19{K|tYU=&VTZ46Yjb6?EhPuY}&G1zj#SBW3 z3EbgaD2a?A6(&Yi?>d+gv^z(3r|n7KPGm z5Or0t0zW~S(70knu&%kazOiO`t3S}XwyAMt6LcP^RJ#3cz?jL-{){9J$3INR4G;$0 zW46SmprAFgGU%3lWlUXpQH5)+)=8r;DTZi@HKEchv^#T2HA|=43g)V)GjGf~B{k!Z z@>d6_X($!-AJ{Hk=gLpUXd2x@x8g^5<>ZlCEbcr~fmK(ORn--hRdq#iRb5eERaaD1 z)fM$jbw!<0T~T0FR}@Xv71c&{McGhYQNdGJR83LO`KG#m2=k&4sP0jNQ&-f)aGioa zs2Jn>2|Rs^ig8EOUe!ljRqpAbVO8ZtJycw^pGsP)_R+9medO(-(yGzE9xAJ=74=hj4~>=kvAt9=?3*;sgXzZi&;(FAtfh}8_R!%EVkQqgiSOe{ z!}kIzqJ`w6MKqcg(^NW@j-e$qmrkP-X(`^+(h6Em8)+5ko-y7|9ZI~<=Q@EL`VhR+Fow5W#`XMm~#JVcE&oYvCe zw2msNiB6^j2(^woXhrRivG; zmrm`aC4F>SFD=FM;$Eu3MOZ7lw3n6{O||tt+|2Ez<(KwT-Rxdk(N8PUGPTMYzRDOr zx2%`^#{1Q{tm&uKBGNFm_VXYNQG~|P7Me_55Gh8-QXCr4jTKKouq2&HX$sRe^w_S% z8gH;`igheB=KJ&<1bZHoL??`v7jS2Z{36~ky<~~ZrKZSND}M2H=pxrE#7vQ$Q00i< z0P*({dIx|-V?V9M>;QOO7AgyABse|~sC){j>;x(oLaJRrwB#f7^8Y15O+!Kmy$_7gwLs`PAao-T`YaIo91yw* z2;BmNZUsWO0iip9(C2~Boj~XdK-ZwzQv`ducuH zoJs->7|5PB3<)~)c0f11mrl=8u4Qnrgwwon2sM*&U^WvWWjceCd!N{GknD ziK&HGLZSY>&bz3WRG(iGI3&$(MBj3`W5hFKLk!c0#2{PN`DM1{Ryn}4Or>V zfYX~4qPO7xe*u60HvIcL!0MOqWWRFYbEWOa&dl>;(vq)YKeF(lf2xp2vs z99F)dpY*-duKbfj!7}=ABzaX%JDe*Z$an}c0fHP(bv%jcd9s7i1_z_V!P1NPydH+p zF^tYkj{hy(b{vS?sWhCYf!m|O?R0QE6WmsU+gXF-_6?g`*_23j8U?nl3I?Ym*{OP0 zY8YxiZSJQ?A8moT?x)TcY+IuB=(8A~Ql#?z)P?tZDONSRkK(=5jdshoZ8c-1u{Al0 zN&)>`8prc!G9ODP^YOHlPoOjTL~4T>b@E9{AR`t=P57769QrBTTW$jSrY)6~bMLY^ zcS}I`Q$qF@Nnza*Ptr`MczBg36@0QSk-RsuOW*8$loHQlW}o{go#l1Z=7JYO)I|_= z3I1^FG>E#CKEX9~HrLV?UQS7_qYHV3L*!ki$TJY{-lCu9iTsO07g_SgLl#-3lkOs~ zhRACmasVPXLgcj&c^yO!LS$?|cs)ekFa(j`wncu`49+{P1sv=u%l69_+4QbMU1ltI z+K9{TEMr{TE0WEYov$kOm-zb%t;e|V{`g|xoT||H-mLvgJM~X)9_F`vI4G= z>)ZwA_kj5?0snh||Gn@~_feSdcVOOTV}6<|=JQ4K`x)2lGwI*=rIr?%(4bgta? z(s@?zj9!(0O4R|+w|4;-7+rUw@`X1U5*PN;E=1bQnjAxASd*_ptgk_=2aps!h@@yQ zt>JG_6YoPl(F0TKb*#}=#~Mj?X_T<=OFvVnH%y@pRKhAO)J2ABWeb(#V6$H$lNhwb zMyPoR67Pq^4@2TdAn~^$@uRevAEP)wPCNJsx{9B4NNts@vNtZjv!(vPl=^OKaom(L zvkopxX-1lC!3N`CvquaC3RUCe?*WRZ0mZX`;`@N&IY99|(u^1Ae14HG;g|4d)GyN> ze#HUD-L6XTdmD~k6OIEV$Ay)^bg?4b#eL+e*_^@i!mNzZQ0iA9*K3gL$B^qMkn0V| z^;0PI&!E(ABBOZAA&-?HJD1lZzs7}vKLKZS27FEtuA}zTr7e?@oL?3|f%((MQHLBA z_+&4@Z9YDN76(Dukraq%r;T!#H%39c3U>57(d zMU&>t?V~Ff7EeT}|07@hq6!06wvoq8a5yJBe-v>t?Ppk|=@u}1+lgZ_w(y#O@> zh08mYpfj+|zXxhE8hToWZGH((y#N_D`{*gZSvDD5f<7|Rx1Vlmk&WTa87|cJp8ut0 zH@12{dv>3>)$^ca0FZe!Dtts$SdimThrSI8n85W8TyH|V{JReSErO3G!ZFV0ZBCrr-<{F(gV_C8^A556{-rSg!(lO$;(3*gmAHtz@M^Ovh^>646+o<*Bjn3;rw2Gh}LQQ zBzYZI6>7pPrq=P6{aRel?>##oI?-&-N44QYlIt`}QV!<+dwz98ID1r;G!JYhMrihV qF0i)l`E4UU6YdU;X)Zw+mFD|7DvP4VI4(TKNx@0Ex_GECn#mtf8C8q` diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest$TestTask.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest$TestTask.class deleted file mode 100644 index fa06e21f6b2240fe3590695f1c2713f6c494d9b4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1418 zcmb`H&2G~`5Xb*(2fH;64Jj>@&j0}u2kJ&b;t+&@rbsOj2na!N+c-<>#$6+SK>G+h z4iZqrfd}BB5VOXlJ@i!Pvg;Y|{`NmJyYu_c&tCxUqHV#zm022U7ED~Va1GZj)N#Ya zO%u0FG#HFz%a7|T43B$ZwCVB>+}jfFn7O#{N2w%GjykcWUyX^Y2=9~zN*90Q8!_En}6gsPQtb^3rl@DCR#Rb$-K zE0nXhQ@B{oUY4vh@dNi!6!E>DpFQ6BBis}RAB`9m8viyO%5bOo4mD~nCg{b1RVu2~ z)Od7gs`BJ5XKhF$8?c<#H)zg2^e<#-I7OpDYk-%SqfxD=Rj6Q|W`;9Z$O&t-S|?xG z`3mg^O3rtjcyj=;GW=@-3^LGSW|wmiZNnnYl01;8r5xGKk%rUS2RmRL7F5d)H6|Pb j=SiLtY2X4b5<$ZyvNT*qmF6ma8Ps7<(|nF>RA&AHg2{xL diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/DelayedTaskQueueTest.class deleted file mode 100644 index 7913ad842ed86b6ef7a270f2555d2002f119f667..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2833 zcmb_eU2_vv7=BKg53+1&EvZ;_YLy^q+7L<+fs{Z$K--44w6zfMYnvR(Lb|E@!Iber zubjafFZ5RafipTIZAV7$^&fccjQ_zK^*OtnCO~JLX-sBz-}9dHzUSHJdEc}9+uuL^ z0^l^(l6Vb84FkBCz$FbGxTrki2~22+E9YeFvwwo0nw`>c8CR6`j)r$Jt%frR%xdUH zDaMrQms0ar8zZe@^f!!{hB?ePMj5qwA%SZNEGDodpif(tv?ptZ<48x~z_N7QWy85A zkaQ|{WYw!lTcB8~+iQ7a!=NjF&$xZh$X9Bnv|J}&sauY>F73Q&xze_bnvB$1~%1v4ETv-+9E!{Ql8+p$)Yx!~8HXf8r$K{gblDTFXu4hYug}v5{?H5F$ z$n^0s(=y$nKu2b9MIbs^uX3ZVl4;2~Z+%tT%f@Pro}N;@V$@a)+f?Jw8FlZNWM^W( z2z(k8wFGj=<$WpO*0k0HVlIgq5twZ|d%B?>xAC){3h$!=eT$yun(K1KbWA?jxMkH{ z!!@a%K)1j1U5^CiLzJ3AxhU7`_sNN7R4USO(jzBN3Y0T#<17sBr7uB8Ho1YeRQK$P zykaUj`kpN&h3|x_ARTYwxQ;A_1jgDaPaSC-6F9s_SyC*KlkM@y{k6|RRs-uFc4vE_Wm)hNef;XRPe)X=b zxUB7Vv!+?X2Q9hjF0;z&9#cj$GlMhSOl1y=Yn-Qy%^6)CH@18E6OZQwYcw-6<2U6- zy;ck2*rL5nK_=-Z%Ai`4a(%<~PYrf!rnQv9ljL|NC@){N;Q0#Ad(?&x|A$&`yBpFY z1%;)IR zfg`-Wgf8^qWh~+ZJ$jG=d7R`(V1(%-D!up*_KkBI}HWYs=B0r`G#;{dCRs#MK#ZSznIjo{I6w^5LtP zrUJLgI?#_pl=(#pJ%BeT=23p$NAQ|?vDy@5e|%Grlh;%nsaEw%cN3T;!L zUvUb*GwKh#g}-n{Bym<8#yOG0d2s;+U)5}&T&QYRq!=?wbx#Y8k)u?YFLuU{vV_hM zRh)Bg`}&{V)xST`ACGpm{4aeKxuK%U3^i0H3*gl~Z6D54XZ@sTDPktAgyx~#E zn}%;f-ctB7zH`p^{=V}~-u>Xkmx<_nepjbk>HRAD!)iK4KdqvlN&80{{jpA!^mF<6g}i<# zuRoF3pH^U<{OgymKhx;Xb(%)+%g0}6^p`r-(qGBPUu*O$jsB+M=!ecfoo3PB%EaI4 z^!M~@>HULF|49E-Lp$l8HToAB`B#npt(tyA|E|*l`VSfTPmTUdrzOswzt!l!b!u=< z`yY+|SEuFlfq;FeGci|l8JFwq;|h#%rN)}ZRXVMq_cgB8X)Oq49WB?$%g|{xPZ7we zAc3a=L!J)&c}5l2>O7O{s(6;{Hrpe)tcvG&{qib4Lni0SYaa0BkI8GkY-c*6Mr#UnsqcWWRN?&wBZ6lHLY6a$^l|;&p<{#R9TSsJuizo4xZYWI>Az zwMuWZEN#=cUFS=Ai%ti*Lta~5@R_#@`jQszX~aCV%M97u@LfG% z*}VN>+7m4wz}j8Ec=>hd^jUWEp7#)@qVCvZjt>QdL0muI}(i_?snjXRZ0&TEsaXM zsIL{r4ox3r-G;p_ds~Dfl^a5lkiChiykQxP(buYUW?E+`Vg-^T`>c4cxi1Wk*LFsO zW_YI=56QPX=(C4HAbQ^I9h^>>(+wf7h5~)ssB?gCKH%d^5BN~5PNXqifW&ar& zu5C_=yY062nee_rNPMA8bwt|3p}`@Cv0ahvW-?**XIw1Pyv+I{lUb3po1w4;2R<_k zp0v9Ly5Z+>(sUWU)UunyVdcW1Qq^X*e?<;wKvGF#SH40-Rg86x5oR#&hzvxVHsQa< zyTLAHgxyvm1_K8|TN<1rBcZ5Y;zvtzU;wt(v$eE4i^ZdMG#Cy0ZD<3$q8(2LMZYWe zskKbAif#!5u;TG3%%6J@--fP1(28g9IIoBfc;4hd5==|@mBxFR<2=15nv4gncJLdF zb-hF{K+l!PYS7o{8HsHeFz7bA9SfwtQiWK9J}D#ToLYRvUcASkPthF)-ASJ|=q`EP zEw6j%-ji&TA2<#A41E>>u<%gV`ALLzJguC#*P#38egweM@R^`GHJeOyMk9l={0f5} zpwDT1rNLKmpFt1OLmFRgu*v%j4l-19e&NOr%OzSG^m%%iX=5QwX-{1cAT_oO9$gQ~k-o&#T)10l{5;Q>7u1AT zA%ng`U)6ZO!NVNZc*NicM>UQae1PKyJweBr)|}2!t~w%S6|Qo~O;`vp;m{G~>Xq#^ z$K&SV&6YPIkBYvn3gdSm8ub*8t*;G5_fgKy?r3_ixU8hjhyZty4hQwDvX zZ_)S;;m@50f12+y=zH{g2H(y17mSgC=CVV+>(FVeoN&(%`4~ zX@kEedp^U@8vLBl^L2jS;BW8=jlXH|3qneY(*}=9?_r^JOnT$|qF_3~w5kx1N!|&` zol0;$2c_}141S646QR6p@GJbP!LRYR4gLi z(znqOiCFR0u$f3$3BmGBgWuw}rS}fgg@uPr=6p6W&hfqQ#93F=wQs)_L=rJGJCwBm zACz=#Rzrt7r|5SWj99p~fRx~5adoMT4(e&y2-!?7CTX>0E8>}c!hg~W@{a0+|i z@b$$we=18w2nSBG_%gZjx>t|V*m;&$>~`Q7sS}2k{FUGxeaugJzzZZ|$oy?&6K7=D zoDT$IPhz%mG4i)YM`B2863E|sksl!g33LV8;L3^+;JJ`u3vtwnLZRlZ$h%-M8P-V5 z_U>Cu$>N#a>7r3O4|MJv1=qYvGZwQVNH10vv6Nipb*dh2Km)l|cGMYSs`D%$*UFi1 zK8db);V|dE?uCZDaNpF?(Xpo_6TH~#1m)aVTw1$MZ;fjgCsW$VrjvEF1qoJZtR`hs zO^!#&&J5cu}#bo5Dt>+7s4QlX1 zl4l~nExkgOZNy9D1Qkz5s^AdaikGip+1?f?P{UTa>nj%@3`ga9;~NNvaOIxk=~IC@ zSGyIVeE?ow8j(r$HaWtVu&iMi=_xN#WYIqs7Lr36tPnWROzPQ$<2V? z3{MU!pBJsejWnAEU;vk?T({u{xLmi9iFv++Dro|L6g;PkuXO4p%}8kCh?Rz9HgUV{ zKoZ5fO!*0psc5`%RCKbObF4=0RY12ojCQBw&rmqm>0r;QP$q?$okygX+F+?RWF`XE zA)p4F_kc?g^^x!FGUUz5<%6U2O~PWw6chv1g#n~hnhD7g#Q65+V4Tl9wZcs%<_3qP z9+Ali!7@}FM^s67rh9G{UQX3Qi8cu@xJ5f`?Trfud<}bKwjw6glFVEnDN2w>fQJi! zaqD+(^hgZzSao^QN~-!+RwL99;&Gs>x39CUd2?HLUwc<~-}dJ2-j3dmt^n8}rL*3s z@O6jlf<*h^lIHCQTo&ls73k~j+TPLHm%nUgS9^O;TW??24m|CB-OYhZ+kj6fk&`9` zEURBiIj7+lAXVg{*&Nm-yKqFfHY-bKE6fI|vQzY{Qh{6@aDCVdZ^&Hirq@!9?bySPO zj%rb|Q7uX_szpsjwJ4{k7F80}qPC%06fIPX>I5-;S@j>sxu`FwK1vCy#VtKE|1|07 zqh5vgvqbzDFlMwX8b^s6Ys*qp-dO8PQAOhfRqk$lfy&EANlQ^xBL=G*%TuH`&M5y? zifS6`$Vkytz)jmdO4Cy`1KnC#GGmlxrl<~|vvyb4(VS75o1%G*vYBs;K9-{S(x0EE zGt0;6tTK9)&Q4K1z!&U3K?_G|QHmDhYl*wpIVoC-&jy^bEJcmzE|-})JoE6(03M^X zB1J13XZUE0&OJf?F*+|rs{pxLAXnn4#-rn@@lDVg>~y}{U`RGujA!Wtt;Ng*wHK!8 zB79e%Nz*#?1l<{UrjFA36g7QAarHU812hxwHRQwZoH|`V)9FI0r;G60={jnp^|Xqb z@U?+jX(M&hCNk+_8lg+*MruY)uLWhNR>)>EeTUlVN3?~$j@=AL9`roPIZP)Yt8$SQ zo(h?xL7*Z?&lj}Pv>{C!(Gz~ur)d+KReJ5kX}YAg8Lu?8$VeTvI%93QW9`n^mh4!X zIs`mTTX$v2Tu4cjhL8USG#)M?N~{hL_h1Hvy*3cX77 z(rB4RK~R}ez8FR;G4=|M6bV$N=`ycdou*FooA5wWwxOvLF#xl`3F=DIb|v=9)6|X8 zm4NO++mxnWG*#oYL)2<#XC1XKA1D0sacqn}UPq!0?m>Igbj9ScE1faD8EdXW@b=C1 zjZ+`P{LYr%)r>vO^%eDVp&I+JxL!A#kAL`cVU&+VA2#7+6Xz=S!ppBB0Lr;>tVw_!Hce2HUZ#VJiH&NnDbVC`$Q#UY*|{65pP(1Pz^xAKsmYyWOjY)>uxssrFlzfPe zx(q5$Q&76Tm*54;g{3`X)So8HV_>0pio%|v%*JVeLBL>E<<6mMpm+qFycQO99j&12 zVOuv+C*1_z-V8q8ijmuL)b6Our+gA$&ye`EOJcuA!bz3Q)si6qpCS#yB&By_6w1=! zn9d!b<4$PAUC@ZTX(eR01qJ%;@b^I!-m|9DUzCP7GBjMP42qP!moS(d#B*&YP5Z|Q zcZfVD&r{a-Adq_q$UO|?9szQX!eKp@gVe^NNWJMH#m_p77H&kq^qga~aCAUu$+aAy^%Wd$s0DI2P?;WFL&gYf* z%<9Qb{HYEhhS$*J5XBP^#c_C!C*e7s!r$sVO+#>)3E25v5X=J*$Zhx;(hm3CVnxbNS%M&BwkGN=}V_A$d&R8*~v^)O)WK1Ih zE5Yy9&}%f8UZ=(M27>6DbSX&RL2tt;z60m^F5N=k%@T`b%n6_Q&fskj>x9z#;VR3p zujo3S$6B7>`!BD-?s^&K;M50U!^ zkm7rE26(xMevJ746NQp^XdLCDvsEkd#w?unq_fP?Uh z5;E&2=%^rFH9nkKW-SRwOBrXuxC#V0E=F!g-Hs`CTQtx7e wo#^}Y``BH2?|}k=u7^(D3+jZjAJHEuX8joBW%Lui9IfP#H{j2Ww$t4I1A-+jr~m)} diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/HeartbeatTest.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/HeartbeatTest.class deleted file mode 100644 index 6a2605d81fc017de51393ed34c4dfae3382a61ad..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1929 zcmbVMZBrX%6g^LxWJ%b*45YNRTCLbNq_k@bkyl$N1uIgjkfP9PU6RMLB-zD%arzhh z;P_epg);)3I^$=j|E7OHJvUiGfP7$@nLK;%eeT|K?!9OB^}nzF1~80gS@dH(MVmi0 zn8AmbFff@#3ghB@YT!=lzxbC$7w$@QO75q{y_cMt0@8Qi{3gHOB|anZ`&rClF0PBT zc%SWg6-js?q0je1|LifiC3o|3d|_b0z(WNibn4oRLIpLikQ7fLzu@`LEqlu@Z)mG( z|4_B9vRAKrt`&t&EwCPXxF* z5B6SUmGq?JI^i9K6ZwG^(wOloq;Yz|arI(UU)O%gUav9Iv*4BO+KTNv;>W>cxZwl} z6K{uJ(m_ZDy&<2=8(vhaw1_EOSc+U3$%+#=1e|tVFSJ94%_y8{;B7`EZpB6V_KPff zVOyPZav>8dd5ii*`^|;MD>?{RR1&y~*%w=muPa25yCogyaJiYFkX`m7zpUq+#xz<8 zB_`h7w12Wq6mZ?dN4TPJ^>|A(aTV7Tibu1Lu{*(AG?B-EiAVU-#8+4{u`EXkj}5Gt z_!{4sSe0W9PfR?O{BQBt#CKRy7<(&N>6i17j;)&QKDQpNZ|X8bsr>xBD9EfkwVD%X z>aN0FQC3v<$$(mH&5Sb3H*!-!P3tYnEjYY>NmwI}aBMDmIta8+oUGk2N`6FDCi828 z+1-rIQQQpnif)dqTPl)?*-V}NF=Dd9*`Tukdi}QL$Lhdkej@t$Jvo7d?3q?VHrRM_ z4}7MfL{>1SiDV9$U3725{G(Co2G1TO zd0yoC9=dS}eRv-?88Mp$Vc}zX3bzQBXn_4&`!Vw9 mv5+qx$RdxnMKq@{MIO`SagQpP!3gfRaT)E%Wt4p9j4|JsQP)7kp9rIYwu&C48 zBMnO$OonvOvHkoq!+OK@+9vOEt0T+rOe^u#vx%7j|8r5#il-V{b;6sBOa7YZ@Qspi49w2_w<>{T2fz8A-t8A zhUXey7+A$ihFst3u^$GmV2TQUqP0c;*y1t0Zw8nkXFk}@b3{NcO%0YqoTkMV=IA=k(yh&MIdzoOOVZ!KMKD~VUIhty z0DXSpE7<%uB)-rnfpL1LX%BE6IeJeY@4^Hw(}?y*G)6n qs}#X6q!)Y8&X3@jq{k7YV%BX^NHOo=F10D#LmrcgM)VA|^!p2-5&Ied diff --git a/out/test/clients/org/apache/kafka/clients/consumer/internals/RequestFutureTest$2.class b/out/test/clients/org/apache/kafka/clients/consumer/internals/RequestFutureTest$2.class deleted file mode 100644 index 33dda576b8016d009d4f3371582a02ca7b1ddbde..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1784 zcmb`HTT|0O6vzMDmWIYC1qJVlTD9dCM7&@T#g5E$rV3-3KD!M|8VH-33p0K!A4JCo z$7es3Chu~qBg}oiyU$I_b%YE9)AD2x9SGlaWGH;e-N5`HK1Cu3 zH=;1|#WwY-gj#i^6Rt75DG$l2v>0+VZ$~hUH5@74M+a@;Z}YZGGZPKZ;%yOyT$&s#ha^oaElksOT%cQ<6^rU9sh6d{f%9OvK)p3& z=mGSZ*{@(T-;nu2qYTFAou@sZYrRPCsopM(<06e{e@tVvgN~cl6ze0kF@0|K2-)w* z9brV#vN27M3G?K`LQI@aJn0h+vT-@KXy6L2((Y^Yt?_T>I?ddmnG9|g*U9QFT2WS) siKDDi1iz461_X z93Om1QPy(#?2mH!?VLbT@F_TJW%kUTnf>i=@0oq__dkF931AXGWN-x&2JU7M6FHed z0#h=%m&SdO52TyU;30}Kno07Uw4}@)8JNwW7msEAV*_&;^kK@tlMJ$$kR2uIo=P__ zdln3QV&GE+bJ2CRS8^=h*SPZ+O}5cE!(bM4iC1pw{C4Y)by2W zRm-VZo-Om1G7;?AwDGuH^L7hX-Ky^C!V7EXg;l6Jwsr%*P_4Osc%Z$4?FQO&Eypjc z>X)JRgM}~%J-yCRQDGpUEY%L`HD9lV)vEUWlI0UNwbR_DQ0yLq_S4nK_BNs3B=S@J zn9NG|2s7n<>y=e-EO)oC7I?P1Tbw28^Z&=16{Fwn|YXX-CAG; zHlwU?p$WGiGM|P2i0gu7JB)Y~SH;*~?eMx~cK*kzu8TQ(pINJgUR5vHlKz1+S-T^Y zOS9rZ0+l6pztKu z{mpCM94Q4x>&+Bf>>-H)Z=zYc4mikZm$5 z64mG}%(2pWj7mmIo)@fpA$KKmn{ukhmujQ8H7i5%mF37%HkUsO!5YF6R*7~HVv&_J zvN!)SWLe|!+|rVmOnR0rrR#d9qo=Pz2ah8yftO=XtH05jTvn|Plyg<&3!H4p#F&s3bc z;t^sSuMsbg9wD(oOUVXO&qn)<2GX}1=s84&;w_3SOpf0E3%w0o_~j561v1`*9D>2| zC8RNkerm1|P$2UJe25R4uH1;=TCSvVm--LsQa}4!{Kb%?jRDGNEq}5$Lft6WgxGCL zg|QPFzmrCSy6FxYeL~|Mp>c#Px%DrPjx{jQX7euDjB$GD5QA+JHwg^7oW=;98fTy< z=H!4;td5wK}FFDw%efAKY&Mn_P_D*{pPYWyWL{?X!7LSbIzReUEc5gzB8Lo z|NWQ012~Gc1P&t`!(<$%F_l0Q9*rY~$6}an$4mlkI3x4NMLZ#opG@FucuGcE0<*}; zC@(@6F(+&D2`s=!@Wi65Ep0#n%Q78%g8N*8oT5U&jbOk3eR%t$^6}8-go?6o8 zmb6r^VCbe}r*f8QmzVWY$}krE zO$yvgsJh`jlZCG@Uk0iy<=xTJCf$ zFP0fX%2%_os%ApBbw|NwM*~*GrZc!e`+CjuVXHJ|nQ7ZL=1o0c&8~tY;emSpZh9uE*|uJC(uG1` zo(K=Dn^BP%GZqmNIyE^(Qp}p0b2HIdzePbJYn4kmeb^9=?T9AH1xRw*FW9TN8}}%< z?|-V27*uCk%dMqaXG)m#i#5Th18(M zV$b=#DsHoH6m-Wj72Fv$>!CPs(DTGSnAN2tNj?8fSb5s8hn9;D>BlQi zOd2*pb8+TT&;Bq$Ne2kC9lP%Fq`ivEnI;(`uyH`Dv-eMb1dnQN7^y)}2WgD5gJ?Gv{)eFeezRdR*$x;1&aksVTUA%22UEXfc z#kaD1gE$NcCKU((_j|O@EB$YiPNOR?ga{ zlQGWPuOjZxTHIOUBOYnOmnoBIq1;WmhvP2Fy*SPl)l-WB+{+oS9GYx$P5z5}#pG(w z6&ksMcm*5eV-2dTRM4@8jlr`-1)Cz{&R|@@<~1aHIE^RPa9ahphflW1kZL}`Rt2)F zYUgun*cQ&HVOtnTKam0UQy#z_=;rT1;?;wb=*1ZNFpWcemF>q05q^mnhe59TLEg{v zZ4B*%8}I``z)IW?_;P4k0(N7+9osADqzo1+*jd3HYuHt@w%c0^H+?=FN*Lj64+B=h z=}ux9neL82OWf<>tO7+X5oqp42SYkaXpUh!$#aOIrJ2hN88Aa276`yO=J{oY{stc4 zRh;I%dNKmh<%STYs}Sw1{@KrKt zgjG1o(LE!FfM3{TXuM+(ZXNj^x15$#1gBYKn+c#IU7W{u3SyPRRNA7@NYaOKH}=Q9mG z9}GNK?qj-}sPCW2SVk=VpEF+DF=!y#BW{*)mV@6OYO@;n*Bh77XD~7Z>m)iE+$@`L zj(sqXU2L;`m}BM46RZWoXz} zK2vi9J5Tc~)O>M$n)%Iz$jJq4ktNOi+}9^g_g2u)79EVGdk14_-Sud_MvJdw8+*uI fxa3;i<0adw&}jvwKy2^0ST%4uAH diff --git a/out/test/clients/org/apache/kafka/clients/producer/KafkaProducerTest.class b/out/test/clients/org/apache/kafka/clients/producer/KafkaProducerTest.class deleted file mode 100644 index f999114a959ffc7331d7b927aba4a4736172777e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2598 zcmb7_ZBr9h6vzJ?5;ue(5S3PoHa=8?(X0>nz=J46Xu$vq5v#P-WpfD&o87p(q0$*U zedE+`(%1R|oHEl7&=1x2e>WS5hU!R$z4x9y5BL1exp)8m=g+?Y48Tj^6mF}ylR!j{ zV{wefk(aBxaZJQ98OOIOzDr;V)9vWMy#xxFk+WG9b1Lp9(1Y6vbmM`F`2-eFjN@Sf zhp{LZkK*`Vj*sJb(vB0jCm*z2N^lI!eo*nFik}o57%^?rA63wn>0eY3%{wKbpsQfo zVzyRU7H&~rwkSyz97DGjb=Q>ZP#E=BO;5qKg5#Doy{a3lLR;5Y)^*LWOkw+;R&||H z%@D43Pm1q{tD^AyVFjmsj(NxSe79!!jytKFR?QW8%kjj#@SK`!FtWn`%svsEO2x73 zekWcSqUxKDO{X!2ta4Mmg*AOc*J{3LX^dMHu5SvD3Q5N*O)=-ZQ?q@#pHk3CQ4lqh zbuRe2u|A_$gFFLNcgR`6(RtdK6|rb~CNYfJw&UwEFavf6dt9s0wH79LNG7--$jmZ- z5pEDdLA(J)!R3AO`Y1W>zxVx0Y*ri9xbeBpMwZ5x#z*Ngu@JDHG-aiGTS}FaB$GIe zGYSsxM^(Y4Pj^70_$>H}Sys?q?~$BYqSp^PAt05+Q#@19v+OuNTSTwsJmGE#m#j~D zPDSKQdxKz0xmCyWZJmM@$H^`0?$99rSdfbFUDL?9qUyMQVAtQWO&M65aaip@*_#iQ zRWRda7)g{MRIDUX#%dBKNJwk(^6_?8FxYBQZ(n8eQ@KdR5;LX+y7- zSN*hZr$ysnaZyUU!mqiuC?!$Fa|H)2c_m2rA@DOTo7|<_oRBN~=xU4Z#~Bt`Czdqm zK1;XD+T8M*FevKqgdd*oO$9eI^5-z2L0;UVbS^PRVJk6osgg+I%_ zVt;0uO+vpsH&e4y#i#kXhqFZm*`^DJ$c>uo@(R>+->H~}HWmz1wlB)UWwidibMy&i zUd`M`a6$8`>c|)77A6`oUxw*^c+2f`6YSS-dsP|!d<*%x=$2KiUwLC5dzsrz5*;TK z&uh9xhOx}_wA4&6hZ*5{dYL%SHCHJtKx;{Y@6ig|SizBIgAl)b9W&&&0w?)9#V;$+ z27Xp>hR<)f(l{y{W9PS_yx}B*vwR*1gnc;2d9tp64EniJaGu5yX{=A(qH_xeZ@onqhtVB$KhEx;hlZ(e*v6ru=pVQ+ z6zz*XdsSC|^$tgl2PLlV! zxrfOf;Y$4>O^IZVg7M! z&?3i}7x`YJrwDyc)AACdU!mp^ME<4wsEVs91{CsDaV>-Ed?m5o;K)+)$8BP{(*3ot$4M#v85K&7a#C?7us*T(3WtR`WwU$F%L7I SY`3~(gp&8E-QxQQM*jmJUAH^{ diff --git a/out/test/clients/org/apache/kafka/clients/producer/MockProducerTest.class b/out/test/clients/org/apache/kafka/clients/producer/MockProducerTest.class deleted file mode 100644 index e5e7714b3e1b282072d7f110191595f87a4d8419..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5963 zcmbtY3wRt=75;DbF_X2US4-784nD5nSOm*(NWmu}sP{bl zq>6i_^?Ox}sTfx=q2fLj_p6wchCij^0TmA___T_LB3OsdsQ9d$_?(J|B)WCz&N7~uxb{%Ifr(*I@DH&bw}&h98F-6hgQ}x(#e#WNo6gIJ4$ZOI$2An z=z>x}Mn`g>tewTe#b`A3Z0y;%ffg^Eg<`44dDZPRIomV~yB-1^{_Y#4JCW9dDSgB- z%nU;u-;vEY#<0H2unoq)Ig>FR&7Yg=+TWXH9wq(I&XJ+rrdf_5Wpqp6#F1RDEi+)Q z6j)lpD|2|*%p~)AQj(68s-7pHq|#a2kyaN^Yox^!q57^0GBagoQ+b(`k4S)a&%*GO zc$FI}ACacr(&*4KSuO1s2VrE5>J~S>%+r~gnXL!)6vxa=K5jImc;z5L^IEHDWw@Vx z{<1At+UFGc(3Vc?gIc=T8q5we-HT08VYREaM&N`lM@#MN&_-M$aEXSrd!vG{k*G*W z3LfW*hTUpeCOuBnR(9Dhre%8s<`+3pylQyBEjZek$wX7#hDI`;Ky{Zfn9*d33YeAXbqxcrSO%S^DOn<^2GPCLaM4z58 zhKKcjLv!@>Xd>nFF2feZckrZw??&-Ge4ph$Ma#~nQo3yqWYbZ+4A(^Q11S}|bYQ^N zog(Wiqxd0yG^>r2GENjf#!sU7DSjq#{<&=ZLZJ3CJ!gDK+pi~_AwA)xal*(X7`Ft^ zAIG$az%QeC3crft*Z7T8=JtP!c7Zcm(z<5ljOdnT>!o_Xi{khAgTRt<=qs?1MD~vY zXHF;VvyQ+#_o#JCwAr$>(RTVNmH(vR=_vk;zsSyC1vZphn#xT^hE$i)(spu_KA>gO zrM$Yf+<@n|?I)MZ(*Aj0nsCnuoR;U>97IeJo}UDn?*2^z6J&ysF2ZhRu_5~;4zJI~ zqM}sEEOioYpmLeTbNJNU>F!2vz_4tGF}y0vB8cMec#=-+V3(tXBrT*HvF_sG%oO%@ z5;hOyG{c`;Lg1{zwPrf{qT79GZAgOt4-#mJOLnRK1BNq{&@u^9t;xFlC;mkbTMv#9 z5xw6VO*p13C1=Po4`ibFx3sym5b&bgLOCZ#@gF=ZaC)bi7&a}L&Mp+Tv{~5FVhITm zvU8yDxRp!hB>GoG@n4zWi>3;e2<7t7z4GN*m3O1^q?wv^Q;ASuZUN>?`}PtJ;!<(b z9;=rqc-KvVq1rta4$X=Qo9D*H0OogtLq8q0S?xK#p?#)T3BI)ty|?1 zrRMVQ5KD^{W`uEmAz@lfUNutw74ig|ZzFsKFf)=Ud1zL!Sd62TlD8Y5{&^JTHtJHY zKOIU*^5+Gzu&QT1X!Fo@ci$#p6|d#*sjWhoy)5nI`%z=h|26IAQ*E1kcr{fxCWwAm zvppx5)b>=!#iCn~zsscWziXv{s-MVHN9C#$m9p5|UhdjlsZ7iAJa?nlt^c7$r-g9J z!ByY^>3VhkHG*lIW5Z7UAaiDZg*dt@E|YsQXBNx&uQYjHbgzTK0nJEz?9rlSWhE`l zrIuSmJ01_i;Xce^t21n^oAu-Y`GbT#fT_E!N^@T;OiE&fAi|-Gw-aF6`vC8_Ctzjb2=a-CU!a z))RbsQhV}JJ$b2K&R$NbAm=vYrEG=Sx`I!EEAv3^6G5*Z z^!go>IH!9OOS%K&I5+eFCj<3e-NEHu6IeQq+D;jV1_ojo15r1LdOnu>k8w1t=S(gj zu^NVAeJCD`hZv3Z;n<2uJY0Z2DRGPkJ#gi-61sIv6`&Dw(L@w3!o|e)3jU7p+s`PP zaT7N3t5pl`!X_8lE#4k|Wasirkw#4U9);_XVylX}I{Nt^pq*A1_aLpUad8jPX1$Ah zIFGw0H9&MdsX^wsC)G=wr^9_HhkMmil&WSZ=1JfJGbHdznZW1y37ki9l^k8|jdr?e zy*?OQ6A$`{J+Hn&VjJR6C^r;4pQ*Z5CjA9<@t~je7Z&Cxv93F|ejJSqNmG41Jcf%X zEazDjm&Gz^ph2o^ zinaO@x925pM_%F>19K;i;3$I`bc?AEr!d@GnCxvVhOG?jCH!q?NpukEP8bZb#o)f2 zJaGeA;#LOmPKNF$oWBfO; zS0;>@moLR<^QHI{_PM3lg2$0&D?m>kVL9)nua{#C8MZ?7;5wL0jWE~Rf~z>DaK&bN zxtj$ii&v)PLP)z_@k+b0cs;v4dAq%NyS=pQS?kSP>!mfD)^0=id2Ult;3&u{*zf)w zP{6-~5DG?Xo+krUSDbgH4oIo1pQo)Eg{mkduAH^xG?h>#U!1vFc_LiHa0mE|)_a*2 zIMNY3!ZC1b)nqO&1!8=H^{FsO)fi-uhTM=z4wT1+$>7VwBkG4twyq`E!UfwqNaVXD0A1TH3UNx@|WR~X`BrfvG;3~jmmB180+Qx*&d zN~SI5Ya1)VeZW^N@}x?R!L3E^nsQxtM*WKEFW0%SP+KemLCYuA7$wFX`XysH@R+cdsV;hyQaN5+Txk9eX%Os(Q&$0dZA|f=7v}_J(IMUux-cZ@**Tf zXOMTTMw;q%l9TtHnrnzDQ_9%;w+`oJUKL$PF%0fiNa`4=p@OR_ilE}!h(ZZArr?^2 zaa>n%0~6A^NzEo>3qvYy;kJrNOfkeLzA!{{AP>XHUMrBdqcXwq?806~Oan;#l zPy$KsK4R#}&E$9Cm!XtvasOM0mY^_Ut6FuBd{CY8KA|2Ho+n%?Oyz+Q_B}&?Yr_9x zB`VDqLa%j(NXyO6$|Uii(@FmBU9j@Y^c2FBQ}ocN38J~>{31@_0DV2=F9Ko~*kxh{kx}JdAF#)i3}(=%p3IVLBXM_YG;lO6Rc ziZLx6`)fBQ5vayuTuaAWcK0*Jq+3bTpXy30vZipP$|OB)ko3$F1;A0Vk0FiYIE$0? zAWq>rvM8Ys4{-(+^y3L~ct)S<7trt$1-zqA@+S=93x@Cw!}yL10mTa;jdh9-;VYqL zi0UF<;Rsm~LVQm7GL-)b)+lQRQ9>`2nT8^Iq=h4b!MyWP6Dv~6#8U~|`=|foObxT!L4n^s3Ga8pe?rbjba*T(# z#9%J_k#fXSSA%Xlnr2vcmJT;39nTM3xxOI64ZmdIzJ&*PXyFptEi|FoLJKZfXw|be zT(r=Ew1o*wT6lyh1CK3C;|W9W0acDeXki9vV#?{1J&H3k%FuEUy11bPFENaqHx6}% z2cM-= zW!G8cDs-!tbNciOoeZB)mwZXgd3GGoW3!%EQC)!MvZAChQ;Ui_0%`gLHqy5;)zYHL z29PGc#hNG=YhqojiIMbXf<^}}gQ1gVuTXo~j_3>oh7z z<$j=cCwA>3Sxc_KOI#t@9*65gH+pGBVKnV&pW!M&+fVTYsrapd+;=4QV34Yz!S+x` zDoG=?i~4VL*6Dq)A_n6BT$4vV0-X}c2+3QYxLvoKMZ|HlI>dTJ5WV) ioxBVr-W$jyDO5i#6Bxh^(neZ|J?z{D>|M=^--vOM$4`n!pu>_is zR*;F~S^}jQk8)1_j?3AJwO&;AURN-gfGW+W64;CDaomvY8wtFLx5{u6n(R);q05o1 z0z;b5$j+>snM=ULyzDM0$R$vR^x6g|6Wm}?ftA1!*a_ruOM#=n6;KB(%W!&gn(G?u z?)B`vsgG)oXL_b>2{fl{XEv$jHGR%VE@(3gT2i+c7i}w<*<#r(keJq7V`OH=H9P?^ zC{X4YxwU@moN1Zfd4baAmT`edubnjn_N7eA81@&Z4JV^b=QvW4vUM#tt~sXc7X~BV zoJnvei(tu_hUK}*yklp5-Efj)hHg9Av|(j4hU*bkxyLqRE*iE^Sc+rZ@;M-IX&25+ z%QGBH%el$X(EnJtddzVBoOB+W@jc%m?5FeAAFK+Y4jG=7)jW;f_M|;cUl`KzLD)ft zCaDHyn&kE^=|*1W#8p7}wPU{JF{*LXHAz&DW!atu-7~TREvb3!wwClgGsm=Ay6-roGugLf=p3`H^-YORl`byjY<qOSsq_O|rP?7gp`sZrDvqN=plNppk7r3H zMlSMIw4zN#JK6*;KDvB+G&{@tDP0uPv;=iqpr)Xb2GebzOdD80@;QS7RB;DO3ht_S z8}BH1SH*jHU&RObP@pb7XZyKq!<=^8XkZ%cJ640Sb_P0Be1wk$jun+};k2)-PsJzr zRK;ibT>Ab(!IvuT;VTti;~Rm~j}jH;Fc6!Y?6n=o&wF9EHu>zwt1L;6F5b@a)-*Ss zkfeO8;5!xH%QSaBtp}d(AnM?tjZDw;C{s)uL+cO)s+%{-KJ7U?xZMM?{Pyog>^sY& zdr&%xneJuVqMBLOr-)1j6BeExUMLQcmCKvO@%mXsX;uu;lyp@0*i2qy8DqH_8c3xE z(tW)n!6*OpBMA^n5 zWHsh9$=(E_d;<=*uAroK6{V9Ek-ZV7~h*v zix+T$BWmafC-EYCf?tdsgdsQilbgsjQ>#!XSFk6wioKIR|}DK;54mHvG+1A;0*e47L(`-B52@RqAWxZ#~5v2qUS2~(fSOjJj?lN zl>ANeQUzTKx)q#*Q1A+Ho^ugeU%*AaNt%1u@yTYB3xS&WH&|+aR!otvOkU?3>RP}*~*nsQXw|BmJvwtZwNk$kwEYz7!3ek<#7NK Kv^&Ct?lM5TZ(OcyYNSsQZ& z*l-J&$AXO)HeNDxp7Nj+aP2i1CaWL$DR&d?2d)=NFHuTJ?QVvN)LY@1teUV@h7OYx zg<-(3Tn*K+%OmbJgnPn|PPpp@K4ryjq{4dQ3FZ1y3nh6FyBo>Tkx(BflTZo<mPlrO)_+dbv-fHOa;D9UN*w;?Wtc=0gky5D5fXA^Qb3J2MW`Csx+F)KK zl4RL5#HjXotKERhV|T}#uu38=>f7g@h_oL{O6lI$+&kIjQQ8^Nj}Co5Oq3_y`KJB8 z8$OsabB291=C<^LF!tr~uF#FJZe!8GD=azaMV|u)-45=d$3YSI9rR<&#%l-57-M*p z^7q!39K12nD-Kq%MwH4%-PT4JiZ{h-4JCM;VI?~v&CTNr>1_!f1azvJ_M@|M^%g4K ziMSR2!{)MeYjiU(ivVzCs6?d3vlXbY7;AHZ^Y} z4^w1>$^f6q*5p+*R7_(eJKF;BK|^Yj-n(~Z?zv~~oSC1$zI_KUgGVvMkx^z`nXEDsadcoZ zj@y`u!$3~MojBStt>LbQ8HV;8T^9Cyp$HEvdgvCTKPxnbItkiKWsT&EJ4 z!Zj@E3s-X6GZur*P2oPH%zOjIvtA1ZR%A#hP}A9|*}@kUhOyF1zRQikw`{|7q#3v_ z!5T|WApIxKo-A?StP)I{U$wk+j-k*bm9p@Bg;TJkzN!rl|RrBPk`Z=5up~t)sRfXoXTyF|P^1Mj7>IzOG?*3!Q*{dJaeDigvjojOp=40EY z!}h6XFPfEWzBqm%%`wb2$Cs`+Kcm^7IV~qG$5`C=Mg6Qv7^7j6{(aFIV&>`1Cq>j; zdQ?I^m0fNM4AZOCLGM{KnN)fJ$de>OPpSw#iG^0Q(7y*)z|c!ys}#4AAIwf1gJma= z(UMIaBa$769wVCliq^w0r;ltKjtQbhSmiZstKgj56j3$2OJTrPV= zs7HvLpzS&N=n1sX7%EMiAa)ieRvMp^KMunzoocjGwvI8fy;Mi}2o#f>h>?9ua|cxQ z1I@mp@%Oam03#HSkvH)1jA6Oy N+X6L~E-88X{RYTVt{MOU diff --git a/out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest$StressTestThread.class b/out/test/clients/org/apache/kafka/clients/producer/internals/BufferPoolTest$StressTestThread.class deleted file mode 100644 index 34897e9de2c1220da7163540dd995d91a8cdc65e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1690 zcmbVMT~ixX7=BK&WMSFzX|Wo#tyB#`x>^-$3n+nB(JG;>U$UmEg3tGv4}-oat1bvx$S+I@2LDdG~z0pU?Ze=l4f<9{_j@*DZ|TtO>_L z2Cw&F7;k8CPK!7DsPMLh8N6fRJl@sfg5F-#t644PwAs9g_bd$JeG?@UivpIHs7NMW z7{mgiEMPRl&=)vg38R~iY)W@aIoon`TRN`qsUV4+W)wDBu8JHlplu-i*eSI(H&t|v zmKO!gxaGPkroXAmr}8uDv=YyE+%RxkQA7`pOu`+{br#Z3iAG8Ww9CzUftSn+%odIs zyga$i3KqkL5*Vy_fm&|uY^bO%H+-IqR6oW56zI$jSTOLy#swEL~uB$kyZ$(Nr z1nhDUsA$obajd9OJz`SY<(dC_XSvy^6;Kw^lwtj+DZ@Kw0{wxym2_#HHu#3X%wu7mYMMwwJ&u~LwUi;G3um-H2@~m8 z)lL}QP9t35)aj_@AR+$?Wv$dJeV=?!*!B}V`5#7E4fRjGCu(V&vazD;!DhsvPf1pf zq)XQqj#ik)7pM7fjqqv9Qj$nqjc6|MJFMBoZw7*7d!91Sod<)uK)!eYQM`xD_3Qz% z`!I^*`{*f7?IX9B8jVwqp@;HYntX#Hd`HLM)6oxjfv2ZBZtx;g(7${dR-`MPhz|P_ zCc0rCQQP3%T6Q*H{24u8W9|;jHhRCp0Qb2z`sQ{q!n4lK`m&AwIlUfl!(P^l={Ami zA;j*Vzj0?6Z4B(9w=&hn&>n+;ld{4`ISZZmIoOmx!onU=`H4xkF^vPh<-cH_=S#Rt z{rj}I$GQ5I6ZIf1Y92WjlEs;{fN5gbSli6&iw%N8ROG&fPD@U89r7b0CYtw_0l9Wn9OR56X-AQ(u%}kh` zEe#4PQWX>r?)%m%B7)GQrHCjB3W|b?B8L~?je<9#_5WselWnr0`L#V}zWL_O`~L6! z-+SNN?}`6C_%MLQN+664xJ<<-!wBHAad>YvpoX2-k;k18xlCCfpptEi!p)2)D`oc6oD$T<(;|&xY{15O#;~d71ix zT9}lVcY8cD#a0&3}CKX=`qe)DEB#agK`UHFfkILnn6YwoOCYQ(M#Y1xW zb_m}o=lzK=p2T+r$@f$|rQ-VvCbzqek;!xz8Mhze6Iyv zxJwRxyL_~|(K9QlqinfpneIvjB{j91Y5W>HNo|u_P0QGr?cHWL9s0I3XCke3LQikj z9aEljlR-CS64~-1R;NwFax?Kh$4+Jwh7&g}*KjPIj$57W>N1>7ww)Fd#7dXB0ldmW68s@Jxd41bGFPDTh{27+ni3*VXo0; zJBDvfM@qN07^mnvXiUi1*=HnNgV8s;fR~nZ+UVBP%}#f=m+l*aLR?cs@kZO)MzuYR zjy8{{iS4eQ=wUQ?gm^K24Rem>iTGM0t@j(r92AINw7?^yZZ3@r zGssVEVxQ>2&6At<bn}Y1qVC zB%aO{Yz^d5WG%G5<8QqeH>c(W{In6J*gJY4|ao<-X%v9FwXS z%cSgVIvF$0&Fblx!y5q^N$*{Qil1osDV|gDGY!w<1t#1zB&c5)mb-Y z+2_SFcCQgLGcntWxhW&oW=THSkg>d^W0ds9dr4w5C)fc9>1aX4$kIq z6gZHToASC9Gu>E9&m61axA>ic#zS-LQ@CztO3w;kLv>ukdDyAp{kVW?=tbEfSeA3f z3rdftl-}28SoBT5;m(hpr<*RV7AN|g687(fmOtQ34S$qWm^B=gV-|9Wu|ILIJ~aBV zw(8jnMaOF@{;c6Ivh0!QS|qw&XMztyZ+0BLU#gITQ;z`U$jT@Sy}{z5;jj3chQH$< zEHfJZiGOMMH{R6nAF<^mzjEf|(BomKkjEGmjDmbYjb*Zl1e-{f^mISjcnkkk@jnd* zsFIC}GOidI%gZdvo=dz=@U(5)*(T(@-(uNI!q9Qw7JoD`*wXy-7 zG*sI1^JRTmx5c9NU{~5?^FE~}*Bp58ybe?L` z821Z7Q|(s9h+l{#wJY;+=W&xyd4`_K{c5AJldfV2^P8aMJxtBZ7Zevt2e;cMgHO$s zH3jv>E!ADS*G$Blz1wQGl!k8c^V;F6Q{a|j!i2+cWoGzE+2Lf_;#C%j?5(w%aV(Dt zHt^gc$deAXa2y4sEt8`R>BYP|n=YPfoWXGbn-Suets$?SoU!;N&pdE^-5`{@{Rngp zqGS((9F}(0J%HdK%Jx95i-h)|oReX`pRfnoeV(?hydI4*UT0CyY#xHl!A#8MXddFY z1MlV=&DW3j;JtiOa2D4tk+qc@dDRzohrM?erqcrr^j@6I z98b?J#C$BmNj&ITkHutd2|c$I+i)UGp7XeTa}gSGDUW%s#&YcD0nR;GiG6q{9>*#? zjb^sHtJ#CD!RuJ-F|e2iU-WX0fgoPP**J$xgxDeJ)Lg<QcLhCp{%|?@}ml4?f3pyuw4sE0Z*0`&wBwSsg4y&PIY&~ME_ zKf#ASH&`s_TI(4r6 zh<72Qd=arq7F06#Ku-NEsN%Da+~({Q1P)L*prTL3xlmL%7-7h;`!ho}Fd02!+k6|u z%aSria|6q1|#vXWW(`{{{;hBDC0%LTpY*v08v)c!}I9f21dm) z!dkmzQmF?6IB~(f6bN`EM5q*GxtuIlQppVT7r?8^!K1nyJQ+!# z2l&NQ$_l(n5_vusmSW1EA8!L>S!Z4SF$)H9(pW$)pxO(m@Dc*K^pHR#3wZ>wy0Qjp|;4nX!!#;c>wG8;*{M7UaSm8%e(?TM zhbUc2!aqc=?i?CDiHW&6dO8<2C9J0x@e3ZL@pt}*J+{Jl0T**rLM+dC6*Wji_j}=0 zMqGCh$43}YA0>9#1O1tgaYk}}2|@=5B*dKOPoW_5Ud1Kch~LD_w`BIxH&HceFlWrP d`8ZVu@ChcRY{(_t6k2d9&05B1`Lo2o{U6WO-YozC diff --git a/out/test/clients/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.class b/out/test/clients/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.class deleted file mode 100644 index 012a3adffcb06e43cd0b3c861747df19e03439ae..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2835 zcmb7GTXz#x6#h;!=_KhCN~9EQk*R{TQ3xPZAf?g^NK3iclwzr(ljO9Sb~;mLCSZMr zAE8gKbRA}?KG13Ozo!?DJU|d$eko+6~C+F^ECEhE{%JbS0`WS z_%e+y+*g~g+VM54B#LR2P)x3c27xtmmI zD4wnxoT+nWab$j1%g|f3O9GjxCF`M;b1Zi;R|tLEU8LJ|Gk^N(tZ7t3+sRGYK}dP3 zU@y8>SoNjAg*U`QJ4SBaal&2j^279_mz-)4DvbJev>L0aP`ho&cI|LjAl^SPE1-=L z1A)C$wkvN|SBlb~v5F2)GE-j3a%L^xR{Q#)7FKLtX1qn5H{Zf`nUZTcL2g_wSXC!% zCe9Et0^lIuI=n8|nv#j25Z0onn%D`534ue?RX4O(wj)h$nZMcch=hm|`nU&wZp`(`{EYbp?9u>LwqGzhPHzLK+UUfZ)QI+$E( z`G&O`fe}^h5?{ICRsE8@YAeusTHp1gstyBZ@DX8WmFrkCunfn5i9Uhh-7EXQ&@}Ka z-V^BE?q##eS#6Gb&x#j>Dv=cfM{&%+`-&V_q#pxBL*Zs1iw^{j?c}cspQ=FR%+*1@ ziU$Th#0mDzPL}m5R5reWQ#h?7FmMrkfTIO#MViFM3@TpLDVuAyBn{N5dZK6!=%^Za zh(`kF@5+XvD<4YVELA)&kfvoi(ke^;gd>B%R0F`YUDH>UK*~}u@D0`kF3(h?S!N&l zc9A{lxn@y@kEC?X@R4UWD`hiu&a*l@Ei0@l#Y^*jbP;-uT z)IYx^quMR=18>Yb4yIf^@yN@V=Qu=z50l@bx#d(vJ+b=>G;39j$NMdy*`C_`Xv?$h zv1$l5ChGvFRHss^4Wf}CkZQ0PRg#p>vs>i)T`ur46Ll(H`wJ?#t2Mi1rEYWwh_e zw*9UQvOlA}FsG3jg}DS-M`0a%vYEZ>=u|5E=HkyuDCpD#=^pjlK!!Grb7f_`CQ39! zk^8NpXLxHJ`Y_7LzDd*IIxcHpCVMwk!F!5?SY0~bl@-!68U~&f1bEsBl42C zFLQ?e#pyXnO{0bvKBfi%#06p*Q&_^Mrx^Jl!|0j4>ROsmwEI^a)G?Gf)bpgZ=*cs@ zy_wM-z7hC_-5j##_=YO#ftu%f)NbZ?5qU-*qO6Vl5;?ywja)<;I{(xGGx4GLC2ja{ zXZc5r)PsvHNeyb5o-XY>q`I`k574!z(BjX~+ogSv)MFu@qU~`^JdTT}ul~#&?s?MN z$TZH(z}jjjt=(VaiNRd^nByR^$rG6o(s7Aw?++j;;1 diff --git a/out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1.class b/out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1.class deleted file mode 100644 index b5c768f375baf75819a0fd22f3da937a36117347..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1883 zcmb_dTTc@~6#k|yEUgPIASl!eYUNUp1q2n#y`mujL!f~eo`&r}SC-v%FT@|9uf}+Z z@4oQj3r4_b;-e4#BxBSw1%pW&gDFk+oSD;ezH{bWX1@LW^aa2m?nI%YLz2!IYSE=2 z9m7?0#}Gk}TwRl-HwF!T3i@M+VL-uk1%nLHEpDc`t_Pb8% zbKEILM}5E$TeJgL7k3SLgw~RRN>{xSlukTt>89-&*7}_A3wBIV#--vWZb_426}OR5F@lVOQ59pDWN0jN5*1StKU<3b zA1!*2Jch)PgIvKCJkKy$F)%~3PZ?YEUEz6afw^e-0hfoRMAL+kFxA&bv$1><&D`}@ zwaFtDhDzPsSt^~8{qe_tzx03aMEtKz9l;Jr*wKm-NW;z@M_ANZSsJIqUMxx$gcq1T z!@F{Emus%f%GAo#F{UibO>>oIBGYvY<$qj!snTev7;%THLft?PaxT|t!saW=P#G_1 zqdUKr-hfadA!iUEP6H_=^fjOn45w+8rG1!qJl(wumhRa_D7}yHPEoOmXf-JU&(KV? zxo8~68MM%jjEazSiJ%q9L%{cBAps}CBgy)X#9IufKOpoOkq5-tM^q)NoPAUuY?K35 z!}bte-bT$_&mK-aVQl-?*B!!(qMshI7~#oCEB3b`Iw$)(}R?>SFN(6Ngc^5k?&%KM-P%6kLjYM-|yzCTR#)kRl$TC$Sx8Y1Ke7 NmAL$k(;GQ~mfr(vA-n(p diff --git a/out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1TestCallback.class b/out/test/clients/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest$1TestCallback.class deleted file mode 100644 index b2a57b1b4b52575b1657590ef63a5aa583613d81..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1589 zcmb_cYflqF6g^Wa3+pNss0h9)qP7KDRD2d6r9o4JPY5RZX*wOs!gdyCXNmqS6Gh_( ze}F&AcxQ`%h6Id$=-k=q+;i^Z-1+|F>o)+?sP&>37wvM%g^Qw#63RUocQJv>E+*l- zn8FnYR~=k)FwHQ!!NW0?wCCOlxo(0;y_7-*8?sqfH6DfyF4h>v=U4d#_Y)I@zKE1a zw3f>F+(hj__|??Zm60u}D-7LcC5XqS7;5v8Zuxwjixugw@#Qt{i!dP1*k9LCGZ9kz z0l8_#!`LU{NH?oOB<&<5XuT+7W8r236_|SrZ;Sr{?=LbbEiq(kQByMX%?C<8P1+5q z7kMM3lm7Wg@NkLizAscRvRV@dzM`1dTvuIy%@8ucy&EyYcu zDYCAj4cv=VEo!fa()N_$@j+_W>A(|dc#|7m>69wus^xF1qhhgp+FMr`t{xznKVT_Z z`AcjJ`8`<}+=VF7LOu*^u}AjQ#-tS`^=3|qFp2}!qR3aGri1GqZeYa2QS^E6aKu9n zBMxqQxP{vcXP$3~uMorq>nfXt<%r0ZlQ4Wg>EMos3T7PK^>7dO9X#+*#Vo_Kf0py_ zjbq5AbCs6(Sz}cS!%*IzWJg-n{V(q`U{qW~YOOXRZR$#NBqL|Jp_wJI4(tTDtpRFI zOPVJ#j(LlkGgvI`>fwUX)CrR0uGBhbG|BR9LX%ESFYfZ_Bwi`KCLf#5YC_}gS7}%@ zmBY1Dqs^jDh$>rrZS_|=@M}cs2$ayWO^;4hH6KYrr1_|$Ku@F#hQr9@a#kY%SrQCA zqZ#_;aExp}?P_GZ$o7@TKY^7eJ|R>7jIIyq#Q|Eo2?lHsgS0=MI`<)u6ZAz+DI~)% z?K2pmn}>OQ~rp|CVF1&z#p~jYzQ7Zfk8H$(iGB&^4m0d ioW>a&2A!Ox+-$_>Qp!=BC!56>E?|guHs@Zl$AO$ToIrOn?|KjtI~O<4+p`26<}@_&vw2j z;J&m6XcnI#k7|Q!q@7blzvsDQcpje_;InwX!3zR>HZPQqTAk|(uNoX2l!=Svz>5R4 zgqMurrMyh%`Z2V^UezFP%XMB+M5}pafLHN3@>o5F8+nb+=Nh~=!0WgvK14FFu$*m=-Caz z+cxbdLC@;XLx6Z>eeq2MyjK2RhT>okA;WaJRuR zIZ52$9^X~YG@B$7$)rnUMPdvmd9{3|D(Kj>-cgRdB5~rq4NiH zzLLpov689f;b?SwxNRps8)Gpmu{;`1CaolsFBR{Jv@r$O?g(ERu8xLdoz=~$L?qT( z#}w!ZCsL7ABp#c?1hIKK=6XNofle&mZeiBcwedt}b+{+o)@4=i40r4dSGUExyW_Fy zCQJZLH4&=|Q$3hok#~BF@U_9+9}BU-UM9N}-x)E^lqobTVZnn5O4tX}LWG zr1WkJB5s5DEA!5Frc|%9r(#vW?rI5lV&QmOU&3Yex^NGMi<%>yv2ZG#06S;pjxHLm z+~GY-K*PT%5{smk;II{yn;@j+(3$KNiCInQ?(J5hCA>X~p5WSeTR6HYoQTM`)2XGp zBH(QK2uF%WtXL{p-IIv7r`xPVbtDE^#ljHqMyoBJXs>T;OLwQE;Z!^!Xay%m3hkYdTl~9DL*Ty?n%2bdjjKwd*>NQv$7Wzy}-y3V7sfT-dtXMmasc#Rb zfN{JPdxX6%|L zxhay209y62SUeS0!y;($Ej!YHXte{A@dA^2rIWScL?@tGq(pC9Wk5y&-cB)0BdT83 ziN>T~d5CayMIs!D!LCz1UXO-JwVtEr6Qq6YA+Bmk9fC77B%8pMTFGQbI@%mbrPVor z$W*SoZRtu_;dY2hxkl{lwe4n0tRWCfciYo2tikJ`G0-Pxz^k58BxFoTF;#nb2Zj6^ zQ%+gbRPI?}pEes090uiFA7Yv;=W4Qcr8Z)#-sN3Z+s+kMGz{bPxP*D#$Z1NGbEiBw|@v9Tv=r7X2joXJ#ZHDeNR zj#lJStGzLnjo&jpY_`Q?ZRtb;zOfo^p*zx6U9TGVL`V*R9XLN=u)LT-E+N@O(2}CE37(dPs?3(mh zdeEfL$m3)5UnYH-9)*F3@J;$SeZr*MeKWfswbe~C|mdE{c-zXdBCN3sD zK)YcjCSS+bkFv^f-L^P^(;G~_k#92TAra5Ce0(}7jm|fld<)+yoPNxtJ@h%GX8G0( zNo-^dl=Cos9(hzs+8N2RD3(}>p)_=U(B#kZL$YcQ(}b*Rbu887&+)@1f1d9*`3w9A+=R(r&->@?P#Uxu5qT5SzT82S!nv0}nU5!WquBdNZ=> zmXt;3Crm!TPn!G`e-(DeeM}QKNur!P4|XX-<019#ctWHs8DP@nukq6+f1RIUI(bx$ znEWh%gXxrvk>r&-lF^2As;oV2m8IfkNo49}+vDk2ds#dd?Zv~&>dsYJSNKv)vaA^R zLXK|1`{Ec`Ez~Cx;a&+3pnd!b7~)CG=LinJY4SmS4q+?{3p?VP{4ILaIg)j`&Vh**T(}{tru9uLnQHRN-x1c3g}5BtR?)cTcvr1ib79^kLz1exyc0cg zu$7>s3$WISSp0T*L+78t)Hd3s8FjgK?{JUgmvC;vTPH{ufs3KJn(IISfZ@^~I(d|` z5$sbHl04i(%swX+rSh}wR^`^bcLv<+uq`ZX592_e1PL)L$v;$OP3%JsYgaPrQHV{z zKbcaIO{7*{l15!jT!+UxTCT7d{m?kp&%Hv>u?zzKsxd_6Pz!lwTr9vvgHfU61Q~X+@*=lIh-OAw4)M zq}^V=b5EzDfYSvgD;$yK(#wqFUapc)*!~Fc%Q60S7P8e|8)ZpwJZm^y^UsI&JE9R4 z5S(+xBJpbRxDDx!4%8mtf+J3GPfqU@kQ?VPM~flHW18fS=N{J1QgaDo&1DIRL?O85 zd|RV#BT_&b0m4T)mMsAAqIZRplHe&sU&u7ZhVLqX7bqVoyj`eZlZ;vx_|Ym9maiQN zJ;!18sva={&&NSRIljVhJI-G+60UGIUowUJA?&%`!tW@k1*CAqMel}nyFes@(00V7 z?pPu!-I!ZRlX_La7H3OBerG63;6idq+LHIh?|xe)@o}k~A6Br_cB?ZIa|%^{WrEoM z1Q%mNqc7l>HQucML3jAFaKtWnNxXw2LYxHez{&ZsBx+QmbcF#8)(8n)@@`HtXDm|Bm^VHoo49ER(U}%KsL(}Odx*5NXPNV{i;i)vReP~=FZuV;_&zH3cIm2Xg15_1T|snB%q+xWlX)La>Z20xfbJP6 z^$r*q80)NiPOJ(}>8F$WsGur%vU=gFViR)<@W=aU+O|F_!}RH`zNh45 zLcY0G6JpK#=@kD{;7K}5?9S5WRv*45%h*0TwU5fN_B7%8#C>#nAI-pL1$M6Nqbdj# zYJUvRjWh|**`!f5e$uU>@iYfLb16jgsEW>{Iy#G*Xg-B$0j20{KxZL}Z?*I_j66?^ z=m)fzUZW-S7A@r}TE+{ho*oB9(^eMkr9ONy^+S<`qR4_BG!8ovFBi`GY38y6G^=%j zhFxbrM%9z32HVTAX2>}HpzkxAFpk%<=zOhkzOTerqV1-6v(ZzGKP4{ir?WN(=NCz5 z`Lupou$RsrpoL7=?x)&oe7tAy#S!M$VYa!S7O7Qu7k=!u`)RS;DHDQAi~4Dq%n^`E&s{P^(g92h_C?>fZ;e^8wjQ7&`a4uDeTFF0QMAEz^aLG%!e{Yh zdXk=k(&zJ7`YPsW*!cqh&eP<>-tF{tdWQVi{XBY>zCk*D6DKGIEbUnbG0Pr%4rAZK z*c{RZ!H7?%Z)-X|&+rC^X#OBwLx$U>(+fJisMAYoY#Dt=35e-s`Yzu5VC#Em8OL|f z2!}KdED~*+P(X_WiBoJG7L9zi5X#zFDUL&QF@&&{ z%ISR&=QcVYRwcGlW^>?(P=iW-0CAzB1UnQF7UVt058ZR{%Q%N9Vgd{(*P^D4aFV7G zY+xUM)hnF)LGvtS zL0QDx^ABRaJ?l9@H6L_e$*BBMs2rTotHETR{2>T)=`_fs2P0fl`4&nw{CuB2<}D*USUAw;FC z=_mBzjH){5B;}_3STT)$jFd$z@l{8C<%myWV<^lb2@y9{0P44a!qc$I=MfKWMY_(( z9(y%2CQ1|MbgDR~b;>!3Z&HrvC88sAf=)l>f6a z>|2tfTX9Mf|GlBHRy#l!A*^5QD9PAOQ$oH2w6(P=r0t{k4bZj%+N{*{hJ-3nvN9^A z` zg^HXNljTjV2-jl!ezI_wEQof{iBQya(8KkxuN&YlZ={*XhR;M&vzTr{RJ<83`EFSG zJ%H4w0H=E~`#!oAKW~1T?nktGfSyA}`y0UQPxK&<$1g@D^blg^9-c-I^C|QNuBS)1 z5gDxbi94am1vJ$GUkb2$jebr#))gxI9|ubFm3NRIP(Mh&KvXQkE*C4v1whMQA<5Dj z2d)LYn*Lh_2S2Z%Un#g6h}6HvTOrK^9|C3#|BADe{zpaHtLO!K9ix7nFt%j!QKMvtz-`35IO=ZGf@3JfURQd$%iHCzI2FAK0;$ydxy@^>9;oO zk5Gjjgb&e_LC}`yoL#IT1*aozv50eQFUJdyEwH(M&XXuk86NCZLioBHtegl| z>kD=iNd(F!j=|1p($R$+QY-?YG}4V2sJx-%af);hLqisIup-F7eO+8)vh_ft#^zz%?0X)@j)2R!y70QMu>A3!Gm1X9BR z8S8b-H5+ggb(LE715XH*|uIoh;ViA0FGuVjI0e|4V;x zfVhl|K@iSO{-z$GI-UN=hiK|ynun(B5}n=}1RI%o7^VVra$czYDFd}{0cv7wf0#r) z%O+8xA;M{b3)xJ_ z!`5E9Jaha~|MX)OkUep@fbui(elSryDic}ju*1U-VI4mLhF_sd*M}Oh_ z>971H`Wt>Y|2w})|H$Mfz*@!@e&^VN!LK9bJOG=mq*7=_{*6GM3y28FM6w@TS5;yP>?+b z)j6q4Z6tjK6F4)3w>$U=>Y(5D!85 z|AuQk4A*!B+1z2=jNidcbr815)C-4q1N&$<`{^KJ;TyOMzlHW~9s{+SbQm`Q*9BIh zcp@%P13>&L#uvePR;b7>Ir|Y;?es6#1tK-IU0^$sZ*hDYa*ZwYSE$(Mxf=_OgUI{s zv9}%fSV8`Gs8U^v4^ibibP@cXPJcs$K0;INu>3EYI$93;_Y5>7;`+hkZW|gOX)WAG zA8k^;a5Rk6+NBu*Np%QGsaaS)}piBtnqtVL$F4Q&UXOuKk0UC(873s0xJkf}Yu zfjC%^lea ziw&s!5Uf?eSB(~vNdigXqj<54a#E;}qMV>f71E&Pe@h_^iw^@k!xtAE+v0adT+GL? n7@SF5EdXP;qCExZybkRx_}hv<`S16rJiSRFz6-UVyD9YlQ=M#A diff --git a/out/test/clients/org/apache/kafka/clients/producer/internals/SenderTest.class b/out/test/clients/org/apache/kafka/clients/producer/internals/SenderTest.class deleted file mode 100644 index 80d43e6bedda5d1092ec07a3859b837e3f447998..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7552 zcmbtZ33yc175;BBnKzj{1O~#c42mIyWdK=}02)H530WW$z(6ZLCNIe&lbLZA1eLm5 zm)50fTf1v%ZLO_}WD=#NcCm|gZ|%O=eX+IsrdIl&_hu$TG82h~Z{EA_-gC~q=iL9E z<>u*Eo;VI*rMke2?KrIAgC2ayixNCgjx+IL4IlBsjl=TspoWinQI3b?<74vpxQ2%{ zJW_Vm3Fn^|6Y!`E9P#2Y9F^99hO`%z4%489Aunb);vU!VL>Z?BCV^6q;S*lW#wUg8 zxQ0(@c(PPRJovO1^Y9tz|E#R#b27_Q9(>-5FW`$Fd`Z}zF2^(YvJig7gRjaezvji) z@r`nP6W{Xk@og{ppOt~{c<@~hzE^?o;|D_XLuvg;!;ihF#ZRQ?r?TL~Li=0=eukfG z_=OjB_@x)W!mp+KHy->}B>zr?{@#P-Td&9Qv4Z)5*!Ny$;3aU|ILVYxnNE(sk4kMg06+CtI zTY`J)nwvQsEERJrR5+|1?Trn=QZb~0FDt2|v8ADHd&iy@8Cft%p(GhosBVtNdjdwx z2=$tQKBK$O2!x{j{n1FEBO0?pTa9?qN?OrKt%56Q^>g0h9I2!gP6S$_p}r1MlEWPe zTV^DwaCU)N>>-D|-fokje6NuV^#-lWMZbPCX>=J$L*cA3*fT_A)lGB$Z%w&o{o!Op>2mp6j35)XUB9Um) z5M)w3rzVK^rMS+(MzcE_7w~zL6cDufW3o7Rw7Z*vcvPDB(VcR(==8iT3PmHKR6I_& z1~%Kn6fx~&l@7+TYHb^faos5>>ueNpWh1;+_TAfXAY{gDjWoPSH;|`P#ugQ#QoDfB zq}Iw3JQ@db9TS2{Bh*JE+Qz55w4LTlv=5Io&5$&^c3R2ahIm|%)mS~2614@CaE=gR zmyCv@;eahCQ7fQOH1Wd1;u;Q^A)b(EB&QyAEi;`p6hC?+XQ3-qI5g&xUz6R(rRK!U zk!v>-F#?ZD?=J2QNBGaC##2x>Nr)xHJ)}Z{eOH_lDNN3J;o*t(#z@4B*N2TnLa?x@ zs2*|i(iBmX2yAuSMOMW8Q9?PPCq{oS6l}`ej*=_JEu5kb<-viFEVMZVeOPNgg$Z#)5|(80V7gT&XZ%Hj=cymevO`I zRP+<+|JBh2Q%5&;GKdvt$_2*L>;PoPl;&upM`XOBqX)ejPUt+AN{5Ag3JXTda*RJB z!s+NkSYi5bT%()sGrKM{6206+9sP(XtRF3H93e?lCAunAE-qG;>9`d1MJvW5(b-Fd z)#GOp01(1v9a%0=%q{fXXaD%|DoHTIi+!=GSO51amsm+)tfp5Kc2yTevbZ*r-wD%CViP1n^7 zHB(cwXb9Y+tJ!Lfj(6Z*U7fDZp!w^<;Vg5)pYz>I6=t_Pr#U~H3 z=xV77Fffc*v~8@o9mTr*Mx<+41zjzpIdBK=RPYD$Oq=YDM=y(z*l%WoByG7|q%Ro8 zYF-E*9d&fILYo)R;x9#1LrDi9Bm-yU}X006x^nO=sZoW)m5!pSJ32ROBDX~ zy4s-5XUrcy&ux46nIVF8%5YaMf)WS?wRRF0<}H~xnV)PnN}OI$JX|@B%TC;dvaGtP zN#1iu8Ax1VKn~J&N%y&C2ldHlB$Hh8WEPyisAnHe%9oCsEmMrd?Zh@v&pd{4DZzB2 zgQ1d+xwWmefkrPXyD%f;)7{mLP2=TiYT8wlWrQa!ogqqsvM%Ks;m&hQkJ8eXU4 zY@D^_3K?!-JjE2jRc&v}R4cOCj(nLAP#vlGboUl7Ht2Ups*<_Opf$Rz1P^h9Q56rMqN2T)Nn`2q|;XYYj0@~N0Oh{*%+)%Z$XLzpsvsT`=xkWU-HbbijDpqT@h z#pY}o(Whe05u8z8UdoT@BALHggP1!2KWXL(O$C2G{`|hPN=hd$n8c<{>6<@@vj;#Wxj=J)c6+hq0(4<6iZs!Ta}w~>B(QdfR`N=X)L>82+O5e!$;ZNBUq8f zIfGc4#;OqZdv0Wp82?SkTJR?JDBPU2V96Ea(AagIYh3}D%qC0(dxO|Or9C!MTG%RL>BT~1 z@9fgNW}b!Di|OQ-#`%xq0L^TVDlheClGPj z5Q%dOLFZP&Q&z)MwuzS{ZFEk%ct6rjfLzOqkK5>+?xkxwOy~44ozpSIaGb8`SvsfZ zcuDX)ozu&>OijZ9H4g{XVqC5+#1$%tE7ip~q%8JDah1B9HT1Ovl0a+&+-g2yc?%t) zTR@Qoo4N+K+Rn5_b;<<?wNCIEahrOrUf35s=Vc#G0-n}*w6 z^pG#|2o#2AiImjj72k1`Gsyv8;#;#Qd$ic#$=d0=a0rbLpt5yw8e7t6UXn%&Uqfhp zlnAfDbVh)tj6u&3jB|MvoYKeBFr6yS!cF|c+s$-rx7y0rkajjE0A?#4Y_Qud@dV>fV26xC3&OT*hH=jr7HjklA97bYYVogPFP|0)Pn9Yb4ZrRy&L zAhxb8TQsw52;0`WkDg zRc^UecA%5xR@wU^j;-(o diff --git a/out/test/clients/org/apache/kafka/common/config/AbstractConfigTest$TestConfig.class b/out/test/clients/org/apache/kafka/common/config/AbstractConfigTest$TestConfig.class deleted file mode 100644 index 833ec122f30ee9a6f2b964506bed16ff69a15bdd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1651 zcmb7FZBNrs6n^dsd#Ocm=nzn>f;xt*A}_vdBFh+NW-zlZ7(bD_(v3pbE^TMx&k_?! zH1VS!{87f!b;y92jeh94=k%QCJm;QsZ-4*!{u96oaxo0!kq8NoBS?x6FdxA}6i;GU zL^=v3hNoDH;+g+xIf502aK5~=ReZ~kD%nm$;VrHm=*khVA8|#qn@!syVbzU>lJAI3 zy`Ev1*)CKo#r$5S@V;EB7AkxBQchJ1>fR|@ymZJvamD0TLs31)uo@IKLJ+RUEpL~b zZJlBIe<+*fJTc1}mf@{4#FGJ7+cQjMo3}Fay9`5ld!HaiONOQIw3{{Esq&ghmUzk5 zxVg(6!`HjUA@9I&8N{mYdO;lwQqi(>CvS4s)#=-Mzxs1E*K@emjrWn}&fuuJ;g(~! z2x#;ITZY7WN}pS&w8qc7W>cwk0*a|d!{T1sp=uHrYVWxC^+&Us+1(!@=^I4VZabR3 zW%!#gahYt|Z-|5uT$eD8RfgoC{r3H5BUod|_6fP1wP%YM#+o{ff|hplmhE`D<3{jY z!b`kjKm-{HuR-lj=Va4xJ=w0yTF@NXv*os{%iNU%1XunCk(TfV8w`tAqDG}8`}O;5 z-p_ZbGk!}~O6Si>e~SMS&Q)ndXvDjsR8*@B%D{X(;WS6feK?-wD>nUxa7&}CVy_~U z${!e32Ty$#Oa#LFdYz6Z!>{fX_g)8sWBS{W33|{0DAx{k&C3i9&_k*Rrh6^T4~&z6 zOxLSKH?4pv{~*z75~COc!wp)85GH%PM;FP)==9tqd5b($iVTsArH;WCP7wM+ihu;Y zeOF)`QS4xXR*Aq7w=qo`h2ReE64(r_C6a;V9<5|rhj5=FA7GYrKhYXd@t>(55I&w@ zC>}mWBpy9REG~Y-wXa})X1ZSJ9b_(0Denkpjw~TOq(Uf~xD6vkUqt$U2zaqEN&gQ4 H*^SA+(l*HP diff --git a/out/test/clients/org/apache/kafka/common/config/AbstractConfigTest.class b/out/test/clients/org/apache/kafka/common/config/AbstractConfigTest.class deleted file mode 100644 index 7ed61488886e1a2d10368cd8bf67005581b4ac71..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2464 zcmbVOT~`}b6x}z3ObnsWNTIP(Q=|eRnUs%GOaP?_iUwK%LHr()3k-yrIGI%cQJ?z) zE?LWWef9sS`_2py5-Q5exp!{P-DmG}&YZ~~fBpI!fJyAe(2vWd7G4v>;=W6|2nKfnBW7xnK zQEY11((t7~Y(v)E1>4#+_XOh0mL;8qiczb{nm}a6v`lwSplx_$Qy{!xm!&{g(X`}h z{b)xz8^%tBCyAn6GAf&fW2)!iFzoJ|#F!}B&R))_8l`=iJ2Z9=jak6a@O*b5?anmY1!XYgi@4DY;g+T=Ph7nl+Qb<}J&14cD|S{?_fqI;fL$ zE&%FMK$}LzBu2IF(&^4{@xXXtms@uo)7mR2b%H`3T}ReW)Z))2SyeH~w8NLd z5OpsTE!YC>RmZL}a<3n_?wXa{a~?^@HD!$$5g$Wf?z;G`$r@BbeOg57Tz8Gq;fhi9 zis03Fo9)6*VN(d)R@qp#UNq(-wrDLfEB9BGWQsH~~(*Hrgv zs{1w7k0cXXs_9F{pjP)+_TO(;SQCqv5d0g0+X#!UL*gT=!`&**AY!c7nmX1|9dSOD$Q2++3IIx zYjJ#quT`185y-s`JCK-bDoq^U;yVHTb=Um%%=uk*-u&#A=4O9dH!EdT>hV=OCC}cH zTFP)FAFHzD%5rL7Ix>^;dNAd^8Q#zNIKIaZ%COmXP|{G2L#h&vFP_~rQa+Ol=v28b zQ?8vd{hki+@&aSmXld#S{3eG}~5rOH(DdKw<8sfa=ZaLJ-G8ESlZ~jTqIcw8{e6D*zgK`ItfFI6^M@ zU!kMfKkC|Nsp%PR1twy*s)aw5=lNlAn7;6?VOH>fpS6B|_kb{jnlQ8je460OVT1Pc z3B)UILU@-p#uZ4?c@~4*#eF9{#Cv=a7$QK(1Nf1v0vJsvLMLcTC&Hd>_iW?@S~?N+ z>>Hkq(O!fZK$KVe5y5?icn1kGyG!W%gdQZ3H0?$B(IEUV??f1MglkAmN$!=FKhcJE zu5+j8*gipgC6l-{dW^2EQ*>@;PSCxYP25V{K1O2eCGO?Jr|6;EokZ_3`ZCA3dyM2S zclEiz6H;Jc`;j(<~E59sysq)9E9K zFoj1LBkLr88)x?0_)kP7I`lUW!WtfHn9}e>!!(43_xtF|;Tj11{l5!5({}HYwwGwn zhkb$5C^QKiC%p*@oFtto3VcEid3sG#V8IiZzKN{D6|w|n6{tIe8CG4j8^5RQ(EEt} E1F!FVkpKVy diff --git a/out/test/clients/org/apache/kafka/common/config/ConfigDefTest.class b/out/test/clients/org/apache/kafka/common/config/ConfigDefTest.class deleted file mode 100644 index 270dc7517a2fdaaa0910a5c2c98b7868bea62521..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6953 zcmb_h3w&Eu9sW+5aDu1XNpTkHH`!VWMO94+1i}7#}kKjNMpU0yb z4hAt_hVlg+U)1oJ@2VsAKe_NrL7a^*YxqhKt=KI;zN+JK*?vvO*L8eD#}m>9{rRRG z{g$-#q=u(-d|StNbYz2g8qY{A-_`Lw9pBgStd1XO_@RJ*Sv(ofEf=V2oV7{7yCxO0 z1Zq0tNvkVA)MsUS%)SIijLuZQnb>4z;!<`7y}2#%tiasPRA#WnOq=~%td?!&z&5j` zKQ%OzO0r1~#0Oi}sAjD-&|_tDZ2}W>Y*(Avcz@5OX^R@FV%C7b?27eJhcBH;rMb1C zbE~<-Y{}>1iI%fDWMy)3E6WAm9cF^_HLm#$W||WX<)P36C!EFQOPASV!b}dfM01&V zad&<;r^4{Yd@>gwvNpxDaVBR~GMUPmxp*o`^R+6bt$8N7#fjdK(RU>2 zUOdK}nE6CbK({VQQ=b(Rm{(aE+Arz1($We&sF5nW@`*%I-y}J*A)d|B?Tyy?Y1W15jhb7ucGVn7|Yh8zjN zcu6`9KV}snV+1D4bd(>?MgoAoP8mC6qr$=SCV8G_E|p;teMd;|$3&2{3cE^8$^K9=BMDL`shSnWM1~UX-K43gIuRGZ4aGRcA1Szp2g^PR9l6#w$KF zIX@DKgz$I#g9(V3k)}vrQ>0(RKc$QRA`eTjw=tyQ-y!@5ha~zlk{P};%tW-EV$a5N z+e3H>FB7C8yn_E~cr}E>I6_qdnh+u35mlP-O4%oHT*;&LW_AnVUS`f_8TBq&Fn2ML zTXLi`l^hHSztBR6gB+b)nzDUdosTDCRwg8L5fJdqo5$7CW1sYxE_(-5!SyE1AxaC| zor1>j?Q0-n6)ILFm5dM;k+`fmk%3euLj2{eh*TX6iE74#xfLOxSOj^8P@`R0;xSE3 z2#Fd|8^TrCNzVpTDb8sQL|G}6WukNb|MN0NmvSk3N$=t@$47ytiacW(>|rSpXs&d* zs+ufQr2|xlx1i$vq*;N)RUKVD1uUrQ+L6pzgBC%psZ@9Kx@~CenB^+b&I;Zpm8D+R zd8ah`oL1|zau-`xQeab~OSAoua2F_NWLD*+Mp>Y53oFh>$51+z$(c!c$pzhlcYVjY z^#Y44H(uIJ5Bx+C?@xi3-7yv(2o<}hu1Hssoa?W0r;4ZDR{0)RATHC6@0Fhhv*7LQ z?pi1C&@sqC#VI>>-ut6H8#|=`j~@g7OLg*fpW(5K$lKWwCFA{TyU$(S+0Ht+rgK#^ zO1iJ^?(S?~)y4YAS}SmxJK%!9V$#MjIDz_dkBa&9o9VQbo`iGRmm9bMo3NR!z+1T4BR6|C zLW7q>*B*eUcR#8O?=XD&DEw9YtDVS?QE0=^jlc+kM)fd4jxfQPIE)&j))9|0jA2YF z&eaX0z9=*dV{%cLGK_FhINmKxbqSpZa01;qaYK{gHBK7Aw9QRMwU6G|UBigjow_=l zJc{YI&|u6M!6|lsiZRpf9B-UzccvPRvNNY?1hcxlRMl*FAI5ACMlq)jO{19GY*ZHp zjM`>nVnOgfiW$AkOtjZ)R7ZP##>D6-TAB@`u)ubr*{CZF7&DrUhJrv_^O}t*1(AuR zczi*$XQviKd45&u{C40(_TPlLn2ysi183qCp59Eva4NQ=5m#asc4IbwI%>imG~-UL ze1Kqh2=nk5f0}s`3-Bxf^BflOOTrR@<%9OEheMgh2JTTKT?0S(k~F}hsEwd(a2f?T7td7ZA1v6 z06fM-2kM#F5=9!0Zep-#qKizDCL(SV1xz2nd;)g?K{m!%7p9@xWh`=ZV_S;G&T))Q zcNwcG7*lL52xII;iJl&pz3E4{7cbZobD*o>UYN~7P^&$Pg@k@Bp)Vxz7e#yhHrk`T zS{;^So4scQ%Z$_cF@oh(SiZZSymK2b zpRMt#+48|jJ~ddonDnO0<&$BTK4XQ9W~KDAgu0Um9k4d8B-*be-ft(?_Yv(+5$Vqp z>m``H;L2nH={x0jP^OMmRkZQYg4DIOsjKr;?mD)au48*m0lBhh zaQzOn`w>vtyjs%E9h|(2 zZQ#l~i`L};!CM=Ut&Mk?5YE(A%XUA`qK+Ep((1JRCD=Der;QB`3Ebnh5@VfQ+9mgw zC^3=c9Jq|IxE#B1g)6G7T~QS%;E}&wI23q=CpUq6_ZHJQr@5(+vsQoDE75;&iC(oN zP#>t*??!W2s}C$_4Hn2z9j+!hf;KtA{&2MB8!%zhC#f8EZE6=^<7xXG_>Pta$+Z$ek~dQ9v=2}l&(#_x5%-*MUL&IbZIueDjMDEyL5VGpw+-=WvBwt#OGAM1q&+g65|Ne|xX5AgrGJoqIse=FPH Ma{Os{_%aFn4=N7)hyVZp diff --git a/out/test/clients/org/apache/kafka/common/metrics/FakeMetricsReporter.class b/out/test/clients/org/apache/kafka/common/metrics/FakeMetricsReporter.class deleted file mode 100644 index d97d6bba9527984ba6f8a3126062487a5c19be64..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1214 zcmbVL(QeZ)6g_U&hLu9Mj+Me-Fg8RP0wckmS`CRwXcB=vH;R3!CdwOXIZ`JhCB0}o zjzT%sX>7bi9;%T(vg1#B62+;Gy8_Ez6rLD=8Z&>#JylNLW$lA zJqda^75bALiVn>r{RL-te?WYvK?&C=@-ooGGR3;xfEupTjBLjZ+Qr~^^p!Zrz;X__ zo&j5SaRy}QO*AO3&~8i0nN`X(Gom#b-NO1j^7a+vE!-}Ux6nQ(zeA(D*r3SD_p*XB m<@qCTTtR*h_X#SndG6dZqF|H$acwi_8gr&G|7(LsTCWLY^MTLqtL@2a0wBoghB7&D`afUKD{Bs*(3A9N~QU-l1|GS$KmUIH2C#?^3dWF^Fs~pc+Jb@v@?xOLSP*Sd zLP^FW8H)<~@mRtW1%qPAQ#_OLT*eCtWd$noGL~e#l(8(~m4w#}$tBY=y=8`Yc6OT~ zv1;!c48s-EGB!G`y5Vf`dXo^fV(YxQ%^g#WLong(nJz=IVml3ux4FJ&Xb1f30oQc9 z)v_(EWq6LMyV}Rr(WcS19gp}7*DEumJ!*ZpVY;5fJ=3-rq%eSCHj=X%a1&S3H7wV5 z7_yN;HGd_=NI^`AL0LP}jkeg|m9WAvzS*%nvt?|Xu1Qi?EX(%%^hoK5zrlWoEYm_E zO3m&#y0LBwmqsrLE$s6{p29dX3^P$Cg@PDzQ7g~wlENFTG7R~)7aZeThkSKYc!0MI zlUET1TW#V!)m9E064q0AhxcUC z8DXj5k$262lB|vt7O-svK}w*h-sF2!k!PgEf@_Psdq^onw5Vjy`6f<}QWfW;HM!N$ zKGpXPouG6NIx7r@0lvF?TEPs9(S;Ts=4nh>T&|^TrAJ<)n_)G|(RF@SvNM;2UqVP5 zO-C6KaMGnGSqz`AmeVVga^&5r8r=#1{iyEM(|^i$)r?jM~1r z6R@BD^aTBrJ~)Qk)K3Kqa0hp3#Bh&-<9_g8^wIkV{OBUKbAos!*F}OleaA@VROuLU zHQz=5j;g2wCrEWMNRvay7!J@8^_q{Wgr*72siPFKG4F?54?|8T1N6oPw3qQTJ-5IF zWXd^Bxn?Na1TaTy&69>zhR zU>r&*f||?2P!MLX;RhXyUvc|FSY3YJ<(BPxo-Zv=M1f<6)=}R!MHsDq7u;shyi}eZ zvr+k_8tI%no$Y!x>C{dGhV{RZX*ANmrfJm~^iw|$Y;oWuBPo4(@%kn2aRWoh8^}S^ zu*9(YzmDmGMAUM*>{uu5OJPS0=Fgg+zhLcWa60;AXS>1d9GYj`nZd1%~k|QkeLJF@i$TRv#jG#mj!zjuFl^P*c%4N+vC^zINNCa{z m;VmUE(?tXrkP~kV<7qvM2}}}0!BoneMv<&ewG8G^K>h;}aJ@bN diff --git a/out/test/clients/org/apache/kafka/common/metrics/MetricsTest.class b/out/test/clients/org/apache/kafka/common/metrics/MetricsTest.class deleted file mode 100644 index 66eaf2da41be28201b2383aa5647757456cdc9d7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 10894 zcmb_i33yx8l|Dz7^j;Ljc7z>b!XnayIJRTQ&V~~w*ogy4u$>S)iwOl@q-R@+ED1@@ z0-;crQd)M}u$9uL>5bIv{cIakVCe|i2zB3jIE1n6=08SD>`hka9M2O9>P22Tl)U*G|Qrv_*W`vM%~ zX#w(au^^QgTx#-klS3w-U~rkqjG3OqzetM z4^X`<8w_4Fg%|UZ0CZYv@-l+!JdTo4D?6}V4 z^`gTDvB9|pw*{!1+YRpUuX^&$55M%>FK!e&Y&3b3zd57{O+L@xJUh4EY;vc+dFhqv z-Q+HlxA>dqs0&Q~xXBlqe38i)n{0`|JLMxXm7{FSc|9ig zn)H2>-Zkkb;@&>_h>5_v#8$ig&H4nAx44+IUoaAmZWI-}?>0E;Kl*EpDS)`iDU&ZT zIUV2(XN6)w4&Gz(UV-d0dB4Gz`kQBJ8vo0=#Y+wslP~i(H|kxJ514$pzj^b^>OFUK zgXs7PgRhuMkBKF(WD2cGCNfzok!`oF%s|@O8Mm2&>k|n(y(VsDGIoZkXziv>CT?Y# zmc!m^#Rq`s&BprisBBB7d+V%}73s6V7NNruEj@{Pra9wsDWHPbo3(mzW=Y#FYmcR@ zP}gpyu;EEGFwGb#=KfSp)FmVFoAhhkO7zxs!fc7&6~{tWf$Ox+tQFaf6O_tG5fm1K zKZz2-RIzy=A#U0l%fw)mmP8_%wX!j|ovBo5vugmxt#i~a5w$yG{i(POrdc3y4FA?M z1^Qhf(mbvmMtM)z&dG&MA;=&!qWU8uC&#AKPGpj4v0Mh_X*LoSlC0equ~SM1gRf#b zQDj~hv(r{O(icMnXd>`8QdZhdWZ^hx1?&5P6ORI<7k~tM(^ev?z&Iy`El&zyPefJ` zS@lMk%H@>RR#Y>`adO&z=#gSF?R_ablC`4}A69a?VNxB6N0X`_Fix`rdE-7XR zY-2pyYGti;R;DAlwxGb#X3`;bnuvB^$HV6~WHV(c7L!Pt*=(nh>8zcGzbB&D(7!K-ZSd7WzJ{-b&itt$ zJx7B~v&W;?D(K?sO{f~{>rE#IkWW*ib|#XJsT_w%3mK)$APVemd%vXc9%)4Na@=(Y z=0dea-1ea!q6h~0(|o-ke1>Vcwq95c%Y-9o8<7_c#}eV}-efo$Lzqjo2=BG_XTq7j zA2=9$$`@&WtoCHZiwWM&eCk!D~r_3?y zv10Kc-^4e=Dg~X!s}eYBJkx1C&{=hhlvo?1$igDgSQ@EY>n`2i-n5jsrmj9a+>^E< zQb$EtX%eD>l}g!(sGWuxV%bc%Cz%ds(Au#VkE6W`dac;=mLT8Cx4~DEm9;uegYc0*ECF0JMgXL`DJ9&>&>x8}m_7*aCdI zYTyM>Js4=i=I0hQC)7HxxxjGa!1!*Qh8&Jg%iOFpZuiMm5>Jvksv>hY~x}6+K8UenjBEq7O#XPCINXsL}h?_9hC;Z5abv6r69k|uSf*H>g+pF z8|2scC8oKZy6E>=d+cy;GMlwg4#StoC=%WiOU8BaV_G&*a;fQMYmOp6_vI>-nl^Ni zOvyaqESD`2`@fD<7>~aW(h=Hk+__5)XVVJJ4~>uMghGRHov&M|T+8ZO(rNTLSl!CB z#l&!PD|fhNSIu|Dd<{|*f`%E$9{;|VizY3nGi8qEFz%G4Z;R@tDA&BUN|!j}-hYn8 zackj_3A>`y0F*q_&D7D+#bTt^YKV70OzYgF$03hPf4KyTET!ULeeY_)B|8nlf^yqV zXV=QG0k1TE^kL&m4(?u=Gf;KLU?GQ3z&NArehDktT@Rx!~2%Jr?SlQhD3+N{`d+zeDS)Q>M_ zE)muw`%}2k7Yy&m?4C>`NWGOxeqSbT+bHec$_=8<^h$SemePVE*J~rR&TuEeN0l_6 z8m&N9Ilq-@rkl*kl-5>tG-`gKpEz3+Tf)i8iNEU0r*CM4PENxqE@YDKROOZH9o0ux zXn&<6?o_+*%?eXJ>@}g+bjfmBmeAmc@!D|CW20z-{jqpFmVwg}QIv2P72kL)n%MCp z*t9C&;(Jo5inx$894Gw-@no=leOuf5e1Qqb-A;eQ!?dY#!vyu_xD}r}k)hN)ggFOa z+GM)=EZn)O5YiLcSCJvAeQE^z$YPyyi6)tt^$43YT|2&&HRkd5GF3e%RB(h8lGA2? zGHq)M-9ARj9UsHLeuC2Jw1XbUj|xR}5H5bO~Hz2S=A76)i8Owhp6Z|@?z=h9wt9t4etwNb{7rP6ubmF57X4DXURK6!RKh& zQy}%w^Y{#r7oT(SIiE_Xik8yPX&8{8W=~(E7x2dPB98XR(Hz9l@(W1sFco))NAn{z)Lj%hp|jf?D(f7g^0r}`fm2UxuL+$HDmy|mi)dQ`y=I6iIzlH^FE~uI3Q&lv znSg#<=;Xt6%0uK^?iHJ?C5xhTDufh~tMF40Q7xTJbu@<-;0Ek=UIy1Ww1!@xSE1G_DE}H@Mf5s2i|Tss4eY%MNCkP0f~D7> zzcuLZ4ElR~5*zdu)7P=>2f;V+WcmkKsYt9;)BwxEN*-&NPJ<=FZ7}{crR?dU*`bdO z(VXf*nk&H28G|&hy+-X-hGq}Z{9&pB3HEC=bb$k{1+*>?UFbmT0d2@b7dgSmxdsrZYj+I7G|y`zxG%wV|O+-KEhoT3JSCJxwj}@oI^t)d(;6eT_mdqt+oh z8*b0T&vD@E1U^I?2I*X-WE+(1SfEJTos&8s!bY65po}&R(&nP2-g0kg;1HG8lzXoO z;2JMKbo5sev}M#eNLxZ%fhf!donX?8j?i`weAhil=a zWs{oe6l$S)w3=2T#yhE%V#toG=xjPf=O9Gav4`4_$n88EKN`=cO}vQCN9d^c_9SMeM12Xwxtly-Q|qzgP3)5ld#EygdOHB?6bNdF|+<*}UXGCke&P5Ks6 zVV-9>eVe|6-*JPyh5i|F%Ox#BU7(EaSvyb}ZZhlqZD?yye~s z1U!;p&sdmyHRb~0xZGFaQ<(ckVIsNr*Wlj*{HyTFhYufsURrd7F3ZJpqaQdUAtoqB zLIOfR5)>3}BSFO_GmBMv9Ka$nnn`Xam|FxQGfmez7@6tUuZ7OB3oVOKYy-5D!W2PS zjw0}E+KR#(r(U{(`sg46?-AOC^z27zO`vKfQ7KahkW1(t%FqXth35wlihKAp+RF`e zDX*Z*5HJV$Lb{xH(I|MIqe_ z9?~_}swlf$qEM|q;fj%V1-Zfn=}?d>bgk4SxK&Mv2 zLST?SiJ-Zvy}G6Zf+D1@9;9nVyp++k)iwFO>vTL!j8RZPKb=-uu zeKShZE!0J~B35ow%7(RsxsrnT`Z@gvw3tbo=|AbepzO%5`-{AG?>X9CK3cn{>C=OB z{jmy)I#7Mf#A-|)v!Jk^R1QMnJ7^Z&NtJXL=9Rl?CEWv6zCh7 zzoK8mhe3>AzoGwz4@;0|zg6vH4lPpZ`9}HmcX{=a;3nhlN(eSBbSpAWRn=Ras+t>@ z)b4GgcXM+_!7s%sz@ETF`BiGd++7%WS2_ZZkwgEJlZIQ!Pz)Z*OXGB*+Z}n-iH_;f z?GMxE2I=#IbWr!IJI25&K1_EyIC>hYF2Mv8x=T&L-u5!ON6yL(KKh)|qridcGP)O3 zc--eEehS9kfx_H}$UBG$-~l}K#IOoZAlxzn_+@z^ZW&#;VeG*8dNJk+8-srzEEB`% ze={cLgP4`?#k72g-T;T!;Py9_-DZG0AprD6Apo4;tI8yE^Gxy{1rv|z_3y~U{2^T? zAEO-ozd6%-E}-eSUJ!3DjW!KaQ@1gz4{b~4PyfjI^UKPJWprrN{CU3v*K;YFuQRVc z;J_br=F;}OdmeTwpRUihE+|5yAEZa6$)o$K={09}eW+vi$CgW)8>aEH`HzqUjaha#M}Eri{AKrowlG@KagIVLE^mt?;|v9;`6(Z#f1f72aaF`Whks zO|gr(*hO0GB9t7^zTZmaSign0ucL{78_oDTbT#I*>*;Oy_Z@iqyJ(@`qxaEVe~IVs z=v_>WKj2b&k58o^@jUu5FTisV{glt5pYgf$K5s*_kzDFQME%g24oyBCS0R}bQT%zk z6DLJz@qYk}arTF3$bUi}V4x54)V*}p;GotJrgFX7Mo-F@NHT*|Jmb-0uSlXb6y{G|l4fc?X08ik5l{Vr3 JR^HoO@&8p47vnYM%FReD@cHRVxMNk#XPf^ zWB12X5>|JZRxF|oYSDf@GNAPhXpyLyOqxa{soPdOnu^e(or-LUr))E3Y015M%5J1D zHv92K`WNT_cKVc}q9l#xZi*Y7nX%Gt7`oM(&{8R#HXG+^vvOP)E$7g0TSEO@l?bOh zZ7n*mT^n>TU{=UP31xfIh8<7p`{JoM@o6y(lOe~6wS;`f%F#4&j$|S)7LjK&G47az zf}5|pH<{S+iOOy6- zL1w9+;+hp>a*A#uO$J(;(Q76J?Hyfdo{Q{p)*5+-WxOaM*lDJ%sD6i|v2dKmBEu`h zAqBg!M?%XS@5apR2}M7U5SgPQaV)9ehxm~K1H%#;=BPkKqF@9UWQ;0!31br0wy4KN z-B5e`*oGQZmB(H+UQkN{boARt0uKU)i#~ozPOdLRpv**%eW}CydvTDtQM`u zJgz%(>A}HSwD8h9N+c|571VOgY1WWx_NhJT=zwmk^|NiRnr*W|K?2P6#yR3sIgR-O z@K+UF!jE|crc-cJI-cm&EeUGcAROXmsN|Ds>Die8lVerEPw-RG{ESYkF^vdmYHz%+ zPq%c#=7Cf?VXI*w@f!K7p3qY2U_y)Py(&)&4H7(RwSu4H7c4&0`77g>3SP%$2}*Zu zeO=vhmCd1bGJd7t*Z7T$-zs|3W+t*Gx?PETF`oV&WS=cpdFWxS=}ZTwZi-|%PP|fZA={ z(rlBU`oryQZK6K-Ej?;ly}aIGXPy&oujZ@gb6D%Ha+)uqa0_w^pA(Bufg;`&%ihc) zkziTyhud6*KHFpn?nypP3&S&u%hs*JvX3q}FDNrW1$<7$OQw<62HmN>$+_wosxl;D z2`#7Jvu?M|wjHodx9~{_g=Y#<|0LKuhgh9Sf3KM)4#lq3xEa~S%4t)Lt|fV($bo(8 zGRyIx5K0iA;+q9XSSInW#j>;^fvuM6aGu|)DW%#?e`Cg7r{W4R+Fme`a`#{$KA!oK zY<$jvEF*YG?$=T~^t%`mQ*n?9G@Qv)JTf*}%*Ae-nRdQ)ctDWEvFb20yCFL?+Sw<< zoss}ofL8quAQV`C6FeqAHB+OVifbco<7eh(e!H;(UyS0d~s z_P%8!kX~>^_Oca1z^(W&Tg4T?KJ4dA!T}!kIEViQ897|*8iS`RUkSk*f^P(J#~1=# zH6zH~p8ubwioiMKEw8wD7{OCs>HLlNyiFDP!E;zrQ;~CI6!SdbW(bNILSA$j1?(=M z&=Gx0IOK(JXbE!JRwAFD&1Ib3hNb*lf?9;Jnjgk{*dM?$$Kh4*p&SM1!a<&naMvBt z(M%BeXh%2hWH4p?_&$sy3`nE?669P*xr`neQR#i;UxUOhj9%^uNOblR`o;lxjXw@N zTstNJ2u#FZSd?26LTDHzr!Qb(VQC0uoZph2R|GlpW@op$yT0svaR@5+KaiO(6T0OE zF0v!2Tt;9kCo<&%vx13@paOLSq#lh}iEaGC@8-{-0m3tgH4ZQ(uCo~s^3X$xeg-r( zy!d!{I|-x+uV@TaUHL&KX#`6;ra7>H7z67tpJofOev+Z0jG_6&uA7cbHRKU*Cf5WD zuh5X6eYbZ*dCe%6O8oh$JUx^!HiLay;$QLLv_~(mKD|vV|n>7Y719zP(6%D zVI2p>1hKZTo}*!`EI(cB6Y!t%kaf>ZP-&54Y9o2wj1XE#rA?&RMv|p{oIWA$8N4=L z?l!>WocKLH|Xj)+_Q`;C(lQ+e28#+tb;V&WaP8{S7<8Y@8!ljWjoe@EL_6SNu1RYKUS?}NdKfOOQL5d!fGJzeu&)ZpJb|A(Y zd5jm~W8~)p>>njXo+3S7Wq+B;zrq~9$$IiO4ocNHBsJhpsR@UrZR~gBh|~>DI?4VF zdZY(gOdjSX6UDTTT(Gg&sYpIN#tWJv&3v?VnCnMaPGg8Wl=4bzaEmJ^?aj*3PPZ#Z ztGX*En1wWsQ+q(VA182EhK{88f3a7_kc_*pWAS)dyOkvMzK@1~BXA9yrfn{o*d)_G eN~IoxaESdP-iBrZ^btRK&OSzxSFt~e-2Va7W_36K diff --git a/out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer$1.class b/out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer$1.class deleted file mode 100644 index 3015a854885981fe02487865479d01a81e6858d3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1857 zcmbVNZBrXn6n<_UnDbo&pFRIdG5LAoV)wSKfnA2;5y7^oW-Cl zH{)o>kcQznMlc#j6k{63#kduRhTEduk!3=BSy}FiV^WqWS*FGNiH4kpdm8R5#5av{ z(s#`@D^O_5uNs?1#<7A-A%sR08iSJUC$DOlVeq6`atoHXX?Y5I&T%X+TQ+>(@)d^i zuD6mg)(x{{W!8-4H6vrXm5S>K-InXEWeQf=G6UBuT7Hl`W|BQdZO0DA6=qT=W_cs^ zo;ihR)?Km`5_#LP9#kufmRB?u%S6uPUDGJf8J;cn0T`8|DvX>ohh#s`(%@AcuBmW) z-`?hCDyI-zd>TlZBJ&e8Y-hd7dUw{6w%yEKBQSCT76Q*QDpC`-8iYaTUkjQI(W9tS zg}^Y^W{mal2%HSgL)hI1Y%m*EaI2naP1$nhw)zHKkxsKRla5(-ecM@?v4WDjq~juz zI?mx89dF@n9T$*PI2Vq^n`vs8)$u7FO4w%#9ra=LiRvhzC_$g=n8O!3zLez=9_yIL zS317Nf{rJmeIvUYvgg@DvfS+Cx2Clo*e>fjbiyG^_A9i8jAg@F$rMYh@{)!{g~1at z9%gvR=Wf*|D-rH;e1#NFuUJ7{Z&}V%&rwh79Bg_=>OUPaO9`^n*HnU!Y3e=X={)9n z%_BrDwx4x*bX0fNLU`ZEAi(D4MoBJFvo_ zXeuX2EJP@m%T>QbDO1InOaGth;yu3O+WF3lw6@Ch0hAm?LHJxi7ZkdwWob81CVKl` zLiMI!BGS8uh8JPryIdO?2sJ=LkVgKT!29@sQZ%9Dt&~x(5vrd^r_uNP6{3%Ne@0{% zu{|{AUqL%CoA%H=Q}e`sK&Ga({D@0^yU@qGer+0#_H{;gaVp)li_>XQ6X{*F?(F|r zi*4J%<@^pV=8wz{n(f1}BRe+S*cto%1(PrGS}foU6Dnen(!?q%2lfV0AKQ3FdxM(T z4Uj`GqmzfeiG&(Lw;IMZHG)AkicvL&2{n#sb&D_Y+jyw%ps1!WuX0#W(|Drp@ghvX zB(_SdLd+qyp(f!I+a~sm*msoQtNUS5uOo&*oW~`6$lXWTt;lx2GanG5uu)E37R-W;qPD8OuTsj diff --git a/out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer.class b/out/test/clients/org/apache/kafka/common/network/SelectorTest$EchoServer.class deleted file mode 100644 index 62b8b110b7fb37a080ee788e67589a96b4c0813b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2168 zcmbVOT~`}b6x|mB6Ot(qzA7zLN1;7L?_*EPQ zEX(C<4J$EprOr4M&BtV-K8d05kMParmDmfV8n?OC3Ho^u?_n=Ko@Z}|d|UDpc) zM2;F?nG{@e+oI@fe#_W1vW^vG!@HqkN)_xBY<1mV&+M7(vCe z1cpwir^9pEa5l5c%E#1<^yjOP^mpma7Ze6%qwJffZO0B~1R9g66{axj78$H9Z#&jP zWoN_kmW_=vB^`O!G|DT6XUlU{7zs)?BVEtC-e%U=HO!Kg-8LR=8(Guc*>N42@VBnF z&7{kg8MxlE;{1KV|c4fhGdt2hG9VHN_3Fo`j*%H+B_97MaYNW!JZ6T^7r%Mp_qWQc{*H(C~mj3U0+Sty{LtHc=0FR7TN}LRz5v zjdhgDhdM4|P=|pH9dBbmpi|kZdfxOr(lYc1JYOCWUG zXcUV?QFX2IUEJ6rjH%R`z^N}Q&BQp z&!HI*DMUmX?P3JyMFuHx8IxiZm#NnomvJzPET32Sl}N_;eUE5_Lh~bKjns^V!Sy_! z@sSre!yk*6XkAT@9HMPLbAWa#Vwq6!^wsbAXUch&t`l@1;S3^5IOB+l33Q69=oj<2 zC>D?ri^z&4ToDBYYk>Aq5;V?VK7oF;;VRxIIRsIKI?2682?zTcO*pt!?mV6(+F^2B68II}5^c}ZSAR&NLS(J-l2{?4+ZYme6p9|Y>}T>4g^WF17dwuI zzYx>#!Jn-75pM~`=6j@yxna8Tt=Uy%N+pf}-8|8nCc}ccF=Uh>QD3 zigi-;0OR7}F-X}z(^Y~dPJ-;N1KCVy34F+PhWRw$BV6FKgS$T5Wv)^FNl>4E0A7Um AjQ{`u diff --git a/out/test/clients/org/apache/kafka/common/network/SelectorTest.class b/out/test/clients/org/apache/kafka/common/network/SelectorTest.class deleted file mode 100644 index d7fa444adccd7d54ed953dc0f1036b6ce9353d95..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 9432 zcmbVS3w%`7ng4$?liZuh<&hyF1_T3G2#FBh3PF?v)F1>TJPar@%v{L8WG2i^c&N~7 z@zK`Wms+)3U%M^a?bd3mKuUG3?e?{6TerJiYrEUl*V=ZiwQH?)Rrdeg$IQ)3`dfZ* z@44rE=X<}-8NUDS*B%G3K<rmEIec4mj-?n#TYyjMFpPIti7D1&BPxR{D1W0b#EO}6Mxi+H+0vZ6w{j~{%qnc9saM0w@tib;$0J` z41p<9fsllvm@8pJBHSothLlHft{D0-O{w5p`RtOYA(c^_PYs9_#RV$M1v1K%(fS@^ z%2?fAMYJ4 zrYtaJq0TMR)pJcb&s(eQt(3*G#FQ3YSnAEr7;DJ+hAdOgFHpuVjAEZGH>5QxZL&hA zR~mAWAi6Tqo$a)9`|O;+YR_csTw6Mx&)a!HW$VU^E?T+%s?PQ;D+SUn7}b`|A$b4J`TkZ62IT6dN&O=VJrWrA?S z%uOWQmQ9lEnANF_-O<;x!_I9`KtXJEHW5#6isw@L?M+4s-6`s2_FqWSm6Jrv@^+!G zSCxGrVfPkNRH7jl3+mSQWmN7>seFo5mS-~ALflz)7@#tD_mSJ?R(oeQ=g3{KUR`6ZNaYh*S~F3gkQqnF*y`Q!eevdWJhQ90vye+=c5%z7 z&O$t~XHC4<5t05O7#qN>hVl`&rBimM5JWsd>9x@vI_*qybE?qYo>`a6?#kJDYRR+@ z^pd=t6if+FZcnG}UGX%@7VM%rNMh1pT-m+?y-g(`r<7OMJF@FJZEwh4lFd`F8W(RW z+uXi3h^xlIJvNclx=83PbWe=~cY!R84T7)B z<_b(Of^jY*xt@4>ZLghk?8WFv(YJb2c0vPZ+}41=MO7O)uBP_Ux#pbKfjZoPS?}&3 zFOek0=POT53#OkIfm(#y%G3$SQ6y*Y)mJ>3WSpN_61kmny8qCV~4%jvL0f zkQ#i=bP_IB5U7_9ta6U>$8A|UvS9qcpgGv<2hO5_XqFCDPhh4!Q?Pf@pO>{Tu`|~j za)}}BOveTuWa=!sc_z6!p4(-wcbR2G??};4#;SQ%_VgAG78lNNrVh~XG`sIp*-U%V zkX4jcSJ%+17>_z^OK0=;+O+Ds1S<%osP{2D()1c)DdIk_A;7t3>Y+0;hfdLDRKGq# zj}(1+7=)UsqKTZ%kWsq|I)?!oN@C_|y~J}f&sofP-jzVGqTw&y>$~W%B1Sv2eYu2v zQA$&4RWSR{)?8%aBe+Fy?*FrFShxWvV8~ia*2!g-tj8^ubjk)pHd?YtHd}JJbXl@RHdYW^9<&jGEg^L`%=sgIZJlS9%hDFb98f>-AhL{pr^dAC`bF*M4~U3vy-ze z=~3-xB+IO3Nw4g+B&WGx_UR<_qScEy3;&9LqwmbJq#%7H-dQwXDpOye$=EdX)X&fg zFe{-6+Rn{jer)Tu6MLNXyc3+3?31)Q!hW*gFUAWFgjccXKlys5y=3}e{SMpC)aUH| zL`X+EAO|hETDCDqTXKzTW)AgH*o1P*NWDkgl56FVuVF`P}hmAv|r#hh?vzesj^mU3+9GOg5uj zy7i3GtDpOyzVS4C7!i)At5SPdDKx0m->;RASaOTT;lkmz2DkO*?0uyOA}@Om~7yd)9J@^7q?=Jb3=ixQtQEj-P*Tv zr~2>5+O2tkO z+u&7Nue3P*`RI~eZFF1z6LuG-~mOPKW=p&9}NOy!BdDi`nVby~4m!_XbN z!>btVpR0ZQ1;UKeRYf&=>7AN1Q^i;r@9%P~?#B@K8BEO`_5m_Yg{xIF&c{G1p4BzD zdebSw4sb|MrkTSYTFTubAy_<&oB_qWr&k-j@q_7XoQ-R(FGprTCZZr_Z#JLOvo9Md z-+3>!4^l4LhW%ZTyzj+B>8sMb+X+P(&*g0vnL$C|3rzns_UT2&+MViR0}SmSLP)>2 zj|c|z`6CNE97mOUlt^W~gW>>x8AcG7xstz}H6$fz8`Z2`&PKfwVS>oTGs$d^yOAEC z^@Wn9^3EwpWgj4!N0TnxmvKR-4k6)P+o-d-cP{EWY1?Khuka#HbCF5YXgSfSZch!D8J@wgTZ%M<8jEt z4%n@H>a`~@0}+{qk8xzVK=?R5!7srli6HDCxSFtv;KIg}2z4Dt_)$a}A3@|8QJzFO zXN*T-a#+z-jp#{OoEb#`GdB7p#_(h8aa28uSc&#{=)K=uRAL@tWN;D|@S1aBfR773 zK0Zab2&vzO+c_%d=nlRGpDuFO#xLcriWFo07{{vU@K#+SH=2R~i+Lfugg1aK0Wwu1 zlDX3%b61hf!(>Ah))A{&-Pm*-<72grO|jbJIO7EBT#6H14kohK>BpHN+>MydRAW*< z&I;q`spp1lR35`nb4(|9Kr6pi!o)?WMmr{A6=va5EJX*FV=dNTox}ThmlKcoFd^>7 zXQ+g~A_iVFzm16oK3k8Wy1Rz}f_sZXJxe>NP=|e?lwUGmA5ta8DBzUinA(qN?sqlL zb|i^SuW&_Wf8LJf{8@ag{KO$Y;6QD;a7wJp$NFC0tG= zbkR4qU@Ep^Hm*P$uH@`Cq;VDYVml5Y9#F>NqB5@WlrfU@_Z6i-KE~C)fU{2?*4YOU{+j?{4L4w_5{}iJ#N4jfJdPueVE%C| zs2U6Y?Z=`rlyu=Hl|#%~hXaJCo{sS6Na*uL!r!GrBP4to?Q#<5c7>1Qyfx<>!{TMi zNt46Kl1T$u(3O4^L+`F3p~w9#rD&T1c$ELF^(O}ykdpZRk&bLToTqe-#R zWu$1k4$pQLqx2~Lmcdhn3-JYJ<**ZdqY!!*O^iMRe>`+D6{zzb7_^l ze!eSPNy#1IMUk4w#0OmK*F;WW>8ya_k3@u7V`%gJ>4-sWPt}x9JhS@&`J!&9aKn9t7w*Bxb``!- zW1TGq#v%|FT#@&!DEzsFDTB|MJ@ z@hZNIckqx{cvvR#vaSVR4LsTAGfJZqBp*A!P$G{#1E94EFpm#>FMqa_!I4Z+B z`3+7It{h*drnHQ@P>&U%9wU^N(JqWWMTW}^$|VK`1KvXYDeBbprwly7zv8Scd>e}l zeB&M5Z{SG--<^=B-93!%`6W%(cdq<~#yM?01}zc9t(C$M7; zE9W_bE2i5?Jw@;zP&YrMmVd+o^t7Y%dKXDCd5!my*JxJRZ&Rxyv+|uHD;vp*b_g{^ zRuZg;iNS2d2nIYuffbW{4+-E!s_!ML?`1X$zh;y5TlUDWu;=_8 z-Tzf&@OxInKTv(IGc^Cm@OuN_!JinYZwA!;f~Rf`nlgkJpVcy=ga4DSbX-G6QLmcs zdlfl0tC8);-mueodW9$FN>VKA>>f)l;Pzeb5Y@X(6+ksZak7LlPa*-|n_u+3IrwL% zOZFY3bfnO>IHW&f@Tn)JAJd!lfU;f5J3&|S0>A&$rQq1}02YmoSzaZa!hu%^BXvA2 zUDot@L>ucOtcVXCJ@xaWr}RI&xZVF`80^dqj1iMfOcXPu604YaJ7pBM@a^{^t9X9V z)Tq(6-IGj5ZvU^NA3A}p))*5>PdaIf-k!n%Ie)z_rJ5XfWzY0XfDLQIBy6E@kE4@d3W|%p79veLu|q E10u;kt^fc4 diff --git a/out/test/clients/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.class b/out/test/clients/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.class deleted file mode 100644 index b11f8d4e9f0307a3b573073406452709a35048b1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5359 zcmbtY33wD|8Ga|b$xN03!g8hpZYk8wxjRLwN>g}4{E(iEJ<2gq~g_j*Lv5pTJNh~t>v1Rm9}XLMi4Zp!c&V%}GjOA z;MF=_6F?BJHE@-I*9B05*Oy}x-k{@+0nEdj0(diSmfc$dcq?uR;B9z&0Pn!925t-B zop_gyBO-9S@Vq;~-FtjzcV<_Xo;`c^8Ry<7RvLJ(f%h4Be*hoA2MydI>xT?{*uY1` z+($+5s66hJ$HxrZ6~M>w2_2s_@F{WU(+2LApN`4-P=LTcBaeFw92Z@~ay%ltQ8~WX zzZY?XcQT?%Ktx-dq~os9;0ssn)Zky2>ajIUZ+DV*N4CG)PIp?} z37!PoQ!y*C(@Hx+=MMdDpOewhRxIG{Is8$ZY7)+2%XL!8PCMfgNNL7)vjZ9`cVv^U z({Jx|G7c@ZCX*>w@zD@LFf#kHL>y_ezF55*9XruOp{Y^VitXQO z4X9}9c#vhlPGP~awlqu;-0SVQl}#|S<>^#5+2f`ihA0{?)>J3!r1?U?RTmyH^-@$C z>L#U|b(8Ae$1>{9#^ZKc!)#A=(n&?y4!L%lI;OM!v~Be;%PiU0ewNJt?E;InENy30 za01a(HXXA!I#L1)FIr{wk~tGw(LpyS)ngiJi{w|yrMphjU7_Q1CQf6YnEX6_=h?C* z6JG!WCPc$>6JN{|O^qhLB!o1xlk7F|W%W}xqnI)AkRq&f+B(EiD-ukruy2ct{fB8X zE`=w=S4?cimPrIbO$(9ys*bOj_&UB};+y!Ejz@&}wvI<-{f>_B%KAMv#4;VR!+z0tl@HqjH}@CaE{JQ&K1pE;paJe1@} z2pzN%S-W1xZ%q6azthlEWMt2q4Qzv+R7?W&dkyn*wtHhm(HLT{S zydO)cYgv=GHZ1qnrbaexv~$Pijw^Z4*44Qos$uEm2At?#9-d?%JfUKJ$lVr1Xj&l< zGNd)=st>mDEjBaUHva7(KXZEf_;S{8O?X@(7m3tdj6%8nV2bo^b!PZIB(yVZF zJPLJ67&8ito0G2HD=(Zm;mzY&bLV6p=VW^-*(H?NB zHR_3f8SxjJk9>Fe=>BBh(4O4*(2m`U4iw!5o|19(^*>jW!CH2DR1hmT-J-obns5cTb_05?8 z^d5yMZ4Ww0-V0}jbIvQ1`AcKf#5g95lh?<*6C@()+QV3tS($bx!=x|aWG#%2yu;p+ zk`T6LeyncdfAFKPMD9mw$D0aQoi`L-8~NNuD#DeQ7L-|f2&N)+MNS>UG({SUoIZpKMOJv+Gfp7br4-EcjyNKEFgs^s1apLq%W=#l z&Uszc_h9Y_DuaIiC@vYsbAmcI^JP=c%>vm>35;T)Y^DaEE6a3QLb97d6PI@RgfU9` zI2KXeqTnp=IgHDKv&#w`UH(#G>?$qG@l#}1A6}QmI5iI83nDOWlI_eRD z;1WXe_U0^HPJcaOS}>g7ROL6U4N*|x8%0fS8?0qOMo}kZAtCz3T<S% z8;)WfKRwnLg2xbhWtMtEEbYv9!JBu?9y23ap)=w}LOI zs8$uULVBxR8jH12xiz>_o!16u4`bCRnpr0eg#^x_HDDLf??w%}n7BPm*xGr~C>A#yYB`OySRno6TIpa7T*YPi478anJy%}R(dKA7!&qfX(t---QnSb2v zXC_K0{X8VlPc(~}rzFq(JmqI$4(DzBMY2>|T~j-P^&PbjV#5(s)}F!!{u{-{t#xDD zj^LHGb&Q|H&a0P8n03%+opz_uawV-UrnMSc%GU>VkKwBw!&jTbr?X)W5S5P}0tU1* NSgc9`Y0@Rg{1-!z#9{yd diff --git a/out/test/clients/org/apache/kafka/common/record/MemoryRecordsTest.class b/out/test/clients/org/apache/kafka/common/record/MemoryRecordsTest.class deleted file mode 100644 index 18a5de92b1ded855e515ff741b84793eadeed231..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3851 zcmb7H=~o-q75_bHq!D7`U>jkGx42#^5TMvjNDUYVut@=7yM{n9Y3e}?Fd$}>nUNck zrYmmJCTW{4Y29>-)5LDNC2fKsuH7~L(0`*}@~u7nl;e}e{k@R{0^3K&=V;#Cci-Lb zefN(3`O&-A0Njs%L=nfC3N%2EVjSrxYB8b0h+r~`DNILTMiIhH1hWxjBAAQ7io!;= z5|6`?iz`_?NtBD?EIucpCnRoOE>FtkDHWfWq)%7k3;3d3z9i94NAYERMa5U6*ntJP zIVYj7N$43V=2;1SUBx$|*o!k!Jcs91ydWigQ^mIgD%18{)-hbyv@L=AQnoXd)U$ee z+DOjole2n~=X17|bd0p^OeDKksfXsXMw>vm-Ly=vL*QWiPMPb61(a@k!VtJSWm?AZ z+}ya~4C&(;Vm77hw4NE(9aHWLNX45rU4aATGz}PYwljaqx7i&sT#tro9^bv5;pm?2 z2n6F&y&&}Qa? zHmx|>LD7dwBuHLLSymt+HdU|$1&-&J|JX!N9zOCX#fx1&U%QNSUVbe8P51p1} zd%oX{T9t3%OpbM!EF@c*&^=wCHeO$1Pq&@P7-<@6Ym4Ij%R zGP*UDJTX2)#n5Ugn%guq;Q=<~9!RQyH5Dg2dX>z|g?mxjOL?`*y6olG znPnXF<4VronxZM|N1-~cyT^?u7y^#y(`2>$O?8oeZ8)k{i?fq0)FE3`_EN_ia&l5} z{IqmGY)?+QGOU!@$4bkS9D6*cOSeMtzCNFdlapUL9PCtfyMW|rl7-l^Srj5sYS+!9 zLyE67ESE!LSkIW$QJJ@)hZfMz<}&KmeR#>wt@=B@&U9a&pz7EyOqQYExf;w{-gNBZ zM{b-Sy#fI_2M7fTH=&tplDi(_1AIptE<-dV-a+6k?gF@v>ju&Sd!X<`v;zCEpSzko zH4fk)cjWV>auD#d`T?F%X5e+G#6z6isIc7z8{S4>2})|sZXkRWq0!(KE}O!OP&Y-m zT;`x2ReW4j;wp={kdMz(yWw4JBgAxw_CI zHZ~T3O-tA;5!HMzBDSE2i#Og&ETV?bmXSsRe721EB10GOQs^pbM}rC4(>O?7%cv7* z4b_E~agV@xWF@p!P+4tjxQ-rdYgL0SkyxZ?PFJ1klP+PqgnZH+trfAzxw?u)>|DaV zO|gh?uiWoi!pHpgB6crCXxj1$`<{z8p7*Vtk(p8a%$&t`!VSz&0^69?c4qV>bNVRZ zGt7ZSs;8K*1?K5x?q6YcUSodVKnn-&gPfg*IR0AkKHBhaa{mv9g^CWb8ArqpJS3XY zDO%7j+R-g~a8&f-kT}6wkrj|dC1u}BNe}YhN*W#b1f?nD-A##Ytd#$v&&Ne5dxX*t zvmV~({O`aKggLLz;~~i*E2L0I8$=g6c`C3?+>b6o0s8SDbmJ&%@EEnqy30%Jp|t$j z!#w*W&sq?*)Oe+I8AKK;AHQ9_PCP zCm6w?e+)4MU z5x9YkP*n7*7!vgC1MI{jJPBZU#fj702QlVjAN8@7l=K-skEQWhjPfAL%?Q4WPw%Vv EKTcE$jQ{`u diff --git a/out/test/clients/org/apache/kafka/common/record/RecordTest.class b/out/test/clients/org/apache/kafka/common/record/RecordTest.class deleted file mode 100644 index 08ca276dbf8a11754ea9a61233e903568f746525..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4539 zcmbVPiCq2GCLU;yPu(jRc|x%VvhoO8c(?&JDP zPhA190sjheNCYu}WDzzYCBvW$4`@gS;YUUmwout1j^Jnz$1o(z<1)NSG~X=LTZH?d zP;ZstZ8E%FhIhztLgY>gbxMYZMEJB&?+nuX!$Q4FL>>{wqq2M~fX8L=ZVm4d&Uqw!$E|Q(9jgXM}qh$J{G`cVST(9pTH-J@hN;-hR+007sO|AM#JZX z^LY(lP~h!1hZHPoIcyv;B1tP1X&kc6#%xcInQl18R<8A`i%(r1F2*rZN^jS?#Rx9^nsy4lZN~*M?vLmy89gyqO#RWTJ|;t zn?s$AosF~UO+T}$O@X>I)lKi`wOC2BH9OE{rVkih3GS4(q~b=R%}86a&Tp!ApOsNi zJ=;mHhX>4zO)ov2jSA+(Y$M*k#~5@hF|QiFs9{(^(5BQb%S?3Bus0+=mG93cZEL`6 zvoaRX-jPhEY{O<=DN*7aeK^awBYFF}SlHQT#``nb0Y(&04f4Jei+@qVlgMDEWOHwi zf_bhJzv}J_lxwY>VaOuWaf~5H0!JST)J6$ZqX)Hxfp&H?p_uFwbe&)}Y}D4PVmm zWd%hst2b%b*);FBb_Ta*GV)bk)jlI_44Af=rlWc^nKaWo6GkRua#=I!UeZ}%Wg-*e z)^W3Y<-{t($b+#|HXS#2Spv0r1-Pu1=oKuz$y$V%Xt(S>>aMtnIqj$h9ox`Mbm{mS zzOG|4w&WEn(fM2cSC-|v?jk81(O%hsGw;NQ`vA8)X;M4Il z{9KOvg^pk1R}$>=8h)+gH~6iB1qBT5NT-b<0ha=3_??d5OAvp+9|anPSkyW?@i*3~V`35#b^IOwP*63Cy9FpHSUrpEw0<`3_tX}xU~VBAH;~f8MwDPS z>{MF8+RoVue%f&?tbZy9i;r$Qnho~s2uIV#Ap0A;b-pJ|^_Ae97oYktC|JirePxND znUaEAJ6ND;UY{zOMPMPvzbD>}74U@EZ6G0Web^s*-1@?*Doyqfs%gS(!E|C3Y`2l?T%D@T*LK zJfs=NY42c`p_GTBGv08qtGvrdWT;(iCNu2EdBds{I=?k0-%ZR;Tgl!AIU`^s5`2|y z>MaRHXM#{oUVB=iXAUyz*<5F4zs%fnT3sH-eSdA(~=*=25k`s2%!A`u6E1pPB6TJ|sj+Dot#8)qN zk#2=LhB;6*+@&Gx{N4?Pt1${p-Ss#!QNGMGGkMZ(H1m}0wEY5ok&{n{&oiL&2#la8 zhaf5MRcPukRBsN&E<-1k=eZkmCM3&5VFYtexr}+IawsWs>G?vFRNZB%tD|0p=%@vq z1tM2SVi-ZXwE=UnkqO)6B)yMQkP+AA!VbNcAbeg=4vQz$mgG>b zZC&PCTY3g-!$OW?*`Asa+_tQwX#}roCrP>6IV~@($YBK&-Fi|{hR3d-qmRdNf`Pp{ zkEceI(90c|&p;~~U>$MO$e+83j{Ud`_mk>FlveaWR?o68O0rEDe7^G ztBgiLGXZ|h%||=R@Os>ij)?$ia-y&*<=c`61{ic3vEw1#K7RcyeXkR|fHN?DM?|M! z&^;^28ARJWrnrEylDz%NuqvV#9kjQd2z!Da3U;5H?|t#~p7GEQ)GqZlR@poi(~CCd761{N-HewOQNIP4^B15Gf ziYQf2%r$YZ8ADFb^U~5YILJNF)(F3A-@q$*BUe(+?#_Lrq@3@8|0Qg35O=?ZL;uaw zhsIbZMHAZ&amX1Bh0q>j5B5#t0{@2R2Jg0uW<7{cfV7uuDG}MVoZVs`ewcrCar84* NNFgxxufV*c=zr-jb4>sM diff --git a/out/test/clients/org/apache/kafka/common/requests/RequestResponseTest.class b/out/test/clients/org/apache/kafka/common/requests/RequestResponseTest.class deleted file mode 100644 index cbee421f473244561e892c57afa80b52f0ce8490..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 11233 zcmcgy33yZ2l|ILk{lr6nk&TTLASMJbF-ELmhuExkj9Fwz6++3g{A>$k$w;yZBqV7H zl+w^7-PP^L=UlQ=0#h<_FUJw>1AF%@1|@5&d{F z{e=E^GX0c(Ce6=vs$kM-KC_G}WaI%r_WHO|<0_3coo-{F#*=i`d9ucSofgp>KAwV} zr^>@=KCYG@({&DTjgM=E(+v4hr*XZHXX@B>gT^=Mw3=_$c$SZ6>$H~VXnc#tjXs{M z(?-5krwu$$W`NxY1g>u6S3RBCOAS8BXUr!e24@oFEh@$p)n4)QvU*XwkcH)y<3 z<4r!^>|=;BxK-meA8*mAkGE>PO{XX?(s;X0M|p?FJ2h_CDb5`_4f0(Ycj|OZIPcOa z#a$Zj)_9LjBfM9qVczHCkbvsec)v~$@BxkQ*69?#P2;dm5A!`bALM&AKBUv5!sP8b zo#w+D->1`AHZ<72|*bUM$y8k_d+7efD!ai7NhIz1r-158yLqVZ^Y6O*TD(E%p! zmPD_~G_5llH@k)hd(33F(G$Z=pfeFMVh4<5RDRo&-t<5;#dJq!BH14_hK$I789ZwA z9W{cH#Nc2e9!#3YhRswu72IbH_L-@nL_B47V{$#y^fdaA37)a&Nh2Li!~v*hxUbJl zGR^2bVvHEUcr+1gJCQcq)HD|Pd(D)+M96Nk$YMskAL9CAW+WZlWu^xby+yRWOzXx6 z+S-#!CyhwD2rzhguo&bRO2HE)ou4j%puWeO5>wU4f}@U>FA(&AexH8ey#C%BCQk-^HnPV zM~2}CLEA7jk)&y)OY6z;XyMA+Znw^+OTCa zt-KDY(94{1Y%S^VXp3OGnT`zPgiY#`pIaecfSE?K2ZMlU4V!il)Z7!S2*`PmZN1Fxq_>D$q$t?k| z&eAjto=msFuX5V1AM?P$ZZGLbMB_V>iQ%E*C9Vg~5}TQX-|oJ?l$jT!C)96C4G-Kf zHH?v5LKQQu9lJ>#{15)W04r;x85XiJ;b&VSKkwl6#??_Vr>ysO;W?bsmo$fB(TITn zx#i$^#GYshiAIaWhEr)tIdjV{*kX}DK$V$HCX!(9&a4R74VFUH$q-5>qw)Uokv5b} zq!W=uEVxbaQc=}lX~c4-EjZAkE|-!6E}FJn>eLOhTnX7Foa&}hKpiu6TrW74{4C`S zB$8>R65D+(Sni~R6UGgqC==ai1r>-Zp&6_qKH zmFc)2sBFpFS(i>vV}}RXI*pNC)@FkH!l(KzPkL5}Vqu-R#AIRvBWb`L8`coPGxs3jjUeh??=OI3Z zO3KekM)`Ea&uJd^^9UdJ^Zk6n&nNi-KR?K){QM9Nh7$9fxk%*Hepri?*ziJ#xY=QV!J&lmW;$jc7R zfH9J%Y8)_90}&)~kUuWlzQ9i~Eh%TRWKpf})OQ50;U}4b6If`4h_zCdy>BAUR8_I{?d<6;=g%n93B<<`nM2m22sQt4$P=L6Dw5N|iif~tJn2~FZeVp>p+ zNnuF{c7k28`w-Lma;tJWcjw}Kh)ZG+Hr z)AF~%&DxmY_{2nTxT;n4K3ov^R&=9`Ny}S-K$ADnwAp_`L3?Q;&|Cr>cdb$mq*#2C z>CSO?=PpSV$U?JC1jQH@qM(aRt#3(qDW_qYnoozFiFm)CKgFLG_Mc%|aUJZLmX5nB zU-Bw_h_siHE6c633sR=}Fsi#SbWtNX@Y-Z=r8yYa$(O8&TO$Ge-6PT>8` zs(sX3M6P_B2+E=Y5(%^4OsjV@XTwbUn%Z4&c-;fZ#BsTsT>lGQQdt$5+E!~aX`H}o zWtEYV%!a^jI_MH;6>N}}-?xy-8*enX|2kY=<)1rx0gZ;@s>{3|TygPw502Kj`cdo9 zE0x(I03j7UY3}Y*#O)AZ#A3LPO-rrb)NV^CEHFjH@g#tBCxplXQA8pazEw%{cr=c? z(HcjxB62QUeNeko#ySKFnwo@==^)r@Mq)_u9_wJQ!ATb%XsXuWy6HD|50VJo6B z7q>I>>Avt85J%%9iKE#1TDKTmRc{|jivVz(h`<;UQhN6Q* zF|!MWG|KA&t=J4;p>)C;!*eBkDGcz3|qiP}#c~b@L<3>q@Xy z@+zA;R6ivj9I8I{3=Km+Yuw9hGrI76>{Qd#;f@_C)S#VOcuL;PE@q`>lO=an z&Unhv#<^4+=dtW7=(ft1L`*&+;ga9hpxUf!R#wz@Zfo7TZQtRp-Q9;<545&-wzhS` zX;w6qjS1(z(zR*%s36Shnrw%0>t7xe#6fw-D7xZPJYCliA-TR&W^fxv&B8kKODNoh zF=wv&QXrKx!oH;dC*_6evuqu)+#jL+AbtnYGy2NgGsKrLsGtv_tx}V7 z>BAWNEdu%oeH6c#K8D2}S?rld8vYaisCxD_st9|)!xIX7(Ro8*;aC|8S4ro|QdKxZ z+9>%#VGT2rLSY{|Jru4M0=4g(OojH4TrZ zqOV3jP5MHTYD`VXqv_}a=mXLhlGI?T29Ijd*P@>xeIZF5re@$#J^DKIGo>#iX~0xH z9^Hg~Ci|TsfN%Md`WG-N;fm@T6~peF?9Y62D2GAXcb*8qcrCV-LlTx(%>DX zMrr2WjmcXB^SsySwy-BKKQu}U)>SrCGA0&=Jq=z=G-2(!sw^#n4Q>xKkJ4gUGfHR| z2bQ22rC>`#)fHM=hGv;fGfKaQ8UK#$wv!l8a%CznrJru z_8>^N(;axWi5AmdYN0SKAp^5%T1pSnGCEDm;bbf51(uxzDaB8 z+q90pM+gx7y@3yZcu-B7cph!$X1bGCP%CezHtxWmL3Ghp9-wV}41aELoObXj+R0B+ zJHJF7{8hS(zeAn;ee^$|UHn7pQjWcesz865o}#D83+i5ahMuKLtl0xR&*A9+U7+VN z@`#gL0RTxa(+l(>0^t)@aM&|1Vdj&VnMxHu<^396)aX+`2IZ%fj+j1!byi@#g1$oT zV6j4CaIGC~)3UTOT&;TptE%-3-SN62%SYp0S2Rwuqp?D6acwp$0aHR*&(dnJ$dOE|FAlc8-{+8(Z@9f9@?bzG&pJao79 zurvQ~m+N7cc84pfdjor#GqhK5c$D7;;J*Zq^idDIH$rXj0!eyxmRuW?k9N}M=?g$* z+Z^;Va|JUMa_Md3J%f3{ySfnXJ_}MPe3`mOX@4ypsMZn28G4%qJWJs)VBM34HC0@W z2EZ9~!Df1|eLgMEQlAYCd#t<$KS6$lv-Er6EI5_ak52Ui(lRtqEa{|6 z((;@ne7LYBowA)VSkkyrI3vuOp(Bn>kA^E?)R>5|e9bcIL(sy*@U=&jILpt1wm2U$ z=dTtSRc5{n2o>`A#D;Q#@K;;lySXHkK}QeQC?578My^moG@PL!2ir06w`4K2(|~pc z3HYoF+U6Xz)noikW`12k!$%z(8j>nTM73E;h28eeP`X&-=Up1F&1oFphHPFxp()@- zG$@J0i#+1_F7ikxxNh9F*wY-6L01a~SsE5u8^ai5qZp2e^+&>ih_`uhONNeH`@er3 z+(2a#IU)lmF4IXzBM*cdybYBZdN8k)1vCl5-;0d?IMvV-R8LRREYwkrsGsga=I%xY zF|Y;67qV>twKp-94%2VZSFnv5x`)1s)2sq3_tI};38UNk$JkQfqoD6uFT~X zYv%VcQz75SY(>0zUOA=m;HnHg-2|!QXyaKY)B^*L?R?zcWPFt&PsDv=p7E8N5$RVSq$TOz<2?=c+mx;Ip=Qk z#<-iz{IP<8?}avuornZ+w>eolE$OW|?v>l0DR#VzN}^TR&dE6*K0w-B9|baruazY> zm!-3ozO7wm=v~e(&&e*|UA)Ve!Rr-NK(D%XxiYuQ#bb6UGk>ObiEp7co;OprI!YV} z=fmFVHRk~1J;g9C0mfyTgU_`2l9({xULcze-SR2M8b_PVQ9b@w>J!Yq! zTDp*-#~nmZh*F*`Mw0=WD<}`Iy3jZ|*N2j^_+*orZz?q3f>|o)uPK9m9q??Y-870Y g3Zb-=K1xSbKS;+^e+tP~=FieO)jx(n{wr$uKNlK;Gynhq diff --git a/out/test/clients/org/apache/kafka/common/serialization/SerializationTest$SerDeser.class b/out/test/clients/org/apache/kafka/common/serialization/SerializationTest$SerDeser.class deleted file mode 100644 index 17b0fc6fe02d0d879259ef980ecdd1187add7da3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1255 zcmbtUO>fgc6r4@c1{>3~1;Tepp@%deETKvjB#ID4AfyNmv3lRQOS-LNSGI!$7k(3m z3W)X$x3=Jq%?!j+Mav(gYpPYj?@w z9I3pAQLUQUicNh0wt^) zwcsB#i7dtxK8+o&1V3txl{Hp;<6P!kYyE&|Jvm3I_4FL&?@W~m1vW_xY;k-Di*44h zjSXfR8I1ck8Ij)+pIO6ufzTe Dt_)}v diff --git a/out/test/clients/org/apache/kafka/common/serialization/SerializationTest.class b/out/test/clients/org/apache/kafka/common/serialization/SerializationTest.class deleted file mode 100644 index 37af49e55455723a3e6cd9357e9c7e94b25d79ff..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4681 zcmcIo`*R!B89mpMd}Vp_l4B==QUs*IZzX9kiEIpUz=;#v0qi(9Zc8@OT3%aTiPb8> zX(^P_7kyCZ1JcqLw4^-RLQ4s*r%b0mb@~sqe@dq_ZF=tRN?OJCjLb}D?7e&Me)rt( zJKt+pfB)~FJ_2wUe~jTE%qSM4Sdz%c5LUD0R#+%RQB>m-t*}v27e}HT!vS1sK|e05 z%aal-G3-=(uBh?z>hgtFJcTc|;%Ph+!)&EX;47j3Jpax=CB7QN zZ7?Oi7Q^id;OkL*L*n@u?ozXFw&Gj(w#0X0xErQId?AYOO1v0DkD5P?7bISaQHSqI zd|%=R0?{eMe$XfxwtzNfS%!T)ua`}maCo-6V3LPpQe=(efm{$ki!Fo|(=ykO^2dQs0T7^x+Feo0Sd3d_p{E5#E{J#Svo z9g{OEQJYO0C1+4zx5IJDu}v#id91XIa;LBG$j3$BM?e>Y$Ha)|o~^(SZ4uG1iJ)Z_ ze^rdb4Pv+o_`8S_HLOe_OJfzIi~1!!mDjCY$|Hc!LnVg_%H*_bg)Cgb~Za+31YSc6EL5nRKPcue9I ziK_y;az-s%-JJq6odHYyNbYLz@QN8O*NWU~{LupG2EXEUSE-tM#|zfHnInNa0$!(6 zDc_b2q-a&{o_e_;UB0pb_0L>fG&0Vh;;&`O%vriqwpsla|1TzPpo%&;HdQFw8DrE` zb+=<3L;b33XlzjTP{-w!BrAqnPg+z=PJ3me%`Gi%r8s8W`id$SEbeEgM~^V#HT(!a zX6fg&@8E!jSMd`jfrg*rXBvKvU$E5NT$(g|4v%XX#{C)|z%hY6H%p9^&3sn*>g?1) zp`6bq87WC;!AKTtGiO?Q-d8lK&pU>ltT@zld{Vcv$(r-LwaccnkgSxyq=whws-o`ruC;XF3nJt7a?>cM0!KbBMHNomm^!JK7MOC3f+b_6zv4>6?ytB@ z;&&R}#d`t+jhXFbVvR2izsDa~mKA<2Fajsgl&v7w^Z7u7TQ_ZvD*fJXI2tNuRf}9* zf_WuJV0$fRy>yM&7RHz|x>H~n9oU-cD)J6Bwxyn;keZ1Y8|w{mqoG~paPv6psz3gU2`J>t%11Pc;noNZ+g8cI~CvF>%uX`1v(2H;A@{*n}8)xR(8al^<=U4i*z_wc8xLF!1mZ`&$;GsV68yvBI zi411krt7xCHXp_<1gh_zSKke0=N212->Dni5sb^ht{L3H&#Z&|LIXd@gldDl4siBq zj)G&P`#QvXylKL{yv8^JdntSij_^+Ngm4sT&IASt(4+vw+mLAC+FS9a@px!r4dL1A zh)njyn^z&5aIP;N^>0?uB9IOz!mEf0T*Vc2*DCNAXz56IB61zu(#?tH_pv=K)vzNS zUBgbI?TW`gKwBch%LmxqlZf^vWN*(Z+Cx}HM+hHcPepKVKu~ysqE7JIjc1jWVb{u0 z?Be}?g!oC=j81Gv7xthVN%XR5_hFP@gA+($n$#6>2v;~hMe?rF`qw$WK?dF=l49sl zL@9p(L%5Gv6PQ(PpWICI341@E4r2i$IF5*7%V#o#Yj}`rOl5ZIhj`aSEnX$k32Jqc zlHJtR!yT*Qo~q)WCaxzxPWcJS--@OiG$kT2DRIXAdsyNT;NN(Ls(>H@Qyc|Vt2lAO zcjD#9pOKgi#c%l#w@%H5~r0Yi`@4Z4d08$eXf&S zQGN;$ebV()*k!$ge(;{nR-c`xCltd;#qeQ7U54jehCd_mS&2CzfPeAlgdojL>K)L} z6%}{QM3`Q~U8Hf}L~s04jN|t#}Oh2Tr5bcuIYWqK&ntC&xODb}aN z`e*x8>ZIN!Zpe-5{cb)^()Kfm<6&m-Be;_;@1es7$jbfXWsE$is8w81AD%$j^xlV-)M*F86?5SZ)bAoOJ=EQFRXL(eZ+QOWMS z?b_k0z_aW{5}bit$?fob8{S(UpucFl)>gA#w)~PTJM;_`y^3^pq;D&`?To7f2O!*R}i&M+Sk#==ufNCjZVz;e;bFe45}>Ys>n` zyC2NvTH74yR?U~?J*%Sbzjr&7mD&s0Xlv7z&GFU+p65%C|J}dZA@d1hdfRLI6>HO0 zcQSTfU(Bk`)RDm@9RqlvqYwQ$(&$NGPRB#c>sY`e9gCPqU`b%D^VU0cI|8$vWf0J^ zO47d5Pl&x2*!w&cqkJ1$)uPrz*}}PEC#2&%^<3_r>#VqXIhf#!PVwn$h$&&fmZcR# zFF&dUWHL&>$j8pdW%>tc?Q_;R4;s@)5XLve4DAS-F>{3Y7y2~}aU4LL<0308FxwLU zUdAvz6Rm$Sf-AI`C8{B!)i^0)xXSe~XB8X4HO^`^9&xn@uaor#Gqrd=po=zO7>A@d zMEn?C0-rIe*2XJ3U&r4N@EF}O92j4bIN^Q&@`#^;Whvp`!WYQ8YLC dX%JIwxo**DVco_gLop^AtMyAfYgZ4Mg!D>AXr`NFic@!G7~40F1YWL zc;mG^)A-RqH>P!b zY#@bc12-{~#V7bw$E<-i%&FJBfzPm@q+3e*TuB8bePLh`UuLm{f{tY!w*}0STaAj5 z6UrzEIW79W6E1qPT6L-dZS$`0MhgPz{OG2DwixU=0vAiJ@2u9!yH2}Kv!Z{AuL6aM$IKnfx;W$ z8EVZEx9WeD?@6V>dH7#p=g)z;zzsJ4FNpuzxMMw%_R)&0#6{3?hm9dHaIfY^ZrRy% zt1b&v@ckf?ksJ7|Pge}(u*O1L_1t%>+%J_Y(MhuaeJ#uyS`x{hqi)5vwtMK<(VXJk z?i^PrJ9`3GTUMoxme$#cq%A1xi*PDKHuzp)F&#w{U*T(}!9)ow0=Hhj7hkEp3btxu4R>{XW8xmxO>E#>6PwsF@g44( z*v6KQ9TVT?4kXV`l6W;#UeOT;f~Z z-@s*Pba;!BnK)qpZ*wMS$@_QunKC&d&gR2eoFcIrC#i-6U4aq2jWS!;lxxz>(#+YHGA+8;!XFoNG?b=J*2yJ=VEYs(^{cqNHMHIZp6DWpjB7H>gUn+PxeNs@vEwJ-|Os Cg!5zo diff --git a/out/test/clients/org/apache/kafka/common/utils/CrcTest.class b/out/test/clients/org/apache/kafka/common/utils/CrcTest.class deleted file mode 100644 index d385cd4a390df9d046c55a8f56995d69730e199a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1544 zcmah}+fv(B6kW%bC2&M8P#R33RYL%Civp&k;gS}UkeC{11EJGq`fzMyERZE4OJsQO zm-MC2eQ}s}CLho*=~F+T4^7vR+{t9p4z{)TK3hk7ueJ7(fB*Z}*8mprERP$odvG_8 zc`O(x<)Oh=W>J~Xda$JYy*xh0{XCYjqRgtwt*PjNGG7?@(!jcazUumJ^gtk6oH`KD zHiAP(;9Ax7ot?NYyZd+io8iYr-JeBpPW1q;ziL~oM zs}=Zm9JyZG-U#daPCHr=$Vc?gP7h_olEURx5y;ifnHI=AFB696_=Hgp>+=HRmlrNA zu&mm?kh+pU-(Do^C)@HgL75O8rAz{ar?DToE$6^(yCkvh`#~flH}Ki~deZ7RrWke? zY~Ne36+a@*GwH>|t=Hm4!wCfjtH<(8+P)juWl|_7X~l9ch{L+G=_=vOF8e!buZeyP z2n?+I=hj{ny8e-M9>ms5=|=|2CN}U;VC+&gOA96*VN;pM0;B9;CD__lyBWmZp;dFN zsOea3*>X%gLB+t9iLX#Kux(-oyC%L?<{LaV@e~hD>|vjBU)oqHL#-0pp7f9G-P*BJ ze>au#66iT{A|(sO)rwPUwtC#1yit5!u2hoqR!p4QdyuNjbGXG83Q9VS^>*6{*}{}b z@aRP>J=VNgOw~_dq&-)*wp7i5|CkElNYCRDRtIm0P5)wFv4CkwtL7@+-tE1O=>=cZpl{K6oEL0UaY4&g(_X1W14F~k*5 zCm+tJhs7fI8O*)x&vcNj&b(gx4cd3FklX3{W$oV}H#d_V=|Ime>4W;nPv|dbGlP1m z1LH?s{QTz6LXP)y9(OQeFv9DQtNxz34tfh3%?0w^RW4mAxY-2mGOjWnA)Hb4;uA8p zFig&4{7&OGN*Ko~eHjy|ViMnD3QgWQ!%RZ)32Dr+(l9<^4HD{-)$xK=M@hODK0fB% z3~M*o!D8oO{J&FUw@5Dtq^l+_ z_;`-F%^TT2R*ef>*`8(8f%)xws?H@h`U=`?2UmGrGv1Tx?JOqEIhaX diff --git a/out/test/clients/org/apache/kafka/common/utils/MockTime.class b/out/test/clients/org/apache/kafka/common/utils/MockTime.class deleted file mode 100644 index c38384e56072df158be18831cbd880d471bb57a9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 900 zcma))NpI6o6ot>rt9BoGoRgBrQXBBc^b$9=SJJXDTTi66y+ zib*W^0sJV$c}^O&EDDQv{f2wKd+wEg{r>S2z+-IbDC2eo8t&*=!DOgnI+L%TOPmR>5bakW-;c1hR&FiK@8P*!bCURN@zng%Bm zdE+FJQEIEVx12u5vb*CrJN;MNz3$7tK-0N0pL48sUvzu3e2vgknGn=6JO9wOJ6PvG zQiec|CGnsAk($5(V2<=9m8f$(ylK diff --git a/out/test/clients/org/apache/kafka/common/utils/UtilsTest.class b/out/test/clients/org/apache/kafka/common/utils/UtilsTest.class deleted file mode 100644 index fb12f9738ed5a8da45ad06685933c07af61ebb78..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3389 zcmb7GX;&L%6n-WN$uON-m_{rvg{}q?NkX8cfoel-l^6;wloqkofedtLGJ}&zF|Jix zx4P7Q-}il|IUd^|;5mNr$N16XeP<@gfca42oSA#?dza_E+vJabe!C2y4;KaO!*L!b zc-$@E9>@Y_F~=jvL*X&cV}Zv>9%>V5(_hD4rnD$v8%{CJd{usdN5P?ee>p^34-bv0 z>da_~$Grj!6nU78FY!1Xz_Nh*Q08%e01tT2Py}}LpyL2Sl}I!3LshO^B|59bM3wL| z#fPgL1$qQ$csweg6OS?GarQXN9_IuM;|WGR$sSKJ!>6l;o&cT+;5-NakgBQX%^bX; zt|<<_eR@veupz5z%E3}$Rxu{!**r<4tUf2_r({EAyd(6PC)FZ{uB>h>MCB!U?xYeu zCC{IdqjP$ppli{RspgB(BkVb;6wN^n>r8szrkJN*4Gg(GgH4Xtrre{eQD3)fK2M(J; z*+u!Z9Mx1kI$SoD;nMuPVhpmI`)4hh!_^H{$O}a%ZpD5M&GFuWo*4bdNux%mlCfAUfM-QKhvx%$ zLBs{T$U!WWb9zBmwI0eYha1MT>5+qz`;SegqEYt|nI~gZ&{T@N3N!RNo{IM-`Z#Rw zjm6@r+-x$H?3WX%WG<14C*r$PNx4@^B?e-tfka=D!+_iK%$3+2PxbE_NQ!s~FN=5u zuQJ51QH2`73jZ&Muj367Z{jTu2;glI@8DeyTtvisc%KNmV2f4IysRxmCrm@t76SM{ z#E1BZN)nIsMiK<0t}>#T%7S8u_!ysv_!OV9$L9fjA>vEC%3)(#D|gy{I?H;gb6M6* z5nthJ5#Qij5#Ql^5kKHZ5r;7$;wQ%bj9*0jiWLqwUf~?9LOnLSsLWAsZo2|wYOjUr z?Cf-U*>&~7F1vT#B8tTv+jEpI78N!+qHGcS?=8uB>fIaNu53JmE>DhxO$iHcPv0sV zJ7cTDJDH}D=T*&|lJg~+0$MU7RF_aDV>vs`q+|0O+Cn34=ylum^Q-53w;QPw)rGWb z=yv94(O_F`z*bUppPtWC((OgltQ3~aa#m#vLJB#imKu}M1$>LvLK$*mukql{RD(R! z!rP5%8K(_HE>qt9a?uXEBQ)c>MK#~H$c@A+vwh~HY*8AUjr#Vv&cU{3v5ky$v0c^P z_|!VvXk}%!9hE4r7F5ayPxW>doG)bQ^Kw2iSXbydipKlX53D+Xe(d@8R9tHLB<}-c#hu8 zxJ&X#-U@sbG)aCb;848O>`;QV&Zg=C+Q2@(Tj>YDHUzO9o3R5S>~sa}asp;akHHzW zf*-5}Uxv>Lz9$HgMZanV8_reG5=5)C(ai=Kj{0evw5iUn1=NiV2osnHZleDo^tb@+ zX$WYn2Iv8JEkJ3>D|uQw_Eykd!L=+W8zp~h$AvoIEb*ngZjx}k#23{i*g5;~{7QpZFETWZp+{!7?4 z?WH=lPfYuotXGqty+pN9j>2ZB?M4ayL8|f?x-m}h4^cKJ=~73K#S~_66!SP{U8&QC>RgFHlim@! zNE_Kqksce$W3@b=C7xZaTN+qFG8?{J!R}FpD|Ixog2AKRk&74#SFmRVH;-CO9o<&& zWO)~*_&BYB6OeE>u7Ql}k)xSa&_$S7l5IrY4kACDO%o3KveGCqY#zFUdy4O@p*wHw zBjAuAcz>rYaSJ6~5So5pH|-5#@A$MYi1@f8CV_PzG!`?>X`kR|*DqnOnngE4(C diff --git a/out/test/clients/org/apache/kafka/test/MetricsBench.class b/out/test/clients/org/apache/kafka/test/MetricsBench.class deleted file mode 100644 index 9c9a086f8fa544689bdab740ef71c593a913453c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3354 zcmbtWX?GJ<7=CV(=1zx!N}z>^(1Ha6*rY`T+X8lTDYgYotwdC~q|vLOK}B5lF}6n z%W*`(sEP$JIQFSn;&Y5CNONRVER!}&4ok%f3~*$tupc&uqreTd_mE%or(zXy3dTe2 zJ5LuD>hXZIbU?*H9Fo-GDxw}#@DRttDjvb3DjvfT6-RN5<2XZTvtb(UR)#=y?I1(2 z%No)ds(TGn@6ToS>h^%xo2Hd;uay$%L17zmURVseqlUxK&}-Qvagi0NQ9V8;hQ~zQ z)g3q9r@OY1ayoT0HQGkHGr}+#YNNY)_la>4PYZJ-o*));q)plmIlAdsHbeAMeJLxG zvCMeJ*BDQDoTQT(^xs3eg*`%^EGna#q0(@5n|9eY23}CXuy$4}1^>xRC@XB;bV*fe z)JPA}=(xhBxuU1jBJ1e1#JU;gCtQ&l>l0ZIAdW6d7)lV1#~IXwm9ta&Erx`1VKFu_ z35SLou#w{l4Nu}JhHEZ?^dblf3N&oR35GSZvHOTg?x#7P(eNyu<9J@f3wV*d)9?~r z*6<2mWmq+nD&Jzi$dLa_E{2hT$8DklJ$PlIk=3`2#Sax?yi=2ESPyQ*#XBtl8a}8hMOATM)Ylh{kV;hF+ zO{YGq+x0$R^|qd}>>(<3|lY;pcLJzNBiY z?mc}uI%~X#sz|u9Ko?vv(rcL`3{_Mgj_%Vn(K6*tmAARv`m8m0zS8wF;=AT5K6|a7 zx8noo9k|1`#X;&DA>s5I4&~qKlGf$p+qTv}t|uYcs9pH@oL`|fAv=(dU2U{jU>Z{r z;-kXp*AI{xYQ^_Z$(Y_`@if2JE(>s<_6FSoLshR$4SOm`E; zA*u=n?V@RqY3g=YS~!kQ;Z?e}_a_EA`Ue==XLZl-O+^i$#1tVIof8b*^1?e9?dk6R zzZPe&fs0-CFm0aM&FMDWWo5F|Nr%qIp{Tbc4mz%$A?!?HS_6i>5-4k_%0qrLq6Ce4 zXAD?5%IF$DaSSWIgRXm*QcM>a0;@=4x0UlA8hl)&bNcpS5+V&n3FkAE3Qgx~XxOqd zf(*5kN!sE!L21nvl>q5o&nC8_xgr=hg+ z2%o}&p2W4A zLV=A+jdHy1m3Oc(60A|4$fNO?!cLz1i*{>6`iZ&|R*P~9v2dI~Zb}B0DwAj?P&edp zos49_i{zbHO!NwH)X{ssp5E{)5v3>jAg;y{G|;8H8h>I9<-j`Di1ln6(S5jN@Ur4w%7j7uLr=Avz}ZgJq8%Odl&5S!C%VWcXHTOWH&d=u zv!l2Lw~|Lo*kRm85@(l={Bc&mZ{vgM8jve0b i4&V%{{ueyR&Y*_!^-kigKms4osFJn)bjtVJBmV(I7NS)E diff --git a/out/test/clients/org/apache/kafka/test/Microbenchmarks$1.class b/out/test/clients/org/apache/kafka/test/Microbenchmarks$1.class deleted file mode 100644 index 5d4024f03f4a6d047c52ba85556bc10b73a9bebd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1745 zcmb7F-%}G;6#g#RWW%y51(AZZqJ^4-;%aN91+)d!77I!ZqF{gBY;Ix6W;e6Dv16a@ zKhhVU?MR$yXWE(e&40oF#0TwnH_Rvv?a+subMCq4p7Whwo2M^+{T;x0RJzfJ4>UQi z$%mR;u+Wbw3m@TQO{TjsgIN<7Ep%flk4u_=qRFS4%vqSnf+m+OT)|ZnpPBevAb(H# zV~JN+0;Q!e+H_JA1A85O4}KMZvDG(3(bZ%6(m7CB$jZ+N+#+hi}Yo`nlxXmb1LE^|Htb49Am$LaV^pyY#Cl}O3)J;yy`rL3S@EgqTMAw2ugS>M=Ut_d zY4TJU8YD;|rTfD;ub&a&r5VMPV*R$WH5c)|O=>jhpsx(F8B7a@dSqb=A`lkNzXT8Ql-&awTy6F!qAYWtD7& zRTE#>Si-W6qZqJp41+e_#E^|)9JeuoaTC{V+`x*BFLBdE*~Thv2@Jk!Z#jwRZocXH zRSorUe4j~0Vc`9!s?(!3)^J;sJ2uv_VdAchukf`d- zij=JGQME@+pns25_(QvG;|GeuzSpJ4MjR{LeQku=Y0l{06gvF;%Z}*A`ARkDvT#km z>X zAC3^je_ZF2u$?5Qr_gCg2ko-%1;jCJXm~y=Hn1Ncy-0;coqaLw%0YIXC5RPEU=%ir zYdFgz($!;`=m#@imn3EG*|C8fs@Rtveztd?3Z=Q06g>As)Vb_6zdAp?m!atiGNW>|GC|O27W8 z-ol|44!3ZGX2)CTrPeT7c;gYXc^B`&VSI=G2byP~vA<8m;F9LC0*iavhnvj5OzTw? zaSKyCU&L)xaR)VSVT_G5&k_t)vp~B+O3Z3EV+-3jO&vdUsPMkX$R*yNgz*9c!VoW6 m^sI>qiY5_}Og@MCoW;)I3=O*o?ku^1a|Cyal9f86fc)PZqOBPK diff --git a/out/test/clients/org/apache/kafka/test/Microbenchmarks$2.class b/out/test/clients/org/apache/kafka/test/Microbenchmarks$2.class deleted file mode 100644 index 356b7d5d82e1a481b47487fe4a778056763603cf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1550 zcmb7E-EI>{6#m9GS;s6*nqMGLLdcKpBy3A*DI`!5C!q;;pmm4@RR!bqU~jy;W9^Qq zxaTQ&0j{YUBefEhK;oj0&_}5%!kJxHO=?<&i#=y%=bZ0+=V#{kx4*mua1nQU(T6!L z=CwGVK|ibvKE-DlT)^kOWb%c9FEdEPO5>8&Uukh!iz^u{V9~%;1J@YRkGMNomA+t@ zE$;9~+-fPuwQT9ztsoG-vbd6-V_Qp+T9(ok++XN8D#s(od{G89i#NGl7gmEmY;f}T zJn2VRp;dkyD&cLB?caW`v>OZqaX^>*HS1PoN7#zadlntvQN(N~`vH6o&!9VIZkr)h zkX6d0uju$a4 zJCGIO+jWlzjd1b;Wtj|GKFO?(ZP>EqD-kfTb%tYQ#qCCkH=|flo4z)1ogq_}t-uy* zj;1yAFVOS4b;N(ww_O=Jeyt=_T~-Y&nOH`_#K#ygF^C}(!x%9!ig5!gCRVX#;D(8B zux?<(#7z_#Mx!~1+EP{lfi1Tjx2g&D7GkPb#!Zy4sl_c5xACok?@W}j#W2$~3-2!i zL;pWvw(5c4)dRocfnXRsU?qQy+cI^iDAZSvJ5=bA?DzZn9w)V+YoCkI4>R)At(s6# z&+ai~yW;=H#4{WTT_Ku8CA*;`?#nv97X&F=1Uwuwbfk5gYOaW+;GVt^yx9~!6=%M4 zA@((9A=i~wr&r{7D!cxF$7vn!q@G9CCrS2@HShz* zNSe`l2&eD~?HJBryo2=y$p~vTxtRYMtc@d2F+rYC;M5(TJh$c|LS@LBO(il>IA;I=2K+?b@GcW}b(|;nBpe$x6Kmu9h SNT-k|*weJikaHhtApH)Z0f=h= diff --git a/out/test/clients/org/apache/kafka/test/Microbenchmarks$3.class b/out/test/clients/org/apache/kafka/test/Microbenchmarks$3.class deleted file mode 100644 index 91368f86148443880b61126d2fc2b73a5299128c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1772 zcma)6U31$+6g_Jzk)x?EmlVGDlVFM&B7>_3V2deV1j-vxyz9!g^cu=-%1@ip^`cfQ3f)fX2Z>v~ousO@#knzCIe*v>QP7l^Mn0l8nt}S zUCoxYC<>Ge={UUNhkkld;5+B=z$=5F7zAzb&4FM_+XBXVRA&xHH~dgt>$GYr-jcO| zJL61L2HP_BwcXz|(uPmw>`EA_cs-CwqG*;5CX%WobvJ!4j%q6O8Z8+&li9OOKOc8O zN=Lsfd;!yoIw9*JFqEV+PPxnS9;v3%Yi`PR|6QvZbz)Cl@inrEf5knc%f=Kghh7jR zez?1-(neG_@s^DZY}z=E6E>c}q>U+@v@wk*ZJffKiEB3A#&sL-;D(8+jV;XCcvqin z+jtM}3!KPWB9P&(TTNreUhDWlUBeyLjgRq(7Scq`hKIVq z+<_JyOpw6X-w$pzVkPN(`cTd+WlQ?x_PBGa*8`8BUeE7{UBBin5i zl8OaqKxzlJ`Zc2CH1Yq>aUqR*Rb^8-2QM(kv~&oY)p4Az?BW0t?c1!WgGKGj^8Of1%VlAmuL@hohr}o zLzJiYkt;9kBVV30_Ff0@V)!UcTu<_L{A&JV~8LB{OumBpD}v#0qpT3T|CzJoKf2A z`&Ji6yBO=@ab7#w#W;5jql;r2V?M*UNMt~Y0y?m;hjDz)Y`-AN+i>tDT-xXH6({d& z4%#W`3l8}<5&;}d|IHW+IHu5%BE$w=w6*Igt z^fvGd`od$M%QR*NX8HsAANo%^v}a|{aIs)`&^c!>=X~F}tp0xa{4W5rxLw9BOc`<9 zh-n+cn6csEgpJQ|(#9#w8gaUeGdOGEoDCZ@C46r5c_ZeGn745O3l_exaFL;OkB1Wt zB?ZG&?FW93d#MgW&sWk<<5)=Tajl}j_ZBm?q*N%loa;I2AR@KZM{ad1A6fnH#E;O4vK z68&&$`9brzRMr^=mQ{mN-&qT!xRyp6B3|PgA?*wkT^_FUI56wp=74Sn34^mDrHGe9 zo+N^nm40$sB-&dI{8()W={F-Dw~~oj%Dxz}EnlTli5s>|i%W9E@Vj!5-{&un*%FzI1R2 zmo0qd;A^Z{xZ>a{#vRnm;i`jc_=aIm*2<8}O|PzFioBEtVZ)FtFL&vlKkVQ-ZWwXX zK^44Ti}Na;BC=JfIqmx!-kL{TkOfQ`SoEFy>7a zHDXiftQ~h5+&=twOh3c6Bov}eRNN~D(alB789DMWyYvWB*)?9PL-%(641so$+P}&t#afmFN KF76@?l>P@cMxW0B diff --git a/out/test/clients/org/apache/kafka/test/Microbenchmarks$5.class b/out/test/clients/org/apache/kafka/test/Microbenchmarks$5.class deleted file mode 100644 index 3012f2cf0b6bf1bce5075ce7866633ddfb63296d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1451 zcma)6U2hvj6g}hJcpZC_rb$C-(ozFymlBFy=(1` zE0t#?Bz^`63XdR?5|H=-`~m(85+7&Qk>aKm>cieU_s;z|d*{sj@z1Zn1Gt1whcS-x zMtDYCFyhrBCa_S%MO-T3GF~&{^%CB|o5S>7aj;lK2@3^WHTo?hmW+72h--Mq!F31A z0>k%ZI1^}D44yrdce@Jfz5i5YVMZIu5Kkt zHW`q=8bpC!61Z5Y4aAy417?9H zizq9mx7ZdKO0`Th*S7>l8d~~oBH!;;Y{b39S2qKL=h&fY=grUv{6^%5aT-Kht4g=x zrh^q1H*nL%F-*F64#!%Aasszl%@(Uqm|t28a^prfvoT{N&}#78dHvEkrj7x(aq!0a;)g=c)w zZY?`jfiusPOr~VxB`6N4QaellGvtBdjR&b#9VSIGir0ufI|vSb^niedC}c$%eBR$B zQ>`sU*JPzO*C$Xmmphx=$~SBrImpdmoUJ?=_0&Ww?wPXv=pgp)a?qONRWtgQUsc0tGe3^TAN16g1q`=Q4H@w#>mypY_Td>Wv2T;NMF z!8gPn8#B}csL8%elYp5f2N{v$H;t1JIK|Zp?IG%^>humo_1q3})%hJ*)f4s(?CK*7 z{g~yvK$&L_@C{B=y4iXPWxPa7;APD0SKMQqP2a{Z$X~OTPE~(~_!_fRIewP0?L9aG z-8~d?*u!uR8(&!B;lF?WgyL@~-FpOgd}J4+EO1(VW%9p_cX4DFW4kzN;8-wlrq~Gh zmd(DyC?4V*^#Z=(P2vyloNNT3e5LCBao|t8J}STdl3FwWnCE)@rpU73uHIY?2Mx+UWO{kG%Kh|Ni&& z-o7W^d*ERJ3&cbXQ?N(F)wo8(wYW~h^|&FQs~a`kgfFW2k_Hufyx1$}o8_=i4!6kd zFKbwZugKx6Ufe3dU-RNNx$|`u-_TGYIalCz3Aw|IJLSh+8or6U<#3M|_o}#0!$j;+ zaleMi=vOhIVo<|W>{s!ChG|G^ScwPa@Q@b|OQdh9ctk^kB>J|7S@@2MM>WjBV=BI@ z;aJC-?@5!sUx3H)123L%p}1v<)bod4JgMR-4aehYFMg!r#~K#nX%){%)Mqs`<0o<; z@8>kM;HL%n8J?HJ&kOJiydZ~Ps(4ZQ>Q^d$Eugo>VrH^6Vx&@LO2D%?91B~`0+T$T3a*B}*%kK=(ZS{o7V@7*C*&Q$vMySUO z^cq`xjeupQtiYOZC>h^k#zH+&BiWm3BKxQjjtNxOZ*JdaY&QZCBi0@0u#(|ecasz^ z!u)F^o{{V(w%H|ccJ`xX6tQC&Z)`qe6gyPX?JQk2FQh|=0Da(v*Ag(m2D;^`w zF7;{)9JF{8`7FcuSz_$ck*~3rD)lBy%QpR?-UA>i7%Zmcu(b{))dzvcK!N99QVL z5?2V+ICU{6ll!th9QEsGI>i5AMQ3ngv_K*_TW{i@4AzlFs^eccD5dEP6|XKl(flA)eptw8;lhV)+wSYYOe*#A+tbf$0(>6ultY=C3L z;ofmvhXF)I9{@tKq(;+7_p#!lHPbWtKorD0{`_QOJn zUd~paiwfb>MWv`>Q_#gx!mo=7qIy*7IITk$HF#BE`WSt4&9mVZb9B!Wi zMI%_!5*{WafHoeTJd;?|PPA;=Ov`3$5=Js*@>9T9pVgX%PH3LYPL(>n-|kSP8Y*V{ z62UIzC}88PF{~bTFWEq36I(Wu=d~-|Cj~hC!|}jLJpL_;FpVhttz*+@Fxz%QB1L$r zOD61(9wU)3V_gEXb16DT_e?J%{~FT2&$ic!JJ!l`XDlKa#(~XJwosb zhvg=@-CIBYYfR>RwL_MV4xWyQn3${qPF?RObn>0F6kC9r-Qnag(G;Ud! zN4A1tn@h)Bb@=0S760VFJP3LF;<%Py0nS1~8wMcm=0d?bjvgCWgOj+{9R!?=_58|5 z2R0Ck#CdhZt0c+g4X$~EP!<=p4Z^j^?P~P+Jk|S9QeQs^cc^W*N8Eb&t$WxY`r+Y} zhT1s*Z&`jnG!j(0&5Nrqi1(wQA9_Cus|QiEkwfvu{6!v{VWmgHj78bRCBwwk=STZd z>d=m$7mr3S%ZW~9Dn0!ucQty`sGw55V1C8~&8N|Xa-Y{vH7n0-^zPPZ#T%sXdHYe7 zWyVoCS(m!;peLxg8Vh^{e(hnh@@rg?ng8V71!VQQpTRL_tW8Z{&G2RHAAjuq;V{1=LRcWjXs~Rq-!6l{S`jnAm+(o{(dZQ z;ndx}jIm`4DKxLF(O)r$MN*05obNP_@AOv;AeaF+4S^Ti;1dXzdF+;XEQ-*|buC|1 zrZ97-q6kN$imxG)I8VcL9D_QR!c1m%15V}FCccB5f!XNcJIF;Wj!XFtat-E!I^q#5 z#G`1$lURgja2#IX1tPrPj z?!t*8f|cSDtP)pawb;w+*=@W=@5gE4acmP$p;tVMh{WqS&etIc)0KIdl}}bVJ+zVZs&WxJxy##uasfTI zncbjB*~Q-jr_t-1lt!G+l?RW~k~4_Mry}tf&V+${UN^U63qq83g6KdOcMIr)gD|m` zznb>qEp*fOh4jZO2xA+H=#}TuiwKJ8o5vC5s)U}n2{FV`N`G8}1Yu?L$~icTt8)6n zKoTjwky+H1-75p>BV;=v322AufIJl+7cTznq4=`HxC>qf;1(+QfCmpHOkOichNel>Pslp~a_dHj{8JrMl^_&p?wuF5r^gQ*2{a zs@%CKjpoi-`>A{YXD1wY@>O;yb1d-(eF;9rL%h0Y`=Q|t^2V|VVUHyOkVQ4 z_cNgo%=72%XLjK_RLRweg2}ov=<&0+)!lmd<#C|RAlh)*uoyHdwwL8{Gm~W>3+NUs z;2gy54E-H+@SQa3E?mgxN10OV8RQAXl1Yk`^1M+;Q_KuQ-ueq nLYe{|`j9Fp_ySc`a1~Z_ZlHNBoWXF;$;`7d^R&!7)tUbTl*|TH diff --git a/out/test/clients/org/apache/kafka/test/MockMetricsReporter.class b/out/test/clients/org/apache/kafka/test/MockMetricsReporter.class deleted file mode 100644 index b733f8f60b70b6303ebb2fac05c33f82690ee543..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1503 zcmb7EYflqF6g|^7r3!-b5JVA_wg`jr_|gW0jgrO!2}O*b%(4t+q1|n^TN8hlnng#T9qG;Kcw=NLP=Jo|*CFf8H3{)&zZMPc-%H1;#>hug%9Mh=p8?L48jx**RTP$pn>3 z$A(=a=2G@Xlr%yjKoD5CQK~@PtUE2%?BgsPSv6nNP*~HJ(F~d$1n>q2LJibk1!gaM zih)_D8d>Fy+OuGm=WM> z&p18z>4{*1r%62Es_U%!KGwbzUwvPSTU5CxYBxE8b9g diff --git a/out/test/clients/org/apache/kafka/test/MockSelector.class b/out/test/clients/org/apache/kafka/test/MockSelector.class deleted file mode 100644 index 0b32da93024c9205d9a55b40c582750e383912c8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3363 zcmbVOe^(Pn6ukpUmcUjD*59;Z1o@?|Qm_h${P7onG3>OX6Dt~4+tGrRG)`en zns20eB+X-Ka?-4K;0ZRQc`D7f9e9RK4c}?_Uc(Op3C}*T1p2a$yK9(-W`55y_RXDr z)5tpq2TsYTcy_UDtaI0-z$v?Ad$#FWg`8C?lm$M{zA#^!YLk(*%O3ZnbM|h@^eQgH z-mQB*6`Q7Hd9NII-;i-9r)4VUKP*~ds;=6?V=HgjFD<5@{owekex4wN%A^I(f6WVY;@Q2a5@B&t|zzy5w19&scHt`#Gz~0vwmrX#q5W&j%a9`zz$z zSfa_%)$(Qv1=lK`QJ#2=+nI`Zhz|<>q->Mu6v<}K?GGBDc%)zH+y5v?S7Z{3f z#<9~TLtKmP+p(>TedrX6d<-s2H%+R?sd%h!nUoC<`K-uZSLHC#u=oKY(5q)&%0l4Q z$rY{o=Ud50BD`^8VNnl(`Tqeoh!E9OXBBv`sE%UX6;q|SE{-BNI5MpLzZO!3!KOB; zMQOWU%ZY=E$Mch_l>C7vA2YJgX@5zL+jCCE&0CAMlxbH~b2nvF9iL)U;BN4`spTzHc1}eDw-|?qE#Z8 zqgn`v@>ZJ~%L!nGwE z-;?Ru{1WFR{n%WvYoBrDG9&(qRz?M8hTlO9|B04o@6h@diEx+xHk*U^C+D&m;Gw-!++iUG!>KXVSL-8zOb!lLEcb&eAB1+BnM z{tTk@iJ9ka8C_=9Vlv-m=FvzkdP8P`+uY$t9APwxMT7;U)WRJVLib+=A$WP5>^RZX zk%|g3Zt*EI_OpiM5jv=mH%L_nT6NH-1|h1mipKj$W16Su0WPs+eQb|xnvb%-hH@Nt zf@D#`5;24=kg&xV!p8B1f_oSDNW99DXj<{UGr5$sv9^A;eNm bcCkXQM9Qe55In3^y~aAeZtD0@Rh#`6((07= diff --git a/out/test/clients/org/apache/kafka/test/MockSerializer.class b/out/test/clients/org/apache/kafka/test/MockSerializer.class deleted file mode 100644 index d3fc40e230bb94381d701bfb5a1ec5f38f0c73e8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1400 zcma)6ZBG+H5PtSrxc01opzwHhEOUXFqhD*zJjO!y~}HWQg@DujRH> zd_Axr(C!Orh7M7GV;?cFEr#3ebdG8*N|Wtw`9Q32r)-v_C!BIFy+}ofmqgj&(r1#q zz~tXPKD~Iol@7x|(=Cj>y`0)!CZ^J@2}>2tZ%o*NO1#S@XV82@tK!zF7%d86%aHH- z&Vg;ooR1yLTa{+ZUSBr3+zHFAR$a$z5$95nbl#Qx_OhynQF0reWxo`v*M04cn^8$5 z(2br161bkg0-hw$rB36RU`Sp0^(m0243k%RE(|^A#S03+Fnh`KKk3ms5{~8BRT_vT zXZMvY{g+;#HY5wFO-#^y8K9wnhC!7bP2-@-jrLBur=)4ULDC}ajP^wO6zmw<2_o;m zlB}VZ)-FV79Yz!*gf)sjWf|DxCT@{LZx;Pzt+3wGr|g*mi1+|17;x8j91oODN98&| zq?Ddv2zLl%n9c#b(v8slDBWlnqZ9RnRu$$s?JA6x{o-pC(IC!rD2{d(N2B=n$VVYf zVv1Ip?Xvsm&@U<*N<0?e_AC*j`{bz@%@ZHR$Vg|hztC}l&Wp?j yLI_5aSq#ZD7IVHo(EkJahsY^n-=LizqpF}ZcPfNWYLL1yM5mxUDWplGA@dhY7$*t< diff --git a/out/test/clients/org/apache/kafka/test/TestUtils.class b/out/test/clients/org/apache/kafka/test/TestUtils.class deleted file mode 100644 index 89a43f8124bf7ea8503f079d3a0b38c4fdceef0e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2885 zcmbVOSyL2O6#jaqXL@KFR1ge^YfuLe+7%a!37a@NEHa27Nk}s^&@yz7J&lUl6SFU7 zPo90rYm8N3LgI5OKOj$eOI7|uQe~3wb`LlPD65>Bx_$52&v(x`-M{|x<2?X}@QsRE z45=u?u!4&*jHn3XSq0Cjh+;^=^D1H(EyfEl6pX1zV_Zcs%pz(`sMv~01yd^43t20M zODd+35%d|cvjxWyOIE?<7+e)Ocq*zeD-2vwa5cg(_WttQU$=zPHL(l{)|`Ut64XTR z=sUDEE@InS`Y zK_fG3#_*zqXh)(eF_4PkC7}~&kGA%7(1D;+C)L^6(b+NDPhxNPvf)|J%IN)uJ?_lV zaCC_qJz?3FcTz$)UNL(o?2BZA?rXx@YD*eSph(y3Nty`W!RLn4V*|XJ&Jr=}KshuS_{nx6)SH znVE5HJ+DVw5zon5X+oPdT+i|>VkALINGMJRwl7%Tl!PPk#A7;;EIP4?1nq`|@ogh( zhSftc5tE38lIsJPKH5nu=aE(UFm!X-CHRj0Pxi!`PWqM-P1qAo%Sjq7NqI(ky4%S5 zSD~Oy!OH}|^)I+>j`%aCc%p8kjhC9UuJZ~bV*xoNXj&i{RGNeeimn8URgp1mVml_G zY6ZJ(*2;{VZVS6dLhM93ldmK-<;=QiQ&c!Bez^#mMCohTj{_Q>0xPa&Mb+eQT0knH zC?hIv%E@^`=@kj(e((6mwz{q{C*YN^Yb`4eVJ1wxN{BSPhSzzYYboW6Ou-u(-oy=| z_Lhbw=o;R}I}$eeF%&du9)#6QKHLKmPPVqScXXcWI-NLkHrd_N+jp)%HE@3L!qD(V zV=O&xPE1Z&m!>l_wv)Z==DgV}SFg=o-?#t3!9$0ej~qRw;9U*x;e8Ds;6n``;bRRq z@ri~{@fq*4aIg?|acr`jCT}J?Zn+9R*YE|t6#0CmVHb9@QXjw>j0M5`TEjl*OH4n{ z-myz&nm`|VobcJC4>HJyr7t%J@m#$}If>lWro);I;&f?5yK zZ>hVu1j>T~WQ#?OT)qtBBdaI{q~(0sw|d^TPiuEF8Dh%;)>0r@Kov0++2%EGxuZwg z1TDC`f-+%(f=pzOwa~dYg^y zIZH!gV|=9`Qz{H*j-!2sb7<1^`dlY#y55|G-GzAO-)*Z}dIL8dFm&(sZK2j8i+ zeAPzz?qvmYSA{)@LqZ+*A=LBSP;lPMb7R4|oM(2jug4i-^(6oY+OLm4^Tt6a3eyM-xpCl+;|sSmArrw z5vS~@xtmddsYG}xWwhJ)+{HwjnCcNG+Rt@}=?WN5Gx8X=;AuuL6Tv+nWwTfmMxC>4hV?7V{~MGbRqa}Cg_Wk3z6!Cq1aNo7g=2c2!T AMF0Q* diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$1.class deleted file mode 100644 index 8fbed45e02a02fb14646766d67aeccb3673c654f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1455 zcmb_cT~8B16g|_Hez8LN5X3J~3zQaF5Z;O;1~F+;G^T_HeHzM8mM*)^?$qdC@mJu3 znD}6P_D31d>@G=-eXHGc=FZ(a=iYnnz5DC;_a6Xuuv0(|7RJ%M+X|A5!LkRPPo*N$u7_^GcQ3uN~FQzbCjbUgLCe|e&UBYEOd zGu`yt(mj@eW6p^(V^S2b8lI?w{tcZs935EEG_*Fb>La>A*$=&kopoCO>oW92zY`?u1m?q5~g`F5C+b?AIA)433HpW zfsHkD)MQM(s4+Q(lQY8U)(&i9MFEfV)Jy z7u^sOrI&=ipTKV?@P%saC&sRkHMpq=cQA`MzCwbT!Gp-^Ar_;336HpE@R&JIQ8M}E Hxbyc1;Y)yk diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$2.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$2.class deleted file mode 100644 index 418a039957dc831cbe54e9ad98974a55fcc810e9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1457 zcmb_cT~8B16g|_Hez8jV5X4W^3Z+FBfd^xe#2_Y3ipG@ipie^?%F<=G*_|5sEB*?6 z5ECDa&;BUmncXF+v2V4T&fK|s=iGbGy?1~8{{92NCN>Mmp6xC6K ztJHq5uR>iX+?L}xdRyRq7N~{U|$}( z)J!$~wsa3<;Fxow%$O7ftcK^QV8@kVsCe8RJ-+ffGfsUXc{9Tz6`1`yob%pQNXtX) zw_=?u$#;W1cT`Z{Hk2MmR0}LshFBy>(XEyaSS(%(~&OjM{3DXbW?=$r%`V1j_3IFGfNUcsbI2 zGl(QabK5=Z>GPOEfpo=8&RkOOe`hMs**=hNpG?!VA&}R8TphM|Ww0lbnqh(8kvzXN zqEs?8Q%dtc4h!rvN*|DlHPzZzi0aBWq`pK(*SThC1U$n8*M;O_2~#{72m@!{k7F7$ zgtSB;2|Di Ll9I_U$DO}FG$etv diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$3.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/FilteredIteratorTest$3.class deleted file mode 100644 index 900b85ad183a08f63936221b7b6e449b23677880..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1454 zcmb_cOK;Oa5dJnv<2(#W+R{>94G`KqN?k~ADV0ztQlz9sq{;zK8)uWaaqY<7Aowf% zigKVx91v%I6k^tKq#$w&mb^PVp800x+Zq4*{rv}kLmVcdp^!oXPf}RHj#_p#6jP)w zX((%`FvR81@(TwH^``3$3_j-OP#7cLA92H%p5T|3GcZO|(XbpTJcrx9@!GOU6ukzm zT)N)5@MVqQjx5KL#|-a^{{gRg!9Q?gA}!mP#i^A#46(Y~6AYQA<%rYCWmkCTylazY zwdtDN?r_gi=TI3_B{Jv@#}QuL=DsiJc(Qzcg(_7X`AFogFUpi*v*r3k_3VkgF;^)pjSIDX7;d>eix`+gBYzop9oqw(C>% zEg^?)Ps5&$r`Xq#L`p{*hK>U~)A1ZthFs8A^F-^|#BG{^rAMGZ@Gq7^VR*TezZyh_ zw509SZEl-ZXSjkCF5qk*TI!`3hFv`-aGu!00BeaWVp0$k3V(9kW=alY&ri>U$b%SSOfU zBrDjdW`i^_bz7uSf!@+NLVK=qjc8UIe@23&*cH1XWfUr0Dlj;saBqT;3W6Lk3z@?f z?hxGFz=LJ7uL%EMh+hryljX`!tXw0ma3cZkd>PXOlObjd_XDd3*beqQ9?~AeBRr@7{;FsNlpj{i7{$JYb_RP03j8Xs!$Xu)YJr8gJ?k79>M_*91?nR;>ZQJ z9dCQpsWbJq7rnAKNFDnD`c3*JYM(uO@`uVyJB4N@`|h9TeRub_?{4z@pTGVF;4;3B z;xw*k7>*)yLH6VH~5hCOFEmTdc-L3hn!jEQX7Sxp;TMsdwdZx|~ZM!MoUrm<<2R?{1*nzl-= z>68q+lAf|`R?X!MOT#TYvu4E|5omX5)foM_6x*t}0lM3_kmRWSoWp245O-T8(e*ct6nvokue5MHR)<| zHEZ*(!-ad&?k4HqLpI6U^R7|c$QfH6Pw#|s8m0x3cdI4W+BD~^ibdSxrBc~7T&r9n z?U+Y2gV)8*p^HQ zO2>UX&{4p=js-l_u&Cope03x%96FLH+{5^xY3;)^v$W347C6>GB5Cl3HacqADWE+v z?5a7l!t_sNdJjnKT5Rurd2E?tVWXmA_M{e?eALt=JoTqubwh~^S`P30S z6HWK39$LOz+L}?BHXjoM-?4c<9wo1ls4d3JW`F(E7|VPaPWl(Rc})7o~)^O z8P;_rz`HohmcV;-4oGKFX0Q)%6IU>90Wdxc1$CEm)3;?aB|6mQEH!faAH{Sta% z7wy-B7xr*W;0Gj(U08U{2BdgXu|NOM>o&4;vti>n<(n6C05GClw@i#M2^$obv>ozE~S zU1b8@U=G|w3b#1<1j{Mh=1;uL62&Lu^Wk3tpU-LXQ+&?W`J~?9qx6!G(n~%{FZn3_ zK0c{`!Dpz>XTj(5?0-(wL<9QC!$ZHrw8=5SX5lj3cK8$XEgs-|{=oc*hfUC*)swOY z-L6A->(F=JfWGJ*l1fT>ND{izOXy0!5BknOLJ!~)b0NTNc#F1^VG^|R6IaGdQ&#^1 DsDWYk diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$1.class deleted file mode 100644 index 15e196920cbd0662d0f4cbaea60f8e7a9da72df0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1298 zcmbtU-%ry}7(I9EDAdjkQ2Z^ZQ^tmrB_=*V7KCU*9BMMM7~d}Cx|Nk~X*&}C75@QW zeDOg{d@wxwM;TAM!b*r{32WBgd-{Fn-0$4)_V=HkzW_W#Z3;Rb8JI-LfQH99p6FOK z2$pp$>8R+a3QU9@Cn&87Y&3d)+md~0ca+tYAG^{DLSM;4$7@^N;m~rtQ2Cy81M77& ze!1yO&+hE1AY3EYy5l)vU0|zxS+b>;fVS1!R|18GKp9IR~#(_MNmMdw=HmztYe=P;k5}LKP z^g$roRZN&oy*!O4-H~($aTz1FBvl~a45i)Ok^Q8s#kA(zy#r;3X~_hMB&V8A+mqqI zCv`g|e3~@I_P+DgzGKT!t*7!&7oLKW)z;~a(d-R;TfK6kjhVa96)RDR+{2fi?e+q? zv!lXJZ(ql{UTfGY1h%ojqdN(1G(@o&8OLLu<=;e}=v9vD z$nXM;O7%NLN8|;ePy_r#;|j?g1WRh9iT~I-mUg1=DOy$N&HU diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$2.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest$2.class deleted file mode 100644 index 21878afe89d0fe3f315e62b0f08a6fa3ad48cf09..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1298 zcmbtU-%ry}7(I6@>!_U@p!i!+r;H7$qR|J)f)GiFLrq2&Vv zFFuHg4~A#|DC22YpoD0aux9POr{8zZ{m%VvfB*UU3&1lx8HbMh2J$Ey(C|RVLmjgQ z!HSMK9aSAQfn3;eg7Ra5l}68RTe2_hj;1=qtJBcx|gY99oVSD&LcCV7+d} zFIRo(*_|yFgiGXFc04Dn3#?VnOE%XM(AIi8N}$+qJoRRMZv*|g@{?OkPuX~_hMB*&Xh+mqqI zCv`g|e3Uf8_P+Dgj$_MEEvNF27M_BV)z;~a(d-R;Tdh0M#!R2-iiM~|?%^xXc6$Nc z*;HYtx1(d;#3L-2FmSh{{G(@Qs8OLLu<=;es=tYj| z$nXM;YVA8j^#?NFILn~OdyX-{QU*y*guaq{j@PkzqoL(^q4GWH2G;9V z{Bqrwp55J2LAXM$RmXF}y1+)|ykra80@_Butptiq$5U^HdppYCk~=Pu>1N-S?zZ%u zXr6#tREU7t@I2*jxH1S7r|aijNO_rc<>X-CeiEoun!9peTCSue4O-Dr{z?j>B{XYg z;e$Z7rrX>?3lALTg9Z!Zs zpVXa{@KMqj+xyN}ZO4|OT2192Ej$G!tF6-;qtze!wtD468#8;RE0&@XxrZ-3+wBK* zXH$jUep|<)iAPv6Vc>!ZGa6Z77fzj6lIRMQPOy=%iA%W5jvb>S=id!mUOq)VvPocZ z4Ax_x1nOgi|7Q`bIOO3uN(Tg0_ zk>Ldx)!KK6>JMbTah5@m_X)-T&r#xib`*sv%y7o|m`yN(#wJ|v856M=YX`{Y2A`p8 zU!Y~b63V6sS7XAtQRp#YkrG#8!6L5W8mX=`FNaKFO_W|xU-S=5?*IS* diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamBranchTest.class deleted file mode 100644 index 6866e0c2854a2f0c6ff05a12aa7d5505096c73cb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4070 zcmbtX>r)d~6#w14ZnCTj2-cu&rM>{g6tq^sQU$a{13`mminT6bfrYS%n+=M6*;f00 z*xK6n%YJbBu`>!<)af^8oavW-=uH2Les`v|=k6vkxG9Y_1A8BP&+B*2J?HGrzkj|C zU=x0j(1;^`C;_a;AV*w+2S>zWh(ndY?Zq(1hy)2oI7TJN&^X2LnS2;zu1GRbjF!b3PNVG1WWo|jN7Gnv&2?s0Kr8IBAT8l#~)5RYT*-@VF9+CnqM7dMI~FF*RCs>83iO z8imAy-=vZ#++t`ck)N$nAuX}CLDlF}1{0QT z$+(i}SI7)nXE#0Os74rUcPfxJ4ON*S1VZEXG^CL=hOQ*ip*^Gsy=*rWJwDo}rcF|# z!lZG|3kIgqe-{HE%xH-rf!xMY1m(EcM8ZTZy>n9488(zc-fO5sT3j(zp_P{I&Lm7N zl~5zYeWQj-KzOxuWN3)uKs}yPaN1Y-JTmz)3&$g9eeJPT%ijo}vpoN8qr^bPM zBb8D{GOni6NrQ;%ix3N7L*2>vcyI1tr#dW988Wm<0cyQdphc5aPdKR4I!Z)K!gHcv zDwT_b{Z1y%5XeKjt>B_$qrA0Xi{eAvoi-8}EroAZ)6%*26~W1=?me^ZEyA`a{H1HU zAZjY{@opt$-8d@;jZn_x*v0WW$8Jh$48eUF-P9)3el4xh<=mm`Nz+Op6cXhYfMXfD zYTGe%t9rHOmJBx~iJ+qgj_F&U3xGX=Yq-DHIDD8ttOowj5&p_ySzX$)-l z7v?D$8?lL@$wB|GPHWGljO``R%#)f{H3=l%pd5Ffe8~L!WwDXGd3S-4@h0A)d{csU z`P)a4Bjatn!||?+_wYW$8V7mNu;utb#uYF?Jx{w zilIzbk9Fy3lbQ@lb8G4f%(t4i)~@=3Ejtp(NS>{y3`!B#+)+ATc0+UO>ODfbd0rX0 z)+q~2_MTYbQ*q}KWeX^&PMu+f8Wl?Fb#si%o)9inQYlrZ?xnF1kEmU53-PeGauOMq z&*gacYrmPaw-~DGoP0>~)*=F(_&axmlLJ|>RZ_g#K9VuDgzf3DW)-!?79exCshv$z z%Wd_xUDVsJr*9AXM1V`wGn%hHWIoC4{)n(HBn{ zZ8gtAB6elrS;#RGxS)F$3&mtn&n(Ktq@u-B?z<|*+@46xhL^Ibz#%)LGu^E5_d-k>LX1=TLJ@G2|`^6RHB zV`*iOCYMq33zpGei{HIDP#vJH^H?49R|jShB=wd@gSEH;|2;6WD-oC>fDxBr;M zL$P%;c(^6t-Yiv1K`)Ut1p`8ZptoAua2YFRQA-RSsa!dYRo31#R!?KiG}fMyh{dl$ z6TiFpBI>45FBFo3eow)&3JpQOw_rn{?twtPpKu1&!B6h3piZHNFkgl3=)f)4`yh74SR6|`|um0xP^V>>wZ>&18fBjvL?hV&z&IOokatlKpO(M zNq!SENq8*lO+RbElO&D47FaEIpq)Hh!=s>uLqz8zo1Uf7g~QhKIc%V@hNPF% ND1Hm=fiSXu{{V#fgc6r4?*xHS$*TPWYOK%n`erV=B$8C)&P^8-O+B<%4X5Z|yfBpXP6To9wv(RzZzzpsg&`{QKU&jN3 z;EIllj;fBDKt9}cg7PDQjqcFzTXH1rU1bgAhk>+$&{uNb@%q+a@@P3;sC-Ylf%UQ% zzkK1iMBb_(Y?5o;@tm+F(5{@9thyzjwTC-OpxAXh^=iDoO&z)I5}EG~ZRu`F--+f4 zs6~Ycm>thke%qBnpg4VY&V`iMSXWMtM(#&}N~OCe52WQvTGF8vedRY(5G|ou&FXuB z>_9PLA@%Y&o^%J&9mi#yvL&ekgPf4uM%l&rQzZ;ak>?AwZcK-tAJU9lRK$US`S+3ql)JDV!p z9q#C;n=o*}go%cUhgcPuJ9Q(G6eyiy6HODBaGA|IMKR95d$qjw7xly;fyNZ9Cl(2` zrU?JfBG@FAsLf{>vo+}m0v#;#3!MS?6QWd#Oybv^q=#Y*jd zzdZERTD`b>snuG78s#6*Q-4nn*>9gS=-^DOq19?NEcWF)`}=*@{q66!KYzXr;7NR~ zA%wANJYR!dh)cYn!H+SuIU{jagY-5pO1z{&!#VZkWeGz=Eyg6qHPop&q4pE&U}91Y zQxek}8ewUe!K_45!)7HVrRKC6GHS@GGv@r%@#ITVv1sWnV#+(sO8tJKU%(1QXRF^>AtU2Fj=1ki%lGa7j z7U+s(?5VJkH4@Wic-EMlHNuHZDwRoxmrogv#a(~eF{ezsm|5Y?8OdUjKxc(`o=U~M zgj;XrhVn_r$|lX|khsBT`!nhD387&Paprtb_fgPxxS+SUN6Vv)5o2{V_=P@7OGt5QEalu68< zTt3!kPAd0K*w&n~pyh9P3Q8+C^tngL)D&6a>-p@mzhk8!o;k%CR6wje1)`;>WzDG^ zu{P!yiP<3|>*k4j#SMubiHj^PtRVu;r}AmXN|~cp&SF0Gq|+J4y`GFyy{pemo_QHo zexXNdl|o*6MDq^QC6>wC36m72W>ZN*b*O@&vkdMR2)LC*M=N#<9Qwy4LPs0g1rAmQ z;9aodnxLZtVP2II{?Y)ha#MEfRaU*KV-NNU?5zN7<)sjKyo9H0tQm#-i>{8>@H#`N z;|*NW@h099*i{~_B&n6wZ%bU(aRpZeHr<;X(m6);v6bhkqD z5iQheRfN|l?ZB!smY?aB*UF9H`g=)TcyjrIv8%|l17MK~^|usxeECtP7maMzOiu{3 zhl;+(ygsLx#UtH&-L|Ymy#I5xlkrFb8$+cyjNhR$0*h4eE<+2V32p-0i|a zQgsg)htSC#dr}JZDWT#Gw%n4mAD2)|_y;2O@)s>$<&VbwfoMDsjK+g98kg14cy&ED z^}%`>9a=&i&(?SPceh@rX4g^U{}t=wRa{lY7SS-=&Xs-(8+>T?-@-;8#+J|+-+cp3 zoq?(Y!Ny?Q?e7q74k(7U#$d-yY+Ax*Vm3ECxQK@oX%Sl%@$e!Zxe^rD?)<(HTbupY zv279CuinIC%4|QB*&SHNaSy8L**2QApVT25eU7GDG&G09_=fYhcm_Yb+K-Fq!!gRWaGZb7(%~n_-E{X<|J_pq zJT=H~Tj9GyQBoozF(ff85tTS8@f;)l2NU5G4+SYO##vzGzlqC^e={ySSBpy@<8qAN t9H3!?^ksx`8K(cDjLS*JCCa$O7?+X%b6k8l&FJ_riWbfqxU}vrPpz8)}xopo(ylv_FdHp%M7;Q`fgYexR(3i zXwxeK#!_od2^_4szPjCgu&RQ)a@C{dP_5OF-ii!dJ&%osPDQ}3`o0R5JlW|ePM1Cu zNw&z$hGe_#Z3&cfwN3d@I-VpeRkE?Jf^xz{R>)U*I$^4kGL4c^gv+ZckZvl*I-GQh zLvg2v((6Vk?rs!ECBcE}biGhuvbVMyx}LKb1ahn9Mkg?}mzb?~pPbFNEm#NXzw^Yg%<~d#uWuvKep%bSl1xD-cx-Y|Sz|?+ye-b|y;!8st&uM+K zIIsH+uhpUIZ>eyjwPvDX<1((;IEajm30$GlBPs6{tBplmwXuY2Ca&99#`iXU0E=T{ zSNAa!0-0S~PkJ^!!{=;T0}%bLo|P@`iQPLsf#LvR_nx1?;PhAUh*rtH%I)K(3k;!TWkh>UMG=Y^L5cJ8$a9?I2UcU4=eseS#!-AhZ|?I3O$oh(o=Iao-aJd p=*)9i+ZfY8_miW;(=mo)#NqNNMBX!)Xq6EsQ^!H@t3LZS`^M?~#RI-!%yv+d`3-=} z@lhBpIH2LyIyB=piQB^n;DA~f5+h;A;vy+A8b%lgB+M{$j7f}#u~@ASD!R1@6PQ#( zN+KP`a%95DA}4W27%PBz5boA+kA{0S+^6Aw z4G(B|P`!Idy?a>Vu*4$*dVeNk+Px_wpEvUYbxt;CB?pbPDG(W$Fs6)H%E*kz;*M=) z#=8WTOq$buX5O?dBV`>jZGo?kc+GGJxQ!9se(wD%e#Dc81a z(nt*(G=u9BG2k4u=!2e01@ew<8fkhUHd%a(S+vH^7^!@0Hx=Q_9Vx@vW90Uld53x| zayXYr0>a{dzknPmSgBFvv`w|hC?RhvGAZlQMlM%7yE#2)oDEZbl0P_7s#i{^B0E~4L6N~sJbz2Wgww?{zf$aZ;147N%H52c?6 zK^>y*g~;+sse~+1wsfgKfp|^GWa2UO0-s%dg*sS?S~Y53#CcQ|I4ABH$w`$1${lWj z8I^ccqF3TERyvk6iQNJXHxx3Cl{Sa1yv2<@nM~GkO9anky{ptjfk_-wXMlQ!-3)8#H?S5l+HZAa36||}y+Ixg+?doMbRf=}QHqe3x{ zygjr;QO3?)EKDjMEix4xD;4XpZDV@Ca$CYO*CFL(UiR|5kh(WN-fQYrz7_jO53ja1 z(^gk^T-(`xM}gNi7LCSTyOcoxxXG8%a+^yEvG93KnMZM~;$^>umvBFPs{QcEr@Dwc zvm3axP+bZF9bE0>PQiIq({YHVi0>HuO|u9jjw5&sAwEhzYR6FbxyyPb#|S?vfYlt= z@MB>eqF9fsNYRTx*o;oDcq69}pAzDe{PL(H?T=Z63Aboq7J7mL&)Flt3Mc&jp+q1s zln4fg5~0x0;4Bt%(~{0$bJJ0(a1?ccuTh`y^T{8d!P3q^L!jjgM0{unet~5^?3=~% zMDuY(J45~s8I`Tye1KSk;`Fyh<;D|OF^iRiJU7xXgYy(=2ItS}Qq=hk zGOD$^08woQ8&06B{8|U9K=(>1}4j*L{yKyyL`U5U;pETv#lG`=>O6;w4|5E%+^d5#_ zElyHKFC)DZpHj+pYPAvHVh7hj%07Z?DKA8cYp{#bB&CAb{u0I>ViY+CPje2D^E&Qo zLEvwyq)A*aaf3u$Vz0!F62lVvc(6{P{ZDjA+$2unVkF2BqHZ^H_F=y}-a?;%b}=(J G^Y;%#dssUF diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest$1.class deleted file mode 100644 index 885f8652eccc5380d304440bb75c40236d04c62b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1585 zcmb_cT~8BH5IwgoEL~S9pNfj2RxB;Bl=_uGNC*Lg7JYzV^zG7KVPV;AcDEQ_{7w2G zB$^nX{ZYnqw}rF_vOcZX@^mudI(c*0!WuSK+1#qj^FuIGz(N3cSi)5$5EUKx)~qDuKSTBX0IxPWf&GS^S?8-1y+^$_sJlP_%?2@&bdnAy{mG|VKv|UM_NDAqi zpKMXBkue>ZzH+E((h;A#LDm8l*4;>8xN%yK9M@h70(n$+Vp>Pq^E|4Z=GyeueqrIf zk41)%-f(t28Px-ZvCv`Umq1OlUVpp2rz%lf;!Tyn=ax{K%1-JK_Y`}prh*G1Xu&*X z_yViNd7ZSm;n#zTT6J{0j9u2bGrA~juobW3`XS4>uA*JPYM^Lg4kZf~u3P9uNnkLp zUpwIzhwrI{XLv4<`M16nMsR~Ob|oX0Ew*`RP%1Ff0eB0i0xKO{`2WBQ^hccgwhw_o zGN)(uTI84gPaHrQlKG)r>s&NPg{HDvW!GQSO&>7M&y@)xL1Z$z0{qA&`R_#^wHZo_ zNbmv7eBm2J{sf7y+$GS@wVO7;GzPeiG^{X)VeV)jqX}BDG(6K-b-p$F=no{{pCGk9 zRX9f1F}mMrf$J9@CS&LUq$tm#2Sx7Z2vLgNj5h$z+>GKTM(L!7R=04Q4({-tb*34k z<}Nh}+{-L6lyRQuP)gM3w9@&)FBr$@(XP*DrGaSFbeC#E-N%F2`XRD>r|<|97^0+^ IO!Lj(UvXroR{#J2 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamFlatMapValuesTest.class deleted file mode 100644 index ae152c5ce0beb4bf7c11e8936a557aa6b883b8a7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3764 zcmbVP>vI!T6#w14?lvsYrVSVY1)NmgYCmh9dHz{mLgg|C;&)&QD{Lbs1bMF4|@3-dw z+={Q^NvaHFOh&MGRKsqnMd}d; z@yhCfj2bo6iEZjM8Eo54`nU>)F|<~LS5$|vYBG8{s%EokQl+V!Rd~QZnvTY|&u<%0 zV?zB2UCRsgoBxNWC_#8~z}!kq(!Aab^}8#b&LtxyVV%y*hc;0zr97q zW02!zDp6F6I7S(KJ8~&QOR8ZltC8LPsZ`o9vzks%t*O;ij`GegqD6|jL+P9zRkvs& z)m9YkEXf10v0aoG89lh2!D-eV86D_k*l^tiMMf988SbbKpx|0-1(B(u-^S2c#$SF| zidGd9lBz}(JXbFoYdVEPUSU{U23IlEv%)P5xmK;t}IX;o`DL&&kF5`22K~=aTknb=_ zSdh*@o`OsyEJ#-%-(`}pAl-pH#g9n1TgI0-ArMY743_C%9+0a$x5Qb&@hK{S4DD4w zTM7e1OBD=7$1)68D2dmjdav2dVtP8s@O1^uiXGQ_va9XVB|H^sY=v@NZcovCsssyT z*0#v7r-JU~{1$)rUZX%>Y)1^r6-8a_TK-U3t3&=UTBU00?@G_z&Yn+_>mrb3XVJEnP=qZ>-C zNW4Oy>8IPOPRT2aYpZSy?$6OpjJinE@UReQpH%6kx76mMLR1W`rd&aBtfE_dJKe7B zu!+V*Pd3qCOiF#iq)hZLP|Km&F51b6u4*|A*5a|9g1u!9j>u^^Pr*fxoF1i9sQJcZ z-9V#CAHe#nIduSGQ0f*1gdKMlV ze9p6|w_(>D8Y1gY!y9nfd$^akojVS{PjK4XyuAGkmds%(LEhl;&EiI}G>c`kSU!st zhd4WV;im@N9>MDjF+BDF40 zGmYHS5(;p8kK~nnoWw5kaj(>6f_bG`tUrU+LlTMjUAWHy;l53%rLhGPT2V(XY9x2H zle4y>kD_rm6!arbyN(b&ib0&9H@{#AzhM}E5j~F))7>$0{scv~4|kD!f5l4kNfK@@ zhCsrfgx61@sKXzGxS7Jyh@UY)sCGh~!4?`FB>hL+L&eT<}RgNr1tAvvRjJVuBs zX@3DDNEijq?H4)kCC(kRs}+t5q>{uD;@HV?A4iyD7sve^5jte&(e)2DaXi4z<7Vt8 Y4j1Y6AW<70GRHme6Lr(7Oq72A0XbAmYybcN diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$1.class deleted file mode 100644 index 21a392e4a29bf51185f86459510493e8f2afe601..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1381 zcmbtUZEq4m5PlXY2iL10R*UbbsJtJoO*DZJYhx1?^#cuw9~W5R%Jq)i9ZmFC=?7_| ziSe^P$~eni%>`^i(!1>J?(Ez%GtbQ4??2yv0@%fF5-J|3SVvyLLlsfvrFfLUV>uTT z6jib)DJUzbD5x^Ty^-PO>kO^7?F@B3;r)@&$Nb}%>#pYrer#Anee5rF!}5erePUgi(#*DF{omfA+~Q11VgHASmNFExF?)L-ZM#)Y1@5ncDZB7 zeV~lVBp9?;mL;5hle?~94A*7#&%eqwl5Xdm$tuA7|5pv;Mr#L3nTS;G#VFr*jlG;ASDUA#iF=QeSkJcNLK{upX2V1@`}Xs+QV(~^7ocSLEVkS6VATD1_N z7f?&(Z(yZUM8484f(*$x*#K2sA$iTW!X~Zcqt4N&OtKgOR+nrfdiDpRA5IY~&#;z> zlc>;`LHR;wVdVN8Ip=@#4>^nLxH0#f#Z6j8(tG6NR^a0)SxX-qrSdN%Dl}$DE&}@@ hQ_|8f0FL4I-2ENgozFSkqdA8AB(Ks?$y%kEp1`^7M4R5n+&l%w<8!*b*aSR@U$hoW8QK|lBv6G?lie4^*R*B zv=a>0Tcw1z=WyQ_w0yag8Tl>pX@w64&L@WTLj8oFaogb(4Dy^}vQwNF(S|eb498~9 z>CaG(S|_3%T%d#^hGau_6%U3Uk$O|dgbIuJ)zo$zkom%^P7y-e=Os1r)@-;#uPxq4 z9q8;#hAY}3%E)Wgc3htVa3F%7+cB_V;Sw%eNTI^8HtY7Qp>#UJV_-Q8Ha0Ch!xlq& zR?os3Zcr62Q-t}8;7PO?Dhn{D!o^TsV8efJ0z$xWuR07H3)M~-C(1%1a3@)! z92EXhWOzVVcokIcVCl4uJ>C5%wG`4My+X4ZBJ=`gsr(hJbdJauT1AkdK2AFNUvZWC z>k}We9A_AfPyo47@`?h^HO yNZk&p$x`_z5+fvMQ4<5nMLHrGqGGr+KJYH?jmP^G3Lu7u)L)}!lD0`BJ%0dNKv7-* diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$3.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$3.class deleted file mode 100644 index d2efc76bb7a7dd2d073d176d9264c914869dcb8e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1336 zcmbtU+iuf95Ivi?F>wtIrQ9zqZ78`wjTGJzS*Vet`z-SmU5#fRMK39HZF_PG^Cf#3t_b*=vR(UM*y0*||)^`aHOeC|u{ zK!j19m>SZPk1QLIr-3w$_q-!YU*&0~JTEtiuRR5@u`Ya3dOGPUo>9%iepuh2e_JZ8`SJ7HcK|C`o`i-6IwtT?M+uKKJl60;r?9M{ zqM@pxCQt~vmS1`%&}h0|$CQ29?kck<-}R*F2cD8gmeVnN@zk`OKzWX|eRI1NecW;_ zXIJ?_oupOEv4V!cTKQtA%AP=e-91nO(@o1!Zw5#E%G;IuHYKx7w=L~G=~>}A5$3}~ z2pF4=qr7!n`o7}w<%QfRt?*3avfsB)1a`~KLwPJsTQV4%3_?eF^^}OgV3_KaEEc!b z$)2lZXox&t!4P4b0;Td-Wo%T3Y!7 z9H&EIIRkT8x^d_QRx|9e{&i)r{L5jh*D^qchIujcrR52^c^BH%@@v1yL-9 zDdIhv<6~fo(#xDRFvdTiS8Jajs$Venk*hIGb1YB?SVxiL)mRHNnB$825lv8o`Qx{c z^@{J1UZ^d8MeYMm0DPjv5hcDlw2xaCdMw!ro&@fIRMVoaLiaTjQI*Uk$O(X8-^I diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$5.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest$5.class deleted file mode 100644 index e88bb928911fa308d150a3619c4ae9f0ab369b88..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1795 zcmcIkTTc@~6#j-5*49Oqi+I;6ReM2J6vU*Ykr0hY!H3o+#;2i-?a=LRv%4kXXZTO_ zK_w;{p8Wy-9OIeoR#xhzgxF?hcg~#o&h0zr>-SGz04!iBfm4`CB7wAqn@QZl?IbeD zYRDz&B(Gss!yOHC3?qI+dg;3i#ggOJP2T3UhA^A_b(5Q(?+V_McHL}tKTT=-!nL{O znd{}?@R=j+itzjbAuUT=`bCD7%*mj#RfhPgvn?3LO41gaoz|9cD}2i$$wbMiajVK* zsrI2Tu7Y6D*KAw3s}}b>LEA?sawEM+F%9!}+uCEOWJ)`Hmzx%+WUNsV>cTBVM3f9l zQz1Ku#dWb)) zVu4D3R4w$aSSV6p52HRk-`Wv1Kbo#kl8r!yB8G4M!sBFhgDNK6sAIqJKlJrDxc@+0 z`+G)IPLgG(XO!Z0ika7X@W%uvf@)OeWb z1F1Q!mSdY8U(#@Tp;l_4R6){AAAeqa*2g`lL&^y`mSN!UU#hNBRL8BMl-3ohhtM3c8m$BLMKH~creoJRj#LWOP)z$rpXy6>q z2Nn%nz(u-oDY$ppSRor%LVuU&RuDlhe}IwM1H?W+J8GcovL1kqh6dsx+*Mo)U{knG R?>KJII!O{ijegIf50QUd@ diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamJoinTest.class deleted file mode 100644 index bc9607fe21a00ef492f34c73ecf8c94f4cd496c9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6152 zcmdT|eRx|{8ULL$$xUuA>y|Cs;)XEYN7rsko0j3yt-4QW*R70fU1L?eP4Bw9rb(BN zj)H(FLlGE)vQ0#gi6DyTbZs{Z!w>vIQ6?gOgYXys`wVv_?!x_-MCx9 zJt{QZD@P6~xKBl$-T1tM`&Ibt#sdnzpu%rAzNp|q74z-JVFgE2EWkkpM^#)d>o3Xn zLkhm^!NV#R;VZRx1dn>~)ml7;W3urz1&^z!$Jb@?go-Ee4FylBSRpz8rmVjui>GDr zZ8_%|FQLDq;=4Gm;(PeM2Pa$CS@iD+w!9!fx01MY?F~TQl=g^Z!%H> zZC%OKU{D{?V|$I@kiKU~55|(i!^uQ2KTFS;Y;`0u#-NcZUQFP+Q9WMVB@nI<&*rUo zt>klG(xg-ZO{FO5Ov=!QDNk_7CWAeCJZp4XE4bDhF$WWRCYv$@_Ex!kZK={K>1pS- zwbVe?*rtz+RL8fyf&e`g^J*`th*!y(R>T>PKK@_brz2ye^!~U}hMV>_x%&)Snsq>V z)wEQy|JG@X2=h$!tBP65S#4XH(=0LjQUl9#QyHQyxAd?gxvBUU{|`j?563AF)VBE0?v)e0eV_p7aewccDUb2?UeQ)15h%i$9r^!7`Ahr&dgquQL4QXRhW!4 zse~R+2e&dv(C4L#?=;dGGU3UvmTduH^1nks>Cc++0qJ~|S(uHv!Cpj2&A4&Ih-Hic zo;F!@^hr$H78p)fRKjtNPlDwZ^3=Tt%)~(QfH6Rj)+#`D=avzy03qQuMyKO~s5Yru4Eo=lRAe zdFyNVF#0rX#ug3NV2i+-k|0;<-LBzU&=EdR830d|7jyAtuAviK>F_1%OYPx;{=T?b z3iJaFUD$T1P$3Q1VaKJSv}lN2I!ddCU0?{kH04~Sp;w|bSD@mYvE+iDz|xX6Wlf%D zcz!ae;fMGULr{5uU5U6kY%-SGZpawKkZ1=&>nwjvAJFh3ep2WT(Iqte6hG7ObNoVJ zS$W`+C?qZ8B@MsCuN1tj;n(<$z`XOBEyyc+HN1i|viPlrSMfW!+iMzrkJlCaLBk*M zCk21j@E81*mr9h}HHCYd8lz35n!~-#jZw1yH@qQdys6>u=;J^r+#70)GBztM;og?U zXp6ww67H)o_i5Vh?%&5tNuZ$u#GG;jLX}U>$4Pt!|$(xc(IRx~JvRBAVg42?ZDtB9b(LM5?d^r{5vp0EQ zO4`Wm8&R*0)eM!_nOOj^#xspZ`E$CL}-M5(5G&%UMRzl8ANL zc% zeXPs05{KEALZ)|9TWLN3SlK-9`I$0u*xAN%I1(B(XRWk#zYGx8v%`^Z4+bz1A z^LU{b+SjwqtGUqL$u=+9LVGt?@wzOuqZ}79U1IqWjva@S6cB5eglOvX63RF{B)ufnanzF3qMUl!IA)Npll07S%p&cR^z3oWA?=s++;Pk!Jzvs+alGfa zCBR2nHb7;$gj+8oRganMS%5|?#Y%+u0@jW;rcoQnb|*f@nVQWaZotRcqF<7Ihh*P* z4;0i=cf(UyKo%EvDbK0VnwsvY)7c$$xw@lnclWj_T+XrghLy&K6O`rzYMsyHif9d~ zn#csM++N!6lKo-l$|)?0`rbE*_n*RI2g0s^)9P7L=y3;J0XKnWoWjx?1e8-)=D_YL z)JGd9u{`Vzc)b&7s9CN0Ro5{D175l53ctGQBpRo%f_$y?t(rhkc1@sZ0?iW$-Jyyn z&zpXS7F+CADu&B#V6OII|iFUiqKxUzz;ezr9%gneHM5Pzx?k}>hZ0w zOexZkosfq71eUYh1rK&(A&qkhO>-UkX}B>AVgO^B0h*)#SYfHal5zy2Zf0{#eJ+F#9iWX+$~PvUhx7Bi8HKU z$Nl0B92W23h@%dN9ezCISd51q9eBjC6OTIfu#V#~%ZZ-GEK(tI7o*d?ou|arxYF(u z{bDC;2Ia-E}z5}T+&k7FM8k#b_t;l>c+ za8b^GQ2sbyvX?lnr8bgKM5|bX5o*MPqvAEBkY-fOP_M0=Z=+=kXxReVXaQ}sfOdf4 z;6|>@V#YZNTdQD9!A%NoR`3by?-m98yOq;!W7PX6n$Mz6usX+pe8JfYKAE3?#aXDF xA3BRgtln*{mMz@Qt=&|{r&v4iX{)>gn^-rH+4-#H&u>?Jn74m*xmvU9e*?C6B#Zz6 diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest$1.class deleted file mode 100644 index 880daa257d5bd62df476d4d4f63bec95480ce384..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1648 zcmcJP+iuf95QhJ4)3`M*Od5Y8Lr->q8<%*`{U|=5zvs-RqwD))0 zyQ=M{6BA2{DeG7yFgh+gN&Z-O1LdY8P2x&%NsMmtk>&^bqZL2p{ zP??KI8)7*yVgjIj zaGuS2ieAlEk8q)&TCPBI z%o02SyM`qmlG9+PLF975R`EcclMENJzFV%!{kSf(i}77EJYfBI9ZY}hJLcRgv+$J653tO zy(IZs9OV;Q1?1UMd4x=5gsCsc{_P+N)Cgg-v4c#EyN(+n_9jyTWN-_&vA~f|ZH-@E FzX6)m&+q^M diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapTest.class deleted file mode 100644 index 704c4e155f23a62c619c12c48c198e69d8fe7b8f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3706 zcmbVP>vI!T6#v~O>D`7!Xdqw(6|g{CprNfIq=1yCg+h^FG1k{M-Ik?!vDqNSw}9^# zzVL-Qe$erYGtNK(tLTj1^ouio)lc|WFe?7;CV>qJ5bR9v-93BH@0|0y_ndR{%fH{9 z18^rk526i&8XjsuD;}13BnUqS)ytz2j|CwMFNVZW5J3z|B!bW}EMW$5lX@Rf^3f^S z3rh|AB*uc6iDVEdq$M&z%vJ`nYTmB~TMaq2#tF~|_2+LORz8RWcsz)M7}sz}!xI{w z)bNy&KCPtBNIWa?oPge&PMh|Iq>;;+Ie`WzleH2%jFc%5>f2`=Fd|7KJsOEQwv`_3 z7MM0>j&C$`rfnHX>yT*+boXWK(TI^X5_`?am@zVDL=u@)DwB?sRvC`Px8Ag4j+(Yt zS>YWpl3tNOv_`ywOI}^V)vvbFma|sC*U~y5;NOrLW+c=4thBi!pBggl-NsPTjV+Tf zk^=_A;CoT%clKK3U`?e1Imb4Q6d8z&6&@oNqp{OQG8frKNBD8Ok=@>F1I%rO#U>a450af`^1#Pbq85{FsG zSg<5`yqb39(~gxg2dtdMg+1wX#&OF4&);;{t9^MUYDDcf9Sz1Zc{^clwp4D;D9@;N z#hH#ybO{99@}y%amI~fmZL+Uqe4GdQ76iCnZ>0BovYQB z<_<-}5rIV&AXT@OR9`-Cg(!3!#S7dj9WUY~9WUb*fyLF)Dl$?Te^uf&9k1gJf!SAL z7wZ*E-hhrb@s=8n>3AFOC`0e+cn|MO9M|yyK9u-K$H(}DC3;|abh(;3qNLP6&>8Ji zQ&+T0#|eC@R(-}MQ{iJJ)mNjfjE4T9eJq7F?p;}d5NNBxi6UzP>#9`8+I{YJyVr=F zNeO&i1*vL>^&ZhGmFoIBRf=qtdR-YsF=!^TkDW@usN!)ItulJMZ_tt~HlpiUyVS2D zWi9Cjt*xjnqL`cirN*uHu<>%16l*oF1HAL(&HfRASuMS-p3+^a=IocMcc+I1mbG}k z6J9kTuS!8wvM4aO9P!G}nv*Gr1ZK8W;&2C7RWaysp^!h`&RCSFU`v++la7ZAN_odh zMtW@981J*(Qa{5bq+HB|&+#(p-rCqarmH+F^oJf^OKqmBuI{I{vuS^xS1$H}maSWr zLEos!kKyu|iw?1{c|*B|5gj-kpW&KoGp<1ADkqaGu|G z8lo*!cM86?N%-TZ5jcf*^a2p>#!9}C zej2G$M#SIzz0g3}Uy}$DZc5)I^f(Pp-7dcm#(lp2xZmF&4+Q$-_4WNbCUFxNO^XJW zw4J02C(+>l0n_6?KKWu3XpH)s{7cUwREMU(SXQQ{W`#O@OWMvDYqBu(cTIAvVmT;V*(qA9H8yPw6X>DwAf5L zy+qzh#Ccp_z}QBNf@a?(&F$3O!BtD)zetxfiGGQl68A~$l88y{mbjm%=>j_c!4io9 caRIk8R?v>X1Dxv+cgF|G2^bgCfiwU90|S#jQ2+n{ diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest$1.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest$1.class deleted file mode 100644 index 90ec776f0be9a9504f9c213a45a2d710e78d1732..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1285 zcmb`GTW`}q5QWd?=ENA97ATZ!fzajxaSOaPid0376p_-0RC(W|+r*9QM7C3je+3T| z!~^1)AB8w}OmRzvB9%n(?u=)@%$}L~@$>690B`Ur2Mwz_vUs8+ji)-Ep`hWpPWPII zbqz%gC4r1Tw7kNGK%+f&2d13J{!p1Cc{q}$=etTCTh727&9o;mch) z=}LR5ynW^QTg25Y$MWj}&EkLHmAV3{#`r)9EVM00y_+8Ql)Eo`Ha$!2abMb9=~{su zIa9%P1dNvBD7Rrt&r>vauAWO_lebID$;5syuwHB*$rEYXlG;!RO0-pqxmuKIpxofZ zNuW-U>|X~2*5a7G-jVA2=P;2=AlI=5j`XK4Nmh&$16 z+3TJ0)a|RL73jTkS;?!xwOE}Ur*Drv>aeT);rKv9*+2y^4CuIKzzB9PQPp4?*ubVh z{-PWPZs9h&@XAeCI8UERBv73L9Mec(dyWJD9tVL%pHtDarGGXi0?A^~)frnk1AiEr zEe8IPBhp_$3tMOJP++;(x~#jf%302W?E#U`2TF7Dll&RDPVWuw>PYYl=;g{Mi1KG7 zKGI5HfzJ$Mz#`@OT%JW?5jSZuKBNgouqqL&#x-DNE2l_)LFxndaZ)u#%AS$3WZz+a z+a%qMNUwP=Amz%H??|5_a}hO5%3wHyB2)@1;UC_^{cwGNhg?&5gvVIoPG_{nm7ibN CwNFd{ diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamMapValuesTest.class deleted file mode 100644 index 2e442a671b0d322f3a9d299a65892507ddb1f30b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3500 zcmbVPNmN@^82&C9Zb)BSpiMQW4qzvy66kvvxg3{Aw;oWmXdcmMAn?mt|9`0MK%01n_& zKXzf%hp}qx#FG+HKin8qo2Mkk{gAoM(-LR=@MBcstREVlk$BdRO=_gTQ|TilNXl76I+_TiilmJb;pY(>o~>b%5r66Xc9{$$dy zdg6K}V`K!X?Nr)~4eJR*pl)zhpVvciJvkkU*p```?h@EMXDsv@8N)L5xOu^_1iA)O z)^tcu>#-RlG^bC^>7iIEkw_&&g;Tn1a@U`$Ffdc2q<5Uvm}r&2L5;k-x8 zq-h@&a5XlK3b=bxleA>>pqVs=vxy1AI;~H{9otedJwB?_4DRzpw>@Lh2R)?b29zkr;`n(;~H zy#1BP%1bzwV^UTo^mJOWw^zdMs6{QvL0-)`pEhE)F*#r?(AcVR>f_BBCD2+4US1ty zRi~{~%*bR?RHd<$Re2yZl#0!rEF9}Krj+`VmN~D~Z~7OWyaeURUgs#8d#;e;fD`FX?w^rZkT@c7ft7|OMWRo$O_# z)HrI*W|?0hbu)zZM^afUX7rgVrGj}wSL0nBYEvad!(Oxrc%0gzVGkMw4*&0>p`i)Q z0*5LC$oaLnN@!?7t3XEye`!$UEy^bzD?lcQd9m*Nc3+yg|DtpIt`|U+o!wYzk z*QVhmysY6ByehD#JX%Q}DdDe4ysqIQE@^lJZz_PdG`x*>B;M8V9^RL@tl?RB)bhxapiIk$%oX!r;pt8;eQS!3qN#4HPxKwAZ{MQJP0 zTmeJgB?5hA3f7&f-R1O{DJzu__@WG6**;hti>5X-wB{OX^e4aV)AU60Kl^MCx-OY!d#q845i_fx;pUd(Q!}M-EaY6}n zO&k1pn<;q}S!aq(Sx0d+@O`_Bk5Lz@RK4e_N|kMAX8Cnyq1pljLfnn-dfa?&sLR_wfz{D6)eTEeC{s*w3Bd6TFSW zDMb9i$96SozpTJdxDA6V(4rK$aY%mSkGk9=QFo1hB8>T*eNl=lw|?D89Ofei1nLtjvLB3$56wu5kC67mab^RL5`gmz=Iga zVHgPWzkn`wfG}(RQGA9+ut=`&(1V{i|BT~~GpFd<89MeLc_o%n@2q^viFpV6@FOt~ z(b2W|p2%V6roSZi5#H$lF5@BMyLtKw9;OrzW!vbc3@;__q`Y1tA17ix&*w1uh>_!b zbd~cla`y97E8MrJl277<#DK)0#E`_W#E8U6hR{vy_!BJ>kBgfOj#K3DQnv`_Dx7x4 OC$O8dpHz)Ae}4nMz52xf diff --git a/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.class b/out/test/stream/org/apache/kafka/streaming/kstream/internals/KStreamWindowedTest.class deleted file mode 100644 index f9ab049b011061f0559f9c4e2c3644576f149a67..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4011 zcmbVP*>h9Z9sZ8CBgvN~1{<8XNg#_Q+gODpq}av@U{Y+n5G?E}q@>r<#lE&SuC4%Q zBMVu&ByF?r3vGtUOkUb4Hq>pVZ<)zc^4@=;51r{#nhE{Rz1Oy6sU4-B@jd7K&Tn6j zzWnabp98oXpF|PIsW2XiAcmiXp{sRTA{|8tr_>=MVMGye52qz&qR?H zdAre{B%YPHC=js=MKe97=L~`7kvaXWp2+I?nZ$%`nfaMMf#^9ipDCPk zF*eN`3xh_wHxTS^?IBD|b(oMdE@M#-ik%{JE+ zn2RAGO#TlE$mz0~%_!&H)rc%l!aAE3Asjc{G{MdN&rH_o?lrW+<~Jv~3F7oALYDtks^ zK;k)x=h;TsJOtW~m-DunGbYWF$)rD!&lhZ`G?_x19j)fdES`jy!I5-<_(Qgl6WA7W zom95XY=R(5w+mKZoM{&@c@o%K@17#-fcuk4$H9>ab0)9bWsBvwr$5`~23!*1*m2s&L(Lnpcf!lMgr z9)TY<#=uRl%9f=efgcL&uA#3jgz7FJ5UZhaD(*x+Yvve~%xZ2b54^w-uW#)dUc}Ft zA{u^yUkY^8=jVxCt6;_MR~lZz%PjjEnp(YAJpF6@M&h>`UcswOsP)Qt&5iwWQp0O_ zU47or@Fw2U@HXDj@GjodZ~~JO?`!w~9}+Vq@sWm)aY@7P@Oy##8_?!|_U$Ssxk)I{ z*#KA1No>9iAg=~SpvBAZ%7N?5tE$V+n8VnW8XMjF9Nk* zUo)X-71BnjRIvC!T46(lWXZ6TY_4=fY*-1pVJ*YGK|>@ycFM*$Cv9tNC|=?6l~Kgu zs27VyJ|nQJu3~FU-6*ZS3EHbUuD=f31sB9#9jg@WLX`!cWo(CaL{}*5b>~`$o4I@k zu`yJ#@Cb%IvDHI_HK%IE9X$t1d^6g7;ONfg@R>4SSZv<0VaKPye)O%JhjTz@j z#W&wW^ncztR9oq(fE|1k$N3Hpz^6umdJ(G;;H>QA&dQF=)ep91?BVw=o^!bl@f`aSx#y znUf{<;$E(tA{6IS;(V7O5hm!VWoTr%VMP8Vn)3U{Q-Q#EDi|D3g+k+_%h;Ie{1bvp z*wh>7;x5eprodmZIpychKe31##&)#@uA$k7sby?QbuD3QZ?G+>s20)kH?;PK+Jccq zZ1eAxEi!ls3Fp$^-6DIgV*4`MDE-Ffn-EeQGv{5S7Xrq8N+Nd}N z__Gc;g>C$HGb%l3XN=;69$*xQ8M`DSH;sLm#eST@0EgCoywBS|;vl}jA^aVK_$Q9w z-xv}*a71)qL=0e5oWPiv!??JBV@^Ii%%6qx%z=IAW&Hm|DejJW;N?hIoJSw}nJeug zLn#BymjmKH+=u&_pm8xpsRyZXQgl=9Aaie8+>FC`fWJO}O#N>qmWzL66@Q5MF4i!y zjuI=5z;!ATmKc#3l^By4mpCTzpu};e>^Io`Ehfb`RCR*rA*z$)>Qm#{O<=r0n4hp4 z{rkJF;ZDIBzWJh0aNi*?b@}=~+CSs1>sq%&k`8?nIc&rfwj<@ZL@7)zp80Awg`beO kj~L21KF2J#kknm@cgh3_-?#9I7Cgj*4<`wJ*xCK`KfY&R^#A|> diff --git a/out/test/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.class b/out/test/stream/org/apache/kafka/streaming/processor/internals/MinTimestampTrackerTest.class deleted file mode 100644 index ea0b062309de5a94a14c3aa0c7fd44fb3766c370..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2134 zcmbtV&302&7~NlPk{i?17D_7$6%mp2$3_7QnA%dXr46ZZnNq2&qAth!0KS0_-~%Z8oFq~rWf{%j-hF<)efIZ#`=0#r=l4GWIE|$w zTuiz+lfc;|A~@&by(D60DjVYt^D8Iu3Z~j4-3Hg&V66>GJ$MQ4o5yq#GdOS1tcwe= zzgj;R7X|St&-cPI0>q$(>a@XWY184YiE zb-_Zu7OZE~mMY!QnN4+lQ)TL5pjFlL*E3r|t)%PqT9EPlPzS!M)H6$-zwA|YJyg}L z~kVtTlp?p7)H)4)5WN2~+tL4&kt1vP-kId?A1#_&{)>OM+);VkS?#&}>Hv zuj5z>bC^%z5*A!6rjWzs6!MrC%yfBXrxgVAT{1jFwW+vZDE&_><;=IARORvswv*ojOV^>;KSul6A!P|f)Pv@+FUh7=l(txA9=v;f} zIS|u(K8HEZ@Xc-_Ho60OKoG&Je8%nMK^!4YH8{ z=kW*sMTOT&FD}V27GwmMqrjIsYIzJ*$i<==e_Oa2h6 diff --git a/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$1.class b/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$1.class deleted file mode 100644 index cb4398453048de6bc9e41b7bd9d500aba9494242..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1081 zcmbtT+iuf95IviiCNYMlDNybdYC>Wp>>wTxNrRvQQ7I`8iQ4ClyG^!nZOgmnzmT|z z2R?w0Ld-fKKzTy3WY3=C)y&R0JKukN{RZF(yc(*wZli`9Rot|34YzD!ZdY-~fb9zI zR&bADP0yrq1BT~=B%As?<>5^DbAB@CzS5cCXEL7pX_kaSsU-7dtVI^{Ncp4U@0hFk zSSa14R6Q9>-Dh~&S}|AqkiptZjs?T|K*r)AKbweb%qI~sjX@If=#Xd9;9sO=+F)?@ z<5*;S5m!nOe6iX&+zxeB8_8+Rb)He_$>5Z~<9@DXM=D#XGGWAK6S8*66ii!<|o+ZZ>>^QJ--hl(xc$%a#>^QiO zrh^A){iYr}XroD9uVl5`Vc1v_f?;>1tPJIr@!jU3xoU0@(50NNe3O>V&~#hd7n}88fEVT*cVuzaPG)!tSwN*p!c-1gmwBhVA1y(lbz-h8vS9<8 hgeY%Om1YQ+2$gV|c9^(@s}wDof0GNei_uAo=O-`pF;xHn diff --git a/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$2.class b/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest$2.class deleted file mode 100644 index bdef3ba95ae9a10a4c8e6a5e3a0beb2da33c67f3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1275 zcmbtU>rN9v6#k}Ex|9lKxp>E0#Zq9Go1j)DP(jh6CN%hax}DOY+nr{1O04hVL-<2Y z{NV%mP{y+@AcO>B*ktEi_RM$Ane)y5{`2z}fJH3#VF-7VuyHSe`xY`59wf;%Vd0^L ztc6JnISW%3rY+1^a4fhMW)ql8V4k7Z)MS{MWmqXHz3cKO_iDne^J<;Dq0xdjWU%Wt zwemz5D(%X^2pw=gbhkTSOFXQXL}>B^^-KoRykJ<%UJz??hap~26~QoAl!18HYLta8 z@v=`IyQn1r*1Wxav^RxESYk*=h!t@hGyk8MFHh`F?{mK;j`BxE z+FjSlEW<`Nh#9=mbi^m4xn~rGPM^wcD9G7TUcGb}zy2?LsH~JBBJyUGa9?N|>wq6I zpr%6ebbfAmj$ypqs#b*-6-NtC=}PD{g?5@;8yOuA16v@1pD~8>`JJJ=G7^p3Z5ipi zAGD$rMn)`qej#9no@~VT{!o|$)HcK8Y_XHf=XCpsB*c*XY0u;_*ZcCG@Jt?e=w7Fpk?K3&NXj^Z)<= diff --git a/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest.class b/out/test/stream/org/apache/kafka/streaming/processor/internals/StreamTaskTest.class deleted file mode 100644 index 19fe162e12c70106951c3b1d84e37aa7ce39aa13..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7533 zcmcIp33wFc8GirCG08AU3=xa>fw}=?Au1k0F^QpVL|HAyY=XOY7cvBt39>0K7IQ1{pZ-7VJ9Kbr{UR|`M>$@|GWO- z`Tsoi41nYDZY|EiHS%(;h7PP(@o^26ST7+PR9vS)^%mEwxIu&FEpAk?u?C;eFc~*V z%$sX)i@bkQ#jP4{!>2S%#U=^4UBw+5_Q#zn?o#n-6?bbm5cg=f7oX8E9rsD3`&B%k z;y9M za?t1>G?J<8&`>s$%=uV`V{uiYX%?vMv<5PUlebNQv-X(5f{sjAIU% zcJpcEI)$p8VFqHQa>q}QcbJNd+V_HNOr?5995Yw4USR$NxtC}btZ0??X~Tx?SkjyG z1QKI%>B~xXN%l4Ti66#muWQViq?7)aXYdLk!u}x#h;>Hi#*_4^fW$h$)-Lx}C zI+t7;z~};(2RT=#vYCEsKw$QG+&TlkZ7w$HYDkyM56LK+TTYouTV}?|G1NUI*CNuo z6r_&Zd((5}9GYByp7Bv&l0b}7ae8%t6YTj?0NL4oFSOUL)P=eI5rOII(j zGM00iKt-alTcEOyd&{isurlU(`JrCZ?lOAQgw%IrQ%1VmU=iWk2Ua>OEk@+Rz1N_N zkwq=4a;B3X7MQ*)pK+`qv)js96w{K)WF5B@Q3^F+(OQ+Ky2)0vKWobvtdpThXUecz zV)w}{m}Hx&tligbr1R8US&6=pId}eNS zG*WWsX3U*CS70EfJWe!{-Cb=g-!-|*<&{ghgyJ+`_B<(D^D5h!mW=hOr3{gL=g7{b z6t<*ee_X|4ShYJEwla3y_+ShIE;fAE*MIA5Uo08^RI+kOFz>G3^F`FX^ zzAbQi8SeOj;8%dG9W)Vv1$)avb~o0vJtU8Y%na9C-%;^h9pA(ErBy%B@gW!jGsj`< zF=3}-a5xr{ni{ zP4a$S$EEnNjz8d!I^Mu*D&ExbC;VB*U+`BQf5YEZyrtu9iTw|}qvN0W7w4@qp%b-7 zmK3(6(m1@Nca@nUXj&rFLTEqZpt)T(We?ZQFE^FhPpXYvhb8$O#cIAQtDqOT6cUYn zXu8kbg|%hVX^P}f+AR4G#uVPz^FX)^#f%-hr;*ZHW2Ehg?_8DlE0vs*G4xA~IW+8y zoNElTDkc1O<(foq#{3CO0wHtr)cyiALXMY|+V~R{n0g+R4mD~3P8iGV} zb)^?%VItS!$<{qQwDTkq!@GTPIrnmij26x4>+|X}^LSxgh{w7z#H>9SyCpl7K_h2N zmQR_h-npTL-$QIxOVR=O0B3ezn zGxy|D5h8%IevY44yi=%SSQ#?jFYFPo&NnwMm$tg{yKH$xsZ1XLl6V9pCg>!^;?;J$jJLdxT z&gNdk?@y>dkUx;MKvE)KGuw)mX8`A zCyheqGKxBHwXe6T=W1$C)zd_%n7I{oojsMj)pu@3Ljn7ZVjAD?-$PKX z{1xz?Q7GjgLC|!`DYoce$89LVR0$vlJhZ44!Y-G zTm+jdfgF9WNZ;dZRLOTZfZ`M7p4*0GNgItcV8l#EcZn-?^0+#DS501d-UU_%5|lTo zIq8%|lu#||wC$L?1M@1-!Sx*e>Sk6yji#Q8oxI59HXP^Oj@Njv_CY1OSp}THEh$HD z!2B>lLm1Ug0t zB03%$At>p%4GY#b;MDC{NNts^IDIYsET4sZuJxYhcvvlCVEGh2)=42OY8~Ph#q-~| zG8^Dixl+S;{&Nggk+&b@Nocj_0CBx!%fR^a9Tr z@A9NE3;kjl4=oqK68%^uYz&IadFtMPjJOS1aUU)cTVRXlkP|N=FJ8uK@ft1`Z()Q{ zyGGdumnsM0G9`h_mHD_rIRjU^!9UFlYCrhvloN0fwHm*+%6Py@{>z-;qStAnK z!UPOruZ+0aT{3GziICWaMLTe&K#(O8K<@QBaF)XVaXFjM~$d)+sJV&*MvfR(naPF4a`!8dd`}LwN#HpxD zJTIB#r9@tjBbX{{naJyy$k#BDuf=72z8V{_3D+@^ugAY}gE$s9iY{yr%W;!fiJOIk zTf~*PRcypGMGEj{15d#0 mP$#a!M`eE}@!D8oF#%#S|ClRc9nR&wp6kPTmwyL(xBmie6o}{m diff --git a/out/test/stream/org/apache/kafka/test/KStreamTestDriver.class b/out/test/stream/org/apache/kafka/test/KStreamTestDriver.class deleted file mode 100644 index b4e2ddd5a4741806d524d89d64d2d83b0853ecf8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2796 zcmcguU31$+6g_J@QEU_vQV7&gT1ZPB$EoTzDQ%O6e6>L9Bn6w4J}QdpsQ!>gl2gi0 z;5YEZ3o|^?8S2c?8F=SMF`QjFk&E2m2?Gya?cLpb?zwwE)_?r@>u&(&@GyhdaU+8< z+{|DBi)o6tG%RH>gioZfoWTlKrMM=CbvfLY!yP$%s^K#Yp9_oxZp*E^)u#fhCD*GO zX3MnpY@=pAs+oo#c(&PanpLCaxt8twu4g#S!1kJE-8b$Zu2s4i3j&(ua{rq^pb&=( zY(FqIT&s4ZxfY(L;_#x=bb=*;Yq_{$#G@8YjeBK9ZME&xcWrNBdRrj1=I+`8VRg#9xt3YqHa$nm-NjU}=lBBCabt>Q7r#Qr>z=bu6ZCB-v?g#huJw-; z1-?ItXK_oG+i18=!?)>A-TBrG97^S`M4|&~tlNhO-TTW#vcQ!AQF`nO=(|0b0&@dm zyu?Ic^5t+nZ5@Veen7m4(_(&!ar<~cyL#m~#lj+9=vmp?d2CxjpB0IcW>aviO|zkjCk7*stl4ZB6LCC&p?$O7 zrru~dFs<5#*;4hU;V#Mfc2IF7XexI%v_r`c%tlK<@W5!*4yYE`<-nC!e!GtkeF`=z zy{Jn?Fie0{Q@N5UyKT?1?>Mq4OvWnwl5|wZn>Ztfdm2hQHn6E<9B&BZW6klE-At-) z>6pMN4fl0?fiHEGQPJ@gwskzf*E%kvsI!aD3ta4v8)GLh9t9$dp1`SSsj}y}Pr`ne z%*o3w#(<-H>fc2fJk{X@)^h`1^YM>?)2C$qtM0XYjJr`<9K)ASkgxJXVC&?07a0hX zZIf>jGjOEU_V?smV5QHcM3Fg`ssdAG{>b&7nBFear)xaUqn|?zz6(; zF~l?)(lxF#F%)I7w+4DV_mF@-splKs@6-fwwtk)cQGS#CnJFWOxpz zk)=cpyo>jUBGJxpPojNC^c26@`~edAnGTZAk;=co(8I#)0fwJLllV!6KcP-eaXb$l z7l@lvcJd*@k)3SU4pH$w&MFpJoFl6wJ|tSmOYTh_?Vab|1@0yA5&2yta_H?(Tp!|k zG5-rR{!*kk(m`5^h1m`=a?-(T4}Qe>&D5cAI`#WALXDB$6qTn4Ex(}~SqyVN!`)f# z8zgp_yN2u^J?<2b+$mxjdD0;XY84daB6SPIyTnxu2}^^a^N559d9QFY$xTJ{YzO+^ z!!*Zm%qx50b)=8%rEvvU3B$*pCqlSev?_zH;CKb2KT`;!KFlM)^;bY$Q`GrhtMk|Y E0vV+UhyVZp diff --git a/out/test/stream/org/apache/kafka/test/MockProcessorContext.class b/out/test/stream/org/apache/kafka/test/MockProcessorContext.class deleted file mode 100644 index bbb3e073720134940a4fe12794380e4d7942cbab..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5842 zcmbtY`+F4C8GdK8$!s>01R}S9O0X6-o0wGu1QJt$fCd7niJ@pO4x33bB-vSJXSuc2 zTClCX*o(c`i?nJhw#8Oj5*n$}+tVNVJpDiVH~91^eZMoayOZn$Rth{bXU_S)_q&|$ zd(T<^`QP_G0I&srPa=);I$lg7hVzZ+#Fuq^C5d|f5hA2;DA_^CWz({Vx8ekN-_ zm(lAY@J0e}Ch&^{ewn~q;?PC0^|p+DC68b0_>GQBI^NNcELg5>PTOZKSHtEZ$DPQS zc{4j{Wv0yWDKnFGW@emRrX*#0Hou0;I=vcXa<=1DVC^z7-(F4r~B z3|J?MN@&=g$eCWzrNz-mp}zmmkA8ZRD0q%*9W?VAT0=HC!Nrk-C0kf*HwTV|LP>9+R_$4aTxvXfP0DvQ;TjWaI!{s~Z{Z)$!ZXDSJW3 z?=(EvcVK6=jnRm?{uEINA@)->c{(VfN#*KbZp=DO@+TE+`N$R(PJ{5J?I}W$I`W*n z&0<(lYKQP4#RTO|*R$nTz|+PVA15^_O&&gpe6cX;r*2n`Rln4^F7fGkC-0*@64iWB z1+x?!lQrxV-K-^jt)V^Ykm-@$H_!`ab4AoW!oX%cs9||!?J=`3$-+0V4G$UEft?1r zai4)6WLR@bP9^WDT%nlHJFaJq9nM>>x>M~vowf3Q8rutUsFQnIENh&%6U zSQ9Wz)w+hYAsVPsHgQ(RWdpy*9}HZ-{*1p!fDah>E8aKoHwh?R70{A57M>=R?e|$j!`3SOsN^rc{E&ev6*tH; z#U$-(xTAtUGU+;}{42o9FntXB7fb1-<_d(mpw{J!Y}@T&>7{A0n#I`hWO)r6BLIR! zR26=iFB6KZT#Ka9R^k{awUqkHZRn*NQ&S5==4YSbG zE0CB)GTk+crgZl#jLVd-LqDH)L+7&xaiq{n57*MS4fJXwtGJ6^eS)hUK0Y47E`HIH z@?TRUi6DmE9IxWrU)w|fWYmKJ&iT>o<=j3z%BT3;ANbsmUQfToo0#%Oes#*vJbF`*$MHmjC&pq=hL!|*lAa9L2J*yB zK}PTtp(H0;xTlhn6MuqQBqI_JFhG)6J%=^(XxDJ)zIm+G_#Z*{bFfaMw|DN0r`zLm zxT`(h)jfy1({osV7C{&ks{o7mjjh_?;f)jO`*_Z}oTcY*}sQo^Im zNfO#AN}Ob*r?CfTl*eh(%?74>f&>#6ugZBCz$9#XE_25@3R8=$O_kTCxhC7N3Gx4+ zS;x!`GFZKioDN6*o6(VnrsKqiX#9}Eeha!PEd_o{&Hg(`%jO_0cg>?icJ0=AkR+^O z2%D4(o$Bo;!DrOx$mY*e;swf_!@W39iE|A1MU? z+!KNDawNj%%Iv+6H9i)B@Jb}Y=kbL|AMTAncr_AXM;Sp*uo9am-(iyd!^l5GUJEIg z*j$raW?+6lrDN2Qu8()k;p1}3Zmb+>KEb@s&Ul0Wd;6wBtL}ce07%F3&Klqz32@g5 z?i@C;D2LMiIK?q-bXBOuH}V#ma1rfzJ7i>CP-9J4CihZ;$K(PY7zv+a8!K>rr&G|6#kZ$hQ@%En|Be5P_7X~Pzop%L9hZ^21ouaVSzwNI!VgNGx#*lC^|ah z1NcymXOq&4=`eLVv)Oa@?03HNos;zUpC7*fOd=OYD;}$8!(<#!Fr{ExMHG{AdD?V10zo89yil|ZugE>} zM^kIkr3S3G%oFmlHNrFUwkz>c@^~$r@;?i|=m*hKb9T`&#Ih-etE;9zG{9rBI1LG0 z)6foG!IFlTSZ26hsXFI5)TPCuncowRh84Ub(lc_is$mVU6=XHM!McVGEGyX5@D}eF zCL4Q&YKM^_QAr$hhhZ#)bZOTa`a>7$zP++H!RU!S(k4*7=$Uza!EyLm)^t4*Fvbf7 zVNt+noyi%jYE_L+bg?ErI80v_-{RRp6hmh!97u>AzBaWCX=wPKq+gHS4)wo0eiYPJ z5jSgF2lQU}gS@d%*fg3|#mM6{dg+_m2Kr2aB@%K<=u_KFb~`#~?IddxP4olQq2aG! zL*LN!nXCx9XpK<}kVH4F-9Z$t;}%&I@1rr?roHCJsdRRSEQY(dS3+MQTcWdnK~q(r zG^{01h92Cfs})dr7J&BR0YOR1pNL+Bp2?vvU|$gViRg!KXdXUCtX(-r%Lr}G5kH6e zzLHk;d5jRqsGs_N08vhzq&oZjI+A!uyC$DRB}$TzS|rh5jvAn-6nUF4NWLKq(<%w> zln5pQf>(OJL#xMTocc0RB}AgkCV>&+SuKP-3xpVhUdl5^xnzez>Q4KhL0c}=4Qm=H Wk$;47zlaIkq^C-dt@P)51MDvWhR7ZO diff --git a/out/test/stream/org/apache/kafka/test/MockProcessorDef.class b/out/test/stream/org/apache/kafka/test/MockProcessorDef.class deleted file mode 100644 index 0a7f80b8c5a95ddfd7f2526590f4d3bfecefd7e5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1161 zcmb7E+iuf95IviOI1cG8ZOWw-8d};Um1;nJB2gq%KteSaDUqc1?Z1c9#@3U576sDnNa0ueBngZN4uGnDKze!;C!%f9teDSm0o zK-VdzCA%Kip%M($X)%o$>~pVcwX~96x89_@{2=sP9daF!@BAy#_LH=1LwZs-8Pb)i z!;m@fkE!s2Ej{r*JnsnA=AAwvMca3I-{DF|=LF2?Q%Pf0Y+rRPKH%=DuzLKY$1N=a zZHKe~8pHN%xeSZLCWea^i^K42BKNN<)7T8VGw(;$L785b30h}*YfI6< zvLQT%#R@O4tqyhiqjiW;9NAg0d_X*NbE%?Zi~i73;_ Wd=K%6PAP0q6QKDS{abRl4fYcYa4cv5 diff --git a/out/test/stream/org/apache/kafka/test/MockSourceNode.class b/out/test/stream/org/apache/kafka/test/MockSourceNode.class deleted file mode 100644 index a36245fddcb54739c6b9026d40b797296932cc7f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2292 zcmbVOYf~Fl7=8{31j4r1v{0?37mFnUl2uzTT#TlHY9xRHO{JeaY>w%cWS7}&n86u; zf`7#swGA_N#vkC1a(vEi2#Jg0knTf@MRJP@|q~=<10bo`2}x0JweX>)b$TY>pdxaHe*({dfF<$22SP3gN0+cFEm zRdRe)Qy#61JFUjHvXp(Kssf@!p9ku=DKOb%rr>$?--kl!Cj!f!NO$80LkmUa6F>D#1M z)hW#Y2GPk>7-Z3{GJuJ)?WoOGV_$i@a=%Vas_a^_UXk=jy2Fh7&us>BKkCg_P2b#b zt%Du6fs!4x;VRG<`<(17eYF% z?O?j0-_{_74jm7KSbNnaPaJx#rJAhcY`SNjw)S5r%U_Oiv`&nL>>t05IL9q$Z;CLZ z`mE^LNQddws-r7bz0=yUv}QC`)|aAQD(CR>(KwsFr(}bJZ+4<^J=5k8dycF(&9mm~ zsc$mmKDKpRU+wSFoZcq_*Z;Xz9<}UxRe1*PVo_iwGLYwVlz_3ZRb0>RY(3d7u4N5u z;cEle@uq=qu&v1*1G{*lZB`7tg&RZ@_zgBxceNOJ8#fKy*2hzDk7dq;E?8sQ^q#(E-tWZl6K!kj({p<-F&_U;Lw$+M z^KHEGl#hu8{Wyb}V#uJoBo>&`ER&P#fjHfTIE@eRAaFj-e;p6{AZn=)qL%uwEA>$q s;>Q@pCk$T%X`TrH0>*1x*^8eb^&3VsB2GTfkwE`;S9b}^MJqJt#|%MMl;3i~Ql_JAR~S#2`p4)u*> zm}sa_o_E_VX|6>pAY`hcT@f^eQE}b_bM{U}3|kFtJT5xIy_4J*x4z(3MwXvx*S}V6 z3Prnf^k|K6tu}iMj!YAV-Oa|mcn~}gp~tVRQK7f@3%O|BOV!dv~>Fycw)H>M_$~bvO zK3{tSdr6*4WD5y6hf#{lX#x2$${5BmK{h5|l30#>vHpq7E3(fNXVB*_CHz`i)8kV$ XDws?(iow;bYz0 z)6FNkxu3>37E&xbAjZcbn*>#BrsWhCJ&|M zNO#}*YWJC{g=K-OdN~NCA8y-CB_P%Xu4tKZ>!*@Lcwh(l`6QMF#=mwQyJ?52{*CR{ zz3&CgHP==CvLk~)1p*JNp1*I&maH8pt0DIq(h5}&S{L+I)L#CAh*%O8+qJ_b0W~|i znbSkIL))>|LgmZQ^UJ*lUtcfm2pG#=oo61e+OB%qZtg07TkblPOjf;`batd~>v>0L z=s*ZmM#h$(=OJT1+!eUq52Zl%yxXHz*Y%Hm_V>XNguaqZoB3_|UQGpo=UcYRD7exI ztgTQsTdF=#uW@R^+E7P3(rK&8_C|Sor@R#Xn4vw}p*ewSp9~GpVjUYMs<|DDSuHhJ-SObqFtS9gyufk3ej^;l&4)GhLKslA z*4u0o)B^eT4%?fNr5xVnNPjs^&}cn}TX+vKyiaMCy0OJZ zVEEaF4V66hJc^J@jJ6{#18SU`z`80Io7#Cw|y8t*D48gHstdI|Fdu4@1B$p7mB rGNWLmI)1gc^Ae+D{0=Ox#c&t%>6x}zErX2>Tn8vpREt;m#6y=>#h4PTL0+LcFia1Sf+M($rYcfIjq~G+% z_(9eZin5kJz#paVGm{sYq2`0>TDgxocb|RE-RDgD*WZ8q3E()KF-&05Kn6=GEE|Yp zNp`;6gKM~+!j0sQG?S0P-LH~A{1bdqxGAZU@XW;Z>B%<=>4p;c&kcCD~%J6pQ4 z%>AOmSjpaSEU)Eig`Xmg=9cU3M!{-Wb+cJ_3%28F z*Rg8NLdmnX8@jUVxvc=Au%sWaTD6v*TUjcutQP0}KZOx*({4`AD&)h?J>B#Q17Tm( zRmL(hXFIkxuW&Rs{IRK3CTXEwVf_1<8GWs_y{_FAYrV#u1IzWYRa>=OTh{H(gpgA> zJG76<0k))?c^UX|qXT|nXw**1X2R$R$paZiq#^3+Zgehjq0UQue<`8dx}dz zEz*>lQ8<|65Q#SD3ySB2#rZ0IC>S!opsRJ!dp?R+BozIO(X;EVkOFg~ok}pv`raeY zblk@qE#5^g!dE!RMag{fp;Uy#w@SZ;c;L)`lyEHZpU$^=7Sqw4a4XeYZdqTprB>b;&>?GbMsCr> z`#5MKiv<#o>Ov;o#yct8H}M@Tg?+uNU#>eFCXV71wH#_fujm7V#Ws1&4CfKoge zf{BuHsF$UnO;kZLS!|lH@gRjQ6E$oroDM+_$xLCq7iv%ztd$U2kKoJqpm1Ur+g@WH ztZs!-S8s?QSP#`9TLgQ$r%NnRUEED6J+C)eskFY_a>{-?8Ra|5uW|Oij(YvKQD5!r z2r@(FvKZA(!YW_%t7v+cymfNic`>nxH7uK%Dh*Y@Q@dqn5Sta0i9q!^#;pqFIuY?$ z3OE;v;1zjdjap@SLr>1m@^v~te1?k=Ee419TO|Y5I%51|*>e6af5XJ^-uSrGIp=%0 zm!+k`2VC9fua+3ciTw0asQeCM`4@<WI( zW=Kb(sbB;Njz)eujnV85j9@X>^bHgkn9w$G44)89`~*RYmH3fo z1$;`YAfv}QI}znMo5r39&-4GQ=aXH})ppFLA9J+tg+AYAH)^vRwHb<9tb(Y;sz2(O zH2ikHA5jn^%uM_~r&<+kf!|S&PwN|1;SNb?*x|}jy zP8pn|E~Gv(eum;cr$9YoA8-xCAG{9YfF3PH(E0Cn-dp{yaHF5z;U)jUUuGZz1a p-~F|x`>5J|gsk);N4CV`^gu6T7o+mtpgyKWNu+%u@@3}qTs%JNSSE^o>Aytz~`gMjC zEBd(9Ch_j5OX6HZD)DkW>a`>RlU8%>D$%KC(X~lA5BXjT4|xNB>~=&rGSc_NuLRwNA8Us}wDSAtRONbLEin9eN{T&+}cu zhm?uk54(04CK0nu*;CvfyQF=19_^GXgFBA!ykezNVW?Zt_2n#TBta^t0@$nABV7MDm|EElbxTR10&Uv<{jdl7qQ(#EbwWKSjPc}h)6Bp^eK%=*a z(H}6%-w`Wse*^nMqd0POU!WG)AjKxKC?ZdzLijcoiKl-?KxMc@eUq4aW)aKOGU}~N zR(V3LewHkM1^Y~$Z~<)%*b2a*I5`0y - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - 1.6 - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java b/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java deleted file mode 100644 index 16492b29ec2c2..0000000000000 --- a/src/main/java/io/confluent/streaming/ProcessorKStreamJob.java +++ /dev/null @@ -1,59 +0,0 @@ -package io.confluent.streaming; - -/** - * Created by guozhang on 7/14/15. - */ -public abstract class ProcessorKStreamJob extends KStreamTopology implements Processor { - -<<<<<<< HEAD -<<<<<<< HEAD -======= ->>>>>>> fix examples - private final String[] topics; - - public ProcessorKStreamJob(String... topics) { - this.topics = topics; - } -<<<<<<< HEAD - @SuppressWarnings("unchecked") - @Override -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - public void init(KStreamContext context) { -<<<<<<< HEAD -======= - this.streamContext = context; ->>>>>>> add an example for using local state store - context.from().process((Processor) this); -======= - public void init(KStreamInitializer initializer) { - initializer.from().process((Processor) this); ->>>>>>> new api model -======= - public void topology() { -<<<<<<< HEAD - ((KStream)from()).process(this); ->>>>>>> wip -======= - ((KStream)from(topics)).process(this); ->>>>>>> fix examples -======= -======= ->>>>>>> fix examples - @SuppressWarnings("unchecked") - @Override - public void init(KStreamInitializer initializer) { - initializer.from().process((Processor) this); ->>>>>>> new api model -======= - public void topology() { -<<<<<<< HEAD - ((KStream)from()).process(this); ->>>>>>> wip -======= - ((KStream)from(topics)).process(this); ->>>>>>> fix examples - } - -} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index a67ea68c8e68a..09dda65fc3cb6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -54,35 +54,8 @@ public Iterable apply(String value) { final int[] expectedKeys = {0, 1, 2, 3}; -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology initializer = new MockKStreamTopology(); ->>>>>>> wip - KStreamSource stream; - MockProcessor processor; -======= - KStreamTopology initializer = new MockKStreamTopology(); - KStreamSource stream; -======= KStream stream; -<<<<<<< HEAD ->>>>>>> wip - MockProcessor processor; ->>>>>>> compile and test passed -======= MockProcessorDef processor; ->>>>>>> kstream test fix processor = new MockProcessorDef<>(); stream = builder.from(keyDeserializer, valDeserializer, topicName); @@ -101,4 +74,4 @@ public Iterable apply(String value) { assertEquals(expected[i], processor.processed.get(i)); } } -} +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java index 3b4abf9d3f408..bec524f37397a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -50,35 +50,8 @@ public KeyValue apply(Integer key, String value) { final int[] expectedKeys = new int[]{0, 1, 2, 3}; -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD - KStreamTopology initializer = new MockKStreamTopology(); -======= - KStreamInitializer initializer = new KStreamInitializerImpl(null, null, null, null); ->>>>>>> new api model -======= - KStreamInitializer initializer = new KStreamInitializerImpl(); ->>>>>>> wip -======= - KStreamTopology initializer = new MockKStreamTopology(); ->>>>>>> wip - KStreamSource stream; - MockProcessor processor; -======= - KStreamTopology initializer = new MockKStreamTopology(); - KStreamSource stream; -======= KStream stream; -<<<<<<< HEAD ->>>>>>> wip - MockProcessor processor; ->>>>>>> compile and test passed -======= MockProcessorDef processor; ->>>>>>> kstream test fix processor = new MockProcessorDef<>(); stream = builder.from(keyDeserializer, valDeserializer, topicName); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 270123edc4678..28673cc54636f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -82,7 +82,7 @@ public class StreamTaskTest { @Before public void setup() { - consumer.subscribe(partition1, partition2); + consumer.assign(Arrays.asList(partition1, partition2)); } @SuppressWarnings("unchecked") diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 06cb829c9e70a..4466051d25c2f 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -17,85 +17,10 @@ package org.apache.kafka.test; -<<<<<<< HEAD:stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD:stream/src/test/java/org/apache/kafka/test/MockKStreamContext.java -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.KStreamContext; -import io.confluent.streaming.Processor; -<<<<<<< HEAD -<<<<<<< HEAD -======= -import io.confluent.streaming.PunctuationScheduler; ->>>>>>> new api model -======= ->>>>>>> removed ProcessorContext -import io.confluent.streaming.RecordCollector; -<<<<<<< HEAD -import io.confluent.streaming.StorageEngine; -======= -import io.confluent.streaming.StateStore; -<<<<<<< HEAD -<<<<<<< HEAD -import io.confluent.streaming.Coordinator; ->>>>>>> wip -import io.confluent.streaming.internal.StreamGroup; -======= -import io.confluent.streaming.internal.PunctuationQueue; -======= -import io.confluent.streaming.internal.PunctuationQueue; -<<<<<<< HEAD -import io.confluent.streaming.internal.PunctuationSchedulerImpl; ->>>>>>> new api model -======= ->>>>>>> removed ProcessorContext -import io.confluent.streaming.kv.internals.RestoreFunc; ->>>>>>> new api model -======= -import org.apache.kafka.stream.topology.Processor; -======= -import org.apache.kafka.stream.Processor; ->>>>>>> wip -import org.apache.kafka.stream.KStreamContext; -<<<<<<< HEAD -import org.apache.kafka.stream.RecordCollector; -import org.apache.kafka.stream.RestoreFunc; -import org.apache.kafka.stream.StateStore; ->>>>>>> removing io.confluent imports: wip -======= -======= -import org.apache.kafka.clients.processor.KafkaProcessor; -import org.apache.kafka.clients.processor.ProcessorContext; ->>>>>>> wip:stream/src/test/java/org/apache/kafka/test/MockProcessorContext.java -import org.apache.kafka.clients.processor.RecordCollector; -import org.apache.kafka.clients.processor.RestoreFunc; -import org.apache.kafka.clients.processor.StateStore; ->>>>>>> wip -======= -import org.apache.kafka.stream.processor.KafkaProcessor; -import org.apache.kafka.stream.processor.ProcessorContext; -import org.apache.kafka.stream.processor.RecordCollector; -import org.apache.kafka.stream.processor.RestoreFunc; -import org.apache.kafka.stream.processor.StateStore; ->>>>>>> Refactor Processor and KStream APIs -======= -import org.apache.kafka.streaming.processor.KafkaProcessor; -======= -import org.apache.kafka.streaming.processor.Processor; ->>>>>>> wip: refactor StreamTask and ProcessorContext, fix RecordQueue timestamp tracking -import org.apache.kafka.streaming.processor.ProcessorContext; -import org.apache.kafka.streaming.processor.RestoreFunc; -import org.apache.kafka.streaming.processor.StateStore; ->>>>>>> Some package and class renaming, fix KafkaSource constrcution at builder -======= import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.RestoreFunc; import org.apache.kafka.streams.processor.StateStore; ->>>>>>> fix ProcessorJob; rename streaming to streams:streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; @@ -109,115 +34,12 @@ public class MockProcessorContext implements ProcessorContext { -<<<<<<< HEAD -<<<<<<< HEAD - Serializer serializer; - Deserializer deserializer; - private final PunctuationQueue punctuationQueue = new PunctuationQueue(); - - public MockKStreamContext(Serializer serializer, Deserializer deserializer) { - this.serializer = serializer; - this.deserializer = deserializer; - } - - @Override - public int id() { return -1; } - - @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 RecordCollector recordCollector() { throw new UnsupportedOperationException("recordCollector() not supported."); } - - @Override - public Map getContext() { throw new UnsupportedOperationException("getContext() not supported."); } - - @Override - public File stateDir() { throw new UnsupportedOperationException("stateDir() not supported."); } - - @Override - public Metrics metrics() { throw new UnsupportedOperationException("metrics() not supported."); } - - @Override -<<<<<<< HEAD - public void restore(StateStore store, RestoreFunc func) { throw new UnsupportedOperationException("restore() not supported."); } -<<<<<<< HEAD - - public void register(StateStore store) { throw new UnsupportedOperationException("restore() not supported."); } -======= ->>>>>>> new api model - - public void register(StateStore store) { throw new UnsupportedOperationException("restore() not supported."); } -======= - public void register(StateStore store, RestoreFunc func) { throw new UnsupportedOperationException("restore() not supported."); } ->>>>>>> removing io.confluent imports: wip - - @Override - public void flush() { throw new UnsupportedOperationException("flush() not supported."); } - - @Override -<<<<<<< HEAD -<<<<<<< HEAD - public void restore(StateStore engine) throws Exception { throw new UnsupportedOperationException("restore() not supported."); } -======= - public void send(String topic, Object key, Object value) { throw new UnsupportedOperationException("send() not supported."); } ->>>>>>> new api model -======= - public void send(String topic, Object key, Object value) { throw new UnsupportedOperationException("send() not supported."); } ->>>>>>> new api model - - @Override - public void send(String topic, Object key, Object value, Serializer keySerializer, Serializer valSerializer) { throw new UnsupportedOperationException("send() not supported."); } - - @Override -<<<<<<< HEAD -<<<<<<< HEAD - public void schedule(Processor processor, long interval) { - throw new UnsupportedOperationException("schedule() not supported"); -======= - public PunctuationScheduler getPunctuationScheduler(Processor processor) { - return new PunctuationSchedulerImpl(punctuationQueue, processor); ->>>>>>> new api model -======= - public void schedule(Processor processor, long interval) { - throw new UnsupportedOperationException("schedule() not supported"); ->>>>>>> removed ProcessorContext - } - - @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() { throw new UnsupportedOperationException("timestamp() not supported."); } -======= - Serializer serializer; - Deserializer deserializer; -======= private Serializer serializer; private Deserializer deserializer; private ProcessorNode node; private final ArrayDeque nodeStack = new ArrayDeque(); private Map storeMap = new HashMap<>(); ->>>>>>> kstream test fix long timestamp = -1L; @@ -334,7 +156,6 @@ public long offset() { public long timestamp() { return this.timestamp; } ->>>>>>> compile and test passed public void pushNode(ProcessorNode node) { nodeStack.push(node); diff --git a/temp b/temp deleted file mode 100644 index a743ad0965b15..0000000000000 --- a/temp +++ /dev/null @@ -1,6 +0,0 @@ -build.gradle: needs merge -settings.gradle: needs merge -stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java: needs merge -stream/src/main/java/org/apache/kafka/streaming/kstream/internals/WindowSupport.java: needs merge -streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java: needs merge -streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java: needs merge diff --git a/temp.sh b/temp.sh deleted file mode 100755 index 74e9f056bd9b5..0000000000000 --- a/temp.sh +++ /dev/null @@ -1,4 +0,0 @@ -#!/bin/bash - -cat temp | awk -F":" '{print $1}' > temp2 -cat temp2 | while read in; do git add "$1"; done diff --git a/temp2 b/temp2 deleted file mode 100644 index c4c6bdeea2c36..0000000000000 --- a/temp2 +++ /dev/null @@ -1,6 +0,0 @@ -build.gradle -settings.gradle -stream/src/main/java/org/apache/kafka/stream/topology/internals/WindowSupport.java -stream/src/main/java/org/apache/kafka/streaming/kstream/internals/WindowSupport.java -streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java -streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java From 32c870cd1be46a4e4d7eb3a401268e0fd9da91e7 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 10 Sep 2015 11:16:09 -0700 Subject: [PATCH 224/275] hide node stack in streamtask --- .../internals/ProcessorContextImpl.java | 31 +------ .../processor/internals/PunctuationQueue.java | 19 +---- .../internals/PunctuationSchedule.java | 8 +- .../processor/internals/Punctuator.java | 24 ++++++ .../processor/internals/StreamTask.java | 52 +++++++++++- .../internals/PunctuationQueueTest.java | 85 +++++++++++++++++++ 6 files changed, 168 insertions(+), 51 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 63694dbdd1922..5705f3dcc1180 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -37,7 +37,6 @@ import java.io.File; import java.io.IOException; -import java.util.ArrayDeque; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -53,7 +52,6 @@ public class ProcessorContextImpl implements ProcessorContext { private final Metrics metrics; private final RecordCollector collector; private final ProcessorStateManager stateMgr; - private final ArrayDeque nodeStack = new ArrayDeque(); private final Serializer keySerializer; private final Serializer valSerializer; @@ -214,38 +212,13 @@ public long timestamp() { } @Override - @SuppressWarnings("unchecked") public void forward(K key, V value) { - for (ProcessorNode childNode : (List>) task.node().children()) { - pushNode(childNode); - try { - childNode.process(key, value); - } finally { - popNode(); - } - } + task.forward(key, value); } @Override - @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { - ProcessorNode childNode = (ProcessorNode) task.node().children().get(childIndex); - pushNode(childNode); - try { - childNode.process(key, value); - } finally { - popNode(); - } - } - - private void pushNode(ProcessorNode node) { - nodeStack.push(node); - task.node(node); - } - - private void popNode() { - nodeStack.pop(); - task.node(nodeStack.peek()); + task.forward(key, value, childIndex); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java index 58113f7d0ff43..b972a576c37cc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java @@ -17,20 +17,12 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.common.KafkaException; - import java.util.PriorityQueue; public class PunctuationQueue { private PriorityQueue pq = new PriorityQueue<>(); - private final StreamTask task; - - public PunctuationQueue(StreamTask task) { - this.task = task; - } - public void schedule(PunctuationSchedule sched) { synchronized (pq) { pq.add(sched); @@ -43,20 +35,13 @@ public void close() { } } - public void mayPunctuate(long streamTime) { + public void mayPunctuate(long streamTime, Punctuator punctuator) { synchronized (pq) { PunctuationSchedule top = pq.peek(); while (top != null && top.timestamp <= streamTime) { PunctuationSchedule sched = top; pq.poll(); - - // record the current node - if (task.node() != null) - throw new KafkaException("This should not happen"); - - ProcessorNode scheduled = sched.node(); - task.node(scheduled); - scheduled.processor().punctuate(streamTime); + punctuator.punctuate(sched.node(), streamTime); pq.add(sched.next()); top = pq.peek(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java index af1df5a7ab1cd..cdc3ceb876510 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java @@ -22,7 +22,11 @@ public class PunctuationSchedule extends Stamped { final long interval; public PunctuationSchedule(ProcessorNode node, long interval) { - super(node, System.currentTimeMillis() + interval); + this(node, System.currentTimeMillis(), interval); + } + + public PunctuationSchedule(ProcessorNode node, long time, long interval) { + super(node, time + interval); this.interval = interval; } @@ -31,7 +35,7 @@ public ProcessorNode node() { } public PunctuationSchedule next() { - return new PunctuationSchedule(value, timestamp + interval); + return new PunctuationSchedule(value, timestamp , interval); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java new file mode 100644 index 0000000000000..d99e2ae960415 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java @@ -0,0 +1,24 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +public interface Punctuator { + + void punctuate(ProcessorNode node, long streamTime); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 0687b5914bb6a..39ef5f26e297a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -31,15 +31,17 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayDeque; 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 { +public class StreamTask implements Punctuator { private static final Logger log = LoggerFactory.getLogger(StreamTask.class); @@ -59,6 +61,8 @@ public class StreamTask { private boolean commitOffsetNeeded = false; private StampedRecord currRecord = null; private ProcessorNode currNode = null; + private final ArrayDeque nodeStack = new ArrayDeque<>(); + /** * Create {@link StreamTask} with its assigned partitions @@ -80,7 +84,7 @@ public StreamTask(int id, this.id = id; this.consumer = consumer; - this.punctuationQueue = new PunctuationQueue(this); + this.punctuationQueue = new PunctuationQueue(); this.maxBufferedSize = config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); this.topology = topology; @@ -183,12 +187,19 @@ public int process() { // possibly trigger registered punctuation functions if // partition group's time has reached the defined stamp long timestamp = partitionGroup.timestamp(); - punctuationQueue.mayPunctuate(timestamp); + punctuationQueue.mayPunctuate(timestamp, this); return partitionGroup.numBuffered(); } } + @Override + public void punctuate(ProcessorNode node, long streamTime) { + pushNode(node); + node.processor().punctuate(streamTime); + popNode(); + } + public StampedRecord record() { return this.currRecord; } @@ -197,6 +208,17 @@ public ProcessorNode node() { return this.currNode; } + private void pushNode(ProcessorNode node) { + nodeStack.push(node); + currNode = node; + } + + private void popNode() { + nodeStack.pop(); + currNode = nodeStack.peek(); + } + + public void node(ProcessorNode node) { this.currNode = node; } @@ -249,4 +271,28 @@ public void close() { private RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { return new RecordQueue(partition, source); } + + @SuppressWarnings("unchecked") + public void forward(K key, V value) { + for (ProcessorNode childNode : (List>) currNode.children()) { + pushNode(childNode); + try { + childNode.process(key, value); + } finally { + popNode(); + } + } + } + + @SuppressWarnings("unchecked") + public void forward(K key, V value, int childIndex) { + ProcessorNode childNode = (ProcessorNode) currNode.children().get(childIndex); + pushNode(childNode); + try { + childNode.process(key, value); + } finally { + popNode(); + } + } + } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java new file mode 100644 index 0000000000000..b1403bd02f44c --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.junit.Test; + +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +public class PunctuationQueueTest { + + @Test + public void testPunctuationInterval() { + TestProcessor processor = new TestProcessor(); + ProcessorNode node = new ProcessorNode<>("test", processor); + PunctuationQueue queue = new PunctuationQueue(); + + PunctuationSchedule sched = new PunctuationSchedule(node, 100L); + final long now = sched.timestamp - 100L; + + queue.schedule(sched); + + Punctuator punctuator = new Punctuator() { + public void punctuate(ProcessorNode node, long time) { + node.processor().punctuate(time); + } + }; + + queue.mayPunctuate(now, punctuator); + assertEquals(0, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 99L, punctuator); + assertEquals(0, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 100L, punctuator); + assertEquals(1, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 199L, punctuator); + assertEquals(1, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 200L, punctuator); + assertEquals(2, processor.punctuatedAt.size()); + } + + private static class TestProcessor implements Processor { + + public final ArrayList punctuatedAt = new ArrayList<>(); + + @Override + public void init(ProcessorContext context) { + } + + @Override + public void process(String key, String value) { + } + + @Override + public void punctuate(long streamTime) { + punctuatedAt.add(streamTime); + } + + @Override + public void close() { + } + } + +} From d05c562022b228d432787bfa926dba07251eeb35 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 10 Sep 2015 11:47:29 -0700 Subject: [PATCH 225/275] fix whitespace --- .../kafka/streams/processor/internals/PunctuationSchedule.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java index cdc3ceb876510..dc9a50d32c249 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java @@ -35,7 +35,7 @@ public ProcessorNode node() { } public PunctuationSchedule next() { - return new PunctuationSchedule(value, timestamp , interval); + return new PunctuationSchedule(value, timestamp, interval); } } From 2c645b67049a84e5e2b1084943721fd79cdf90d1 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Thu, 10 Sep 2015 15:34:22 -0700 Subject: [PATCH 226/275] change timestampextractor signature --- .../streams/examples/WallclockTimestampExtractor.java | 3 ++- .../kafka/streams/processor/TimestampExtractor.java | 10 +++++----- .../kafka/streams/processor/internals/RecordQueue.java | 10 ++++++---- .../org/apache/kafka/test/MockTimestampExtractor.java | 5 +++-- 4 files changed, 16 insertions(+), 12 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java index 8b39946bdf09c..26281d69d0293 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java @@ -17,11 +17,12 @@ 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(String topic, Object key, Object value) { + public long extract(ConsumerRecord record) { return System.currentTimeMillis(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java index 6bb0c4f4a55ca..62098f21d56df 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java @@ -17,18 +17,18 @@ 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 key-value pair from a topic + * Extracts a timestamp from a message * - * @param topic the topic name - * @param key the key object - * @param value the value object + * @param record ConsumerRecord * @return timestamp */ - long extract(String topic, Object key, Object value); + long extract(ConsumerRecord record); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index a4893e75d5c80..66f78d2150994 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -75,12 +75,14 @@ public int addRawRecords(Iterable> rawRecords, Ti // 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()); - long timestamp = timestampExtractor.extract(rawRecord.topic(), key, value); - StampedRecord record = new StampedRecord(new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), key, value), timestamp); + ConsumerRecord record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), key, value); + long timestamp = timestampExtractor.extract(record); - fifoQueue.addLast(record); - timeTracker.addElement(record); + StampedRecord stampedRecord = new StampedRecord(record, timestamp); + + fifoQueue.addLast(stampedRecord); + timeTracker.addElement(stampedRecord); } return size(); diff --git a/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java index 0d854fb1f6d67..a496314031d1d 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java +++ b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java @@ -17,12 +17,13 @@ package org.apache.kafka.test; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.streams.processor.TimestampExtractor; public class MockTimestampExtractor implements TimestampExtractor { @Override - public long extract(String topic, Object key, Object value) { - return ((Integer) key).longValue(); + public long extract(ConsumerRecord record) { + return ((Integer) record.key()).longValue(); } } From 39c8256f571f494284b2af61067719904de5ed57 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 11 Sep 2015 11:41:10 -0700 Subject: [PATCH 227/275] remove nodestack --- .../processor/internals/StreamTask.java | 95 +++++++++---------- 1 file changed, 44 insertions(+), 51 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 39ef5f26e297a..47356860e8074 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -31,7 +31,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.ArrayDeque; import java.util.Collection; import java.util.HashMap; import java.util.List; @@ -61,8 +60,6 @@ public class StreamTask implements Punctuator { private boolean commitOffsetNeeded = false; private StampedRecord currRecord = null; private ProcessorNode currNode = null; - private final ArrayDeque nodeStack = new ArrayDeque<>(); - /** * Create {@link StreamTask} with its assigned partitions @@ -158,46 +155,55 @@ public int process() { if (queue == null) return 0; - // get a record from the queue and process it - // by passing to the source node of the topology - this.currRecord = partitionGroup.getRecord(queue); - this.currNode = queue.source(); + try { + // get a record from the queue and process it + // by passing to the source node of the topology + this.currRecord = partitionGroup.getRecord(queue); + this.currNode = queue.source(); - log.debug("Start processing one record [" + currRecord + "]"); + log.debug("Start processing one record [" + currRecord + "]"); - this.currNode.process(currRecord.key(), currRecord.value()); + this.currNode.process(currRecord.key(), currRecord.value()); - log.debug("Completed processing one record [" + currRecord + "]"); + log.debug("Completed processing one record [" + currRecord + "]"); - // update the consumed offset map after processing is done - consumedOffsets.put(queue.partition(), currRecord.offset()); - commitOffsetNeeded = true; + // update the consumed offset map after processing is done + consumedOffsets.put(queue.partition(), currRecord.offset()); + commitOffsetNeeded = true; - // commit the current task state if requested during the processing - if (commitRequested) { - commit(); - } - - // if after processing this record, its partition queue's buffered size has been - // decreased to the threshold, we can then resume the consumption on this partition - if (partitionGroup.numBuffered(queue.partition()) == this.maxBufferedSize) { - consumer.resume(queue.partition()); - } + // commit the current task state if requested during the processing + if (commitRequested) { + commit(); + } - // possibly trigger registered punctuation functions if - // partition group's time has reached the defined stamp - long timestamp = partitionGroup.timestamp(); - punctuationQueue.mayPunctuate(timestamp, this); + // if after processing this record, its partition queue's buffered size has been + // decreased to the threshold, we can then resume the consumption on this partition + if (partitionGroup.numBuffered(queue.partition()) == this.maxBufferedSize) { + consumer.resume(queue.partition()); + } + // possibly trigger registered punctuation functions if + // partition group's time has reached the defined stamp + long timestamp = partitionGroup.timestamp(); + punctuationQueue.mayPunctuate(timestamp, this); + } finally { + this.currRecord = null; + this.currNode = null; + } return partitionGroup.numBuffered(); } } @Override public void punctuate(ProcessorNode node, long streamTime) { - pushNode(node); - node.processor().punctuate(streamTime); - popNode(); + if (currNode != null) throw new IllegalStateException("Current node is not null"); + + currNode = node; + try { + node.processor().punctuate(streamTime); + } finally { + currNode = null; + } } public StampedRecord record() { @@ -208,21 +214,6 @@ public ProcessorNode node() { return this.currNode; } - private void pushNode(ProcessorNode node) { - nodeStack.push(node); - currNode = node; - } - - private void popNode() { - nodeStack.pop(); - currNode = nodeStack.peek(); - } - - - public void node(ProcessorNode node) { - this.currNode = node; - } - public ProcessorTopology topology() { return this.topology; } @@ -274,24 +265,26 @@ private RecordQueue createRecordQueue(TopicPartition partition, SourceNode sourc @SuppressWarnings("unchecked") public void forward(K key, V value) { - for (ProcessorNode childNode : (List>) currNode.children()) { - pushNode(childNode); + ProcessorNode thisNode = currNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + currNode = childNode; try { childNode.process(key, value); } finally { - popNode(); + currNode = thisNode; } } } @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { - ProcessorNode childNode = (ProcessorNode) currNode.children().get(childIndex); - pushNode(childNode); + ProcessorNode thisNode = currNode; + ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); + currNode = childNode; try { childNode.process(key, value); } finally { - popNode(); + currNode = thisNode; } } From 96066bd0ca0f979bcfc26b13a321dff07e48b2b3 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 11 Sep 2015 13:01:58 -0700 Subject: [PATCH 228/275] remove nodestack from test code --- .../apache/kafka/test/KStreamTestDriver.java | 7 +--- .../kafka/test/MockProcessorContext.java | 38 +++++++++---------- 2 files changed, 18 insertions(+), 27 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index 7306ec4a57c60..11e5b83b12aea 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -40,12 +40,7 @@ public KStreamTestDriver(KStreamBuilder builder, Serializer serializer, Deser } public void process(String topicName, Object key, Object value) { - context.pushNode(topology.source(topicName)); - try { - context.forward(key, value); - } finally { - context.popNode(); - } + context.process(topology, topicName, key, value); } public void setTime(long timestamp) { diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 4466051d25c2f..83cfd608bbd84 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -25,9 +25,9 @@ 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 java.io.File; -import java.util.ArrayDeque; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -36,8 +36,7 @@ public class MockProcessorContext implements ProcessorContext { private Serializer serializer; private Deserializer deserializer; - private ProcessorNode node; - private final ArrayDeque nodeStack = new ArrayDeque(); + private ProcessorNode currNode; private Map storeMap = new HashMap<>(); @@ -110,12 +109,13 @@ public void schedule(Processor processor, long interval) { @Override @SuppressWarnings("unchecked") public void forward(K key, V value) { - for (ProcessorNode childNode : (List>) node().children()) { - pushNode(childNode); + ProcessorNode thisNode = currNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + currNode = childNode; try { childNode.process(key, value); } finally { - popNode(); + currNode = thisNode; } } } @@ -123,12 +123,13 @@ public void forward(K key, V value) { @Override @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { - ProcessorNode childNode = (ProcessorNode) node().children().get(childIndex); - pushNode(childNode); + ProcessorNode thisNode = currNode; + ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); + currNode = childNode; try { childNode.process(key, value); } finally { - popNode(); + currNode = thisNode; } } @@ -157,18 +158,13 @@ public long timestamp() { return this.timestamp; } - public void pushNode(ProcessorNode node) { - nodeStack.push(node); - this.node = node; - } - - public ProcessorNode popNode() { - ProcessorNode node = nodeStack.pop(); - this.node = nodeStack.peek(); - return node; + public void process(ProcessorTopology topology, String topicName, Object key, Object value) { + currNode = topology.source(topicName); + try { + forward(key, value); + } finally { + currNode = null; + } } - private ProcessorNode node() { - return this.node; - } } From be49a97c5bae163be9752fc915d3d13f576286d1 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 11 Sep 2015 13:39:44 -0700 Subject: [PATCH 229/275] make ProcessorTopology immutable --- .../internals/ProcessorTopology.java | 43 +++-------------- .../processor/internals/StreamTask.java | 25 +++++++++- .../apache/kafka/test/KStreamTestDriver.java | 47 +++++++++++++++++-- .../kafka/test/MockProcessorContext.java | 42 ++++------------- 4 files changed, 82 insertions(+), 75 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java index 72b68c89db04b..739c4deab25fa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java @@ -17,25 +17,24 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.streams.processor.ProcessorContext; - import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; public class ProcessorTopology { - private List processorNodes; - private Map sourceByTopics; - private Map sinkByTopics; + private final List processorNodes; + private final Map sourceByTopics; + private final Map sinkByTopics; public ProcessorTopology(List processorNodes, Map sourceByTopics, Map sinkByTopics) { - this.processorNodes = processorNodes; - this.sourceByTopics = sourceByTopics; - this.sinkByTopics = sinkByTopics; + this.processorNodes = Collections.unmodifiableList(processorNodes); + this.sourceByTopics = Collections.unmodifiableMap(sourceByTopics); + this.sinkByTopics = Collections.unmodifiableMap(sinkByTopics); } public Set sourceTopics() { @@ -66,32 +65,4 @@ public List processors() { return processorNodes; } - /** - * Initialize the processors - */ - public void init(ProcessorContext context) { - for (ProcessorNode node : processorNodes) { - node.init(context); - } - } - - public final void close() { - RuntimeException exception = null; - - // close the processors - // make sure close() is called for each node even when there is a RuntimeException - for (ProcessorNode node : processorNodes) { - try { - node.close(); - } catch (RuntimeException e) { - exception = e; - } - } - - processorNodes.clear(); - sourceByTopics.clear(); - sinkByTopics.clear(); - - if (exception != null) throw exception; - } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 47356860e8074..b71d6a9778ec8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -112,7 +112,14 @@ public StreamTask(int id, } // initialize the task by initializing all its processor nodes in the topology - this.topology.init(this.processorContext); + for (ProcessorNode node : this.topology.processors()) { + this.currNode = node; + try { + node.init(this.processorContext); + } finally { + this.currNode = null; + } + } } public int id() { @@ -256,7 +263,21 @@ public void schedule(ProcessorNode node, long interval) { public void close() { this.partitionGroup.close(); this.consumedOffsets.clear(); - this.topology.close(); + + // 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; } private RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index 11e5b83b12aea..2c42e6c225c90 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -21,12 +21,16 @@ 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); @@ -34,13 +38,25 @@ public KStreamTestDriver(KStreamBuilder builder) { public KStreamTestDriver(KStreamBuilder builder, Serializer serializer, Deserializer deserializer) { this.topology = builder.build(); - this.context = new MockProcessorContext(serializer, deserializer); + this.context = new MockProcessorContext(this, serializer, deserializer); - this.topology.init(context); + for (ProcessorNode node : topology.processors()) { + currNode = node; + try { + node.init(context); + } finally { + currNode = null; + } + } } public void process(String topicName, Object key, Object value) { - context.process(topology, topicName, key, value); + currNode = topology.source(topicName); + try { + forward(key, value); + } finally { + currNode = null; + } } public void setTime(long timestamp) { @@ -51,4 +67,29 @@ public StateStore getStateStore(String name) { return context.getStateStore(name); } + @SuppressWarnings("unchecked") + public void forward(K key, V value) { + ProcessorNode thisNode = currNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + } + } + + @SuppressWarnings("unchecked") + public void forward(K key, V value, int childIndex) { + ProcessorNode thisNode = currNode; + ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + } + } diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 83cfd608bbd84..c2d166d93ea9b 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -24,25 +24,23 @@ 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.processor.internals.ProcessorNode; -import org.apache.kafka.streams.processor.internals.ProcessorTopology; import java.io.File; import java.util.HashMap; -import java.util.List; import java.util.Map; public class MockProcessorContext implements ProcessorContext { - private Serializer serializer; - private Deserializer deserializer; - private ProcessorNode currNode; + private final KStreamTestDriver driver; + private final Serializer serializer; + private final Deserializer deserializer; private Map storeMap = new HashMap<>(); long timestamp = -1L; - public MockProcessorContext(Serializer serializer, Deserializer deserializer) { + public MockProcessorContext(KStreamTestDriver driver, Serializer serializer, Deserializer deserializer) { + this.driver = driver; this.serializer = serializer; this.deserializer = deserializer; } @@ -92,7 +90,7 @@ public Metrics metrics() { @Override public void register(StateStore store, RestoreFunc func) { - if (func != null) new UnsupportedOperationException("RestoreFunc not supported."); + if (func != null) throw new UnsupportedOperationException("RestoreFunc not supported."); storeMap.put(store.name(), store); } @@ -109,28 +107,13 @@ public void schedule(Processor processor, long interval) { @Override @SuppressWarnings("unchecked") public void forward(K key, V value) { - ProcessorNode thisNode = currNode; - for (ProcessorNode childNode : (List>) thisNode.children()) { - currNode = childNode; - try { - childNode.process(key, value); - } finally { - currNode = thisNode; - } - } + driver.forward(key, value); } @Override @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { - ProcessorNode thisNode = currNode; - ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); - currNode = childNode; - try { - childNode.process(key, value); - } finally { - currNode = thisNode; - } + driver.forward(key, value, childIndex); } @Override @@ -158,13 +141,4 @@ public long timestamp() { return this.timestamp; } - public void process(ProcessorTopology topology, String topicName, Object key, Object value) { - currNode = topology.source(topicName); - try { - forward(key, value); - } finally { - currNode = null; - } - } - } From 3dc28f5d927468463ed0ac7ccdf79d5970eac104 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 11 Sep 2015 13:58:16 -0700 Subject: [PATCH 230/275] flush records before offset commit --- .../kafka/streams/processor/internals/StreamTask.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 47356860e8074..cb8f34cb40159 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -225,15 +225,16 @@ public void commit() { // 1) flush local state ((ProcessorContextImpl) processorContext).stateManager().flush(); - // 2) commit consumed offsets if it is dirty already + // 2) flush produced records in the downstream + // TODO: this will actually block on all produced records across the tasks + recordCollector.flush(); + + // 3) commit consumed offsets if it is dirty already if (commitOffsetNeeded) { consumer.commit(consumedOffsets, CommitType.SYNC); commitOffsetNeeded = false; } - - // 3) flush produced records in the downstream - // TODO: this will actually block on all produced records across the tasks - recordCollector.flush(); + commitRequested = false; } /** From c459e9e43adadecb197355291e53a2c5003a93dd Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 11 Sep 2015 16:00:49 -0700 Subject: [PATCH 231/275] use restoredOffset if ack'ed offset is not available --- .../internals/ProcessorStateManager.java | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 7c9bad284f59b..8f46ba4fb49ca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -51,10 +51,9 @@ public ProcessorStateManager(int id, File baseDir, Consumer rest this.stores = new HashMap<>(); this.restoreConsumer = restoreConsumer; this.restoredOffsets = new HashMap<>(); - this.checkpointedOffsets = new HashMap<>(); OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); - this.checkpointedOffsets.putAll(checkpoint.read()); + this.checkpointedOffsets = new HashMap<>(checkpoint.read()); // delete the checkpoint file after finish loading its stored offsets checkpoint.delete(); @@ -167,17 +166,23 @@ public void close(Map ackedOffsets) throws IOException { entry.getValue().close(); } - Map checkpointOffsets = new HashMap(restoredOffsets); + Map checkpointOffsets = new HashMap<>(); for (String storeName : stores.keySet()) { TopicPartition part = new TopicPartition(storeName, id); // only checkpoint the offset to the offsets file if it is persistent; if (stores.get(storeName).persistent()) { - if (ackedOffsets.containsKey(part)) - // store the last ack'd offset + 1 (the log position after restoration) - checkpointOffsets.put(part, ackedOffsets.get(part) + 1); - } else { - checkpointOffsets.remove(part); + 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); + } } } From 6fdad6d695651da725942e9cb8035db6740cefdf Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 11 Sep 2015 16:28:25 -0700 Subject: [PATCH 232/275] remove processor arg from schedule call --- .../apache/kafka/streams/examples/ProcessorJob.java | 2 +- .../kafka/streams/processor/ProcessorContext.java | 2 +- .../processor/internals/ProcessorContextImpl.java | 12 ++---------- .../streams/processor/internals/StreamTask.java | 7 ++++--- .../org/apache/kafka/test/MockProcessorContext.java | 3 +-- 5 files changed, 9 insertions(+), 17 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java index fb0a1a0f181df..7d9773557585b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java @@ -48,7 +48,7 @@ public Processor instance() { @Override public void init(ProcessorContext context) { this.context = context; - this.context.schedule(this, 1000); + this.context.schedule(1000); this.kvStore = new InMemoryKeyValueStore<>("local-state", context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index aa0b4e1d644c4..6b32b83abbff0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -88,7 +88,7 @@ public interface ProcessorContext { StateStore getStateStore(String name); - void schedule(Processor processor, long interval); + void schedule(long interval); void forward(K key, V value); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 5705f3dcc1180..69f4e8aa3b774 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -28,10 +28,8 @@ 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.Processor; import org.apache.kafka.streams.processor.RestoreFunc; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -227,13 +225,7 @@ public void commit() { } @Override - public void schedule(Processor processor, long interval) { - // find the processor node corresponding to this processor - for (ProcessorNode node : task.topology().processors()) { - if (node.processor().equals(processor)) { - task.schedule(node, interval); - break; - } - } + public void schedule(long interval) { + task.schedule(interval); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index df7a2a550f67b..617312786aad6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -254,11 +254,12 @@ public void needCommit() { /** * Schedules a punctuation for the processor * - * @param node the processor node requesting scheduler * @param interval the interval in milliseconds */ - public void schedule(ProcessorNode node, long interval) { - punctuationQueue.schedule(new PunctuationSchedule(node, interval)); + public void schedule(long interval) { + if (currNode == null) throw new IllegalStateException("Current node is null"); + + punctuationQueue.schedule(new PunctuationSchedule(currNode, interval)); } public void close() { diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index c2d166d93ea9b..3fdfc82c1ff76 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -17,7 +17,6 @@ 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.RestoreFunc; import org.apache.kafka.streams.processor.StateStore; @@ -100,7 +99,7 @@ public StateStore getStateStore(String name) { } @Override - public void schedule(Processor processor, long interval) { + public void schedule(long interval) { throw new UnsupportedOperationException("schedule() not supported"); } From 395ca48567585d6c4b9700c101c913c1b4bfa4d5 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 14 Sep 2015 11:59:49 -0700 Subject: [PATCH 233/275] call statemgr.close() --- .../kafka/streams/processor/internals/StreamTask.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 617312786aad6..bf177f304e7bf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -108,7 +108,7 @@ public StreamTask(int id, try { this.processorContext = new ProcessorContextImpl(id, this, config, recordCollector, new Metrics()); } catch (IOException e) { - throw new KafkaException("Error while creating the state manager in processor context."); + throw new KafkaException("Error while creating the state manager in processor context", e); } // initialize the task by initializing all its processor nodes in the topology @@ -280,6 +280,12 @@ public void close() { } } if (exception != null) throw exception; + + try { + ((ProcessorContextImpl) processorContext).stateManager().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) { From 9dab4c63aef1eee35ef4aba8a6d677fd411570f5 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 14 Sep 2015 12:34:39 -0700 Subject: [PATCH 234/275] ProcessorTopology unit tests --- .../streams/processor/TopologyBuilder.java | 23 +++-- .../streams/processor/TopologyException.java | 38 +++++++ .../internals/ProcessorTopology.java | 10 +- .../processor/TopologyBuilderTest.java | 99 +++++++++++++++++++ .../internals/ProcessorTopologyTest.java | 64 ++++++++++++ 5 files changed, 220 insertions(+), 14 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/TopologyException.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index 93cb4765937c3..232cc71832275 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -49,6 +49,7 @@ public class TopologyBuilder { private Set nodeNames = new HashSet<>(); private Set sourceTopicNames = new HashSet<>(); + private Set sinkTopicNames = new HashSet<>(); private interface NodeFactory { ProcessorNode build(); @@ -148,11 +149,11 @@ public final TopologyBuilder addSource(String name, String... topics) { */ public final TopologyBuilder addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { if (nodeNames.contains(name)) - throw new IllegalArgumentException("Processor " + name + " is already added."); + throw new TopologyException("Processor " + name + " is already added."); for (String topic : topics) { if (sourceTopicNames.contains(topic)) - throw new IllegalArgumentException("Topic " + topic + " has already been registered by another processor."); + throw new TopologyException("Topic " + topic + " has already been registered by another source."); sourceTopicNames.add(topic); } @@ -194,19 +195,23 @@ public final TopologyBuilder addSink(String name, String topic, String... parent */ public final TopologyBuilder addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, String... parentNames) { if (nodeNames.contains(name)) - throw new IllegalArgumentException("Processor " + name + " is already added."); + throw new TopologyException("Processor " + name + " is already added."); + + if (sinkTopicNames.contains(topic)) + throw new TopologyException("Topic " + topic + " has already been registered by another sink."); if (parentNames != null) { for (String parent : parentNames) { if (parent.equals(name)) { - throw new IllegalArgumentException("Processor " + name + " cannot be a parent of itself"); + throw new TopologyException("Processor " + name + " cannot be a parent of itself."); } if (!nodeNames.contains(parent)) { - throw new IllegalArgumentException("Parent processor " + parent + " is not added yet."); + throw new TopologyException("Parent processor " + parent + " is not added yet."); } } } + sinkTopicNames.add(topic); nodeNames.add(name); nodeFactories.add(new SinkNodeFactory(name, parentNames, topic, keySerializer, valSerializer)); return this; @@ -223,15 +228,15 @@ public final TopologyBuilder addSink(String name, String topic, Serializer keySe */ public final TopologyBuilder addProcessor(String name, ProcessorDef definition, String... parentNames) { if (nodeNames.contains(name)) - throw new IllegalArgumentException("Processor " + name + " is already added."); + throw new TopologyException("Processor " + name + " is already added."); if (parentNames != null) { for (String parent : parentNames) { if (parent.equals(name)) { - throw new IllegalArgumentException("Processor " + name + " cannot be a parent of itself"); + throw new TopologyException("Processor " + name + " cannot be a parent of itself."); } if (!nodeNames.contains(parent)) { - throw new IllegalArgumentException("Parent processor " + parent + " is not added yet."); + throw new TopologyException("Parent processor " + parent + " is not added yet."); } } } @@ -277,7 +282,7 @@ public ProcessorTopology build() { processorMap.get(parent).addChild(node); } } else { - throw new IllegalStateException("unknown definition class: " + factory.getClass().getName()); + throw new TopologyException("Unknown definition class: " + factory.getClass().getName()); } } } catch (Exception e) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyException.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyException.java new file mode 100644 index 0000000000000..99d14058a1f7a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyException.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor; + +import org.apache.kafka.common.KafkaException; + +public class TopologyException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public TopologyException(String message) { + super(message); + } + + public TopologyException(String name, Object value) { + this(name, value, null); + } + + public TopologyException(String name, Object value, String message) { + super("Invalid topology building" + (message == null ? "" : ": " + message)); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java index 739c4deab25fa..ab4cfddc55151 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java @@ -17,8 +17,8 @@ package org.apache.kafka.streams.processor.internals; -import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -53,12 +53,12 @@ public SinkNode sink(String topic) { return sinkByTopics.get(topic); } - public Collection sources() { - return sourceByTopics.values(); + public Set sources() { + return new HashSet<>(sourceByTopics.values()); } - public Collection sinks() { - return sinkByTopics.values(); + public Set sinks() { + return new HashSet<>(sinkByTopics.values()); } public List processors() { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java new file mode 100644 index 0000000000000..57a78ff87a896 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor; + +import static org.junit.Assert.assertEquals; + +import org.apache.kafka.test.MockProcessorDef; +import org.junit.Test; + +public class TopologyBuilderTest { + + @Test(expected = TopologyException.class) + public void testAddSourceWithSameName() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addSource("source", "topic-2"); + } + + @Test(expected = TopologyException.class) + public void testAddSourceWithSameTopic() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addSource("source-2", "topic-1"); + } + + @Test(expected = TopologyException.class) + public void testAddProcessorWithSameName() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addProcessor("processor", new MockProcessorDef(), "source"); + builder.addProcessor("processor", new MockProcessorDef(), "source"); + } + + @Test(expected = TopologyException.class) + public void testAddProcessorWithWrongParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addProcessor("processor", new MockProcessorDef(), "source"); + } + + @Test(expected = TopologyException.class) + public void testAddProcessorWithSelfParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addProcessor("processor", new MockProcessorDef(), "processor"); + } + + @Test(expected = TopologyException.class) + public void testAddSinkWithSameName() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addSink("sink", "topic-2", "source"); + builder.addSink("sink", "topic-3", "source"); + } + + @Test(expected = TopologyException.class) + public void testAddSinkWithWrongParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSink("sink", "topic-2", "source"); + } + + @Test(expected = TopologyException.class) + public void testAddSinkWithSelfParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSink("sink", "topic-2", "sink"); + } + + @Test + public void testSourceTopics() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source-1", "topic-1"); + builder.addSource("source-2", "topic-2"); + builder.addSource("source-3", "topic-3"); + + assertEquals(builder.sourceTopics().size(), 3); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java new file mode 100644 index 0000000000000..f0d6aecc841f2 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.test.MockProcessorDef; +import org.junit.Test; + +public class ProcessorTopologyTest { + + @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(2, topology.sinks().size()); + + assertEquals(3, topology.sourceTopics().size()); + + assertEquals(2, topology.sinkTopics().size()); + + assertNotNull(topology.source("topic-1")); + + assertNotNull(topology.source("topic-2")); + + assertNotNull(topology.source("topic-3")); + + assertNotNull(topology.sink("topic-3")); + + assertNotNull(topology.sink("topic-4")); + + assertEquals(topology.source("topic-2"), topology.source("topic-3")); + } +} From 2ab6576baa4cf06fb95743e75127f31b61e78439 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 14 Sep 2015 14:23:05 -0700 Subject: [PATCH 235/275] address comments --- .../streams/processor/TopologyBuilder.java | 11 +---------- .../processor/internals/ProcessorTopology.java | 17 +---------------- .../internals/ProcessorTopologyTest.java | 8 -------- .../processor/internals/StreamTaskTest.java | 5 +---- 4 files changed, 3 insertions(+), 38 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index 232cc71832275..a254c131c1a47 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -49,7 +49,6 @@ public class TopologyBuilder { private Set nodeNames = new HashSet<>(); private Set sourceTopicNames = new HashSet<>(); - private Set sinkTopicNames = new HashSet<>(); private interface NodeFactory { ProcessorNode build(); @@ -197,9 +196,6 @@ public final TopologyBuilder addSink(String name, String topic, Serializer keySe if (nodeNames.contains(name)) throw new TopologyException("Processor " + name + " is already added."); - if (sinkTopicNames.contains(topic)) - throw new TopologyException("Topic " + topic + " has already been registered by another sink."); - if (parentNames != null) { for (String parent : parentNames) { if (parent.equals(name)) { @@ -211,7 +207,6 @@ public final TopologyBuilder addSink(String name, String topic, Serializer keySe } } - sinkTopicNames.add(topic); nodeNames.add(name); nodeFactories.add(new SinkNodeFactory(name, parentNames, topic, keySerializer, valSerializer)); return this; @@ -257,7 +252,6 @@ public ProcessorTopology build() { List processorNodes = new ArrayList<>(nodeFactories.size()); Map processorMap = new HashMap<>(); Map topicSourceMap = new HashMap<>(); - Map topicSinkMap = new HashMap<>(); try { // create processor nodes in a topological order ("nodeFactories" is already topologically sorted) @@ -275,9 +269,6 @@ public ProcessorTopology build() { topicSourceMap.put(topic, (SourceNode) node); } } else if (factory instanceof SinkNodeFactory) { - String topic = ((SinkNodeFactory) factory).topic; - topicSinkMap.put(topic, (SinkNode) node); - for (String parent : ((SinkNodeFactory) factory).parents) { processorMap.get(parent).addChild(node); } @@ -289,7 +280,7 @@ public ProcessorTopology build() { throw new KafkaException("ProcessorNode construction failed: this should not happen."); } - return new ProcessorTopology(processorNodes, topicSourceMap, topicSinkMap); + return new ProcessorTopology(processorNodes, topicSourceMap); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java index ab4cfddc55151..3efae6518f63d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java @@ -27,40 +27,25 @@ public class ProcessorTopology { private final List processorNodes; private final Map sourceByTopics; - private final Map sinkByTopics; public ProcessorTopology(List processorNodes, - Map sourceByTopics, - Map sinkByTopics) { + Map sourceByTopics) { this.processorNodes = Collections.unmodifiableList(processorNodes); this.sourceByTopics = Collections.unmodifiableMap(sourceByTopics); - this.sinkByTopics = Collections.unmodifiableMap(sinkByTopics); } public Set sourceTopics() { return sourceByTopics.keySet(); } - public Set sinkTopics() { - return sinkByTopics.keySet(); - } - public SourceNode source(String topic) { return sourceByTopics.get(topic); } - public SinkNode sink(String topic) { - return sinkByTopics.get(topic); - } - public Set sources() { return new HashSet<>(sourceByTopics.values()); } - public Set sinks() { - return new HashSet<>(sinkByTopics.values()); - } - public List processors() { return processorNodes; } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index f0d6aecc841f2..0fa2c9de2e58e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -43,22 +43,14 @@ public void testTopologyMetadata() { assertEquals(2, topology.sources().size()); - assertEquals(2, topology.sinks().size()); - assertEquals(3, topology.sourceTopics().size()); - assertEquals(2, topology.sinkTopics().size()); - assertNotNull(topology.source("topic-1")); assertNotNull(topology.source("topic-2")); assertNotNull(topology.source("topic-3")); - assertNotNull(topology.sink("topic-3")); - - assertNotNull(topology.sink("topic-4")); - assertEquals(topology.source("topic-2"), topology.source("topic-3")); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 28673cc54636f..9c5826e9486b3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -33,7 +33,6 @@ import org.junit.Before; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Properties; @@ -60,9 +59,7 @@ public class StreamTaskTest { put("topic1", source1); put("topic2", source2); } - }, - Collections.emptyMap() - ); + }); private final StreamingConfig config = new StreamingConfig(new Properties() { { From 60a26a2e51546b24ae7823d0755641c14eb4cf0a Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 14 Sep 2015 14:46:42 -0700 Subject: [PATCH 236/275] fix state directory cleanup --- .../internals/ProcessorStateManager.java | 35 +++++++++++++++++++ .../processor/internals/StreamTask.java | 2 +- .../processor/internals/StreamThread.java | 28 ++++++++++++--- .../processor/internals/StreamTaskTest.java | 9 ++++- 4 files changed, 68 insertions(+), 6 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 8f46ba4fb49ca..5c778fd5f5607 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -29,6 +29,10 @@ 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; @@ -37,9 +41,11 @@ public class ProcessorStateManager { private static final Logger log = LoggerFactory.getLogger(ProcessorStateManager.class); private static final String CHECKPOINT_FILE_NAME = ".checkpoint"; + public static final String LOCK_FILE_NAME = ".lock"; private final int id; private final File baseDir; + private final FileLock directoryLock; private final Map stores; private final Consumer restoreConsumer; private final Map restoredOffsets; @@ -52,6 +58,16 @@ public ProcessorStateManager(int id, File baseDir, Consumer rest 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()); @@ -59,6 +75,22 @@ public ProcessorStateManager(int id, File baseDir, Consumer rest 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; } @@ -193,6 +225,9 @@ public void close(Map ackedOffsets) throws IOException { // close the restore consumer restoreConsumer.close(); + + // release the state directory directoryLock + directoryLock.release(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 617312786aad6..224ca4c365d67 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -108,7 +108,7 @@ public StreamTask(int id, try { this.processorContext = new ProcessorContextImpl(id, this, config, recordCollector, new Metrics()); } catch (IOException e) { - throw new KafkaException("Error while creating the state manager in processor context."); + throw new KafkaException("Error while creating the state manager in processor context", e); } // initialize the task by initializing all its processor nodes in the topology diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 990f21b155788..17da52d07c9aa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -43,6 +43,8 @@ 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.HashMap; @@ -80,12 +82,14 @@ public class StreamThread extends Thread { @Override public void onPartitionsAssigned(Consumer consumer, Collection assignment) { addPartitions(assignment); + lastClean = time.milliseconds(); // start the cleaning cycle } @Override public void onPartitionsRevoked(Consumer consumer, Collection assignment) { commitAll(time.milliseconds()); removePartitions(); + lastClean = Long.MAX_VALUE; // stop the cleaning cycle until partitions are assigned } }; @@ -120,7 +124,7 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce this.cleanTimeMs = config.getLong(StreamingConfig.STATE_CLEANUP_DELAY_MS_CONFIG); this.totalRecordsToProcess = config.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); - this.lastClean = 0; + this.lastClean = Long.MAX_VALUE; // the cleaning cycle won't start until partition assignment this.lastCommit = 0; this.recordsProcessed = 0; this.time = new SystemTime(); @@ -267,9 +271,25 @@ private void maybeClean() { for (File dir : stateDirs) { try { Integer id = Integer.parseInt(dir.getName()); - if (!tasks.containsKey(id)) { - log.info("Deleting obsolete state directory {} after delayed {} ms.", dir.getAbsolutePath(), cleanTimeMs); - Utils.delete(dir); + + // 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", 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, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 28673cc54636f..85b9c908b8169 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -78,7 +78,6 @@ public class StreamTaskTest { private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); private final MockProducer producer = new MockProducer<>(false, bytesSerializer, bytesSerializer); - private final StreamTask task = new StreamTask(0, consumer, producer, partitions, topology, config); @Before public void setup() { @@ -88,6 +87,8 @@ public void setup() { @SuppressWarnings("unchecked") @Test public void testProcessOrder() { + StreamTask task = new StreamTask(0, consumer, producer, partitions, topology, config); + byte[] recordValue = intSerializer.serialize(null, 10); task.addRecords(partition1, records( @@ -125,11 +126,15 @@ public void testProcessOrder() { 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, partitions, topology, config); + byte[] recordValue = intSerializer.serialize(null, 10); task.addRecords(partition1, records( @@ -173,6 +178,8 @@ public void testPauseResume() { assertEquals(source2.numReceived, 1); assertEquals(consumer.paused().size(), 0); + + task.close(); } private Iterable> records(ConsumerRecord... recs) { From 21640022fae6e44308b781d1f36916d6386b3610 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 14 Sep 2015 15:27:49 -0700 Subject: [PATCH 237/275] change error messages --- .../streams/processor/internals/ProcessorStateManager.java | 2 +- .../apache/kafka/streams/processor/internals/StreamThread.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 5c778fd5f5607..cec6ecef393a2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -64,7 +64,7 @@ public ProcessorStateManager(int id, File baseDir, Consumer rest // 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()); + throw new IOException("Failed to lock the state directory: " + baseDir.getCanonicalPath()); } // load the checkpoint information diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 17da52d07c9aa..8af18e7075e3e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -281,7 +281,7 @@ private void maybeClean() { Utils.delete(dir); } } catch (IOException e) { - log.error("Failed to lock the state directory", e); + log.error("Failed to lock the state directory due to an unexpected exception", e); } finally { if (directoryLock != null) { try { From 0fc5c0b1a128e4bcdeb4cd9256c32a30c4a8c479 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 14 Sep 2015 17:08:17 -0700 Subject: [PATCH 238/275] add unit tests for PartitionGroup and RecordQueue --- .../internals/PartitionGroupTest.java | 99 +++++++++++++++ .../processor/internals/RecordQueueTest.java | 116 ++++++++++++++++++ .../processor/internals/StreamTaskTest.java | 38 +++--- .../kafka/test/MockTimestampExtractor.java | 3 +- 4 files changed, 236 insertions(+), 20 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java new file mode 100644 index 0000000000000..8cece285f5703 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import static org.junit.Assert.assertEquals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.test.MockSourceNode; +import org.apache.kafka.test.MockTimestampExtractor; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; + +public class PartitionGroupTest { + private final Serializer intSerializer = new IntegerSerializer(); + private final Deserializer intDeserializer = new IntegerDeserializer(); + private final TimestampExtractor timestampExtractor = new MockTimestampExtractor(); + private final TopicPartition partition1 = new TopicPartition("topic", 1); + private final TopicPartition partition2 = new TopicPartition("topic", 2); + private final RecordQueue queue1 = new RecordQueue(partition1, new MockSourceNode<>(intDeserializer, intDeserializer)); + private final RecordQueue queue2 = new RecordQueue(partition2, new MockSourceNode<>(intDeserializer, intDeserializer)); + + private final byte[] recordValue = intSerializer.serialize(null, 10); + private final byte[] recordKey = intSerializer.serialize(null, 1); + + private final PartitionGroup group = new PartitionGroup(new HashMap() { + { + put(partition1, queue1); + put(partition2, queue2); + } + }, timestampExtractor); + + @Test + public void testTimeTracking() { + assertEquals(0, group.numBuffered()); + + // add three 3 records with timestamp 1, 3, 5 to partition-1 + List> list1 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, recordKey, recordValue)); + + group.addRawRecords(partition1, list1); + + // add three 3 records with timestamp 2, 4, 6 to partition-2 + List> list2 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, recordKey, recordValue)); + + group.addRawRecords(partition2, list2); + + assertEquals(6, group.numBuffered()); + assertEquals(3, group.numBuffered(partition1)); + assertEquals(3, group.numBuffered(partition2)); + assertEquals(TimestampTracker.NOT_KNOWN, group.timestamp()); + + // get one record + assertEquals(partition1, group.nextQueue().partition()); + + assertEquals(1L, group.getRecord(group.nextQueue()).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 + assertEquals(partition2, group.nextQueue().partition()); + + assertEquals(2L, group.getRecord(group.nextQueue()).timestamp); + assertEquals(4, group.numBuffered()); + assertEquals(2, group.numBuffered(partition1)); + assertEquals(2, group.numBuffered(partition2)); + assertEquals(3L, group.timestamp()); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java new file mode 100644 index 0000000000000..6e86410b19ed6 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.test.MockSourceNode; +import org.apache.kafka.test.MockTimestampExtractor; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; + +public class RecordQueueTest { + private final Serializer intSerializer = new IntegerSerializer(); + private final Deserializer intDeserializer = new IntegerDeserializer(); + private final TimestampExtractor timestampExtractor = new MockTimestampExtractor(); + private final RecordQueue queue = new RecordQueue(new TopicPartition("topic", 1), new MockSourceNode<>(intDeserializer, intDeserializer)); + + private final byte[] recordValue = intSerializer.serialize(null, 10); + private final byte[] recordKey = intSerializer.serialize(null, 1); + + @Test + public void testTimeTracking() { + + assertTrue(queue.isEmpty()); + + // add three 3 out-of-order records with timestamp 2, 1, 3 + List> list1 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, recordKey, recordValue)); + + queue.addRawRecords(list1, timestampExtractor); + + assertEquals(3, queue.size()); + assertEquals(TimestampTracker.NOT_KNOWN, queue.timestamp()); + + // poll the first record, now with 1, 3 + assertEquals(2L, queue.poll().timestamp); + assertEquals(2, queue.size()); + assertEquals(1L, queue.timestamp()); + + // poll the second record, now with 3 + assertEquals(1L, queue.poll().timestamp); + assertEquals(1, queue.size()); + assertEquals(3L, queue.timestamp()); + + // add three 3 out-of-order records with timestamp 4, 1, 2 + // now with 3, 4, 1, 2 + List> list2 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue)); + + queue.addRawRecords(list2, timestampExtractor); + + assertEquals(4, queue.size()); + assertEquals(3L, queue.timestamp()); + + // poll the third record, now with 4, 1, 2 + assertEquals(3L, queue.poll().timestamp); + assertEquals(3, queue.size()); + assertEquals(3L, queue.timestamp()); + + // poll the rest records + assertEquals(4L, queue.poll().timestamp); + assertEquals(3L, queue.timestamp()); + + assertEquals(1L, queue.poll().timestamp); + assertEquals(3L, queue.timestamp()); + + assertEquals(2L, queue.poll().timestamp); + assertEquals(0, queue.size()); + assertEquals(3L, queue.timestamp()); + + // add three more records with 4, 5, 6 + List> list3 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, recordKey, recordValue)); + + queue.addRawRecords(list3, timestampExtractor); + + assertEquals(3, queue.size()); + assertEquals(3L, queue.timestamp()); + + // poll one record again, the timestamp should advance now + assertEquals(4L, queue.poll().timestamp); + assertEquals(2, queue.size()); + assertEquals(5L, queue.timestamp()); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 9c5826e9486b3..02874b6604b3c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -77,6 +77,10 @@ public class StreamTaskTest { private final MockProducer producer = new MockProducer<>(false, bytesSerializer, bytesSerializer); private final StreamTask task = new StreamTask(0, consumer, producer, partitions, topology, config); + 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)); @@ -85,18 +89,16 @@ public void setup() { @SuppressWarnings("unchecked") @Test public void testProcessOrder() { - byte[] recordValue = intSerializer.serialize(null, 10); - task.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 1, intSerializer.serialize(partition1.topic(), 10), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 2, intSerializer.serialize(partition1.topic(), 20), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 3, intSerializer.serialize(partition1.topic(), 30), recordValue) + 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(), 1, intSerializer.serialize(partition1.topic(), 25), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 2, intSerializer.serialize(partition1.topic(), 35), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 3, intSerializer.serialize(partition1.topic(), 45), recordValue) + 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); @@ -127,18 +129,16 @@ public void testProcessOrder() { @SuppressWarnings("unchecked") @Test public void testPauseResume() { - byte[] recordValue = intSerializer.serialize(null, 10); - task.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 1, intSerializer.serialize(partition1.topic(), 10), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 2, intSerializer.serialize(partition1.topic(), 20), recordValue) + 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(), 3, intSerializer.serialize(partition1.topic(), 35), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 4, intSerializer.serialize(partition1.topic(), 45), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 5, intSerializer.serialize(partition1.topic(), 55), recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 6, intSerializer.serialize(partition1.topic(), 65), recordValue) + 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); @@ -149,9 +149,9 @@ public void testPauseResume() { assertTrue(consumer.paused().contains(partition2)); task.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 3, intSerializer.serialize(partition1.topic(), 30), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 4, intSerializer.serialize(partition1.topic(), 40), recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 5, intSerializer.serialize(partition1.topic(), 50), recordValue) + 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); diff --git a/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java index a496314031d1d..274e7b5bd7d4c 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java +++ b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java @@ -20,10 +20,11 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.streams.processor.TimestampExtractor; +/* Extract the timestamp as the offset of the record */ public class MockTimestampExtractor implements TimestampExtractor { @Override public long extract(ConsumerRecord record) { - return ((Integer) record.key()).longValue(); + return record.offset(); } } From 1306509faba8a9779c8396ff0fe883ea53a5e8ad Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 15 Sep 2015 13:50:18 -0700 Subject: [PATCH 239/275] merge nextQueue and getRecord into nextRecord --- .../processor/internals/PartitionGroup.java | 43 +++++++++++-------- .../processor/internals/StreamTask.java | 24 ++++++----- .../internals/PartitionGroupTest.java | 15 ++++--- 3 files changed, 46 insertions(+), 36 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index bd9f6e284cfae..bf2dd8be6160e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -39,6 +39,18 @@ public class PartitionGroup { 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; @@ -64,36 +76,29 @@ public int compare(RecordQueue queue1, RecordQueue queue2) { } /** - * Get one record from the specified partition queue + * Get the next record and queue * * @return StampedRecord */ - public StampedRecord getRecord(RecordQueue queue) { - // get the first record from this queue. - StampedRecord record = queue.poll(); + public StampedRecord nextRecord(RecordInfo info) { + StampedRecord record = null; - // update the partition's timestamp and re-order it against other partitions. - queuesByTime.remove(queue); + RecordQueue queue = queuesByTime.poll(); + if (queue != null) { + // get the first record from this queue. + record = queue.poll(); - if (queue.size() > 0) { - queuesByTime.offer(queue); + if (queue.size() > 0) { + queuesByTime.offer(queue); + } } + info.queue = queue; - totalBuffered--; + if (record != null) totalBuffered--; return record; } - /** - * Get the next partition queue that has the lowest timestamp to process - * - * @return RecordQueue - */ - public RecordQueue nextQueue() { - // get the partition with the lowest timestamp - return queuesByTime.peek(); - } - /** * Adds raw records to this partition group * diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index bf177f304e7bf..c64f0638a628d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -49,6 +49,7 @@ public class StreamTask implements Punctuator { private final Consumer consumer; private final PartitionGroup partitionGroup; + private final PartitionGroup.RecordInfo recordInfo = new PartitionGroup.RecordInfo(); private final PunctuationQueue punctuationQueue; private final ProcessorContext processorContext; private final ProcessorTopology topology; @@ -155,18 +156,19 @@ public void addRecords(TopicPartition partition, Iterable Date: Wed, 16 Sep 2015 09:46:48 -0700 Subject: [PATCH 240/275] move mayPunctuate call out of a try block --- .../processor/internals/StreamTask.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index c64f0638a628d..ebe19e26db60b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -156,16 +156,15 @@ public void addRecords(TopicPartition partition, Iterable Date: Thu, 17 Sep 2015 11:15:48 -0700 Subject: [PATCH 241/275] add ProcessorStateManagerTest --- .../internals/ProcessorStateManager.java | 13 +- .../internals/ProcessorStateManagerTest.java | 427 ++++++++++++++++++ 2 files changed, 434 insertions(+), 6 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index cec6ecef393a2..f56b9c8763ceb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -40,7 +40,8 @@ public class ProcessorStateManager { private static final Logger log = LoggerFactory.getLogger(ProcessorStateManager.class); - private static final String CHECKPOINT_FILE_NAME = ".checkpoint"; + + public static final String CHECKPOINT_FILE_NAME = ".checkpoint"; public static final String LOCK_FILE_NAME = ".lock"; private final int id; @@ -148,7 +149,7 @@ public void register(StateStore store, RestoreFunc restoreFunc) { // restore its state from changelog records; while restoring the log end offset // should not change since it is only written by this thread. while (true) { - for (ConsumerRecord record : restoreConsumer.poll(100)) { + for (ConsumerRecord record : restoreConsumer.poll(100).records(storePartition)) { restoreFunc.apply(record.key(), record.value()); } @@ -163,8 +164,8 @@ public void register(StateStore store, RestoreFunc restoreFunc) { long newOffset = restoreConsumer.position(storePartition); restoredOffsets.put(storePartition, newOffset); - // un-subscribe the change log partition - restoreConsumer.subscribe(Collections.emptyList()); + // un-assign the change log partition + restoreConsumer.assign(Collections.emptyList()); } public StateStore getStore(String name) { @@ -184,8 +185,8 @@ public void cleanup() throws IOException { public void flush() { if (!this.stores.isEmpty()) { log.debug("Flushing stores."); - for (StateStore engine : this.stores.values()) - engine.flush(); + for (StateStore store : this.stores.values()) + store.flush(); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java new file mode 100644 index 0000000000000..c983c6b1a3a6e --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -0,0 +1,427 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.processor.RestoreFunc; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.OffsetCheckpoint; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileLock; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; + +public class ProcessorStateManagerTest { + + private static class MockStateStore implements StateStore { + private final String name; + private final boolean persistent; + + public boolean flushed = false; + public boolean closed = false; + public final ArrayList keys = new ArrayList<>(); + + public MockStateStore(String name, boolean persistent) { + this.name = name; + this.persistent = persistent; + } + @Override + public String name() { + return name; + } + @Override + public void flush() { + flushed = true; + } + @Override + public void close() { + closed = true; + } + @Override + public boolean persistent() { + return persistent; + } + + public final RestoreFunc restoreFunc = new RestoreFunc() { + private final Deserializer deserializer = new IntegerDeserializer(); + + @Override + public void apply(byte[] key, byte[] value) { + keys.add(deserializer.deserialize("", key)); + } + }; + } + + private static class MockRestoreConsumer extends MockConsumer { + private final Serializer serializer = new IntegerSerializer(); + + public TopicPartition assignedPartition = null; + public TopicPartition seekPartition = null; + public long seekOffset = -1L; + public boolean seekToBeginingCalled = false; + public boolean seekToEndCalled = false; + private long endOffset = -1L; + private long currentOffset = -1L; + + private ArrayList> recordBuffer = new ArrayList<>(); + + MockRestoreConsumer() { + super(OffsetResetStrategy.EARLIEST); + } + + // prepares this mock restore consumer for a state store registration + public void prepare() { + assignedPartition = null; + seekOffset = -1L; + seekToBeginingCalled = false; + seekToEndCalled = false; + endOffset = -1L; + recordBuffer.clear(); + } + + // buffer a record (we cannot use addRecord because we need to add records before asigning a partition) + public void bufferRecord(ConsumerRecord record) { + recordBuffer.add( + new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), + serializer.serialize(record.topic(), record.key()), + serializer.serialize(record.topic(), record.value()))); + endOffset = record.offset(); + } + + @Override + public synchronized void assign(List partitions) { + int numPartitions = partitions.size(); + if (numPartitions > 1) + throw new IllegalArgumentException("RestoreConsumer: more than one partition specified"); + + if (numPartitions == 1) { + if (assignedPartition != null) + throw new IllegalStateException("RestoreConsumer: partition already assigned"); + assignedPartition = partitions.get(0); + } + + super.assign(partitions); + } + + @Override + public ConsumerRecords poll(long timeout) { + // add buffered records to MockConsumer + for (ConsumerRecord record : recordBuffer) { + super.addRecord(record); + } + recordBuffer.clear(); + + ConsumerRecords records = super.poll(timeout); + + // set the current offset + Iterable> partitionRecords = records.records(assignedPartition); + for (ConsumerRecord record : partitionRecords) { + currentOffset = record.offset(); + } + + return records; + } + + @Override + public synchronized long position(TopicPartition partition) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: unassigned partition"); + + return currentOffset; + } + + @Override + public synchronized void seek(TopicPartition partition, long offset) { + if (offset < 0) + throw new IllegalArgumentException("RestoreConsumer: offset should not be negative"); + + if (seekOffset >= 0) + throw new IllegalStateException("RestoreConsumer: offset already seeked"); + + seekPartition = partition; + seekOffset = offset; + currentOffset = offset; + super.seek(partition, offset); + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + if (seekToBeginingCalled) + throw new IllegalStateException("RestoreConsumer: offset already seeked to beginning"); + + seekToBeginingCalled = true; + } + + @Override + public synchronized void seekToEnd(TopicPartition... partitions) { + if (seekToEndCalled) + throw new IllegalStateException("RestoreConsumer: offset already seeked to beginning"); + + seekToEndCalled = true; + currentOffset = endOffset; + } + } + + @Test + public void testLockStateDirectory() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + FileLock lock; + + // the state manager locks the directory + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, new MockRestoreConsumer()); + + try { + // this should not get the lock + lock = ProcessorStateManager.lockStateDirectory(baseDir); + assertNull(lock); + } finally { + // by closing the state manager, release the lock + stateMgr.close(Collections.emptyMap()); + } + + // now, this should get the lock + lock = ProcessorStateManager.lockStateDirectory(baseDir); + try { + assertNotNull(lock); + } finally { + if (lock != null) lock.release(); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test(expected = IllegalStateException.class) + public void testNoTopic() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + MockStateStore mockStateStore = new MockStateStore("mockStore", false); + + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, new MockRestoreConsumer()); + try { + stateMgr.register(mockStateStore, mockStateStore.restoreFunc); + } finally { + stateMgr.close(Collections.emptyMap()); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testRegisterPersistentStore() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + long lastCheckpointedOffset = 10L; + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME)); + checkpoint.write(Collections.singletonMap(new TopicPartition("persistentStore", 2), lastCheckpointedOffset)); + + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("persistentStore", Arrays.asList( + new PartitionInfo("persistentStore", 1, Node.noNode(), new Node[0], new Node[0]), + new PartitionInfo("persistentStore", 2, Node.noNode(), new Node[0], new Node[0]) + )); + + MockStateStore persistentStore = new MockStateStore("persistentStore", false); // non persistent store + + ProcessorStateManager stateMgr = new ProcessorStateManager(2, baseDir, restoreConsumer); + try { + restoreConsumer.prepare(); + + ArrayList expectedKeys = new ArrayList<>(); + for (int i = 1; i <= 3; i++) { + long offset = (long) i; + int key = i * 10; + expectedKeys.add(key); + restoreConsumer.bufferRecord( + new ConsumerRecord<>("persistentStore", 2, offset, key, 0) + ); + } + + stateMgr.register(persistentStore, persistentStore.restoreFunc); + + assertEquals(new TopicPartition("persistentStore", 2), restoreConsumer.assignedPartition); + assertEquals(lastCheckpointedOffset, restoreConsumer.seekOffset); + assertFalse(restoreConsumer.seekToBeginingCalled); + assertTrue(restoreConsumer.seekToEndCalled); + assertEquals(expectedKeys, persistentStore.keys); + + } finally { + stateMgr.close(Collections.emptyMap()); + } + + } finally { + Utils.delete(baseDir); + } + } + @Test + public void testRegisterNonPersistentStore() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + long lastCheckpointedOffset = 10L; + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME)); + checkpoint.write(Collections.singletonMap(new TopicPartition("persistentStore", 2), lastCheckpointedOffset)); + + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("nonPersistentStore", Arrays.asList( + new PartitionInfo("nonPersistentStore", 1, Node.noNode(), new Node[0], new Node[0]), + new PartitionInfo("nonPersistentStore", 2, Node.noNode(), new Node[0], new Node[0]) + )); + + MockStateStore nonPersistentStore = new MockStateStore("nonPersistentStore", true); // persistent store + + ProcessorStateManager stateMgr = new ProcessorStateManager(2, baseDir, restoreConsumer); + try { + restoreConsumer.prepare(); + + ArrayList expectedKeys = new ArrayList<>(); + for (int i = 1; i <= 3; i++) { + long offset = (long) (i + 100); + int key = i; + expectedKeys.add(i); + restoreConsumer.bufferRecord( + new ConsumerRecord<>("nonPersistentStore", 2, offset, key, 0) + ); + } + + stateMgr.register(nonPersistentStore, nonPersistentStore.restoreFunc); + + assertEquals(new TopicPartition("nonPersistentStore", 2), restoreConsumer.assignedPartition); + assertEquals(-1L, restoreConsumer.seekOffset); + assertTrue(restoreConsumer.seekToBeginingCalled); + assertTrue(restoreConsumer.seekToEndCalled); + assertEquals(expectedKeys, nonPersistentStore.keys); + } finally { + stateMgr.close(Collections.emptyMap()); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testGetStore() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("mockStore", Arrays.asList( + new PartitionInfo("mockStore", 1, Node.noNode(), new Node[0], new Node[0]) + )); + + MockStateStore mockStateStore = new MockStateStore("mockStore", false); + + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, restoreConsumer); + try { + stateMgr.register(mockStateStore, mockStateStore.restoreFunc); + + assertNull(stateMgr.getStore("noSuchStore")); + assertEquals(mockStateStore, stateMgr.getStore("mockStore")); + + } finally { + stateMgr.close(Collections.emptyMap()); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testClose() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + File checkpointFile = new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME); + try { + // write an empty checkpoint file + OffsetCheckpoint oldCheckpoint = new OffsetCheckpoint(checkpointFile); + oldCheckpoint.write(Collections.emptyMap()); + + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("persistentStore", Arrays.asList( + new PartitionInfo("persistentStore", 1, Node.noNode(), new Node[0], new Node[0]) + )); + restoreConsumer.updatePartitions("nonPersistentStore", Arrays.asList( + new PartitionInfo("nonPersistentStore", 1, Node.noNode(), new Node[0], new Node[0]) + )); + + // set up ack'ed offsets + HashMap ackedOffsets = new HashMap<>(); + ackedOffsets.put(new TopicPartition("persistentStore", 1), 123L); + ackedOffsets.put(new TopicPartition("nonPersistentStore", 1), 456L); + ackedOffsets.put(new TopicPartition("otherTopic", 1), 789L); + + MockStateStore persistentStore = new MockStateStore("persistentStore", true); + MockStateStore nonPersistentStore = new MockStateStore("nonPersistentStore", false); + + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, restoreConsumer); + try { + // make sure the checkpoint file is deleted + assertFalse(checkpointFile.exists()); + + restoreConsumer.prepare(); + stateMgr.register(persistentStore, persistentStore.restoreFunc); + + restoreConsumer.prepare(); + stateMgr.register(nonPersistentStore, nonPersistentStore.restoreFunc); + } finally { + // close the state manager with the ack'ed offsets + stateMgr.close(ackedOffsets); + } + + // make sure all stores are closed, and the checkpoint file is written. + assertTrue(persistentStore.flushed); + assertTrue(persistentStore.closed); + assertTrue(nonPersistentStore.flushed); + assertTrue(nonPersistentStore.closed); + assertTrue(checkpointFile.exists()); + + // the checkpoint file should contain an offset from the persistent store only. + OffsetCheckpoint newCheckpoint = new OffsetCheckpoint(checkpointFile); + Map checkpointedOffsets = newCheckpoint.read(); + assertEquals(1, checkpointedOffsets.size()); + assertEquals(new Long(123L + 1L), checkpointedOffsets.get(new TopicPartition("persistentStore", 1))); + } finally { + Utils.delete(baseDir); + } + } + +} From fa508ef37e4cc64c319d2b4b889be26101165ad0 Mon Sep 17 00:00:00 2001 From: Jesse Anderson Date: Thu, 17 Sep 2015 16:48:51 -0700 Subject: [PATCH 242/275] The greater than cannot be used in JavaDoc. --- .../src/main/java/org/apache/kafka/streams/KafkaStreaming.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java index b943752081967..c00cf4d6fba16 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java @@ -51,7 +51,7 @@ * StreamingConfig config = new StreamingConfig(props); * * KStreamBuilder builder = new KStreamBuilder(); - * builder.from("topic1").mapValue(value -> value.length()).sendTo("topic2"); + * builder.from("topic1").mapValue(value -> value.length()).sendTo("topic2"); * * KafkaStreaming streaming = new KafkaStreaming(builder, config); * streaming.start(); From bb09ba9c3cc876d8eb084b4e57afed9799dd290f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 18 Sep 2015 16:54:55 -0500 Subject: [PATCH 243/275] address some comments --- .../apache/kafka/streams/kstream/SlidingWindowDef.java | 3 ++- .../streams/processor/internals/PartitionGroup.java | 3 ++- .../kafka/streams/processor/internals/StreamTask.java | 10 +++++++--- 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowDef.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowDef.java index 51937fb2db330..cc03541796cd0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowDef.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowDef.java @@ -31,6 +31,7 @@ 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; @@ -113,7 +114,7 @@ private Iterator find(K key, final long startTime, final long endTime) { final ValueList values = map.get(key); if (values == null) { - return null; + return Collections.emptyIterator(); } else { return new FilteredIterator>(values.iterator()) { @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index bf2dd8be6160e..4632798dc65da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.processor.TimestampExtractor; import java.util.Comparator; +import java.util.HashSet; import java.util.Map; import java.util.PriorityQueue; import java.util.Set; @@ -123,7 +124,7 @@ public int addRawRecords(TopicPartition partition, Iterable partitions() { - return partitionQueues.keySet(); + return new HashSet<>(partitionQueues.keySet()); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index ebe19e26db60b..1a6fe295001fc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -205,7 +205,8 @@ public int process() { @Override public void punctuate(ProcessorNode node, long streamTime) { - if (currNode != null) throw new IllegalStateException("Current node is not null"); + if (currNode != null) + throw new IllegalStateException("Current node is not null"); currNode = node; try { @@ -259,7 +260,8 @@ public void needCommit() { * @param interval the interval in milliseconds */ public void schedule(long interval) { - if (currNode == null) throw new IllegalStateException("Current node is null"); + if (currNode == null) + throw new IllegalStateException("Current node is null"); punctuationQueue.schedule(new PunctuationSchedule(currNode, interval)); } @@ -281,7 +283,9 @@ public void close() { currNode = null; } } - if (exception != null) throw exception; + + if (exception != null) + throw exception; try { ((ProcessorContextImpl) processorContext).stateManager().close(recordCollector.offsets()); From 789a33f75fc121d5954e10129952e3254a39e12f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 21 Sep 2015 10:34:23 -0700 Subject: [PATCH 244/275] Yasu's comments --- .../kafka/streams/processor/internals/PartitionGroup.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index 4632798dc65da..44a6c5c1a2f3d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -22,8 +22,8 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.processor.TimestampExtractor; +import java.util.Collections; import java.util.Comparator; -import java.util.HashSet; import java.util.Map; import java.util.PriorityQueue; import java.util.Set; @@ -124,7 +124,7 @@ public int addRawRecords(TopicPartition partition, Iterable partitions() { - return new HashSet<>(partitionQueues.keySet()); + return Collections.unmodifiableSet(partitionQueues.keySet()); } /** From f438e8b2ec96b2907ae03647d84b5edf83155b03 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 21 Sep 2015 11:08:37 -0700 Subject: [PATCH 245/275] add StreamThreadTest --- .../processor/internals/StreamThread.java | 46 ++++-- .../processor/internals/StreamThreadTest.java | 140 ++++++++++++++++++ 2 files changed, 171 insertions(+), 15 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 8af18e7075e3e..5c52553884ef2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -47,6 +47,7 @@ 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.Map; @@ -78,7 +79,7 @@ public class StreamThread extends Thread { private long lastCommit; private long recordsProcessed; - protected final ConsumerRebalanceListener rebalanceCallback = new ConsumerRebalanceListener() { + final ConsumerRebalanceListener rebalanceListener = new ConsumerRebalanceListener() { @Override public void onPartitionsAssigned(Consumer consumer, Collection assignment) { addPartitions(assignment); @@ -93,25 +94,22 @@ public void onPartitionsRevoked(Consumer consumer, Collection producer, + Consumer consumer) throws Exception { super("StreamThread-" + nextThreadNumber.getAndIncrement()); this.config = config; this.builder = builder; - // create the producer and consumer clients - log.info("Creating producer client for stream thread [" + this.getName() + "]"); - - this.producer = new KafkaProducer<>(config.getProducerConfigs(), - new ByteArraySerializer(), - new ByteArraySerializer()); - - log.info("Creating consumer client for stream thread [" + this.getName() + "]"); - - this.consumer = new KafkaConsumer<>(config.getConsumerConfigs(), - new ByteArrayDeserializer(), - new ByteArrayDeserializer()); + // set the producer and consumer clients + this.producer = (producer != null) ? producer : createProducer(); + this.consumer = (consumer != null) ? consumer : createConsumer(); // initialize the task list this.tasks = new HashMap<>(); @@ -134,6 +132,20 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce this.running = new AtomicBoolean(true); } + private Producer createProducer() { + log.info("Creating producer client for stream thread [" + this.getName() + "]"); + return new KafkaProducer<>(config.getProducerConfigs(), + new ByteArraySerializer(), + new ByteArraySerializer()); + } + + private Consumer createConsumer() { + log.info("Creating consumer client for stream thread [" + this.getName() + "]"); + return new KafkaConsumer<>(config.getConsumerConfigs(), + new ByteArrayDeserializer(), + new ByteArrayDeserializer()); + } + /** * Execute the stream processors */ @@ -158,6 +170,10 @@ public void close() { running.set(false); } + public Map tasks() { + return Collections.unmodifiableMap(tasks); + } + private void shutdown() { log.info("Shutting down stream thread [" + this.getName() + "]"); @@ -190,7 +206,7 @@ private void runLoop() { try { int totalNumBuffered = 0; - consumer.subscribe(new ArrayList<>(builder.sourceTopics()), rebalanceCallback); + consumer.subscribe(new ArrayList<>(builder.sourceTopics()), rebalanceListener); while (stillRunning()) { // try to fetch some records if necessary diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java new file mode 100644 index 0000000000000..6de55e661aa2b --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.clients.consumer.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.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Properties; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +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 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 ByteArraySerializer serializer = new ByteArraySerializer(); + + @SuppressWarnings("unchecked") + @Test + public void testPartitionAssignmentChange() throws Exception { + MockProducer producer = new MockProducer(true, serializer, serializer); + MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + + TopologyBuilder builder = new TopologyBuilder(); + builder.addSource("source1", "topic1"); + builder.addSource("source2", "topic2"); + + StreamThread thread = new StreamThread(builder, config, producer, consumer); + + ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; + + assertTrue(thread.tasks().isEmpty()); + + List revokedPartitions; + List assignedPartitions; + Set expectedGroup1; + Set expectedGroup2; + + revokedPartitions = Collections.emptyList(); + assignedPartitions = Collections.singletonList(t1p1); + expectedGroup1 = new HashSet<>(Arrays.asList(t1p1)); + + rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); + rebalanceListener.onPartitionsAssigned(consumer, 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(consumer, revokedPartitions); + rebalanceListener.onPartitionsAssigned(consumer, 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(consumer, revokedPartitions); + rebalanceListener.onPartitionsAssigned(consumer, 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(consumer, revokedPartitions); + rebalanceListener.onPartitionsAssigned(consumer, 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(consumer, revokedPartitions); + rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + + assertTrue(thread.tasks().isEmpty()); + } +} From 91db6712d6b2bba388355d333eccffbf6c91175f Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 21 Sep 2015 11:18:43 -0700 Subject: [PATCH 246/275] fix generics --- .../kafka/streams/processor/internals/StreamThreadTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 6de55e661aa2b..150265466e6c4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -61,7 +61,7 @@ public class StreamThreadTest { @SuppressWarnings("unchecked") @Test public void testPartitionAssignmentChange() throws Exception { - MockProducer producer = new MockProducer(true, serializer, serializer); + MockProducer producer = new MockProducer<>(true, serializer, serializer); MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); TopologyBuilder builder = new TopologyBuilder(); From edc218604c1998ed1e0e11c920dc4a377d6b95db Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 21 Sep 2015 11:23:53 -0700 Subject: [PATCH 247/275] rebase on apache trunk --- .../streams/processor/internals/ProcessorStateManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index f56b9c8763ceb..3c7e34e953507 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -165,7 +165,7 @@ public void register(StateStore store, RestoreFunc restoreFunc) { restoredOffsets.put(storePartition, newOffset); // un-assign the change log partition - restoreConsumer.assign(Collections.emptyList()); + restoreConsumer.unsubscribe(); } public StateStore getStore(String name) { From 37db015068a13e935d943a4b52b6860896830aae Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 21 Sep 2015 11:50:30 -0700 Subject: [PATCH 248/275] revork changes of unsubscribe() --- .../streams/processor/internals/ProcessorStateManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 3c7e34e953507..f56b9c8763ceb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -165,7 +165,7 @@ public void register(StateStore store, RestoreFunc restoreFunc) { restoredOffsets.put(storePartition, newOffset); // un-assign the change log partition - restoreConsumer.unsubscribe(); + restoreConsumer.assign(Collections.emptyList()); } public StateStore getStore(String name) { From b4e1f19a302aaf851d59a7979b59a09070dde294 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 21 Sep 2015 11:53:12 -0700 Subject: [PATCH 249/275] use commitSync() --- .../apache/kafka/streams/processor/internals/StreamTask.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 1a6fe295001fc..39ec04904eae9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.consumer.CommitType; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.Producer; @@ -241,7 +240,7 @@ public void commit() { // 3) commit consumed offsets if it is dirty already if (commitOffsetNeeded) { - consumer.commit(consumedOffsets, CommitType.SYNC); + consumer.commitSync(consumedOffsets); commitOffsetNeeded = false; } commitRequested = false; From 1aff413635071d2e38954c443aa837378e02f594 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 21 Sep 2015 13:46:32 -0700 Subject: [PATCH 250/275] add a test for statedir cleanup and fix a bug --- .../processor/internals/StreamThread.java | 13 +- .../processor/internals/StreamThreadTest.java | 144 ++++++++++++++++-- 2 files changed, 139 insertions(+), 18 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 5c52553884ef2..099065dcfa814 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -95,13 +95,14 @@ public void onPartitionsRevoked(Consumer consumer, Collection producer, - Consumer consumer) throws Exception { + Consumer consumer, + Time time) throws Exception { super("StreamThread-" + nextThreadNumber.getAndIncrement()); this.config = config; @@ -125,7 +126,7 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce this.lastClean = Long.MAX_VALUE; // the cleaning cycle won't start until partition assignment this.lastCommit = 0; this.recordsProcessed = 0; - this.time = new SystemTime(); + this.time = time; this.metrics = new KafkaStreamingMetrics(); @@ -278,10 +279,10 @@ private void commitAll(long now) { /** * Cleanup any states of the tasks that have been removed from this thread */ - private void maybeClean() { + protected void maybeClean() { long now = time.milliseconds(); - if (now > lastClean) { + if (now > lastClean + cleanTimeMs) { File[] stateDirs = stateDir.listFiles(); if (stateDirs != null) { for (File dir : stateDirs) { @@ -343,7 +344,7 @@ private void addPartitions(Collection assignment) { } } - lastClean = time.milliseconds() + cleanTimeMs; + lastClean = time.milliseconds(); } private void removePartitions() { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 150265466e6c4..b364e47f14f82 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -23,10 +23,15 @@ 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.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.Collections; import java.util.HashSet; @@ -35,6 +40,7 @@ import java.util.Set; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class StreamThreadTest { @@ -44,23 +50,27 @@ public class StreamThreadTest { private TopicPartition t2p1 = new TopicPartition("topic2", 1); private TopicPartition t2p2 = new TopicPartition("topic2", 2); - 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 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 ByteArraySerializer serializer = new ByteArraySerializer(); @SuppressWarnings("unchecked") @Test public void testPartitionAssignmentChange() throws Exception { + StreamingConfig config = new StreamingConfig(configProps()); + MockProducer producer = new MockProducer<>(true, serializer, serializer); MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); @@ -68,7 +78,7 @@ public void testPartitionAssignmentChange() throws Exception { builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); - StreamThread thread = new StreamThread(builder, config, producer, consumer); + StreamThread thread = new StreamThread(builder, config, producer, consumer, new SystemTime()); ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; @@ -137,4 +147,114 @@ public void testPartitionAssignmentChange() throws Exception { assertTrue(thread.tasks().isEmpty()); } + + @Test + public void testMaybeClean() throws Exception { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + final long cleanupDelay = 1000L; + Properties props = configProps(); + props.setProperty(StreamingConfig.STATE_CLEANUP_DELAY_MS_CONFIG, Long.toString(cleanupDelay)); + props.setProperty(StreamingConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); + + StreamingConfig config = new StreamingConfig(props); + + File stateDir1 = new File(baseDir, "1"); + File stateDir2 = new File(baseDir, "2"); + File stateDir3 = new File(baseDir, "3"); + File extraDir = new File(baseDir, "X"); + stateDir1.mkdir(); + stateDir2.mkdir(); + stateDir3.mkdir(); + extraDir.mkdir(); + + MockProducer producer = new MockProducer<>(true, serializer, serializer); + MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + MockTime mockTime = new MockTime(); + + TopologyBuilder builder = new TopologyBuilder(); + builder.addSource("source1", "topic1"); + + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockTime) { + @Override + public void maybeClean() { + super.maybeClean(); + } + }; + + ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; + + assertTrue(thread.tasks().isEmpty()); + mockTime.sleep(cleanupDelay); + + // all directories exist since an assignment didn't happen + assertTrue(stateDir1.exists()); + assertTrue(stateDir2.exists()); + assertTrue(stateDir3.exists()); + assertTrue(extraDir.exists()); + + List revokedPartitions; + List assignedPartitions; + + revokedPartitions = Collections.emptyList(); + assignedPartitions = Arrays.asList(t1p1, t1p2); + + rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); + rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + + // 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()); + + revokedPartitions = Collections.emptyList(); + assignedPartitions = Arrays.asList(t1p1, t1p2); + + rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); + rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + + revokedPartitions = assignedPartitions; + assignedPartitions = Collections.emptyList(); + + rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); + rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + + // 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); + + } + } } From 4ff39daef1f8ea9090b86451461b47f9325525d3 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 21 Sep 2015 14:29:36 -0700 Subject: [PATCH 251/275] check commits --- .../processor/internals/StreamThread.java | 16 ++++--- .../processor/internals/StreamThreadTest.java | 47 ++++++++++++++++--- 2 files changed, 51 insertions(+), 12 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 099065dcfa814..4109e047a0347 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -61,19 +61,19 @@ public class StreamThread extends Thread { private final AtomicBoolean running; - private final TopologyBuilder builder; - private final Producer producer; - private final Consumer consumer; + protected final StreamingConfig config; + protected final TopologyBuilder builder; + protected final Producer producer; + protected final Consumer consumer; + private final Map tasks; private final Time time; - private final File stateDir; private final long pollTimeMs; private final long cleanTimeMs; private final long commitTimeMs; private final long totalRecordsToProcess; private final KafkaStreamingMetrics metrics; - private final StreamingConfig config; private long lastClean; private long lastCommit; @@ -319,6 +319,10 @@ protected void maybeClean() { } } + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + return new StreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); + } + private void addPartitions(Collection assignment) { HashSet partitions = new HashSet<>(assignment); @@ -335,7 +339,7 @@ private void addPartitions(Collection assignment) { // create the task try { - task = new StreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); + task = createStreamTask(id, partitionsForTask); } catch (Exception e) { log.error("Failed to create a task #" + id + " in thread [" + this.getName() + "]: ", e); throw e; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index b364e47f14f82..71e5b350479f9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -17,10 +17,12 @@ 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.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; @@ -33,9 +35,12 @@ 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; @@ -64,6 +69,25 @@ private Properties configProps() { }; } + private static class TestStreamTask extends StreamTask { + public boolean committed = false; + + public TestStreamTask(int id, + Consumer consumer, + Producer producer, + Collection partitions, + ProcessorTopology topology, + StreamingConfig config) { + super(id, consumer, producer, partitions, topology, config); + } + + @Override + public void commit() { + super.commit(); + committed = true; + } + } + private ByteArraySerializer serializer = new ByteArraySerializer(); @SuppressWarnings("unchecked") @@ -180,6 +204,10 @@ public void testMaybeClean() throws Exception { public void maybeClean() { super.maybeClean(); } + @Override + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + return new TestStreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); + } }; ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; @@ -195,13 +223,18 @@ public void maybeClean() { List revokedPartitions; List assignedPartitions; + Map prevTasks; revokedPartitions = Collections.emptyList(); assignedPartitions = Arrays.asList(t1p1, t1p2); + prevTasks = new HashMap(thread.tasks()); rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + // there shouldn't be any previous task + assertTrue(prevTasks.isEmpty()); + // task 1 & 2 are created assertEquals(2, thread.tasks().size()); @@ -221,18 +254,20 @@ public void maybeClean() { assertFalse(stateDir3.exists()); assertTrue(extraDir.exists()); - revokedPartitions = Collections.emptyList(); - assignedPartitions = Arrays.asList(t1p1, t1p2); - - rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); - rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); - revokedPartitions = assignedPartitions; assignedPartitions = Collections.emptyList(); + prevTasks = new HashMap(thread.tasks()); rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); rebalanceListener.onPartitionsAssigned(consumer, 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()); From 516d0c9de5354131f21ec5452521861c9ed97c39 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 21 Sep 2015 14:33:32 -0700 Subject: [PATCH 252/275] remove redundant lines --- .../processor/internals/StreamThreadTest.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index b364e47f14f82..b80fda99a73eb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -196,6 +196,9 @@ public void maybeClean() { List revokedPartitions; List assignedPartitions; + // + // Assign t1p1 and t1p2. This should create Task 1 & 2 + // revokedPartitions = Collections.emptyList(); assignedPartitions = Arrays.asList(t1p1, t1p2); @@ -221,12 +224,9 @@ public void maybeClean() { assertFalse(stateDir3.exists()); assertTrue(extraDir.exists()); - revokedPartitions = Collections.emptyList(); - assignedPartitions = Arrays.asList(t1p1, t1p2); - - rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); - rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); - + // + // Revoke t1p1 and t1p2. This should remove Task 1 & 2 + // revokedPartitions = assignedPartitions; assignedPartitions = Collections.emptyList(); From f0453c9e27cea276274f96d9089286778407d271 Mon Sep 17 00:00:00 2001 From: Jesse Anderson Date: Mon, 21 Sep 2015 14:55:21 -0700 Subject: [PATCH 253/275] Added serializer and deserializers for longs. --- .../serialization/LongDeserializer.java | 44 +++++++++++++++++++ .../common/serialization/LongSerializer.java | 42 ++++++++++++++++++ 2 files changed, 86 insertions(+) create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java new file mode 100644 index 0000000000000..37983e4a4c031 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.serialization; + +import org.apache.kafka.common.errors.SerializationException; + +import java.util.Map; + +public class LongDeserializer implements Deserializer { + + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + public Long deserialize(String topic, byte[] data) { + if (data == null) + return null; + if (data.length != 8) { + throw new SerializationException("Size of data received by LongDeserializer is " + + "not 8"); + } + + long value = 0; + for (byte b : data) { + value <<= 8; + value |= b & 0xFF; + } + return value; + } + + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java new file mode 100644 index 0000000000000..31005299e7a41 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.serialization; + +import java.util.Map; + +public class LongSerializer implements Serializer { + + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + public byte[] serialize(String topic, Long data) { + if (data == null) + return null; + + return new byte[] { + (byte) (data >>> 56), + (byte) (data >>> 48), + (byte) (data >>> 40), + (byte) (data >>> 32), + (byte) (data >>> 24), + (byte) (data >>> 16), + (byte) (data >>> 8), + data.byteValue() + }; + } + + public void close() { + // nothing to do + } +} \ No newline at end of file From e8e98104dfdd588c40284ea0bd8173414c6524f3 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 21 Sep 2015 14:57:20 -0700 Subject: [PATCH 254/275] added testMaybeCommit --- .../processor/internals/StreamThread.java | 2 +- .../processor/internals/StreamThreadTest.java | 79 +++++++++++++++++++ 2 files changed, 80 insertions(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 4109e047a0347..40fe483836e0a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -248,7 +248,7 @@ private boolean stillRunning() { return true; } - private void maybeCommit() { + protected void maybeCommit() { long now = time.milliseconds(); if (commitTimeMs >= 0 && lastCommit + commitTimeMs < now) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index b2d3bf36b49b5..e4a310932c44d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -295,7 +295,86 @@ protected StreamTask createStreamTask(int id, Collection partiti } finally { Utils.delete(baseDir); + } + } + + @Test + public void testMaybeCommit() throws Exception { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + final long commitInterval = 1000L; + Properties props = configProps(); + props.setProperty(StreamingConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); + props.setProperty(StreamingConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(commitInterval)); + + StreamingConfig config = new StreamingConfig(props); + + MockProducer producer = new MockProducer<>(true, serializer, serializer); + MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + MockTime mockTime = new MockTime(); + + TopologyBuilder builder = new TopologyBuilder(); + builder.addSource("source1", "topic1"); + + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockTime) { + @Override + public void maybeCommit() { + super.maybeCommit(); + } + @Override + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + return new TestStreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); + } + }; + + ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; + + List revokedPartitions; + List assignedPartitions; + + // + // Assign t1p1 and t1p2. This should create Task 1 & 2 + // + revokedPartitions = Collections.emptyList(); + assignedPartitions = Arrays.asList(t1p1, t1p2); + + rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); + rebalanceListener.onPartitionsAssigned(consumer, 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); } } } From 3c3338380d8e0817529198f50732bbf029dbd3ef Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 21 Sep 2015 18:18:38 -0700 Subject: [PATCH 255/275] add unit tests for KStreamImpl --- .../kafka/streams/examples/KStreamJob.java | 5 +- .../kafka/streams/examples/ProcessorJob.java | 5 +- .../apache/kafka/streams/kstream/KStream.java | 12 +- .../kstream/internals/KStreamImpl.java | 5 + .../streams/kstream/KStreamBuilderTest.java | 34 +++++ .../kstream/internals/KStreamImplTest.java | 138 ++++++++++++++++++ 6 files changed, 192 insertions(+), 7 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java index c7c7bbb47f6e6..891398d28d607 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java @@ -28,13 +28,12 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Predicate; -import java.util.HashMap; -import java.util.Map; +import java.util.Properties; public class KStreamJob { public static void main(String[] args) throws Exception { - Map props = new HashMap<>(); + Properties props = new Properties(); 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); diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java index 7d9773557585b..87b312c3b8230 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java @@ -32,8 +32,7 @@ import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import java.util.HashMap; -import java.util.Map; +import java.util.Properties; public class ProcessorJob { @@ -87,7 +86,7 @@ public void close() { } public static void main(String[] args) throws Exception { - Map props = new HashMap<>(); + Properties props = new Properties(); 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); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index ca2f809065883..14b2f14d563e1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -101,7 +101,17 @@ public interface KStream { /** * Sends key-value to a topic, also creates a new stream from the topic. - * The created stream is added to the default synchronization group. + * This is equivalent to calling sendTo(topic) and from(topic). + * + * @param topic the topic name + * @param the key type of the new stream + * @param the value type of the new stream + * @return KStream + */ + KStream through(String topic); + + /** + * Sends key-value to a topic, also creates a new stream from the topic. * This is equivalent to calling sendTo(topic) and from(topic). * * @param topic the topic name diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 7750568d0db30..6a67cb750f7ce 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -171,6 +171,11 @@ public KStream through(String topic, return new KStreamImpl<>(topology, sourceName); } + @Override + public KStream through(String topic) { + return through(topic, (Serializer) null, (Serializer) null, (Deserializer) null, (Deserializer) null); + } + @Override public void sendTo(String topic) { String name = SEND_NAME + INDEX.getAndIncrement(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java new file mode 100644 index 0000000000000..49171e340b356 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream; + +import org.apache.kafka.streams.kstream.internals.KStreamImpl; +import org.apache.kafka.streams.processor.TopologyException; +import org.junit.Test; + +public class KStreamBuilderTest { + + @Test(expected = TopologyException.class) + public void testFrom() { + final KStreamBuilder builder = new KStreamBuilder(); + + builder.from("topic-1", "topic-2"); + + builder.addSource(KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.decrementAndGet(), "topic-3"); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java new file mode 100644 index 0000000000000..2f82cf339dea5 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -0,0 +1,138 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.test.MockProcessorDef; +import org.apache.kafka.test.UnlimitedWindowDef; +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; + + +public class KStreamImplTest { + + @Test + public void testNumProcesses() { + final Deserializer deserializer = new StringDeserializer(); + final KStreamBuilder builder = new KStreamBuilder(); + + KStream source1 = builder.from(deserializer, deserializer, "topic-1", "topic-2"); + + KStream source2 = builder.from(deserializer, deserializer, "topic-3", "topic-4"); + + KStream stream1 = + source1.filter(new Predicate() { + @Override + public boolean apply(String key, String value) { + return true; + } + }).filterOut(new Predicate() { + @Override + public boolean apply(String key, String value) { + return false; + } + }); + + KStream stream2 = stream1.mapValues(new ValueMapper() { + @Override + public Integer apply(String value) { + return new Integer(value); + } + }); + + KStream stream3 = source2.flatMapValues(new ValueMapper>() { + @Override + public Iterable apply(String value) { + return Collections.singletonList(new Integer(value)); + } + }); + + KStream[] streams2 = stream2.branch( + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return (value % 2) == 0; + } + }, + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + } + ); + + KStream[] streams3 = stream3.branch( + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return (value % 2) == 0; + } + }, + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + } + ); + + KStream stream4 = streams2[0].with(new UnlimitedWindowDef("window")) + .join(streams3[0].with(new UnlimitedWindowDef("window")), new ValueJoiner() { + @Override + public Integer apply(Integer value1, Integer value2) { + return value1 + value2; + } + }); + + KStream stream5 = streams2[1].with(new UnlimitedWindowDef("window")) + .join(streams3[1].with(new UnlimitedWindowDef("window")), new ValueJoiner() { + @Override + public Integer apply(Integer value1, Integer value2) { + return value1 + value2; + } + }); + + stream4.sendTo("topic-5"); + + stream5.through("topic-6").process(new MockProcessorDef<>()).sendTo("topic-7"); + + assertEquals(2 + // sources + 2 + // stream1 + 1 + // stream2 + 1 + // stream3 + 1 + 2 + // streams2 + 1 + 2 + // streams3 + 2 + 3 + // stream4 + 2 + 3 + // stream5 + 1 + // sendTo + 2 + // through + 1 + // process + 1, // sendTo + builder.build().processors().size()); + } +} From e9a526eb2eb09483e9572463890be268e4924bbc Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 22 Sep 2015 09:16:42 -0700 Subject: [PATCH 256/275] move statemgr construction to streamtask --- .../internals/ProcessorContextImpl.java | 27 +++---------------- .../processor/internals/StreamTask.java | 25 +++++++++++++---- 2 files changed, 23 insertions(+), 29 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 69f4e8aa3b774..b01dbc1d05667 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -17,12 +17,9 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.KafkaConsumer; 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.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.StreamingConfig; @@ -34,7 +31,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -63,43 +59,26 @@ public ProcessorContextImpl(int id, StreamTask task, StreamingConfig config, RecordCollector collector, - Metrics metrics) throws IOException { + 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); - File stateFile = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG), Integer.toString(id)); - - log.info("Creating restoration consumer client for stream task [" + task.id() + "]"); - - Consumer restoreConsumer = new KafkaConsumer<>( - config.getConsumerConfigs(), - new ByteArrayDeserializer(), - new ByteArrayDeserializer()); - - this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); - this.initialized = false; } - public ProcessorStateManager stateManager() { - return this.stateMgr; - } - public RecordCollector recordCollector() { return this.collector; } - public StreamTask task() { - return this.task; - } - @Override public boolean joinable() { Set partitions = this.task.partitions(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 39ec04904eae9..be155186054de 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -19,16 +19,19 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaConsumer; 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.common.serialization.ByteArrayDeserializer; 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; @@ -55,6 +58,7 @@ public class StreamTask implements Punctuator { private final Map consumedOffsets; private final RecordCollector recordCollector; + private final ProcessorStateManager stateMgr; private boolean commitRequested = false; private boolean commitOffsetNeeded = false; @@ -104,13 +108,24 @@ public StreamTask(int id, // create the record recordCollector that maintains the produced offsets this.recordCollector = new RecordCollector(producer); - // initialize the topology with its own context + log.info("Creating restoration consumer client for stream task [" + id + "]"); + + Consumer restoreConsumer = new KafkaConsumer<>( + config.getConsumerConfigs(), + new ByteArrayDeserializer(), + new ByteArrayDeserializer()); + + // create the processor state manager try { - this.processorContext = new ProcessorContextImpl(id, this, config, recordCollector, new Metrics()); + 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 in processor context", 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; @@ -232,7 +247,7 @@ public ProcessorTopology topology() { */ public void commit() { // 1) flush local state - ((ProcessorContextImpl) processorContext).stateManager().flush(); + stateMgr.flush(); // 2) flush produced records in the downstream // TODO: this will actually block on all produced records across the tasks @@ -287,7 +302,7 @@ public void close() { throw exception; try { - ((ProcessorContextImpl) processorContext).stateManager().close(recordCollector.offsets()); + stateMgr.close(recordCollector.offsets()); } catch (IOException e) { throw new KafkaException("Error while closing the state manager in processor context", e); } From 4a2e1881bdb88054b6f81361cdfc9bfcfd31f8fc Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 22 Sep 2015 11:23:40 -0700 Subject: [PATCH 257/275] rename sendTo -> to --- .../java/org/apache/kafka/streams/KafkaStreaming.java | 2 +- .../org/apache/kafka/streams/examples/KStreamJob.java | 4 ++-- .../java/org/apache/kafka/streams/kstream/KStream.java | 8 ++++---- .../kafka/streams/kstream/internals/KStreamImpl.java | 4 ++-- .../kafka/streams/kstream/internals/KStreamImplTest.java | 8 ++++---- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java index c00cf4d6fba16..f3a99e0168caa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java @@ -51,7 +51,7 @@ * StreamingConfig config = new StreamingConfig(props); * * KStreamBuilder builder = new KStreamBuilder(); - * builder.from("topic1").mapValue(value -> value.length()).sendTo("topic2"); + * builder.from("topic1").mapValue(value -> value.length()).to("topic2"); * * KafkaStreaming streaming = new KafkaStreaming(builder, config); * streaming.start(); diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java index 891398d28d607..40945420cd75b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java @@ -74,8 +74,8 @@ public boolean apply(String key, Integer value) { } ); - streams[0].sendTo("topic2"); - streams[1].sendTo("topic3"); + streams[0].to("topic2"); + streams[1].to("topic3"); KafkaStreaming kstream = new KafkaStreaming(builder, config); kstream.start(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 14b2f14d563e1..e266be6e5864f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -101,7 +101,7 @@ public interface KStream { /** * Sends key-value to a topic, also creates a new stream from the topic. - * This is equivalent to calling sendTo(topic) and from(topic). + * This is equivalent to calling to(topic) and from(topic). * * @param topic the topic name * @param the key type of the new stream @@ -112,7 +112,7 @@ public interface KStream { /** * Sends key-value to a topic, also creates a new stream from the topic. - * This is equivalent to calling sendTo(topic) and from(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, @@ -134,7 +134,7 @@ public interface KStream { * * @param topic the topic name */ - void sendTo(String topic); + void to(String topic); /** * Sends key-value to a topic. @@ -145,7 +145,7 @@ public interface KStream { * @param valSerializer value serializer used to send key-value pairs, * if not specified the default serializer defined in the configs will be used */ - void sendTo(String topic, Serializer keySerializer, Serializer valSerializer); + void to(String topic, Serializer keySerializer, Serializer valSerializer); /** * Processes all elements in this stream by applying a processor. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 6a67cb750f7ce..f972a551087dd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -177,14 +177,14 @@ public KStream through(String topic) { } @Override - public void sendTo(String topic) { + public void to(String topic) { String name = SEND_NAME + INDEX.getAndIncrement(); topology.addSink(name, topic, this.name); } @Override - public void sendTo(String topic, Serializer keySerializer, Serializer valSerializer) { + public void to(String topic, Serializer keySerializer, Serializer valSerializer) { String name = SEND_NAME + INDEX.getAndIncrement(); topology.addSink(name, topic, keySerializer, valSerializer, this.name); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 2f82cf339dea5..0660ddd87a54f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -117,9 +117,9 @@ public Integer apply(Integer value1, Integer value2) { } }); - stream4.sendTo("topic-5"); + stream4.to("topic-5"); - stream5.through("topic-6").process(new MockProcessorDef<>()).sendTo("topic-7"); + stream5.through("topic-6").process(new MockProcessorDef<>()).to("topic-7"); assertEquals(2 + // sources 2 + // stream1 @@ -129,10 +129,10 @@ public Integer apply(Integer value1, Integer value2) { 1 + 2 + // streams3 2 + 3 + // stream4 2 + 3 + // stream5 - 1 + // sendTo + 1 + // to 2 + // through 1 + // process - 1, // sendTo + 1, // to builder.build().processors().size()); } } From 2c26b9fb284b8a1eabef083a9abc970a439deb7e Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 22 Sep 2015 12:56:32 -0700 Subject: [PATCH 258/275] set the initialized flag in context --- .../streams/processor/internals/ProcessorContextImpl.java | 4 ++++ .../apache/kafka/streams/processor/internals/StreamTask.java | 5 +++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index b01dbc1d05667..b3502220cf3ed 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -79,6 +79,10 @@ public RecordCollector recordCollector() { return this.collector; } + public void initialized() { + this.initialized = true; + } + @Override public boolean joinable() { Set partitions = this.task.partitions(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index be155186054de..86dc735e14ae2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.ByteArrayDeserializer; 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; @@ -53,7 +52,7 @@ public class StreamTask implements Punctuator { private final PartitionGroup partitionGroup; private final PartitionGroup.RecordInfo recordInfo = new PartitionGroup.RecordInfo(); private final PunctuationQueue punctuationQueue; - private final ProcessorContext processorContext; + private final ProcessorContextImpl processorContext; private final ProcessorTopology topology; private final Map consumedOffsets; @@ -135,6 +134,8 @@ public StreamTask(int id, this.currNode = null; } } + + this.processorContext.initialized(); } public int id() { From 8b484406b11c5b1cbd88a4c0f7efdfeec5d2e0ef Mon Sep 17 00:00:00 2001 From: Jesse Anderson Date: Tue, 22 Sep 2015 13:16:20 -0700 Subject: [PATCH 259/275] Docs say that iterators must be closed. Maybe need to emphasize with a try/finally. --- .../java/org/apache/kafka/streams/examples/ProcessorJob.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java index 7d9773557585b..7937867211b18 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java @@ -76,6 +76,8 @@ public void punctuate(long streamTime) { context.forward(entry.key(), entry.value()); } + + iter.close(); } @Override From 0f85cf7a162c1d6193ad3f9ab79fc5dccdcd8a76 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 22 Sep 2015 13:52:55 -0700 Subject: [PATCH 260/275] fix metrics names --- .../apache/kafka/streams/KafkaStreaming.java | 2 +- .../apache/kafka/streams/StreamingConfig.java | 10 +- .../kafka/streams/examples/KStreamJob.java | 1 + .../kafka/streams/examples/ProcessorJob.java | 1 + .../processor/internals/StreamTask.java | 13 +- .../processor/internals/StreamThread.java | 138 ++++++++++++------ .../processor/internals/StreamThreadTest.java | 6 +- 7 files changed, 119 insertions(+), 52 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java index f3a99e0168caa..0633f9199832a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java @@ -73,7 +73,7 @@ public class KafkaStreaming { 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); + this.threads[i] = new StreamThread(i, builder, config); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java index 3d2a1a8e5f666..dce69b6000cfd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java @@ -68,6 +68,9 @@ public class StreamingConfig extends AbstractConfig { public static final String TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "timestamp.extractor"; private static final String TIMESTAMP_EXTRACTOR_CLASS_DOC = "Timestamp extractor class that implements the TimestampExtractor interface."; + /** client.id */ + public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; + /** key.serializer */ public static final String KEY_SERIALIZER_CLASS_CONFIG = ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; @@ -88,7 +91,12 @@ public class StreamingConfig extends AbstractConfig { private static final String SYSTEM_TEMP_DIRECTORY = System.getProperty("java.io.tmpdir"); static { - CONFIG = new ConfigDef().define(STATE_DIR_CONFIG, + 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, diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java index 40945420cd75b..feb4ee7d41f2e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java @@ -34,6 +34,7 @@ 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); diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java index 87b312c3b8230..6b449d3565e66 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java @@ -87,6 +87,7 @@ public void 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); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 39ec04904eae9..3b683c8568309 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -178,11 +178,6 @@ public int process() { consumedOffsets.put(partition, currRecord.offset()); commitOffsetNeeded = true; - // commit the current task state if requested during the processing - if (commitRequested) { - commit(); - } - // 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) { @@ -243,9 +238,17 @@ public void commit() { consumer.commitSync(consumedOffsets); commitOffsetNeeded = false; } + commitRequested = false; } + /** + * Whether or not a request has been made to commit the current state + */ + public boolean commitNeeded() { + return this.commitOffsetNeeded; + } + /** * Request committing the current task's state */ diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 40fe483836e0a..9eee7871e45fc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -50,14 +50,13 @@ 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; @@ -66,6 +65,7 @@ public class StreamThread extends Thread { protected final Producer producer; protected final Consumer consumer; + private final int threadId; private final Map tasks; private final Time time; private final File stateDir; @@ -88,23 +88,25 @@ public void onPartitionsAssigned(Consumer consumer, Collection consumer, Collection assignment) { - commitAll(time.milliseconds()); + 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, new SystemTime()); + public StreamThread(int threadId, TopologyBuilder builder, StreamingConfig config) throws Exception { + this(threadId, builder, config, null , null, new SystemTime()); } @SuppressWarnings("unchecked") - StreamThread(TopologyBuilder builder, StreamingConfig config, + StreamThread(int threadId, + TopologyBuilder builder, StreamingConfig config, Producer producer, Consumer consumer, Time time) throws Exception { - super("StreamThread-" + nextThreadNumber.getAndIncrement()); + super("StreamThread-" + threadId); + this.threadId = threadId; this.config = config; this.builder = builder; @@ -180,7 +182,7 @@ private void shutdown() { // Exceptions should not prevent this call from going through all shutdown steps. try { - commitAll(time.milliseconds()); + commitAll(); } catch (Throwable e) { // already logged in commitAll() } @@ -210,6 +212,8 @@ private void runLoop() { consumer.subscribe(new ArrayList<>(builder.sourceTopics()), rebalanceListener); while (stillRunning()) { + long startPoll = time.milliseconds(); + // try to fetch some records if necessary ConsumerRecords records = consumer.poll(totalNumBuffered == 0 ? this.pollTimeMs : 0); @@ -219,11 +223,18 @@ private void runLoop() { } } + 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); } maybeClean(); @@ -248,32 +259,67 @@ private boolean stillRunning() { return true; } + private void maybePunctuate() { + long now = time.milliseconds(); + + 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; + } + } + } + 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(now); + + 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 - * @param now */ - private void commitAll(long now) { + private void commitAll() { for (StreamTask task : tasks.values()) { try { - task.commit(); + commitOne(task, time.milliseconds()); } catch (Exception e) { log.error("Failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); throw e; } } + } - metrics.commitTime.record(now - time.milliseconds()); + /** + * 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; + } - lastCommit = now; + metrics.commitTimeSensor.record(time.milliseconds() - now); } /** @@ -362,7 +408,7 @@ private void removePartitions() { log.error("Failed to close a task #" + task.id() + " in thread [" + this.getName() + "]: ", e); throw e; } - metrics.processorDestruction.record(); + metrics.taskDestructionSensor.record(); } tasks.clear(); } @@ -370,37 +416,45 @@ private void removePartitions() { private class KafkaStreamingMetrics { final Metrics metrics; - final Sensor commitTime; - final Sensor processTime; - final Sensor windowTime; - final Sensor processorCreation; - final Sensor processorDestruction; + final Sensor commitTimeSensor; + final Sensor pollTimeSensor; + final Sensor processTimeSensor; + final Sensor punctuateTimeSensor; + final Sensor taskCreationSensor; + final Sensor taskDestructionSensor; public KafkaStreamingMetrics() { - String group = "kafka-streaming"; + String metricGrpName = "streaming-metrics"; this.metrics = new Metrics(); - - this.commitTime = metrics.sensor("commit-time"); - this.commitTime.add(new MetricName(group, "commit-time-avg-ms"), new Avg()); - this.commitTime.add(new MetricName(group, "commit-time-max-ms"), new Max()); - this.commitTime.add(new MetricName(group, "commit-per-second"), new Rate(new Count())); - - this.processTime = metrics.sensor("process-time"); - this.processTime.add(new MetricName(group, "process-time-avg-ms"), new Avg()); - this.processTime.add(new MetricName(group, "process-time-max-ms"), new Max()); - this.processTime.add(new MetricName(group, "process-calls-per-second"), new Rate(new Count())); - - this.windowTime = metrics.sensor("window-time"); - this.windowTime.add(new MetricName(group, "window-time-avg-ms"), new Avg()); - this.windowTime.add(new MetricName(group, "window-time-max-ms"), new Max()); - this.windowTime.add(new MetricName(group, "window-calls-per-second"), new Rate(new Count())); - - this.processorCreation = metrics.sensor("processor-creation"); - this.processorCreation.add(new MetricName(group, "processor-creation"), new Rate(new Count())); - - this.processorDestruction = metrics.sensor("processor-destruction"); - this.processorDestruction.add(new MetricName(group, "processor-destruction"), new Rate(new Count())); + Map metricTags = new LinkedHashMap(); + metricTags.put("client-id", config.getString(StreamingConfig.CLIENT_ID_CONFIG) + "-thread" + threadId); + + 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.commitTimeSensor.add(new MetricName("poll-time-avg", metricGrpName, "The average poll time in ms", metricTags), new Avg()); + this.commitTimeSensor.add(new MetricName("poll-time-max", metricGrpName, "The maximum poll time in ms", metricTags), new Max()); + this.commitTimeSensor.add(new MetricName("poll-rate", metricGrpName, "The average per-second number of record-poll calls", metricTags), new Rate(new Count())); + + this.processTimeSensor = metrics.sensor("process-time"); + this.processTimeSensor.add(new MetricName("process-time-avg-ms", metricGrpName, "The average process time in ms", metricTags), new Avg()); + this.processTimeSensor.add(new MetricName("process-time-max-ms", metricGrpName, "The maximum process time in ms", metricTags), new Max()); + this.processTimeSensor.add(new MetricName("process-calls-rate", metricGrpName, "The average per-second number of process calls", metricTags), new Rate(new Count())); + + this.punctuateTimeSensor = metrics.sensor("punctuate-time"); + this.punctuateTimeSensor.add(new MetricName("punctuate-time-avg", metricGrpName, "The average punctuate time in ms", metricTags), new Avg()); + this.punctuateTimeSensor.add(new MetricName("punctuate-time-max", metricGrpName, "The maximum punctuate time in ms", metricTags), new Max()); + this.punctuateTimeSensor.add(new MetricName("punctuate-calls-rate", metricGrpName, "The average per-second number of punctuate calls", metricTags), new Rate(new Count())); + + this.taskCreationSensor = metrics.sensor("task-creation"); + this.taskCreationSensor.add(new MetricName("task-creation-rate", metricGrpName, "The average per-second number of newly created tasks", metricTags), new Rate(new Count())); + + this.taskDestructionSensor = metrics.sensor("task-destruction"); + this.taskDestructionSensor.add(new MetricName("task-destruction-rate", metricGrpName, "The average per-second number of destructed tasks", metricTags), new Rate(new Count())); } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index e4a310932c44d..51507b49f67a9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -102,7 +102,7 @@ public void testPartitionAssignmentChange() throws Exception { builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); - StreamThread thread = new StreamThread(builder, config, producer, consumer, new SystemTime()); + StreamThread thread = new StreamThread(0, builder, config, producer, consumer, new SystemTime()); ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; @@ -199,7 +199,7 @@ public void testMaybeClean() throws Exception { TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); - StreamThread thread = new StreamThread(builder, config, producer, consumer, mockTime) { + StreamThread thread = new StreamThread(0, builder, config, producer, consumer, mockTime) { @Override public void maybeClean() { super.maybeClean(); @@ -316,7 +316,7 @@ public void testMaybeCommit() throws Exception { TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); - StreamThread thread = new StreamThread(builder, config, producer, consumer, mockTime) { + StreamThread thread = new StreamThread(0, builder, config, producer, consumer, mockTime) { @Override public void maybeCommit() { super.maybeCommit(); From 8243789c6b55ddb5b582c1a8df5390f2156c368d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 22 Sep 2015 14:28:16 -0700 Subject: [PATCH 261/275] finer process / punctuate metric sensors --- .../kafka/streams/examples/ProcessorJob.java | 2 +- .../kafka/streams/processor/Processor.java | 2 +- .../processor/internals/PunctuationQueue.java | 10 +++++-- .../processor/internals/StreamTask.java | 23 +++++++++------ .../processor/internals/StreamThread.java | 28 +++++++++++-------- 5 files changed, 39 insertions(+), 26 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java index 6b449d3565e66..8285590be6939 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java @@ -65,7 +65,7 @@ public void process(String key, String value) { } @Override - public void punctuate(long streamTime) { + public void punctuate(long timestamp) { KeyValueIterator iter = this.kvStore.all(); while (iter.hasNext()) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java index c09f38381863f..ff6c2e01bfd5d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java @@ -23,7 +23,7 @@ public interface Processor { void process(K key, V value); - void punctuate(long streamTime); + void punctuate(long timestamp); void close(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java index b972a576c37cc..b4b7afe4f03e3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java @@ -35,17 +35,21 @@ public void close() { } } - public void mayPunctuate(long streamTime, Punctuator punctuator) { + public boolean mayPunctuate(long timestamp, Punctuator punctuator) { synchronized (pq) { + boolean punctuated = false; PunctuationSchedule top = pq.peek(); - while (top != null && top.timestamp <= streamTime) { + while (top != null && top.timestamp <= timestamp) { PunctuationSchedule sched = top; pq.poll(); - punctuator.punctuate(sched.node(), streamTime); + punctuator.punctuate(sched.node(), timestamp); pq.add(sched.next()); + punctuated = true; top = pq.peek(); } + + return punctuated; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index c46e18b9dd50a..5e7d11a1c1ccd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -164,7 +164,7 @@ public void addRecords(TopicPartition partition, Iterable partitionsForTask) { + metrics.taskCreationSensor.record(); + return new StreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); } @@ -413,7 +417,7 @@ private void removePartitions() { tasks.clear(); } - private class KafkaStreamingMetrics { + private class StreamingMetrics { final Metrics metrics; final Sensor commitTimeSensor; @@ -423,7 +427,7 @@ private class KafkaStreamingMetrics { final Sensor taskCreationSensor; final Sensor taskDestructionSensor; - public KafkaStreamingMetrics() { + public StreamingMetrics() { String metricGrpName = "streaming-metrics"; this.metrics = new Metrics(); @@ -436,9 +440,9 @@ public KafkaStreamingMetrics() { 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.commitTimeSensor.add(new MetricName("poll-time-avg", metricGrpName, "The average poll time in ms", metricTags), new Avg()); - this.commitTimeSensor.add(new MetricName("poll-time-max", metricGrpName, "The maximum poll time in ms", metricTags), new Max()); - this.commitTimeSensor.add(new MetricName("poll-rate", metricGrpName, "The average per-second number of record-poll calls", metricTags), new Rate(new Count())); + 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()); From 5bac8ca829d6dc55eabfad4b788f798226e8b9bb Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 22 Sep 2015 15:06:42 -0700 Subject: [PATCH 262/275] comments addressed --- build.gradle | 2 +- .../org/apache/kafka/streams/kstream/KStream.java | 2 +- .../kstream/internals/KStreamFlatMapValues.java | 1 - .../streams/kstream/internals/KStreamImpl.java | 14 +++++++------- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/build.gradle b/build.gradle index 3de9101e7cde6..b0e1b58c20dd6 100644 --- a/build.gradle +++ b/build.gradle @@ -538,7 +538,7 @@ project(':streams') { compile "$slf4jlog4j" compile 'org.rocksdb:rocksdbjni:3.10.1' - testCompile 'junit:junit:4.6' + testCompile '$junit' testCompile project(path: ':clients', configuration: 'archives') } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index e266be6e5864f..7f101ab48af91 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -53,7 +53,7 @@ public interface KStream { KStream map(KeyValueMapper> mapper); /** - * Creates a new stream by transforming valuesa by a mapper to all values of this stream + * Creates a new stream by transforming values by a mapper to all values of this stream * * @param mapper the instance of ValueMapper * @param the value type of the new stream diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java index a56d1d625c34e..4f3e4755281b4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -25,7 +25,6 @@ class KStreamFlatMapValues implements ProcessorDef { private final ValueMapper> mapper; - @SuppressWarnings("unchecked") KStreamFlatMapValues(ValueMapper> mapper) { this.mapper = mapper; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index f972a551087dd..69366481cd7b2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -52,6 +52,8 @@ public class KStreamImpl implements KStream { 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-"; @@ -60,12 +62,10 @@ public class KStreamImpl implements KStream { public static final String SOURCE_NAME = "KAFKA-SOURCE-"; - public static final String SEND_NAME = "KAFKA-SEND-"; - public static final AtomicInteger INDEX = new AtomicInteger(1); - protected TopologyBuilder topology; - protected String name; + protected final TopologyBuilder topology; + protected final String name; public KStreamImpl(TopologyBuilder topology, String name) { this.topology = topology; @@ -160,7 +160,7 @@ public KStream through(String topic, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer) { - String sendName = SEND_NAME + INDEX.getAndIncrement(); + String sendName = SINK_NAME + INDEX.getAndIncrement(); topology.addSink(sendName, topic, keySerializer, valSerializer, this.name); @@ -178,14 +178,14 @@ public KStream through(String topic) { @Override public void to(String topic) { - String name = SEND_NAME + INDEX.getAndIncrement(); + String name = SINK_NAME + INDEX.getAndIncrement(); topology.addSink(name, topic, this.name); } @Override public void to(String topic, Serializer keySerializer, Serializer valSerializer) { - String name = SEND_NAME + INDEX.getAndIncrement(); + String name = SINK_NAME + INDEX.getAndIncrement(); topology.addSink(name, topic, keySerializer, valSerializer, this.name); } From b7554bae71c25aa24a00a36babfd26b72791afe6 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 22 Sep 2015 15:34:07 -0700 Subject: [PATCH 263/275] revert to use static thread ids --- .../org/apache/kafka/streams/KafkaStreaming.java | 2 +- .../streams/processor/internals/StreamThread.java | 15 +++++++-------- .../processor/internals/StreamThreadTest.java | 6 +++--- 3 files changed, 11 insertions(+), 12 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java index 0633f9199832a..f3a99e0168caa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java @@ -73,7 +73,7 @@ public class KafkaStreaming { 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(i, builder, config); + this.threads[i] = new StreamThread(builder, config); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 7707f8ff14f7e..c9e65ff6d5e84 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -53,10 +53,12 @@ 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; @@ -65,7 +67,6 @@ public class StreamThread extends Thread { protected final Producer producer; protected final Consumer consumer; - private final int threadId; private final Map tasks; private final Time time; private final File stateDir; @@ -94,19 +95,17 @@ public void onPartitionsRevoked(Consumer consumer, Collection producer, Consumer consumer, Time time) throws Exception { - super("StreamThread-" + threadId); + super("StreamThread-" + nextThreadNumber.getAndIncrement()); - this.threadId = threadId; this.config = config; this.builder = builder; @@ -432,7 +431,7 @@ public StreamingMetrics() { this.metrics = new Metrics(); Map metricTags = new LinkedHashMap(); - metricTags.put("client-id", config.getString(StreamingConfig.CLIENT_ID_CONFIG) + "-thread" + threadId); + 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()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 51507b49f67a9..e4a310932c44d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -102,7 +102,7 @@ public void testPartitionAssignmentChange() throws Exception { builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); - StreamThread thread = new StreamThread(0, builder, config, producer, consumer, new SystemTime()); + StreamThread thread = new StreamThread(builder, config, producer, consumer, new SystemTime()); ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; @@ -199,7 +199,7 @@ public void testMaybeClean() throws Exception { TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); - StreamThread thread = new StreamThread(0, builder, config, producer, consumer, mockTime) { + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockTime) { @Override public void maybeClean() { super.maybeClean(); @@ -316,7 +316,7 @@ public void testMaybeCommit() throws Exception { TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); - StreamThread thread = new StreamThread(0, builder, config, producer, consumer, mockTime) { + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockTime) { @Override public void maybeCommit() { super.maybeCommit(); From 2633c136cc2e1024c3e85964f4ddb020f61f822b Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 22 Sep 2015 17:16:00 -0700 Subject: [PATCH 264/275] merge latest commits from apache --- build.gradle | 2 +- .../processor/internals/StreamThread.java | 4 +-- .../processor/internals/StreamThreadTest.java | 32 +++++++++---------- 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/build.gradle b/build.gradle index b0e1b58c20dd6..f4ba921fc7f70 100644 --- a/build.gradle +++ b/build.gradle @@ -538,7 +538,7 @@ project(':streams') { compile "$slf4jlog4j" compile 'org.rocksdb:rocksdbjni:3.10.1' - testCompile '$junit' + testCompile "$junit" testCompile project(path: ':clients', configuration: 'archives') } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index c9e65ff6d5e84..b07614ff3df3e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -82,13 +82,13 @@ public class StreamThread extends Thread { final ConsumerRebalanceListener rebalanceListener = new ConsumerRebalanceListener() { @Override - public void onPartitionsAssigned(Consumer consumer, Collection assignment) { + public void onPartitionsAssigned(Collection assignment) { addPartitions(assignment); lastClean = time.milliseconds(); // start the cleaning cycle } @Override - public void onPartitionsRevoked(Consumer consumer, Collection assignment) { + public void onPartitionsRevoked(Collection assignment) { commitAll(); removePartitions(); lastClean = Long.MAX_VALUE; // stop the cleaning cycle until partitions are assigned diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index e4a310932c44d..5884c16b45268 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -117,8 +117,8 @@ public void testPartitionAssignmentChange() throws Exception { assignedPartitions = Collections.singletonList(t1p1); expectedGroup1 = new HashSet<>(Arrays.asList(t1p1)); - rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); - rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); assertTrue(thread.tasks().containsKey(1)); assertEquals(expectedGroup1, thread.tasks().get(1).partitions()); @@ -128,8 +128,8 @@ public void testPartitionAssignmentChange() throws Exception { assignedPartitions = Collections.singletonList(t1p2); expectedGroup2 = new HashSet<>(Arrays.asList(t1p2)); - rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); - rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); assertTrue(thread.tasks().containsKey(2)); assertEquals(expectedGroup2, thread.tasks().get(2).partitions()); @@ -140,8 +140,8 @@ public void testPartitionAssignmentChange() throws Exception { expectedGroup1 = new HashSet<>(Collections.singleton(t1p1)); expectedGroup2 = new HashSet<>(Collections.singleton(t1p2)); - rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); - rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); assertTrue(thread.tasks().containsKey(1)); assertTrue(thread.tasks().containsKey(2)); @@ -154,8 +154,8 @@ public void testPartitionAssignmentChange() throws Exception { expectedGroup1 = new HashSet<>(Arrays.asList(t1p1, t2p1)); expectedGroup2 = new HashSet<>(Arrays.asList(t1p2, t2p2)); - rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); - rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); assertTrue(thread.tasks().containsKey(1)); assertTrue(thread.tasks().containsKey(2)); @@ -166,8 +166,8 @@ public void testPartitionAssignmentChange() throws Exception { revokedPartitions = assignedPartitions; assignedPartitions = Collections.emptyList(); - rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); - rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); assertTrue(thread.tasks().isEmpty()); } @@ -232,8 +232,8 @@ protected StreamTask createStreamTask(int id, Collection partiti assignedPartitions = Arrays.asList(t1p1, t1p2); prevTasks = new HashMap(thread.tasks()); - rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); - rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); // there shouldn't be any previous task assertTrue(prevTasks.isEmpty()); @@ -264,8 +264,8 @@ protected StreamTask createStreamTask(int id, Collection partiti assignedPartitions = Collections.emptyList(); prevTasks = new HashMap(thread.tasks()); - rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); - rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); // previous tasks should be committed assertEquals(2, prevTasks.size()); @@ -338,8 +338,8 @@ protected StreamTask createStreamTask(int id, Collection partiti revokedPartitions = Collections.emptyList(); assignedPartitions = Arrays.asList(t1p1, t1p2); - rebalanceListener.onPartitionsRevoked(consumer, revokedPartitions); - rebalanceListener.onPartitionsAssigned(consumer, assignedPartitions); + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); assertEquals(2, thread.tasks().size()); From 1c43863e2f97d072fde2b8cdfee8bd6176e09cb9 Mon Sep 17 00:00:00 2001 From: Randall Hauch Date: Wed, 23 Sep 2015 09:59:07 -0500 Subject: [PATCH 265/275] Added a new ProcessorTopologyTestDriver class that makes it easier to unit test a ProcessorTopology. The new test driver class uses a supplied TopologyBuilder and a StreamingConfig to set up a topology using the existing StreamTask, which is the same component used in the Kafka Streams framework for each thread. However, rather than rely upon a real Kafka broker, the test driver simulates a broker with mock consumers and producers. This required a few changes to StreamTask and StreamThread so that the latter can pass in the actual consumer used to restore the state; this allows the test driver to pass a mock consumer into StreamTask. Without this change, the test driver could not run topologies that used state stores. --- .../internals/ProcessorStateManager.java | 3 - .../processor/internals/StreamTask.java | 15 +- .../processor/internals/StreamThread.java | 17 +- .../processor/internals/StreamTaskTest.java | 7 +- .../processor/internals/StreamThreadTest.java | 25 +- .../test/ProcessorTopologyTestDriver.java | 316 +++++++++++++++++ .../test/ProcessorTopologyTestDriverTest.java | 323 ++++++++++++++++++ 7 files changed, 682 insertions(+), 24 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java create mode 100644 streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriverTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index f56b9c8763ceb..97dbb3bf89387 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -224,9 +224,6 @@ public void close(Map ackedOffsets) throws IOException { checkpoint.write(checkpointOffsets); } - // close the restore consumer - restoreConsumer.close(); - // release the state directory directoryLock directoryLock.release(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 5e7d11a1c1ccd..714a732bff771 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -19,13 +19,12 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.KafkaConsumer; 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.common.serialization.ByteArrayDeserializer; 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; @@ -70,14 +69,15 @@ public class StreamTask implements Punctuator { * @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 */ - @SuppressWarnings("unchecked") public StreamTask(int id, Consumer consumer, Producer producer, + Consumer restoreConsumer, Collection partitions, ProcessorTopology topology, StreamingConfig config) { @@ -109,11 +109,6 @@ public StreamTask(int id, log.info("Creating restoration consumer client for stream task [" + id + "]"); - Consumer restoreConsumer = new KafkaConsumer<>( - config.getConsumerConfigs(), - new ByteArrayDeserializer(), - new ByteArrayDeserializer()); - // create the processor state manager try { File stateFile = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG), Integer.toString(id)); @@ -345,5 +340,9 @@ public void forward(K key, V value, int childIndex) { currNode = thisNode; } } + + public ProcessorContext context() { + return processorContext; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index b07614ff3df3e..c13bce17b3e6d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -66,6 +66,7 @@ public class StreamThread extends Thread { protected final TopologyBuilder builder; protected final Producer producer; protected final Consumer consumer; + protected final Consumer restoreConsumer; private final Map tasks; private final Time time; @@ -99,7 +100,6 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce this(builder, config, null , null, new SystemTime()); } - @SuppressWarnings("unchecked") StreamThread(TopologyBuilder builder, StreamingConfig config, Producer producer, Consumer consumer, @@ -112,6 +112,7 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce // set the producer and consumer clients this.producer = (producer != null) ? producer : createProducer(); this.consumer = (consumer != null) ? consumer : createConsumer(); + this.restoreConsumer = createRestoreConsumer(); // initialize the task list this.tasks = new HashMap<>(); @@ -147,6 +148,13 @@ private Consumer createConsumer() { new ByteArrayDeserializer(), new ByteArrayDeserializer()); } + + private Consumer createRestoreConsumer() { + log.info("Creating restore consumer client for stream thread [" + this.getName() + "]"); + return new KafkaConsumer<>(config.getConsumerConfigs(), + new ByteArrayDeserializer(), + new ByteArrayDeserializer()); + } /** * Execute the stream processors @@ -195,6 +203,11 @@ private void shutdown() { } 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) { @@ -369,7 +382,7 @@ protected void maybeClean() { protected StreamTask createStreamTask(int id, Collection partitionsForTask) { metrics.taskCreationSensor.record(); - return new StreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); + return new StreamTask(id, consumer, producer, restoreConsumer, partitionsForTask, builder.build(), config); } private void addPartitions(Collection assignment) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 2b6d82d166f87..8dcfc404f66d7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -75,7 +75,8 @@ public class StreamTaskTest { private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); private final MockProducer producer = new MockProducer<>(false, bytesSerializer, bytesSerializer); - + private final MockConsumer restoreStateConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + private final byte[] recordValue = intSerializer.serialize(null, 10); private final byte[] recordKey = intSerializer.serialize(null, 1); @@ -88,7 +89,7 @@ public void setup() { @SuppressWarnings("unchecked") @Test public void testProcessOrder() { - StreamTask task = new StreamTask(0, consumer, producer, partitions, topology, config); + StreamTask task = new StreamTask(0, consumer, producer, restoreStateConsumer, partitions, topology, config); task.addRecords(partition1, records( new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, recordKey, recordValue), @@ -132,7 +133,7 @@ public void testProcessOrder() { @SuppressWarnings("unchecked") @Test public void testPauseResume() { - StreamTask task = new StreamTask(1, consumer, producer, partitions, topology, config); + StreamTask task = new StreamTask(1, consumer, producer, restoreStateConsumer, partitions, topology, config); task.addRecords(partition1, records( new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, recordKey, recordValue), diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 5884c16b45268..bea15f932da63 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -17,6 +17,10 @@ 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; @@ -44,10 +48,6 @@ import java.util.Properties; import java.util.Set; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - public class StreamThreadTest { private TopicPartition t1p1 = new TopicPartition("topic1", 1); @@ -75,10 +75,11 @@ private static class TestStreamTask extends StreamTask { public TestStreamTask(int id, Consumer consumer, Producer producer, + Consumer restoreConsumer, Collection partitions, ProcessorTopology topology, StreamingConfig config) { - super(id, consumer, producer, partitions, topology, config); + super(id, consumer, producer, restoreConsumer, partitions, topology, config); } @Override @@ -97,12 +98,18 @@ public void testPartitionAssignmentChange() throws Exception { MockProducer producer = new MockProducer<>(true, serializer, serializer); MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + final MockConsumer mockRestoreConsumer = new MockConsumer<>(OffsetResetStrategy.LATEST); TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); - StreamThread thread = new StreamThread(builder, config, producer, consumer, new SystemTime()); + StreamThread thread = new StreamThread(builder, config, producer, consumer, new SystemTime()) { + @Override + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config); + } + }; ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; @@ -194,6 +201,7 @@ public void testMaybeClean() throws Exception { MockProducer producer = new MockProducer<>(true, serializer, serializer); MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + final MockConsumer mockRestoreConsumer = new MockConsumer<>(OffsetResetStrategy.LATEST); MockTime mockTime = new MockTime(); TopologyBuilder builder = new TopologyBuilder(); @@ -206,7 +214,7 @@ public void maybeClean() { } @Override protected StreamTask createStreamTask(int id, Collection partitionsForTask) { - return new TestStreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); + return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config); } }; @@ -311,6 +319,7 @@ public void testMaybeCommit() throws Exception { MockProducer producer = new MockProducer<>(true, serializer, serializer); MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + final MockConsumer mockRestoreConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); MockTime mockTime = new MockTime(); TopologyBuilder builder = new TopologyBuilder(); @@ -323,7 +332,7 @@ public void maybeCommit() { } @Override protected StreamTask createStreamTask(int id, Collection partitionsForTask) { - return new TestStreamTask(id, consumer, producer, partitionsForTask, builder.build(), config); + return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config); } }; diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java new file mode 100644 index 0000000000000..9cbf8bce6b7ee --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -0,0 +1,316 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.test; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; +import org.apache.kafka.streams.processor.internals.StreamTask; +import org.apache.kafka.streams.state.KeyValueStore; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicLong; + +/** + * This class makes it easier to write tests to verify the behavior of topologies created with a {@link TopologyBuilder}. + * You can test simple topologies that have a single processor, or very complex topologies that have multiple sources, processors, + * and sinks. And because it starts with a {@link TopologyBuilder}, you can create topologies specific to your tests or you + * can use and test code you already have that uses a builder to create topologies. Best of all, the class works without a real + * Kafka broker, so the tests execute very quickly with very little overhead. + *

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

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

Driver setup

+ *

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

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

Processing messages

+ *

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

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

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

Processor state

+ *

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

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

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

+ * + * @param name the name of the store + * @return the key value store, or null if no {@link KeyValueStore} has been registered with the given name + * @see #getStateStore(String) + */ + @SuppressWarnings("unchecked") + public KeyValueStore getKeyValueStore(String name) { + StateStore store = getStateStore(name); + return store instanceof KeyValueStore ? (KeyValueStore) getStateStore(name) : null; + } + + /** + * Close the driver, its topology, and all processors. + */ + public void close() { + task.close(); + } + + /** + * Utility method that creates the {@link MockConsumer} used for restoring state, which should not be done by this + * driver object unless this method is overwritten with a functional consumer. + * + * @param id the ID of the stream task + * @param storeNames the names of the stores that this + * @return the mock consumer; never null + */ + protected MockConsumer createRestoreConsumer(int id, String... storeNames) { + MockConsumer consumer = new MockConsumer(OffsetResetStrategy.LATEST) { + @Override + public synchronized void seekToEnd(TopicPartition... partitions) { + // do nothing ... + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + // do nothing ... + } + + @Override + public synchronized long position(TopicPartition partition) { + // do nothing ... + return 0L; + } + }; + // For each store name ... + for (String storeName : storeNames) { + String topicName = storeName; + // Set up the restore-state topic ... + // consumer.subscribe(new TopicPartition(topicName, 1)); + // Set up the partition that matches the ID (which is what ProcessorStateManager expects) ... + List partitionInfos = new ArrayList<>(); + partitionInfos.add(new PartitionInfo(topicName, id, null, null, null)); + consumer.updatePartitions(topicName, partitionInfos); + } + return consumer; + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriverTest.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriverTest.java new file mode 100644 index 0000000000000..d990d474cf6b6 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriverTest.java @@ -0,0 +1,323 @@ +/** + * 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 static org.junit.Assert.assertEquals; +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.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.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 ProcessorTopologyTestDriverTest { + + private static final Serializer STRING_SERIALIZER = new StringSerializer(); + private static final Deserializer STRING_DESERIALIZER = new StringDeserializer(); + private static final File STATE_DIR = new File("build/data").getAbsoluteFile(); + + protected static final String INPUT_TOPIC = "input-topic"; + protected static final String OUTPUT_TOPIC_1 = "output-topic-1"; + protected static final String OUTPUT_TOPIC_2 = "output-topic-2"; + + private static long timestamp = 1000L; + + private ProcessorTopologyTestDriver driver; + private StreamingConfig config; + + @Before + public void setup() { + STATE_DIR.mkdirs(); + Properties props = new Properties(); + props.setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); + props.setProperty(StreamingConfig.STATE_DIR_CONFIG, STATE_DIR.getAbsolutePath()); + props.setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName()); + props.setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + props.setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + props.setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + props.setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + this.config = new StreamingConfig(props); + } + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + if (STATE_DIR.exists()) { + try { + Files.walkFileTree(STATE_DIR.toPath(), new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + Files.delete(file); + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + Files.delete(dir); + return FileVisitResult.CONTINUE; + } + + }); + } catch (IOException e) { + // do nothing + } + } + } + + @Test + public void testDrivingSimpleTopology() { + driver = new ProcessorTopologyTestDriver(config, createSimpleTopology()); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1"); + assertNoOutputRecord(OUTPUT_TOPIC_2); + + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2"); + assertNoOutputRecord(OUTPUT_TOPIC_2); + + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); + assertNoOutputRecord(OUTPUT_TOPIC_2); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5"); + } + + @Test + public void testDrivingMultiplexingTopology() { + driver = new ProcessorTopologyTestDriver(config, createMultiplexingTopology()); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key1", "value1(2)"); + + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2(2)"); + + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key3", "value3(2)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key4", "value4(2)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key5", "value5(2)"); + } + + @Test + public void testDrivingStatefulTopology() { + String storeName = "entries"; + driver = new ProcessorTopologyTestDriver(config, createStatefulTopology(storeName), storeName); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key1", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + assertNoOutputRecord(OUTPUT_TOPIC_1); + + KeyValueStore store = driver.getKeyValueStore("entries"); + assertEquals("value4", store.get("key1")); + assertEquals("value2", store.get("key2")); + assertEquals("value3", store.get("key3")); + assertNull(store.get("key4")); + } + + protected void assertNextOutputRecord(String topic, String key, String value) { + assertProducerRecord(driver.readOutput(topic, STRING_DESERIALIZER, STRING_DESERIALIZER), topic, key, value); + } + + protected void assertNoOutputRecord(String topic) { + assertNull(driver.readOutput(topic)); + } + + private void assertProducerRecord(ProducerRecord record, String topic, String key, String value) { + assertEquals(topic, record.topic()); + assertEquals(key, record.key()); + assertEquals(value, record.value()); + // Kafka Streaming doesn't set the partition, so it's always null + assertNull(record.partition()); + } + + protected TopologyBuilder createSimpleTopology() { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new ForwardingProcessor()), "source") + .addSink("sink", OUTPUT_TOPIC_1, "processor"); + } + + protected TopologyBuilder createMultiplexingTopology() { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new MultiplexingProcessor(2)), "source") + .addSink("sink1", OUTPUT_TOPIC_1, "processor") + .addSink("sink2", OUTPUT_TOPIC_2, "processor"); + } + + protected TopologyBuilder createStatefulTopology(String storeName) { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new StatefulProcessor(storeName)), "source") + .addSink("counts", OUTPUT_TOPIC_1, "processor"); + } + + /** + * Abstract base Processor. + */ + private static abstract class FauxProcessor implements Processor { + + protected ProcessorContext context; + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + @Override + public void punctuate(long streamTime) { + } + + @Override + public void close() { + } + } + + /** + * A processor that simply forwards all messages to all children. + */ + private static class ForwardingProcessor extends FauxProcessor { + @Override + public void init(ProcessorContext context) { + super.init(context); + } + + @Override + public void process(String key, String value) { + this.context.forward(key, value); + } + + @Override + public void punctuate(long streamTime) { + this.context.forward(Long.toString(streamTime), "punctuate"); + } + } + + /** + * A processor that forwards slightly-modified messages to each child. + */ + private static class MultiplexingProcessor extends FauxProcessor { + + private final int numChildren; + + public MultiplexingProcessor(int numChildren) { + this.numChildren = numChildren; + } + + @Override + public void init(ProcessorContext context) { + super.init(context); + } + + @Override + public void process(String key, String value) { + for (int i = 0; i != numChildren; ++i) { + this.context.forward(key, value + "(" + (i + 1) + ")", i); + } + } + + @Override + public void punctuate(long streamTime) { + for (int i = 0; i != numChildren; ++i) { + this.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. + */ + private static class StatefulProcessor extends FauxProcessor { + + private KeyValueStore store; + private final String storeName; + + public StatefulProcessor(String storeName) { + this.storeName = storeName; + } + + @Override + public void init(ProcessorContext context) { + super.init(context); + store = new InMemoryKeyValueStore<>(storeName, context); + } + + @Override + public void process(String key, String value) { + store.put(key, value); + } + + @Override + public void punctuate(long streamTime) { + int count = 0; + for (KeyValueIterator iter = store.all(); iter.hasNext();) { + iter.next(); + ++count; + } + this.context.forward(Long.toString(streamTime), count); + } + } + + private ProcessorDef define(final Processor processor) { + return new ProcessorDef() { + @Override + public Processor instance() { + return processor; + } + }; + } + + public static class CustomTimestampExtractor implements TimestampExtractor { + @Override + public long extract(ConsumerRecord record) { + return timestamp; + } + } +} From fc6cc9a2d32824c7c59ba72ab3a824238d8cd2ff Mon Sep 17 00:00:00 2001 From: Randall Hauch Date: Wed, 23 Sep 2015 11:16:24 -0500 Subject: [PATCH 266/275] Allowed the restore consumer to be passed into the StreamThread instance if needed in tests. --- .../kafka/streams/processor/internals/StreamThread.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index c13bce17b3e6d..f37903f20527f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -21,7 +21,6 @@ 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.streams.StreamingConfig; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.KafkaException; @@ -38,6 +37,7 @@ 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; @@ -97,12 +97,13 @@ public void onPartitionsRevoked(Collection assignment) { }; public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exception { - this(builder, config, null , null, new SystemTime()); + this(builder, config, null , null, null, new SystemTime()); } StreamThread(TopologyBuilder builder, StreamingConfig config, Producer producer, Consumer consumer, + Consumer restoreConsumer, Time time) throws Exception { super("StreamThread-" + nextThreadNumber.getAndIncrement()); @@ -112,7 +113,7 @@ public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exce // set the producer and consumer clients this.producer = (producer != null) ? producer : createProducer(); this.consumer = (consumer != null) ? consumer : createConsumer(); - this.restoreConsumer = createRestoreConsumer(); + this.restoreConsumer = (restoreConsumer != null) ? restoreConsumer : createRestoreConsumer(); // initialize the task list this.tasks = new HashMap<>(); From 5dc7fee53f04efe53a38dd6821c67c07c04ccce3 Mon Sep 17 00:00:00 2001 From: Randall Hauch Date: Wed, 23 Sep 2015 12:24:48 -0500 Subject: [PATCH 267/275] Corrected StreamThreadTest, and moved tests for ProcessorTopologyTestDriver into ProcessorTopologyTest. --- .../internals/ProcessorTopologyTest.java | 299 ++++++++++++++++ .../processor/internals/StreamThreadTest.java | 6 +- .../test/ProcessorTopologyTestDriverTest.java | 323 ------------------ 3 files changed, 302 insertions(+), 326 deletions(-) delete mode 100644 streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriverTest.java diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index 0fa2c9de2e58e..d265c65b833d7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -19,13 +19,95 @@ 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.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorDef; +import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.state.InMemoryKeyValueStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.test.MockProcessorDef; +import org.apache.kafka.test.ProcessorTopologyTestDriver; +import org.junit.After; +import org.junit.Before; import org.junit.Test; +import java.io.File; +import java.io.IOException; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.Properties; + public class ProcessorTopologyTest { + private static final Serializer STRING_SERIALIZER = new StringSerializer(); + private static final Deserializer STRING_DESERIALIZER = new StringDeserializer(); + private static final File STATE_DIR = new File("build/data").getAbsoluteFile(); + + protected static final String INPUT_TOPIC = "input-topic"; + protected static final String OUTPUT_TOPIC_1 = "output-topic-1"; + protected static final String OUTPUT_TOPIC_2 = "output-topic-2"; + + private static long timestamp = 1000L; + + private ProcessorTopologyTestDriver driver; + private StreamingConfig config; + + @Before + public void setup() { + STATE_DIR.mkdirs(); + Properties props = new Properties(); + props.setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); + props.setProperty(StreamingConfig.STATE_DIR_CONFIG, STATE_DIR.getAbsolutePath()); + props.setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName()); + props.setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + props.setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + props.setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + props.setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + this.config = new StreamingConfig(props); + } + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + if (STATE_DIR.exists()) { + try { + Files.walkFileTree(STATE_DIR.toPath(), new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + Files.delete(file); + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + Files.delete(dir); + return FileVisitResult.CONTINUE; + } + + }); + } catch (IOException e) { + // do nothing + } + } + } + @Test public void testTopologyMetadata() { final TopologyBuilder builder = new TopologyBuilder(); @@ -53,4 +135,221 @@ public void testTopologyMetadata() { assertEquals(topology.source("topic-2"), topology.source("topic-3")); } + + @Test + public void testDrivingSimpleTopology() { + driver = new ProcessorTopologyTestDriver(config, createSimpleTopology()); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1"); + assertNoOutputRecord(OUTPUT_TOPIC_2); + + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2"); + assertNoOutputRecord(OUTPUT_TOPIC_2); + + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); + assertNoOutputRecord(OUTPUT_TOPIC_2); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5"); + } + + @Test + public void testDrivingMultiplexingTopology() { + driver = new ProcessorTopologyTestDriver(config, createMultiplexingTopology()); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key1", "value1(2)"); + + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2(2)"); + + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key3", "value3(2)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key4", "value4(2)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key5", "value5(2)"); + } + + @Test + public void testDrivingStatefulTopology() { + String storeName = "entries"; + driver = new ProcessorTopologyTestDriver(config, createStatefulTopology(storeName), storeName); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key1", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + assertNoOutputRecord(OUTPUT_TOPIC_1); + + KeyValueStore store = driver.getKeyValueStore("entries"); + assertEquals("value4", store.get("key1")); + assertEquals("value2", store.get("key2")); + assertEquals("value3", store.get("key3")); + assertNull(store.get("key4")); + } + + protected void assertNextOutputRecord(String topic, String key, String value) { + assertProducerRecord(driver.readOutput(topic, STRING_DESERIALIZER, STRING_DESERIALIZER), topic, key, value); + } + + protected void assertNoOutputRecord(String topic) { + assertNull(driver.readOutput(topic)); + } + + private void assertProducerRecord(ProducerRecord record, String topic, String key, String value) { + assertEquals(topic, record.topic()); + assertEquals(key, record.key()); + assertEquals(value, record.value()); + // Kafka Streaming doesn't set the partition, so it's always null + assertNull(record.partition()); + } + + protected TopologyBuilder createSimpleTopology() { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new ForwardingProcessor()), "source") + .addSink("sink", OUTPUT_TOPIC_1, "processor"); + } + + protected TopologyBuilder createMultiplexingTopology() { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new MultiplexingProcessor(2)), "source") + .addSink("sink1", OUTPUT_TOPIC_1, "processor") + .addSink("sink2", OUTPUT_TOPIC_2, "processor"); + } + + protected TopologyBuilder createStatefulTopology(String storeName) { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new StatefulProcessor(storeName)), "source") + .addSink("counts", OUTPUT_TOPIC_1, "processor"); + } + + /** + * A processor that simply forwards all messages to all children. + */ + protected static abstract class FauxProcessor implements Processor { + + protected ProcessorContext context; + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + @Override + public void punctuate(long streamTime) { + } + + @Override + public void close() { + } + } + + /** + * A processor that simply forwards all messages to all children. + */ + protected static class ForwardingProcessor extends FauxProcessor { + @Override + public void init(ProcessorContext context) { + super.init(context); + } + + @Override + public void process(String key, String value) { + this.context.forward(key, value); + } + + @Override + public void punctuate(long streamTime) { + this.context.forward(Long.toString(streamTime), "punctuate"); + } + } + + /** + * A processor that forwards slightly-modified messages to each child. + */ + protected static class MultiplexingProcessor extends FauxProcessor { + + private final int numChildren; + + public MultiplexingProcessor(int numChildren) { + this.numChildren = numChildren; + } + + @Override + public void init(ProcessorContext context) { + super.init(context); + } + + @Override + public void process(String key, String value) { + for (int i = 0; i != numChildren; ++i) { + this.context.forward(key, value + "(" + (i + 1) + ")", i); + } + } + + @Override + public void punctuate(long streamTime) { + for (int i = 0; i != numChildren; ++i) { + this.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 FauxProcessor { + + private KeyValueStore store; + private final String storeName; + + public StatefulProcessor(String storeName) { + this.storeName = storeName; + } + + @Override + public void init(ProcessorContext context) { + super.init(context); + store = new InMemoryKeyValueStore<>(storeName, context); + } + + @Override + public void process(String key, String value) { + store.put(key, value); + } + + @Override + public void punctuate(long streamTime) { + int count = 0; + for (KeyValueIterator iter = store.all(); iter.hasNext();) { + iter.next(); + ++count; + } + this.context.forward(Long.toString(streamTime), count); + } + } + + protected ProcessorDef define(final Processor processor) { + return new ProcessorDef() { + @Override + public Processor instance() { + return processor; + } + }; + } + + public static class CustomTimestampExtractor implements TimestampExtractor { + @Override + public long extract(ConsumerRecord record) { + return timestamp; + } + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index bea15f932da63..1f3e54135f504 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -104,7 +104,7 @@ public void testPartitionAssignmentChange() throws Exception { builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); - StreamThread thread = new StreamThread(builder, config, producer, consumer, new SystemTime()) { + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, new SystemTime()) { @Override protected StreamTask createStreamTask(int id, Collection partitionsForTask) { return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config); @@ -207,7 +207,7 @@ public void testMaybeClean() throws Exception { TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); - StreamThread thread = new StreamThread(builder, config, producer, consumer, mockTime) { + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, mockTime) { @Override public void maybeClean() { super.maybeClean(); @@ -325,7 +325,7 @@ public void testMaybeCommit() throws Exception { TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); - StreamThread thread = new StreamThread(builder, config, producer, consumer, mockTime) { + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, mockTime) { @Override public void maybeCommit() { super.maybeCommit(); diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriverTest.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriverTest.java deleted file mode 100644 index d990d474cf6b6..0000000000000 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriverTest.java +++ /dev/null @@ -1,323 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.test; - -import static org.junit.Assert.assertEquals; -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.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.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 ProcessorTopologyTestDriverTest { - - private static final Serializer STRING_SERIALIZER = new StringSerializer(); - private static final Deserializer STRING_DESERIALIZER = new StringDeserializer(); - private static final File STATE_DIR = new File("build/data").getAbsoluteFile(); - - protected static final String INPUT_TOPIC = "input-topic"; - protected static final String OUTPUT_TOPIC_1 = "output-topic-1"; - protected static final String OUTPUT_TOPIC_2 = "output-topic-2"; - - private static long timestamp = 1000L; - - private ProcessorTopologyTestDriver driver; - private StreamingConfig config; - - @Before - public void setup() { - STATE_DIR.mkdirs(); - Properties props = new Properties(); - props.setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); - props.setProperty(StreamingConfig.STATE_DIR_CONFIG, STATE_DIR.getAbsolutePath()); - props.setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName()); - props.setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - props.setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - props.setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - props.setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - this.config = new StreamingConfig(props); - } - - @After - public void cleanup() { - if (driver != null) { - driver.close(); - } - driver = null; - if (STATE_DIR.exists()) { - try { - Files.walkFileTree(STATE_DIR.toPath(), new SimpleFileVisitor() { - @Override - public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { - Files.delete(file); - return FileVisitResult.CONTINUE; - } - - @Override - public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { - Files.delete(dir); - return FileVisitResult.CONTINUE; - } - - }); - } catch (IOException e) { - // do nothing - } - } - } - - @Test - public void testDrivingSimpleTopology() { - driver = new ProcessorTopologyTestDriver(config, createSimpleTopology()); - driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); - assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1"); - assertNoOutputRecord(OUTPUT_TOPIC_2); - - driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); - assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2"); - assertNoOutputRecord(OUTPUT_TOPIC_2); - - driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); - driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); - driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); - assertNoOutputRecord(OUTPUT_TOPIC_2); - assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3"); - assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4"); - assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5"); - } - - @Test - public void testDrivingMultiplexingTopology() { - driver = new ProcessorTopologyTestDriver(config, createMultiplexingTopology()); - driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); - assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1(1)"); - assertNextOutputRecord(OUTPUT_TOPIC_2, "key1", "value1(2)"); - - driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); - assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2(1)"); - assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2(2)"); - - driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); - driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); - driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); - assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3(1)"); - assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4(1)"); - assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5(1)"); - assertNextOutputRecord(OUTPUT_TOPIC_2, "key3", "value3(2)"); - assertNextOutputRecord(OUTPUT_TOPIC_2, "key4", "value4(2)"); - assertNextOutputRecord(OUTPUT_TOPIC_2, "key5", "value5(2)"); - } - - @Test - public void testDrivingStatefulTopology() { - String storeName = "entries"; - driver = new ProcessorTopologyTestDriver(config, createStatefulTopology(storeName), storeName); - driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); - driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); - driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); - driver.process(INPUT_TOPIC, "key1", "value4", STRING_SERIALIZER, STRING_SERIALIZER); - assertNoOutputRecord(OUTPUT_TOPIC_1); - - KeyValueStore store = driver.getKeyValueStore("entries"); - assertEquals("value4", store.get("key1")); - assertEquals("value2", store.get("key2")); - assertEquals("value3", store.get("key3")); - assertNull(store.get("key4")); - } - - protected void assertNextOutputRecord(String topic, String key, String value) { - assertProducerRecord(driver.readOutput(topic, STRING_DESERIALIZER, STRING_DESERIALIZER), topic, key, value); - } - - protected void assertNoOutputRecord(String topic) { - assertNull(driver.readOutput(topic)); - } - - private void assertProducerRecord(ProducerRecord record, String topic, String key, String value) { - assertEquals(topic, record.topic()); - assertEquals(key, record.key()); - assertEquals(value, record.value()); - // Kafka Streaming doesn't set the partition, so it's always null - assertNull(record.partition()); - } - - protected TopologyBuilder createSimpleTopology() { - return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) - .addProcessor("processor", define(new ForwardingProcessor()), "source") - .addSink("sink", OUTPUT_TOPIC_1, "processor"); - } - - protected TopologyBuilder createMultiplexingTopology() { - return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) - .addProcessor("processor", define(new MultiplexingProcessor(2)), "source") - .addSink("sink1", OUTPUT_TOPIC_1, "processor") - .addSink("sink2", OUTPUT_TOPIC_2, "processor"); - } - - protected TopologyBuilder createStatefulTopology(String storeName) { - return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) - .addProcessor("processor", define(new StatefulProcessor(storeName)), "source") - .addSink("counts", OUTPUT_TOPIC_1, "processor"); - } - - /** - * Abstract base Processor. - */ - private static abstract class FauxProcessor implements Processor { - - protected ProcessorContext context; - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void punctuate(long streamTime) { - } - - @Override - public void close() { - } - } - - /** - * A processor that simply forwards all messages to all children. - */ - private static class ForwardingProcessor extends FauxProcessor { - @Override - public void init(ProcessorContext context) { - super.init(context); - } - - @Override - public void process(String key, String value) { - this.context.forward(key, value); - } - - @Override - public void punctuate(long streamTime) { - this.context.forward(Long.toString(streamTime), "punctuate"); - } - } - - /** - * A processor that forwards slightly-modified messages to each child. - */ - private static class MultiplexingProcessor extends FauxProcessor { - - private final int numChildren; - - public MultiplexingProcessor(int numChildren) { - this.numChildren = numChildren; - } - - @Override - public void init(ProcessorContext context) { - super.init(context); - } - - @Override - public void process(String key, String value) { - for (int i = 0; i != numChildren; ++i) { - this.context.forward(key, value + "(" + (i + 1) + ")", i); - } - } - - @Override - public void punctuate(long streamTime) { - for (int i = 0; i != numChildren; ++i) { - this.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. - */ - private static class StatefulProcessor extends FauxProcessor { - - private KeyValueStore store; - private final String storeName; - - public StatefulProcessor(String storeName) { - this.storeName = storeName; - } - - @Override - public void init(ProcessorContext context) { - super.init(context); - store = new InMemoryKeyValueStore<>(storeName, context); - } - - @Override - public void process(String key, String value) { - store.put(key, value); - } - - @Override - public void punctuate(long streamTime) { - int count = 0; - for (KeyValueIterator iter = store.all(); iter.hasNext();) { - iter.next(); - ++count; - } - this.context.forward(Long.toString(streamTime), count); - } - } - - private ProcessorDef define(final Processor processor) { - return new ProcessorDef() { - @Override - public Processor instance() { - return processor; - } - }; - } - - public static class CustomTimestampExtractor implements TimestampExtractor { - @Override - public long extract(ConsumerRecord record) { - return timestamp; - } - } -} From 69638c44dfd0b473f45029a0761c81c8435b57c2 Mon Sep 17 00:00:00 2001 From: Randall Hauch Date: Wed, 23 Sep 2015 13:07:38 -0500 Subject: [PATCH 268/275] Corrected incorrect package name in JavaDoc --- .../java/org/apache/kafka/test/ProcessorTopologyTestDriver.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index 9cbf8bce6b7ee..88b7a54748885 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -105,7 +105,7 @@ * for use on both the keys and values. Your test logic can then verify whether these output records are correct. *

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

Processor state

*

From 5b8e74f13bd1c3b687f3cd8c1dd6b670ced35c2e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 23 Sep 2015 11:48:20 -0700 Subject: [PATCH 269/275] Include testJar when uploading to Maven --- build.gradle | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/build.gradle b/build.gradle index f4ba921fc7f70..02b1db538e773 100644 --- a/build.gradle +++ b/build.gradle @@ -572,6 +572,14 @@ project(':streams') { dependsOn 'copyDependantLibs' } + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + checkstyle { configFile = new File(rootDir, "checkstyle/checkstyle.xml") } From c2ddd9a24fed6dbd9239ae3a0dfa070bf135a17e Mon Sep 17 00:00:00 2001 From: Randall Hauch Date: Wed, 23 Sep 2015 14:36:53 -0500 Subject: [PATCH 270/275] Created an AbstractProcessor and used it in ProcessorTopologyTest and the KStream processor implementations. --- .../kstream/internals/KStreamBranch.java | 5 +- .../kstream/internals/KStreamFilter.java | 5 +- .../kstream/internals/KStreamFlatMap.java | 5 +- .../internals/KStreamFlatMapValues.java | 7 +- .../kstream/internals/KStreamJoin.java | 7 +- .../streams/kstream/internals/KStreamMap.java | 5 +- .../kstream/internals/KStreamMapValues.java | 5 +- .../kstream/internals/KStreamPassThrough.java | 5 +- .../kstream/internals/KStreamProcessor.java | 44 ------------ .../kstream/internals/KStreamWindow.java | 8 +-- .../streams/processor/AbstractProcessor.java | 71 +++++++++++++++++++ .../kafka/streams/processor/Processor.java | 30 ++++++++ .../internals/ProcessorTopologyTest.java | 47 +++--------- 13 files changed, 140 insertions(+), 104 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamProcessor.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java index 473adcd363daf..6b661b423e60a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java @@ -17,6 +17,7 @@ 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; @@ -35,14 +36,14 @@ public Processor instance() { return new KStreamBranchProcessor(); } - private class KStreamBranchProcessor extends KStreamProcessor { + private class KStreamBranchProcessor extends AbstractProcessor { @Override public void process(K key, V value) { for (int i = 0; i < predicates.length; i++) { if (predicates[i].apply(key, value)) { // use forward with childIndex here and then break the loop // so that no record is going to be piped to multiple streams - context.forward(key, value, i); + context().forward(key, value, i); break; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java index 181f9f647919e..5444e70b8395e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java @@ -17,6 +17,7 @@ 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; @@ -36,11 +37,11 @@ public Processor instance() { return new KStreamFilterProcessor(); } - private class KStreamFilterProcessor extends KStreamProcessor { + private class KStreamFilterProcessor extends AbstractProcessor { @Override public void process(K key, V value) { if (filterOut ^ predicate.apply(key, value)) { - context.forward(key, value); + context().forward(key, value); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java index c367146c66ab0..410cfda6366f7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java @@ -19,6 +19,7 @@ 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; @@ -35,11 +36,11 @@ public Processor instance() { return new KStreamFlatMapProcessor(); } - private class KStreamFlatMapProcessor extends KStreamProcessor { + private class KStreamFlatMapProcessor extends AbstractProcessor { @Override public void process(K1 key, V1 value) { for (KeyValue newPair : mapper.apply(key, value)) { - context.forward(newPair.key, newPair.value); + context().forward(newPair.key, newPair.value); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java index 4f3e4755281b4..edca42142ed2b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -17,8 +17,9 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorDef; class KStreamFlatMapValues implements ProcessorDef { @@ -34,12 +35,12 @@ public Processor instance() { return new KStreamFlatMapValuesProcessor(); } - private class KStreamFlatMapValuesProcessor extends KStreamProcessor { + private class KStreamFlatMapValuesProcessor extends AbstractProcessor { @Override public void process(K1 key, V1 value) { Iterable newValues = mapper.apply(value); for (V2 v : newValues) { - context.forward(key, v); + context().forward(key, v); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java index f5d96007007f3..4003d29f8a874 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java @@ -19,6 +19,7 @@ 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; @@ -44,7 +45,7 @@ public Processor instance() { return new KStreamJoinProcessor(windowName); } - private class KStreamJoinProcessor extends KStreamProcessor { + private class KStreamJoinProcessor extends AbstractProcessor { private final String windowName; protected Finder finder; @@ -73,11 +74,11 @@ Iterator find(K key, long timestamp) { @Override public void process(K key, V1 value) { - long timestamp = context.timestamp(); + long timestamp = context().timestamp(); Iterator iter = finder.find(key, timestamp); if (iter != null) { while (iter.hasNext()) { - context.forward(key, joiner.apply(value, iter.next())); + context().forward(key, joiner.apply(value, iter.next())); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java index eff5cfe4f0b57..4d3134874f3e2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java @@ -17,6 +17,7 @@ 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; @@ -35,11 +36,11 @@ public Processor instance() { return new KStreamMapProcessor(); } - private class KStreamMapProcessor extends KStreamProcessor { + private class KStreamMapProcessor extends AbstractProcessor { @Override public void process(K1 key, V1 value) { KeyValue newPair = mapper.apply(key, value); - context.forward(newPair.key, newPair.value); + context().forward(newPair.key, newPair.value); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java index 664acb90352d1..dac655096fcc6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java @@ -17,6 +17,7 @@ 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; @@ -34,11 +35,11 @@ public Processor instance() { return new KStreamMapProcessor(); } - private class KStreamMapProcessor extends KStreamProcessor { + private class KStreamMapProcessor extends AbstractProcessor { @Override public void process(K1 key, V1 value) { V2 newValue = mapper.apply(value); - context.forward(key, newValue); + context().forward(key, newValue); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java index 71c2427a9a51a..ea395503534e4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java @@ -17,6 +17,7 @@ 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; @@ -27,10 +28,10 @@ public Processor instance() { return new KStreamPassThroughProcessor(); } - public class KStreamPassThroughProcessor extends KStreamProcessor { + public class KStreamPassThroughProcessor extends AbstractProcessor { @Override public void process(K key, V value) { - context.forward(key, value); + context().forward(key, value); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamProcessor.java deleted file mode 100644 index a128ac6953ea6..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamProcessor.java +++ /dev/null @@ -1,44 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streams.kstream.internals; - -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; - -abstract class KStreamProcessor implements Processor { - - protected ProcessorContext context; - - @Override - abstract public void process(K key, V value); - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void punctuate(long streamTime) { - // do nothing - } - - @Override - public void close() { - // do nothing - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java index e15c82894a254..6ebcbe16e4d1b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java @@ -18,6 +18,7 @@ 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; @@ -40,7 +41,7 @@ public Processor instance() { return new KStreamWindowProcessor(); } - private class KStreamWindowProcessor extends KStreamProcessor { + private class KStreamWindowProcessor extends AbstractProcessor { private Window window; @@ -51,12 +52,11 @@ public void init(ProcessorContext context) { this.window.init(context); } - @SuppressWarnings("unchecked") @Override public void process(K key, V value) { synchronized (this) { - window.put(key, value, context.timestamp()); - context.forward(key, value); + window.put(key, value, context().timestamp()); + context().forward(key, value); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java b/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java new file mode 100644 index 0000000000000..01d0024d64d78 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.processor; + +/** + * An abstract implementation of {@link Processor} that manages the {@link ProcessorContext} instance and provides default no-op + * implementations of {@link #punctuate(long)} and {@link #close()}. + * + * @param the type of keys + * @param the type of values + */ +public abstract class AbstractProcessor implements Processor { + + private ProcessorContext context; + + protected AbstractProcessor() { + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + /** + * Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context + * during {@link #init(ProcessorContext) initialization}. + *

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

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

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

+ */ + @Override + public void close() { + // do nothing + } + + /** + * Get the processor's context set during {@link #init(ProcessorContext) initialization}. + * + * @return the processor context; null only when called prior to {@link #init(ProcessorContext) initialization}. + */ + protected final ProcessorContext context() { + return this.context; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java index ff6c2e01bfd5d..3cade3a907424 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java @@ -17,13 +17,43 @@ package org.apache.kafka.streams.processor; +/** + * A processor of messages. + * + * @param the type of keys + * @param the type of values + */ public interface Processor { + /** + * Initialize this processor with the given context. The framework ensures this is called once per processor when the topology + * that contains it is initialized. + *

+ * If this processor is to be {@link #punctuate(long) called periodically} by the framework, then this method should + * {@link ProcessorContext#schedule(long) schedule itself} with the provided context. + * + * @param context the context; may not be null + */ void init(ProcessorContext context); + /** + * Process the message with the given key and value. + * + * @param key the key for the message + * @param value the value for the message + */ void process(K key, V value); + /** + * Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context + * during {@link #init(ProcessorContext) initialization}. + * + * @param timestamp the stream time when this method is being called + */ void punctuate(long timestamp); + /** + * Close this processor and clean up any resources. + */ void close(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index d265c65b833d7..1abb989e30de2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -28,6 +28,7 @@ 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; @@ -233,48 +234,23 @@ protected TopologyBuilder createStatefulTopology(String storeName) { /** * A processor that simply forwards all messages to all children. */ - protected static abstract class FauxProcessor implements Processor { - - protected ProcessorContext context; - - @Override - public void init(ProcessorContext context) { - this.context = context; - } - - @Override - public void punctuate(long streamTime) { - } - - @Override - public void close() { - } - } - - /** - * A processor that simply forwards all messages to all children. - */ - protected static class ForwardingProcessor extends FauxProcessor { - @Override - public void init(ProcessorContext context) { - super.init(context); - } + protected static class ForwardingProcessor extends AbstractProcessor { @Override public void process(String key, String value) { - this.context.forward(key, value); + context().forward(key, value); } @Override public void punctuate(long streamTime) { - this.context.forward(Long.toString(streamTime), "punctuate"); + context().forward(Long.toString(streamTime), "punctuate"); } } /** * A processor that forwards slightly-modified messages to each child. */ - protected static class MultiplexingProcessor extends FauxProcessor { + protected static class MultiplexingProcessor extends AbstractProcessor { private final int numChildren; @@ -282,22 +258,17 @@ public MultiplexingProcessor(int numChildren) { this.numChildren = numChildren; } - @Override - public void init(ProcessorContext context) { - super.init(context); - } - @Override public void process(String key, String value) { for (int i = 0; i != numChildren; ++i) { - this.context.forward(key, value + "(" + (i + 1) + ")", i); + context().forward(key, value + "(" + (i + 1) + ")", i); } } @Override public void punctuate(long streamTime) { for (int i = 0; i != numChildren; ++i) { - this.context.forward(Long.toString(streamTime), "punctuate(" + (i + 1) + ")", i); + context().forward(Long.toString(streamTime), "punctuate(" + (i + 1) + ")", i); } } } @@ -306,7 +277,7 @@ public void punctuate(long streamTime) { * 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 FauxProcessor { + protected static class StatefulProcessor extends AbstractProcessor { private KeyValueStore store; private final String storeName; @@ -333,7 +304,7 @@ public void punctuate(long streamTime) { iter.next(); ++count; } - this.context.forward(Long.toString(streamTime), count); + context().forward(Long.toString(streamTime), count); } } From 6466989f3c14a569055d55bd55f1535beb8f84c1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 23 Sep 2015 14:34:14 -0700 Subject: [PATCH 271/275] Fix delete in in-memory store, and let delete return old value --- .../apache/kafka/streams/state/InMemoryKeyValueStore.java | 4 ++-- .../java/org/apache/kafka/streams/state/KeyValueStore.java | 3 ++- .../apache/kafka/streams/state/MeteredKeyValueStore.java | 6 ++++-- .../apache/kafka/streams/state/RocksDBKeyValueStore.java | 4 +++- 4 files changed, 11 insertions(+), 6 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java index c24f1d07c502b..e9aaa20265a53 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java @@ -84,8 +84,8 @@ public void putAll(List> entries) { } @Override - public void delete(K key) { - put(key, null); + public V delete(K key) { + return this.map.remove(key); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java index 83fe459863f61..e4faed1efc576 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java @@ -61,9 +61,10 @@ public interface KeyValueStore extends StateStore { * 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 void delete(K key); + abstract public V delete(K key); /** * Get an iterator over a given range of keys. This iterator MUST be closed after use. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java index d781fdd7b688b..018f1c621c64e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java @@ -173,14 +173,16 @@ public void putAll(List> entries) { } @Override - public void delete(K key) { + public V delete(K key) { long startNs = time.nanoseconds(); try { - this.inner.delete(key); + 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()); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java index 1ae5aaf18fbd0..e0962a206ad71 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java @@ -162,8 +162,10 @@ public void putAll(List> entries) { } @Override - public void delete(byte[] key) { + public byte[] delete(byte[] key) { + byte[] value = get(key); put(key, null); + return value; } @Override From 425df16aace2d53c9686395c46c8ae197b96a175 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 25 Sep 2015 11:13:57 -0700 Subject: [PATCH 272/275] flush records and state changelog before local state --- .../kafka/streams/processor/internals/StreamTask.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 714a732bff771..6ff60b46f99b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -242,13 +242,12 @@ public ProcessorTopology topology() { * Commit the current task state */ public void commit() { - // 1) flush local state - stateMgr.flush(); - - // 2) flush produced records in the downstream - // TODO: this will actually block on all produced records across the tasks + // 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) { consumer.commitSync(consumedOffsets); @@ -340,7 +339,7 @@ public void forward(K key, V value, int childIndex) { currNode = thisNode; } } - + public ProcessorContext context() { return processorContext; } From 8812e12a211e54a312ef3eda9dd6695b2293f1b9 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 25 Sep 2015 15:11:06 -0700 Subject: [PATCH 273/275] rebase on apache, fixes some issues in MockRestoreConsumer --- .../processor/internals/StreamTask.java | 7 +- .../internals/ProcessorStateManagerTest.java | 120 +------------- .../kafka/test/MockRestoreConsumer.java | 146 ++++++++++++++++++ .../test/ProcessorTopologyTestDriver.java | 17 +- 4 files changed, 156 insertions(+), 134 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 6ff60b46f99b3..40fb723577175 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -19,6 +19,7 @@ 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; @@ -250,7 +251,11 @@ public void commit() { // 3) commit consumed offsets if it is dirty already if (commitOffsetNeeded) { - consumer.commitSync(consumedOffsets); + Map consumedOffsetsAndMetadata = new HashMap<>(consumedOffsets.size()); + for (Map.Entry entry : consumedOffsets.entrySet()) { + consumedOffsetsAndMetadata.put(entry.getKey(), new OffsetAndMetadata(entry.getValue())); + } + consumer.commitSync(consumedOffsetsAndMetadata); commitOffsetNeeded = false; } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java index c983c6b1a3a6e..5806901f65e15 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -18,20 +18,16 @@ 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.apache.kafka.test.MockRestoreConsumer; import org.junit.Test; import java.io.File; @@ -42,7 +38,6 @@ 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; @@ -92,116 +87,6 @@ public void apply(byte[] key, byte[] value) { }; } - private static class MockRestoreConsumer extends MockConsumer { - private final Serializer serializer = new IntegerSerializer(); - - public TopicPartition assignedPartition = null; - public TopicPartition seekPartition = null; - public long seekOffset = -1L; - public boolean seekToBeginingCalled = false; - public boolean seekToEndCalled = false; - private long endOffset = -1L; - private long currentOffset = -1L; - - private ArrayList> recordBuffer = new ArrayList<>(); - - MockRestoreConsumer() { - super(OffsetResetStrategy.EARLIEST); - } - - // prepares this mock restore consumer for a state store registration - public void prepare() { - assignedPartition = null; - seekOffset = -1L; - seekToBeginingCalled = false; - seekToEndCalled = false; - endOffset = -1L; - recordBuffer.clear(); - } - - // buffer a record (we cannot use addRecord because we need to add records before asigning a partition) - public void bufferRecord(ConsumerRecord record) { - recordBuffer.add( - new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), - serializer.serialize(record.topic(), record.key()), - serializer.serialize(record.topic(), record.value()))); - endOffset = record.offset(); - } - - @Override - public synchronized void assign(List partitions) { - int numPartitions = partitions.size(); - if (numPartitions > 1) - throw new IllegalArgumentException("RestoreConsumer: more than one partition specified"); - - if (numPartitions == 1) { - if (assignedPartition != null) - throw new IllegalStateException("RestoreConsumer: partition already assigned"); - assignedPartition = partitions.get(0); - } - - super.assign(partitions); - } - - @Override - public ConsumerRecords poll(long timeout) { - // add buffered records to MockConsumer - for (ConsumerRecord record : recordBuffer) { - super.addRecord(record); - } - recordBuffer.clear(); - - ConsumerRecords records = super.poll(timeout); - - // set the current offset - Iterable> partitionRecords = records.records(assignedPartition); - for (ConsumerRecord record : partitionRecords) { - currentOffset = record.offset(); - } - - return records; - } - - @Override - public synchronized long position(TopicPartition partition) { - if (!partition.equals(assignedPartition)) - throw new IllegalStateException("RestoreConsumer: unassigned partition"); - - return currentOffset; - } - - @Override - public synchronized void seek(TopicPartition partition, long offset) { - if (offset < 0) - throw new IllegalArgumentException("RestoreConsumer: offset should not be negative"); - - if (seekOffset >= 0) - throw new IllegalStateException("RestoreConsumer: offset already seeked"); - - seekPartition = partition; - seekOffset = offset; - currentOffset = offset; - super.seek(partition, offset); - } - - @Override - public synchronized void seekToBeginning(TopicPartition... partitions) { - if (seekToBeginingCalled) - throw new IllegalStateException("RestoreConsumer: offset already seeked to beginning"); - - seekToBeginingCalled = true; - } - - @Override - public synchronized void seekToEnd(TopicPartition... partitions) { - if (seekToEndCalled) - throw new IllegalStateException("RestoreConsumer: offset already seeked to beginning"); - - seekToEndCalled = true; - currentOffset = endOffset; - } - } - @Test public void testLockStateDirectory() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); @@ -295,6 +180,7 @@ public void testRegisterPersistentStore() throws IOException { Utils.delete(baseDir); } } + @Test public void testRegisterNonPersistentStore() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); @@ -328,7 +214,7 @@ public void testRegisterNonPersistentStore() throws IOException { stateMgr.register(nonPersistentStore, nonPersistentStore.restoreFunc); assertEquals(new TopicPartition("nonPersistentStore", 2), restoreConsumer.assignedPartition); - assertEquals(-1L, restoreConsumer.seekOffset); + assertEquals(0L, restoreConsumer.seekOffset); assertTrue(restoreConsumer.seekToBeginingCalled); assertTrue(restoreConsumer.seekToEndCalled); assertEquals(expectedKeys, nonPersistentStore.keys); diff --git a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java new file mode 100644 index 0000000000000..23c4504c219ab --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java @@ -0,0 +1,146 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.test; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class MockRestoreConsumer extends MockConsumer { + private final Serializer serializer = new IntegerSerializer(); + + public TopicPartition assignedPartition = null; + public TopicPartition seekPartition = null; + public long seekOffset = -1L; + public boolean seekToBeginingCalled = false; + public boolean seekToEndCalled = false; + private long endOffset = 0L; + private long currentOffset = 0L; + + private ArrayList> recordBuffer = new ArrayList<>(); + + public MockRestoreConsumer() { + super(OffsetResetStrategy.EARLIEST); + } + + // prepares this mock restore consumer for a state store registration + public void prepare() { + assignedPartition = null; + seekOffset = -1L; + seekToBeginingCalled = false; + seekToEndCalled = false; + endOffset = 0L; + recordBuffer.clear(); + } + + // buffer a record (we cannot use addRecord because we need to add records before asigning a partition) + public void bufferRecord(ConsumerRecord record) { + recordBuffer.add( + new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), + serializer.serialize(record.topic(), record.key()), + serializer.serialize(record.topic(), record.value()))); + endOffset = record.offset(); + + super.updateEndOffsets(Collections.singletonMap(assignedPartition, endOffset)); + } + + @Override + public synchronized void assign(List partitions) { + int numPartitions = partitions.size(); + if (numPartitions > 1) + throw new IllegalArgumentException("RestoreConsumer: more than one partition specified"); + + if (numPartitions == 1) { + if (assignedPartition != null) + throw new IllegalStateException("RestoreConsumer: partition already assigned"); + assignedPartition = partitions.get(0); + + // set the beginning offset + super.updateBeginningOffsets(Collections.singletonMap(assignedPartition, 0L)); + super.updateEndOffsets(Collections.singletonMap(assignedPartition, 0L)); + } + + super.assign(partitions); + } + + @Override + public ConsumerRecords poll(long timeout) { + // add buffered records to MockConsumer + for (ConsumerRecord record : recordBuffer) { + super.addRecord(record); + } + recordBuffer.clear(); + + ConsumerRecords records = super.poll(timeout); + + // set the current offset + Iterable> partitionRecords = records.records(assignedPartition); + for (ConsumerRecord record : partitionRecords) { + currentOffset = record.offset(); + } + + return records; + } + + @Override + public synchronized long position(TopicPartition partition) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: unassigned partition"); + + return currentOffset; + } + + @Override + public synchronized void seek(TopicPartition partition, long offset) { + if (offset < 0) + throw new IllegalArgumentException("RestoreConsumer: offset should not be negative"); + + if (seekOffset >= 0) + throw new IllegalStateException("RestoreConsumer: offset already seeked"); + + seekPartition = partition; + seekOffset = offset; + currentOffset = offset; + super.seek(partition, offset); + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + if (seekToBeginingCalled) + throw new IllegalStateException("RestoreConsumer: offset already seeked to beginning"); + + seekToBeginingCalled = true; + } + + @Override + public synchronized void seekToEnd(TopicPartition... partitions) { + if (seekToEndCalled) + throw new IllegalStateException("RestoreConsumer: offset already seeked to beginning"); + + seekToEndCalled = true; + currentOffset = endOffset; + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index 88b7a54748885..1a2f6cd014cec 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -284,23 +284,8 @@ public void close() { * @return the mock consumer; never null */ protected MockConsumer createRestoreConsumer(int id, String... storeNames) { - MockConsumer consumer = new MockConsumer(OffsetResetStrategy.LATEST) { - @Override - public synchronized void seekToEnd(TopicPartition... partitions) { - // do nothing ... - } - - @Override - public synchronized void seekToBeginning(TopicPartition... partitions) { - // do nothing ... - } + MockConsumer consumer = new MockRestoreConsumer(); - @Override - public synchronized long position(TopicPartition partition) { - // do nothing ... - return 0L; - } - }; // For each store name ... for (String storeName : storeNames) { String topicName = storeName; From 20934d1e8401b95314dd0cee570f8488812ddd6f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 25 Sep 2015 16:19:29 -0700 Subject: [PATCH 274/275] revert some changes --- .../internals/ProcessorStateManagerTest.java | 146 +++++++++++++++++- .../kafka/test/MockRestoreConsumer.java | 146 ------------------ .../test/ProcessorTopologyTestDriver.java | 18 ++- 3 files changed, 158 insertions(+), 152 deletions(-) delete mode 100644 streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java index 5806901f65e15..343ed52b7ef54 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -18,16 +18,20 @@ 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.apache.kafka.test.MockRestoreConsumer; import org.junit.Test; import java.io.File; @@ -38,6 +42,7 @@ 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; @@ -87,6 +92,135 @@ public void apply(byte[] key, byte[] value) { }; } + private class MockRestoreConsumer extends MockConsumer { + private final Serializer serializer = new IntegerSerializer(); + + public TopicPartition assignedPartition = null; + public TopicPartition seekPartition = null; + public long seekOffset = -1L; + public boolean seekToBeginingCalled = false; + public boolean seekToEndCalled = false; + private long endOffset = 0L; + private long currentOffset = 0L; + + private ArrayList> recordBuffer = new ArrayList<>(); + + MockRestoreConsumer() { + super(OffsetResetStrategy.EARLIEST); + + reset(); + } + + // reset this mock restore consumer for a state store registration + public void reset() { + assignedPartition = null; + seekOffset = -1L; + seekToBeginingCalled = false; + seekToEndCalled = false; + endOffset = 0L; + recordBuffer.clear(); + } + + // buffer a record (we cannot use addRecord because we need to add records before asigning a partition) + public void bufferRecord(ConsumerRecord record) { + recordBuffer.add( + new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), + serializer.serialize(record.topic(), record.key()), + serializer.serialize(record.topic(), record.value()))); + endOffset = record.offset(); + + super.updateEndOffsets(Collections.singletonMap(assignedPartition, endOffset)); + } + + @Override + public synchronized void assign(List partitions) { + int numPartitions = partitions.size(); + if (numPartitions > 1) + throw new IllegalArgumentException("RestoreConsumer: more than one partition specified"); + + if (numPartitions == 1) { + if (assignedPartition != null) + throw new IllegalStateException("RestoreConsumer: partition already assigned"); + assignedPartition = partitions.get(0); + + // set the beginning offset to 0 + // NOTE: this is users responsible to set the initial lEO. + super.updateBeginningOffsets(Collections.singletonMap(assignedPartition, 0L)); + } + + super.assign(partitions); + } + + @Override + public ConsumerRecords poll(long timeout) { + // add buffered records to MockConsumer + for (ConsumerRecord record : recordBuffer) { + super.addRecord(record); + } + recordBuffer.clear(); + + ConsumerRecords records = super.poll(timeout); + + // set the current offset + Iterable> partitionRecords = records.records(assignedPartition); + for (ConsumerRecord record : partitionRecords) { + currentOffset = record.offset(); + } + + return records; + } + + @Override + public synchronized long position(TopicPartition partition) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: unassigned partition"); + + return currentOffset; + } + + @Override + public synchronized void seek(TopicPartition partition, long offset) { + if (offset < 0) + throw new IllegalArgumentException("RestoreConsumer: offset should not be negative"); + + if (seekOffset >= 0) + throw new IllegalStateException("RestoreConsumer: offset already seeked"); + + seekPartition = partition; + seekOffset = offset; + currentOffset = offset; + super.seek(partition, offset); + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + if (partitions.length != 1) + throw new IllegalStateException("RestoreConsumer: other than one partition specified"); + + for (TopicPartition partition : partitions) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition"); + } + + seekToBeginingCalled = true; + currentOffset = 0L; + } + + @Override + public synchronized void seekToEnd(TopicPartition... partitions) { + if (partitions.length != 1) + throw new IllegalStateException("RestoreConsumer: other than one partition specified"); + + for (TopicPartition partition : partitions) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition"); + } + + seekToEndCalled = true; + currentOffset = endOffset; + } + } + @Test public void testLockStateDirectory() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); @@ -147,12 +281,13 @@ public void testRegisterPersistentStore() throws IOException { 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.prepare(); + restoreConsumer.reset(); ArrayList expectedKeys = new ArrayList<>(); for (int i = 1; i <= 3; i++) { @@ -194,12 +329,13 @@ public void testRegisterNonPersistentStore() throws IOException { 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.prepare(); + restoreConsumer.reset(); ArrayList expectedKeys = new ArrayList<>(); for (int i = 1; i <= 3; i++) { @@ -283,10 +419,10 @@ public void testClose() throws IOException { // make sure the checkpoint file is deleted assertFalse(checkpointFile.exists()); - restoreConsumer.prepare(); + restoreConsumer.reset(); stateMgr.register(persistentStore, persistentStore.restoreFunc); - restoreConsumer.prepare(); + restoreConsumer.reset(); stateMgr.register(nonPersistentStore, nonPersistentStore.restoreFunc); } finally { // close the state manager with the ack'ed offsets diff --git a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java deleted file mode 100644 index 23c4504c219ab..0000000000000 --- a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java +++ /dev/null @@ -1,146 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.test; - -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.TopicPartition; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.Serializer; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -public class MockRestoreConsumer extends MockConsumer { - private final Serializer serializer = new IntegerSerializer(); - - public TopicPartition assignedPartition = null; - public TopicPartition seekPartition = null; - public long seekOffset = -1L; - public boolean seekToBeginingCalled = false; - public boolean seekToEndCalled = false; - private long endOffset = 0L; - private long currentOffset = 0L; - - private ArrayList> recordBuffer = new ArrayList<>(); - - public MockRestoreConsumer() { - super(OffsetResetStrategy.EARLIEST); - } - - // prepares this mock restore consumer for a state store registration - public void prepare() { - assignedPartition = null; - seekOffset = -1L; - seekToBeginingCalled = false; - seekToEndCalled = false; - endOffset = 0L; - recordBuffer.clear(); - } - - // buffer a record (we cannot use addRecord because we need to add records before asigning a partition) - public void bufferRecord(ConsumerRecord record) { - recordBuffer.add( - new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), - serializer.serialize(record.topic(), record.key()), - serializer.serialize(record.topic(), record.value()))); - endOffset = record.offset(); - - super.updateEndOffsets(Collections.singletonMap(assignedPartition, endOffset)); - } - - @Override - public synchronized void assign(List partitions) { - int numPartitions = partitions.size(); - if (numPartitions > 1) - throw new IllegalArgumentException("RestoreConsumer: more than one partition specified"); - - if (numPartitions == 1) { - if (assignedPartition != null) - throw new IllegalStateException("RestoreConsumer: partition already assigned"); - assignedPartition = partitions.get(0); - - // set the beginning offset - super.updateBeginningOffsets(Collections.singletonMap(assignedPartition, 0L)); - super.updateEndOffsets(Collections.singletonMap(assignedPartition, 0L)); - } - - super.assign(partitions); - } - - @Override - public ConsumerRecords poll(long timeout) { - // add buffered records to MockConsumer - for (ConsumerRecord record : recordBuffer) { - super.addRecord(record); - } - recordBuffer.clear(); - - ConsumerRecords records = super.poll(timeout); - - // set the current offset - Iterable> partitionRecords = records.records(assignedPartition); - for (ConsumerRecord record : partitionRecords) { - currentOffset = record.offset(); - } - - return records; - } - - @Override - public synchronized long position(TopicPartition partition) { - if (!partition.equals(assignedPartition)) - throw new IllegalStateException("RestoreConsumer: unassigned partition"); - - return currentOffset; - } - - @Override - public synchronized void seek(TopicPartition partition, long offset) { - if (offset < 0) - throw new IllegalArgumentException("RestoreConsumer: offset should not be negative"); - - if (seekOffset >= 0) - throw new IllegalStateException("RestoreConsumer: offset already seeked"); - - seekPartition = partition; - seekOffset = offset; - currentOffset = offset; - super.seek(partition, offset); - } - - @Override - public synchronized void seekToBeginning(TopicPartition... partitions) { - if (seekToBeginingCalled) - throw new IllegalStateException("RestoreConsumer: offset already seeked to beginning"); - - seekToBeginingCalled = true; - } - - @Override - public synchronized void seekToEnd(TopicPartition... partitions) { - if (seekToEndCalled) - throw new IllegalStateException("RestoreConsumer: offset already seeked to beginning"); - - seekToEndCalled = true; - currentOffset = endOffset; - } -} diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index 1a2f6cd014cec..75f8b4c1e4943 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -284,8 +284,23 @@ public void close() { * @return the mock consumer; never null */ protected MockConsumer createRestoreConsumer(int id, String... storeNames) { - MockConsumer consumer = new MockRestoreConsumer(); + MockConsumer consumer = new MockConsumer(OffsetResetStrategy.LATEST) { + @Override + public synchronized void seekToEnd(TopicPartition... partitions) { + // do nothing ... + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + // do nothing ... + } + @Override + public synchronized long position(TopicPartition partition) { + // do nothing ... + return 0L; + } + }; // For each store name ... for (String storeName : storeNames) { String topicName = storeName; @@ -295,6 +310,7 @@ protected MockConsumer createRestoreConsumer(int id, String... s List partitionInfos = new ArrayList<>(); partitionInfos.add(new PartitionInfo(topicName, id, null, null, null)); consumer.updatePartitions(topicName, partitionInfos); + consumer.updateEndOffsets(Collections.singletonMap(new TopicPartition(topicName, id), 0L)); } return consumer; } From 7871687d4fed1f55c3064232caf1fd08ed2475c2 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 25 Sep 2015 16:27:40 -0700 Subject: [PATCH 275/275] add one TODO to the restoration process --- .../kafka/streams/processor/internals/ProcessorStateManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 97dbb3bf89387..2f1fb35894170 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -143,6 +143,7 @@ public void register(StateStore store, RestoreFunc restoreFunc) { 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); }

D zp8o2-okEC%JcpEvH{oC}%=f|}5)Lhc!{BfUN0f^qSRmm@2}emdnyAM}sFSdeKIY&yr8K%s!cQb@lyJF(pAz8;30F$E ziaxI<@G}C}5ZFZES_0P*xSqfb1a6dYlZ2ns=gkssA$+TZ+X&xI;0^-6AaEyvy9jJ1 zaJPhvgaHZn(CA(XTOE@ zzD3||0`Expjf8h4{8qy6B>Y~&dqn;Nfj>%kpYWdud_dqs0)Hm(7Xlv<_-h&b4gT&G zyzU=F`6q#Y5&2^R|0eJWflmp1M&NS-Ul90`z<&t*m%vv9zDD2?1fU#&$LlHbdWy@% z^KdW%Pl@!D67~{@l%6u_DW`Eoxp>Z_c|AU_rxHYETf_0o8`^`RB@K0p8rneAHpP=Y zwR%Di_ZqdU_3qVrZ8#o{#$&aKWIP=Y$0N1r{)CaLZAXTvegueNZ4K?MEz8;)92r&3 z4Z*rab-}ujEq%ot5Ty&-T9(olqgw0Qf{nq(mSrJ_kr+kK%j+9LYz6J;w<&Ius}D(`X*b*|2rXUBVrHL%Sh5CG7n&yZr{qo)bkD?sJXztR3(k}Ds@xeXg+}I5?L-)7o6T&YT>GkXC$pOm z&z+!M$J?ZW){M@Z&59b4H$$>~HqFw#(jMDdPpHQVzr4AjEnA-L2+{h9g{CdNXnt&= zd26q6h34#iM;+JFxGZ#RTg&oR=g^SA2|09Yr>u?Dg{EnpKWDy0iw7*-eusO)MUntWqr|3BN@~?BS`T##lw1Jg`PBNY!Qpoy=Dr; zj6GkatyVXH4S-w_j@WzZ2@}NhJuk0Im`jcR6t4EQr}gmaW<9|wcs;{Fl()zGl3`=9 zNqYM1oo27n*XkM^05u?HeWNA?#LT_qUu+tYE)ZTcBedG+_j-nFo)I2D9=2ew5iq*~ z@$Ntxzmmq9J|mS5P%AVWb@Eh!sGvn_jbzG<$FN}10&8Tis%veWjV)I=o=h4MJ&k-J zvrF@gqz#tT1^QA(a#pG@k%*W^SHO&=jUFQzSgS|+4BS*t2iBR9NTAaQB=l4YIXcmA zNK5xxDy0(fSPDH>hT$+`X_h^TWFMtu2T}2Pl#YNYipJ*|^5vjdIYfmj?TX**nFxa@YZr!D?~j!sp~vo-M! zsp^g4nL+nqtjOHXWPCL`%Iw1675BzdX)7J*Gp0ukJG_dIm=f_MGR>yN^cjewS9(oU zG@TWvhr>o9ow8FYg`59|HXoVLlWCKa!$>CM$xxUN@H7&}ga8qousPf_17>$XkM-mJ znnPqG#?aDi?mCmmSsF3)E+Yxzuv{L?WIddwWYALq#b@nmrZ5F9B~ge>8kn+SonlL3 zQU+R#x}aalktwtqlb5`OUwLbb%oOh4;9BNdLb1byO7+LW*{s^NCYpGc95|eV8FD*s zD#x>S;wG0fg5mj(7S`Oo4- z0dN$>TADPl-gJd9D4J)i$FGS$k`~2SJ5YsAHkGdhQr6}BRBya5(uLW{p-oc-!emay zyZXX-A`DcOs$jY=8N&>j9hfy2Bh`gTkmh(}e)ggIjqPoyx;Je`jd&lP2#y1d=;-R} zbQ8@-DXMT$sG}+5FIT3LEt+_rv@NrFP~|GexR8rl?hbcFr=BuG@$POmdK?-3CvwyW zKR439dBS#1~z%!EI)Rpg0OT8}2^y0E|Pq*1$R;xDB6N5nr|DUDLp z9J$JI26|>YWE5&4)K;`AHBoxID(xMVGkuEg6P|2qt4j^|15=PRD(@lzbMb1Yq9S0N_6z`_8mu-yM%c;_D2&o)=@o5u-{M zaZ)ZdlX?m)BB=G~teafBpclbQ+jx~qXpFxuU@_R{Ade(}lo2Q=P(hy2$TJ%I+;Ro_ zCijHnzGdXC&VH<4sDte%9+Q`NHOlvq4#ONLDKcK#x$d#KD~a1jf~nS7xjEAoPN|wr zPOgSnRyjTG%Eo$`F26@8ddh1J-w(> z+Q29zS7To1rJj8{U^qE(1OY#RDstyY0;5O{9$)7uf~ep9bw;|m_|+BXWPmC<=-_JF zXEbdwhQc;1g{Qzi*L;WQR9oDFsA8Dd#Gsu#W7U{0>};qd!aD}9Km%|36t#uD1{TRq z0~->(8|}w6*pW~qP`GmgO)cDPrusTl;iQRIK{}3+Ihr(aYLvThVHEXPz=$T&{ee_E zNj(%rRu9CCb&km>VPe9r*dZk+N_{)7z|i#cgP2b{*h8!N$~7KSQ?1P2Wpq@=k{8EO zjH+w}L(X0)4aU>_1oHAk0+VPsnLvPsQ^b2-&v?x_SZaZY#tT@rHO4|HQmRjHMqo@M|pnOu-m`17M8L zamM(nWQ@-@#`x%BjPEYS`1E3o4=cv_WMPbN4#xQGV2m#h#@PHb#?GEGHusFNKWB_B zI%Dk88Dpc(7+YP&*s?Olu9Y!1u8gsFWsL1BW9(oVV-w35`&h=<$}+~TlreUnjIqCD zj4dW%>M|K)qsbV1O~%-6GRBURF*c5ju?=L5eHvrz#Ta8d#uz&?#@LiG#=eX(_Ee0q zTVjko3S(?O7-Rpz7+Vm=*o830Muai;B8;(hV2s@ZV{9N8<7LPg??c9T`7y@(k1<|> zjPVv^j29teybBrQb;uZRLdJL*GRFH5u}1*?CW&wwj%o$g0K_HJ2L;?s(=Z)jDHA8c z4E)v<1I&b3IFc~iUOarWm?(;cs2tobJRRO0qNt;2I|6@kMwIwV2Zgu2qnHVj2xUwt zVM2LERQNR}lrq7W5taU7L>Lsq2gC@HSe%u}{C;P;DsOt^b}_00B@Bvc|7eG63~`la zg^Z<5$ML30t?{(!1STl8CKAUaN5EtfAhQAjWX2SifPIL6s!PDWMA(lB%C7yDW)-B_ z0ZjL@y)`cSG@`TF)AKZ*K^xBGWlG~&r15MfD2;2C9E@JIe~u$(F3Bm$Dn3tfO(w1b zUE&T>^tnhM6bCCE=A+v^9Yq;2rhPyhVh`ry;Ltp&qzo;yf_xYW%VRkl{o!IcA|n=9 z8@abUGSBRz$n2wCdLBb=t#g^ZP$_<-TS`4iaplaS0{Y~s++ee~vLsLC#i$Egp!|Jo zMl2a(fm@44+hg_UvE%GPBMz44SxFoDn_N~lqip{&mz6CP?N%aib{tOvT*qwB0@_F- zF2;kX(lJkB@PAclISFtj8t3N@NjWK31Owk zh_3dIl8orV1FG-)ikTwJ6h)?3-VRw4Sm$^Fo&5=RjH;Mc0>;TPYlzOv zl6lL9guwD=l9r{MWocUGN{Os8Rt@AM+{fFil*k@9>jdU_d~Kc@g+0F3?ag(xWT?m2 zxp~%SM0!Bv-wK#OUNHTb~L`U zfX4pIm_NtMmy)4Bah7eQWv;Xx;^oU}o1eP&y@Ci=G9jmuZ6v{0IfAdw6FlTRSRbOG}2v<61Y*b#@4@#t>X@53a+(4SB1xRe+1c zjd}Vw#wdMm%81GSpF6&|*_M1GO1{M&+>C=;NzU%-{cYs!+u1ED>{ixuKYNGEU3|g( zf(feZ-$^&lU5*Z$^EA%OXsZ@dGwM|EyJ?9lH}Z_!^BtE_VY%Babig)$Gn#*oJs7~j zy|kUH&}3b1RnXmR$U)gc0#Ks;*0A;E$w7OoEx6TIdoYNDZ5eT2@u@q+_5pE!_Hze+ z-d{E#c4mnFz&+x@j5w~H2M_V!dNp{M2N$ZrBRn`n4Iafo5!P5WdMs!3c+TjFoY9jw z%81o!*;70?T@9Y*!7Xa=OCH>#2G8(dD-KX|rR-8QP|Dt*21?nB)IceFs2aGGMF}ot zapY1KM=oV?q?C;*AxhbAtASGXRy9z{-m3;m*=_Ax#j{JN?_5xHS%#jlFZCADHl$ra++k|)H=C0~fwN+*doOAirmmmVqJDXkapmYys=C_P<#RGJYV zmwq6=^d1JK-UyU?F9x4?BUE~?hT-13V1)N!7%7Wjf;=3i$~7=u4#GV7An%iv=?Br_BPzD zeFz!tQyB1-z&*YyxYsuYw)kpc(6<1#`Wj)IZw1`v3&VC_0(SV$fStY#@PO}fc+htv zJmky3!@dXM5#Mw0sP8R!%=ZC2?)wCus4RjfD~H2Vl@sBa%364~vL10GyimCkUaU;O zOO>1Pdkeh6KS@*zPi3tb3AHds6jiPi%heam#VC#!C0|wU?}d2)pNW0G5|Tf^6sLS9 OO7KUI1Chl82mK$HaX{Sw diff --git a/out/production/clients/org/apache/kafka/common/protocol/SecurityProtocol.class b/out/production/clients/org/apache/kafka/common/protocol/SecurityProtocol.class deleted file mode 100644 index 4ed34d00365a64fed25da24e9b4159c2a0e97a46..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2453 zcmbVN+fv(B6kW%%k!|Dv;!-dqX(^$$DRD_iLW@Ij9h0~*Hbv$Zno=Q*5eSPBc4l}= zANmLRgTC~&g`_~4X7bz*=zIU7Pf7QY1&B*JQ`nJ^yk~(UjrD15r7vH zK|I7G5it>&Aj0^XKOXa)EPqV$M=ppK>iE4mS%RJX`0py;KtTB%w^hMBXwiJEAruoOR#6Ds+BdJ(qvm##8R=6vy0yz$wV*4VemT^6iJRa7WSk$z3#uD zd{4K1tEQEqhEJSH!JfeSD%IkqYsVqB3C(ayB}AfB1fPAl7(z-#)i&sKI2BF1oYUp% zW}=Q}@fi*(X4TS)6FQ%V_VWk$CO?=GqWr>%Nm#=73`76>8juhNMY)A#2{+I$;S)q8 zT*Gy~p@~?LP{gW)UVO~0ZcFG!56!E6E1Vv530Jwg<@B<6q;9WKNx0KEL+j~A(z4VY z-e3s>_*6m(Ya()2qpavGOcj%utKW|`7&J01laGrzoE)c9?1$KCLhp>A)Fx{9yV z4!V2xcvSN1aQiuC8Q#;jh~hS{-D1?@Q~s0`gFutZ(C?1oaB_Q}&=j_+WIl3gc{UJ@5w-yW^(%OTMNT$Y~$bIy6 zq9LHY7Cr|+bC}N;$sz*0TVy#%QZK#vFSrZ#8$53y{6Z4LO?o!id!1w-$7hcHFkx|z zpzMtu!1o4TZtpo`KSUa-W>v$ceVR9Va>S<);G=hIY@lZ!4Y8{)UO|j?d0)Z*>q#h? z(gp=>AZUreWem|Dl&rvA=|jy+z*jeL$I8lj{s{xIo?AKMH~11F)o1V!odV`4MN;?= z(dEJ1j~@2(yT5pg1iRZY1Zf>cGqs`@oz#YF#P>QfBrVb;8tB6gt)J5r9#6y~*(Awc cz*poOg6A!!MF@=jLj&n?s+Yn8+Vmj(FQFYqzyJUM diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/ArrayOf.class b/out/production/clients/org/apache/kafka/common/protocol/types/ArrayOf.class deleted file mode 100644 index 5deeb90da021086c0a2afc305a39c2f477bda2a2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2464 zcmb7GZBrXn7(JV8vLP-l0s#u4Vp~gI2%!~Xl@<-y0xd5J2qIRygoSP!vYD_{@pFHO zQ$P5{4|PVvj2+SOqrb#ae?ez#J+R8e4yb&4GR%0M$v|) zC^0|Mu&iN4f$Hp)O$C`5yRwxv%0}U?nZ0jp-Z!!Zdwbh1Wy=-Yu?u!FD}g)NC61#C z!dI=5<%}s9O8l?PnCCko^?>Xl9?&hXh8TA&6MrkWMzkbgwIHN*N zvG_a92G`vv1+j+746=o=3tW%E_U!e0B;2;v@a2y66)80@I!59CtWiE}vSJz=3X+YP zCL1Az1!;bh1pTCX7D;xHO(ajMnZKzK$6jSsg>Tq@eTYrKiF8 zu2tMHEA&TxVIjYo(D4~Qm*EckOX_$N!#YO5gAqSUeYsHuxf7nZMIOj)qoxq~&e=|% zQR?%^uH7D#_SoYHata1di0aFuAbr9FafpIoLXJqA+u8BcL_x~C&UDF9(3P0>y>mQ7 zZjCL|iy@GlF{4~IOZ0uFev^H(z3e|{tH7*f7{{-;z9x|PpD zMVbOiXpzR}oo!sFM0$ zna2$ExOcm~rk>*oiUg2AHCnbKzNuUjj&)2?Rbj> za+K_fE3`+rr$#tcBb;*O<(0{Jkyi-!xzvM4@;3YOc9ZHET=ofz%(6ov;U?mFmJ7R! zSKYe07vNV@=&1#Sikj{btkFG6k0INqc+8( za+49asNhUZQTY!=ED&U54;2^nE>OQKc$+iw@Hxut9h#Uof^>}FKZtq`=y=Y>fQC^G lS3y@?Yst&C?DsQ76u}9nXyA?))#$2IBV1GCUB(CS-oL`Z0SN#A diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Field.class b/out/production/clients/org/apache/kafka/common/protocol/types/Field.class deleted file mode 100644 index c9935f4538b93d0eece48454f4876c34c2275030..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2201 zcmcgs-Ez}b5dMxWTed8dkpAI@Qd%H5vK{dgeoE2+Nl3{wz(vw9TrnrI5=3Mx<0uTg z2yf7PXmLFQGkt)*1usI`U1>Zaoj%RjP$9wkM@7tCBdH4En0Ml5uFoKI^jAEM2 zB{rAYTw!yS%};EuS(w4BMbXb}=Ga^>;RgGgCgx2nC|GyzJzTuCbn~~BHHER2$NGtO zL*3nS?`=Hxy?92U5OmxAQw6oGV03lI|1fVg?giZ~$}{3;JM>>`CF*UtdPjSYe0N)K zZfnYG(e->g@)U}@bi1P!+^p4BO>UU*1P2Z8K{*Y3`8gAL)~|VA|j0ItjH9wTbD-O3?N1?sYc&-kRPBMb^mE z;kxbx{G1*d@uOf@VJa)oQs9Se(xJK zq?->aB=4I-BVz?-O&^bp*~#0aPBnnw-|D1C_<_EDBlI0LNgYy! zqEXh4{85%qJSixi^b=2Fk{@L$$(v9tA=&Tg=`C_}0$g$iP|kQCIj7l2-f8t=I5ne> zf>SH>QFLm>KFnI-50rj~)v6W#Lizqnx|qjFvJNb=6EJWVWz=y5lhl?&O*ROoi8E-? zZ;C$uE>Qq(C!qKdr|5~`g#-7GSVhV;tOsXIe(<$OK#mAxkrP8K}Gs?t- ziL+wsCUpU!>!fr$gj=Musx#hf)r?v}4PY^ioY1fY_yVoeMQL*`NglRY9ky8=wprzk zl{Txqw$f&mcbD2^r5h=}DP1djEo z2%I5s%@poFRba&`4x!Q12CxspT_d;|fpe17OmNJpm4f_A_qc&Ffs9oLIC2nT4kcU{ zh?7a;5TZO}R8AQw@>6MHmprpAIIm#5pdgt~Q4Z&&pq(V71xvJ8=jeoY`jkwOsK3F; X16u1pQR2Roj5sNAjGoR**$eLhc`nxk diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Schema.class b/out/production/clients/org/apache/kafka/common/protocol/types/Schema.class deleted file mode 100644 index 32594be6aedcf51eca7ebaf5bc6381349db73249..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5250 zcmb7HYj+dZ72TuP$g+%$g9!$cibuefYy@t?!v>Prm||i~z<^6kL&mafgk?#QMg$0H zF(FBlGzsKEN|HW8AMS&;(1Iy(d3@_y{RLh9Ct;Pi``j5xW8v$~y4^GO%nF_e@dL@2{AL%%&V?2NWPWo|9zTc3` z`2Z^LrkK4Ym$&8e;{Y%3$f|dByeAJo37`(|%Qhd#$FFt#MuTrS5l;@~ zG(-n_(%F%ykuhS&;?YrKc+`l-(qm)kR5X)Ko9S3O88uI3;<@NfS<$W`Xf5qH)oYB! zHB|Q;H%=JQyqQQw_ZS%(RP-fAQihq&a(3U$*{#-N(nyU&`^;=2HPSx6?YGmvZ)+lz zFt=-HZJcNDO@}miI@3cc>_jTwn;#pDXAc;INsiR^q+>?%kdaNuvpwil$!b_Pzc4ju z34Xe{op~%FFqgZ&k7XD$n5Ao&%c}ZJBQ{D{RXD168%RJ6-dD2;lR<|YEksj^bhP7? z8Slss5683Z4w&~29*@V&c1hiP!bs*-rmAGB3idRE5TSvzPQOm}SAq`D)XV>HivlzdgR*4d{#|9pq z*D=5pa*5ONz09S~_UlBTNs1)~BycJ}rr1tr?naq#kHk$48ydTt%B^FeTRgdBnoB#J zJKD^jzps?)xLQYgwkwIPX`aoX!EdIm^%@p6IzcUZ)dn@xKeAm%K9L+E%jkiON(>po z#7<|11=5v^7B>_sXxLo#3e_i0lKOJWiH0W}0e2>iT&~l|nNCX&TW3rO^;a6!%o+C_ z4HD=}=d-c+&V-0XrPZhF#4d=9_??F4`RHg6>#;$@!jg9GGIGb*=s~pNyFm&j-+*^sCm4+|@@&{{U+T8Z!P654Hitz#;Pf8e-=nvxfbgF$>BhuQI#ACVP@AIafP z5I?|65)LD|io0^TMrnun}&?NVd6qxJ1g=H+;f z<)u2?qE@a$`y`ix*PY4p4A|_1tl;bs_f8OPBc7?0=XsEU0P(;nbAfeI&ADxW$KIZJ zE@zCeM1DRfs*`FnP75{2Evrkp*mgRXSZ|44Wrt(KHpdw*NH#QfIwQ4eTw>V_R@&(O ze|xOpD{++X_Ab_GR>GRAQVOv56y_0WHKAYY%5GhHpU&NABK%>n7ac zDR>Uwg7+qT?rja>8*tyff=r~LwU_t4mK)G-!|%dnte8T@;VA@YTv;2OM3v0(YZ41? zq57VzY?y-UaD$BVQ^U_fermbhV}0U`PC2CvTW@2L8~x{9+Qjtx*BO)>TX(X!9m`cXvDi5y?`*PwrdyrnkMca%eX~?cA&H#UgjhvU&ew*SniEXVGFPA#Og)fo#y;mq9Q2m zho3f;*oAKRXz>ho<9XgK!%_6m#>@GW3OS+3coDgA>_IOhkg>g7>!Rfuynua-MbON` z_Yi&^`?YD3(5FD{({VsXh$?tFIL$cp*)tr6+mCks>o_zmXwPNhDVpf${{|HgxW6W+ zqFtsQK6@F*8C9G+2<>riUv3x5XjXU2$-0`zBv$lIU{{;RS(KFoTa3k|R2Yw47=9ND zBSVhDxLKGlr%-n|+F(>5T6Qka;A%OyUEd=Xd=COh+>lCS>8!y zo?`q>e3p86SfLSB1-Il>til<)3Kht)vIDGwVE=+(U#a7WHlR>`8r}kgM+A4lw~7BY zzOyEYtUXC&9~6+~+k}NIuGz?La&L>wZh_TGDA)ti=5{-%CRqLQHpSY1H(X_a7G0N; zW#jN-SY%l%~3nIKSY!&Z`)Vwx{HT@;s-yRXaN~X#7{zy%m%-e(1x=F0< zx6TO9&=mwCHO(@wRQz6~wo$F&c_4u~rRprjWgOKwM+tj_@_nB5f0Oloi*ocfYkZDu zcn3!)>(US*u3d;7iVso*!%oG|rXOYNSTbG(_d_fswY6=QCh#TI%2Vqijf8Te!dr(P z?33h{x3+oeJ>+e3J)2r|yMF>3+PqF#XksE_Q<4Nqe0Wy>Uv0j&M;6sm0;=z^&P&+h zVb%CYNyNwu1TL_dE}{k>(drZ0eMZYq(Tq!!vWslF%Utm}GyOs#u#0`t%)i}8bKMd= zOG0iHg=EJ_m$*4PfDA?0#Wexiy`s1{TI9|s<)zz3Y9qW4MO;OqSK1Qc)aTp-v=G8+ rn!A*K91ztuYo3$eHx{JqI;$akVG3!>Y~n3;5YJa6-U*h#g;)Ovo{0Tj diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/SchemaException.class b/out/production/clients/org/apache/kafka/common/protocol/types/SchemaException.class deleted file mode 100644 index 2a325b15ccbc39126624a27272311233b6fb0986..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 498 zcmbVIO-sW-6r5Mv24ib$Z7T>0Uc?V@FCs#vm#Ux!FV)_+aZO{B-H>cWf0ie~gFnC@ zCB8)Qpm=c)@9hjT`)1$U`^zhUeXO{cL)$^eL6^|XR2s@?q|z)j`ugmcK<9+|!060M zZAUVisR|qOLxk#4s6*Q)bhf-Zc`rpIbs&Z|4Rz4l9udj|bE60iFVyOC7LQeWCC3pn zEzkHoNb)$wUu4-%!i;d}nKTeGk^V%9sl1&^;hQ)%S|q8l#y63$kBQ2}kOi?kdGJ+Y zId6~P#45{VPz?ALh~N&*EcMlCm`7;;uGgsnX-PwHxJr$Z7SGQo-*tw26SqzY*#;~Z_Bv#`9157f^kYUK8)wUX~ z)w)zo+oUzN(U?}FMjIhD8l!Es+SaaiN$slLt=i>}nqR8@?t6D;9`Mki_7C2?ci+9= zIp;g)o^$Tw#dn@K31B|{<$(kH+?as<^745PO7R6J?snq_>Q#kyYljqMEafw-^UN+;}7NCj|4LN+Pi~20fFkqXsoL` z&>P@$^|nCgwm@}zw5KN;sqT$MXx>x!S=X(y)+aF#g_}LDjy@X1zg2O0eeHVBPcMgF%$_l_xH2~W9tKLVUFZC zM%x47je%H5KC3}{ygNkQ(l;v}>(}+J8LPTdx4GpNV>D7AuPYc|9SVj!T4F1Moq_&v zT%bN};rhxY6;{kUIwh(=Rk|t}SON~+qmobye`c?X2imta1$uRYxlaUav_N21xoP^k zcq|m@vewHU3H0dhN};RiPW)KF)g9>5H4!MVg4`_Nra&&CqXf2$nW}UHJP`>ER$8RX zsi7wKflKP#9gR`tx-qM4VoE{~ZlvzXx22MZ44)Jj$6s?vB_5Gclx|b_Pr+dU)Tc6zg+sEi*Q>svvBK^df*q0kDwETbI zx&1F^pGzO`Glyra{B1VN(RMnWsY?fp}1*%eiB8f)wIPU{I-eRK-zId4&q{5i(uz zs1bPr=djqB5{{Aa1;keV(MFHX-A36QaDb+Z5dYc)ibt%f?+=AL7Fo8rKb#VV%Z zr`V(6XK2yzbF>I7Of&ZyUDc{x?ZMu7C>jx%kU}?jS6q?CrchrW^Nm-p*51x&%-bmo zzPC)nY5YRNdTbDwT;|=`9cu6PvZ{I`QE!LQVi=36G<+P_O0!Rq0-rbu>TUFcvYa;r-R_j8Mj^Hlo!qfPphHLOADQ%~Q zKjSYNc44=C@mJ}q*EQTMU;GVkX!tbV*QePC!$UsS<{#xn;}BRjcbZG_L2)4)pd0BWz`7TcrGvNhpy{hGhaHhL*^z z$NoAMY>Dx7bG)uO6Ng=TmPd>9d;axYz=cDsNaUvx3Z<+t_CbF<6t32zj;z@`L*cMM zdAUV1VnRxQ(`3t}$TaQHOOb%jx)51|m6We}FWQH6%n2RRrI+q7PGGJ;ILs4WoE#S= zk8&sjD|yX_o!4SsC-Mu$B;=q3t2p8{mSHtM#3usiVuXGt1!YhG6QUAx37<3g zt|%qOcMKT=us;Qdg}f=0Hj`*T2a%Pp z4IrC?IsC~V$IAe62a)$QQEWPS6OqaLx%`Y$L*nKkm*0yfVgXHCWFoOXX;eKL(L{Yb zSd3=0Q2z_L-i?g2$aUhP9KK=|RSGxowd6F*=O0AAyo^uT?9pGGKcZoTkd>F~w|efwVVer>|1UG^{m3rJ4G+Ae)+(<06xlRY}=ZvbCU0 zH)iTu7*@I<&Z69jjeN5SmnfYsR7U0{bb1a>`(fB^N0KBP?%ixMGcPGBPl=K=I*>7L zg$$5ZY*y&Y6uR3-s|rtHLW)2G^>P!cJBjK}lGln$t)Pl5pxQ=(GPn@dH}|5?Te(si0)~lUq|$aaD2(CBqfF#)PINLrY7B zRuElQXjIG!?b=b$61sIGWMw89n><{x8%$7EG2e}?R!~%I1k_EVKqa)=hOiYBwVE(V(0?F_Vz$dGnxtV#z5`afB1yHYy^5`}Z!9QG$R~adGRpQPHDCI*QEkQPXzS5OYqB@$S~YK+=Y zYxn0~szWSv6gZ0QCl6wVzt~>jsNvSyq>tD;4#MNl<*r9Jd*TfN+uk7sd%UCJO#5Qe>f0XqYFmu}tLf3(z=R zEOK#~$U~RN$96FuyZC&KD8Tii5I2emxKkA2fGEZxVmv4&;W1&HIF^&94XD6%*uzAy zmVt3SR?s=qN#zY}n{CXl`|%0Bm61A_B>fTGh)JX(&DbwH((~&D?kl>k^DQ1z}GBk;DT*BYWMFj%FhhEOvA!h4Qcs{dd6Zg}n z*q6I`3i0C>66fGaWTQ@;o$sSMakAF!QpDM~k8UNIJmNE>-lo$kDO`liciL#wT{D>U`#Py4q%BsTIN^A>;FC@_b~81>D;UQ6v_T{l%Cn&V^6ZGQ^i)kywg4u^f%!JVZqut`iNo zS**k@e0Qf6RZv(wPBsH9Y<5$d4PO4lfAB$H zf~8enUi~P`p3MjwqzZjV_snef=`*K0^Yho|uK;diQAZl4hA|y6+|Y3ox1_l(w|7z) z*Kk+EJss^ZrMaKN1L-HEnbeWOLmfSs((p*bV}|6U?b^W!@v|X`O zuatzp%1aKNq>G-#oi*;;a<2{(!IoWPFcw@__%jZ#)dX$LqUUd#yvnUDVeasa9d24) zrQ*3})%OC=@*FdGQx!FHmF&Qc35K>;z8z4Jp=@!R?{d?%J+ts85QX~2hVUnvhYoi) z&E?Xzu!0FmHCYP3GMqXxiGg{Nb_f{FrAzvPml<+LSIZrhlC0Tp#qtIfQ%O<8+yX-? z@Kyrfb~hPLW}^_zQ@UhompgWu%Xu@Z1=XOUNx>er9SRvIzr zD_-5V#GEY~*e=x`m1MfcKKt4d)xh>#hSLXin(7qlwo?|qfdxDxb{V^talIhJGm^%r zJbrHA8m>#8MFWG#GK~CV_2!T<ULs`Y@OKU+{O zZ~aa^jBg4G(5UzIaDICCL&FJDolf#hBggc%MK6MHiC3fc_w*@!DZXWV|HQ9h zjGlxvJuiSpKN<#5D6LHLjC|YB9@s}(#L!1J86J+~G_8$h0Qzy3b`0lmo~%q>rsz2N zj{NX<#2Sb z{6Om{xv3DduNkwCnNpYxS23WF#uZX5@%|9TOavpgj}Z&mNL^?m6~+j8sZ%_y9VRj# zQ7NuKm>gl1>L3x49L)bf>>b^QwEFM`sr4^tU*CgH_dAg`8t7=Ca}QnLWRx4bWc;|ysA7#b)|&KA*#`Rn0K_hNd*|kFvZ6(f>H8` PaAeNVN+-k~^3;ysWY%fN diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Type$2.class b/out/production/clients/org/apache/kafka/common/protocol/types/Type$2.class deleted file mode 100644 index e7b6e580052331bdd354250c1be9a19495e330ed..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1720 zcmb7E+foxt6kVN!Oc)2jfY);|-XI~u;0=$$@e)KugFJ)+sq&?nbYNsMotjL%{fYnJ zgT4goSamM1ew5|zP74E4rRRa(JnlKY#h_Zva=ZXd@5T!jz2^F4?$@D{8r_ zKCk64ZQ;6wKWy~FRm+VWZmKw=mRTD){AuF|Zdtf(;SNJ~)(`#o7DKvNT4Bh{$+}<| ztoosN(rK=XXqm4Ebds-1j|VF}^3}IF%*301o55KKLlMmdyxkV`=~iX5;qn&uHif&z z8(ZATzK(}qMBXT|7AFOU=ah8inaz!*3u;-!RrjAgS(XuD#^C}H)5$l%}h}e zv9!RDi)Ag2{BVO|q?m;0ozf+9FL>bBxw_|tgFq5(Ynw8PsU$dCSN? z`h#*K5`$fnoyZgSeKo;;CHJI~gXTLQdVy>c-9-^^%DRJLjL`9}Sek-^yQo;0b8rvy z4(_Ak-~k@ez!sh?Uz%o^{>e$#$O=t9|H>1s*q0&0@kA#rTczU%brCsOz$1pCg5NHL zGA{6fCNincA3L~!QAM@tU<^fuiJxrWGa5trf8>*TZ8e4!9&|`O`gznChBfWI_N2+S zI4edF6;A83=pF@>xJ^2L9V>LT4mW_1@tJz&5p0X)?}g6zK?ufP7fGe3TQ`U2n*=5(ZCY8cfK!v!4|F(#Kw z@_0FgaSc~AT-DJHQ!dw1xGwF4Tqbp-a6?BQZfdxt;Wk5Z(su3OCPO@zUt&m1dlkX3 zuWY;GVY9X@{6)U(&`rARS=?FTzAew{G7+rW4F;p+y277!c%vccXqG*{YVta_R)x9F zSJt^{d9|A7nswg`Jj-*;;8|TX%tczyj!iIhJ@M^;k__g`YkY&7uI-t{XMrd-S5|~S z(Y|!JTQ%pG*Mt>JNUF)@@GHZCU4s~yCuxU((O4?d7rerd-#uG?w~%DRelF%$D47b1 zBIZjBslZzZeA}%u?9W9Z+P73>YJ)p=h08j#yMaW~mfS#8sc@o7y3`nY76NXq&+)nv z{`_A6luQizg4gsdF>A{nc1yuWq#)Gc8P{^W1`(bU!Kzm=(2xBLecN&=5(cJF)G%$} z4rUC@qG;eQ?orc94;RNSGK~LYw-;y}YJBFYCF+6gxeN!lbG1b(Hf^UOd;=xiXXww^ zjg0FB8SZ8jml65tfq`>4FZq@YWRYVS{znGwd1EO2hd=erc7rAEG)X}Ek5H@oL*AV; zO5<2>21JwLP_Cr<-|0f#OjbqPN26J2hVwMAYzre6vXQ#b zWGakdvQnpbTH8tFWkjX80-;9%ITh+45t7Unz9aUQ>LIP(e@5!@XLLW_giiJCNgFNn zw6J#*`@YI7VH5U-dcC5Ky(SIcM7U;*qqnObrl_AHy(4$R@&(IZQj95c|kdJAMH!xpTe% diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Type$4.class b/out/production/clients/org/apache/kafka/common/protocol/types/Type$4.class deleted file mode 100644 index 9420cc087225383150804df8eb0b8cd493019d3e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1714 zcmb7E%Tg0T6g{1U%rFj!!S^#NN( zsO=dYqZ-a?IA@>}mKx6MxS-}SHH;h3anZmwT+(n^!xe_~xZ^p&C5B`!KhKbwkY&NJ zwd8o>PNTXY{5ihh(n+QyZSKx<-%y$33Kj5xY=IWk(n~}uQmZVpqh!mMpW$2m>xV=2hYmw{6 z{}LT}#9++IhHs1Ojw)cMa(hU*LFJwFY**Gv?X(D%WZ6V7wlj3Ed!-bZxQe2N2@}^a zY2rGHCT`#+m22wG+~^3y=s(tZmKveTCm-3O7C6#l*tMRhWlyo;xMks+n8GcRmv!n{ zPX<|@RXT>$@of{wa9jy3ndnE3VelWbx4Mj>@E>@x%caco+-;C`^w%I&y`kv#)@0CX zjX;TH*qyr-)x8}=P^ugLx$cNK%*~p1qezji)Kc1$G{5MM_MGXlg4y-_!PTnZMFc|jlHyE z*oXZzs^A$U=>v2XPJBb6iDVP0yXz$CK6ovyh7?OA45VC}3IE*6^ z`8e%Wyk2Oc;}g;!XdM^#BIMlJ!mM%X5hue@^hHc(X;w_h9{v3Wl2GV#%!Xi ziS89_{i?ErRp<^4dqX9AOD4XHY3*Or+K)V0M^9`R#mt_ol(`UP9;UqN=IH^-(X&x^ eyu9g%Fo_d{Phb#3G^fIz*+VOX7`M?(_4ox;o^Z+l diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Type$5.class b/out/production/clients/org/apache/kafka/common/protocol/types/Type$5.class deleted file mode 100644 index 710bf74ac9e7b3b6eb029cf8aba49eb0e39c780b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2113 zcmb7FTXWk)6#mwZEk#wD#7)ySEnw&+c3el{($eBKP=~Yx90%ewA%sh0Wn-(6B_qqF z<%J*7C*FDNFfA9S)0A*|VTOU>4>7=5NpTfOhT<8YBki93&UbFBU;cc!4d5jh3Xb5S zgjW<4a7n^t36nBjRgl1I3a+3i#FP-P3o$Lst_pEY#v2l56!hbw5O2zOOX!jivkHb$ zmM|ybZHD9}(>A?HhIl5s$j~$8tZ|0^l4Rj8<*Ee*{a2gHA)|#&4Ifi3t-t8uDX$vGzohvXT zK5SCW;rVQby#Px@E5EA}MF~0*P?wp6Q_+HvZc_Od*3(aoVoI zubQI1y`mW7B3V@ZY1^=z7V(?qUfo$!aTLcG1|Fv}@G0RP74xXjiO5#naXp6fPuK$$ zYLbRNeaqlY&va~ti2yrow$hek*SMSZ>bgyWryKe$v(av(TYm1+LCVJky9Ei?RV-pj z#k*Km@g7!Gyf2>S@qvmD@ex&}vM@hWzQ%AQLe@trwoPk|yDC;e(GLpOw&SJsbU=kX ztfH!70?(^>0WUJ-pD?(f!5MP@!=B;fe%5Wzv|8GAA*o5*-ueZG;b?aRd<$98*SmUe zXM@J(kRSc-38Q{%w!KPFxucm%*pj=As4L@S#c(XMQryo%uJIilx=^Sy;UCYmzQID=8b%Mg+b_6J5JvZD(ZxsYI;>;%D~ zTnEbK6OZJw;BjCn{{ZSeS`UIv`8E&f1GI%+Jfg43cM%RlT@FJbhXh%iMV?Og;y}oO zVcgH9NXsPc#7HZNoS&Dz)w^9^L1KUM@rMJeF23SNCh9>$8AgyNE$S0b9ehkU0Qosm zop`r8G|J&oi^38#Msq(R_Bmuh=ANj`7W$VrF+e#POr<(F)WMJ_mGI)%<{)X{SL&4C zsQAA}xQ=$Yj?$O>G&vHvkAtZLSLhNs7kZuu6ObGuABX?RJK9yr5HODO1Qf%|xIntc QZ`G&iNg>2x(p2ET0jLSz>Hq)$ diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Type$6.class b/out/production/clients/org/apache/kafka/common/protocol/types/Type$6.class deleted file mode 100644 index c8c97b2387ebc24c6f0a45959d2e4de1f0edc5db..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2015 zcmb7FT~ixX7=BK^NtYIZK&{Y5tR){LTNSZNTWydk8iJY#;>S#vWPvT4-OOfFt-rv( z;D7L<-h`QUM(eG&&iHxZ*NxtIqvNIHb9R>iC2D1c_nbZFJ6K1%Lw9Q0hRK!TvR&8ARn0Nvcd!_7 zw~e-dvSeAhvtVlNw$4wrY&%=3*3#U8Z8^4U*KJdEZ?^QdT4Ovr zH!l!<*D+j@w4Xtmr`RJ-mXBZ?;!zU zOW7Xa!Luw?bG8^;HpPl17VYR-Lm=CqJ=+f=(l*|w;P4I2q!RhDvPmTY!a=$$2@uBa6p1;Z)ND#qj}M*6fz!7?feR%CbuuPS&ARk>DE@VZ=E z#hQXQP-B}*>$SzIz}$WMqe=rcwZ-@9ddoFzOJIDznf}d{I)>TM9R=%nQ{Z&kXs0dP zO>1eHSfq`L`!*`dthW>tp~`CBR&XAV2^8*AjBi$f{Qt;jUuiPK7taSqD-1X+iBC>? zA=c`+OO`8eGP4x8;vVNqmv)#i^RL!Ca%l80p8tx-x25}h(Y9&S+41o`Gaf;Rnnu$g zaZGEqbc>Ip@E?{s)DIf^P<~+Zqs|Zd$NWQ2XD@%1ElTlEp!|vRmmY+`gPf@x1;=P^ z7vf`1LO8`3^A^9rL!2vq0w!>pUjk>4W|W6NV0M(_)XZIkDuph>bCEoUNEgvHIf<4k#|HA;kknvZK zCGF?-SQM{NK6Ca;79S!iA6tMeIb_Ge!^ifE@cZ?f2j}-I-0K%#XK>VHIg)xh{~1Cb zA|ch?x&vkX4hGkEah&&aVtA;F2f7&E#mJ|U#k0e4PvP&>^9Swlr$;m9L-kZo<8ih} z;0f=PP=>D1bh82Tc_vHdSdJW{_v$|AvseHa#*@qsVGd7mjCkgq=IAhJ>(0$dX+i%3AY>z#-3@L-T}j0zHm(G4xOH0SZJEIXpIM5;dXe}qMfCtV{q%3yQcKR&A4}F zQjLj^uIW)sG2cArmt40^N3UObqCW2Tg&;Rw{WGj89pF;A8@ z!Hp?0WGzgvI7ODMr3r4zqL^RQ3Y7&^p`t_GyVS*J#i^e%=i+$NSOGE0=@ QWJ1n|OodE@3`$J>2KmMvw*UYD diff --git a/out/production/clients/org/apache/kafka/common/record/ByteBufferInputStream.class b/out/production/clients/org/apache/kafka/common/record/ByteBufferInputStream.class deleted file mode 100644 index 65d92dfc1e4af7120b7c8c6b1c589ff40775dce6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 951 zcmbVK%Wl&^6g^`*H8E~Vo3@0uX=#fgacImVVgu?TNR=X`T@X|xx|%qX#3Z&O*Fj># zl8@jUx^NtbLT$J+;h*J`T6Vn4*(4`4CJw$$2II2xQ$(D_N3W2 za0iz3?&`Rwqr$Mzo%Z`8Vpwax;cvMUc%jpL7mH?+eoBC6Uf{)t45iAIi0UbW_B`wf zhNZR_h?A+`712xH9aCnd9lCsc$|Fz4v!oUey>o^q?JycRe8SzKa7Mg8;*J~oei%5B zaKosV8KV_Urg0~Z1ovqojm{o}TB)`e7CSL_N5_0JYpz$$nyr?s#BqiBb61LUdXn^+`Upy-J53Y~`i70j-Ef%1tW1xsWL zgayjbuuVO7aD$@t6pdx9P(=5VuJkM@Vs+NDLl{Q2lKmOVCDf0|DHo})E#vq53#zFZ zhCS-BPcvBwgPGo)F{~w6CX+=XPlx_MBWYByvIONc$bG}Yp>%6oMxxrnCV46{Gi?z-LHQ3$k-1d> diff --git a/out/production/clients/org/apache/kafka/common/record/ByteBufferOutputStream.class b/out/production/clients/org/apache/kafka/common/record/ByteBufferOutputStream.class deleted file mode 100644 index 2a3ff69562a9e1c5db03f757769da8aca119999e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1474 zcmbVMT~iZR7=BLJ5E51p5JW?2HM9*uT>K#Ttt6T@8pw<=^uh~02^&~Rb~BreUw7Vk z(~dL!3;hM{OpA^e{s71RC8y6hyJ|YZO)v89dEa^WeLmi^dH?b4I{+83mcl3&k{H67 z6f7*Na5lyCoI0IX;X(pS#*JS;{F6cf%c^owg-a@2R^dtlR|O{S-Cn6yYo(QqN^R{y zd8M>byC)#Z0*6YT+YO`}Y|2*878tDenoZjmn5;gMkEP{0o>hDn*hO8w!iKAk>jc*Y zrgQst%xwx7C2xzfMyrl%ul3q>+uxA&7AxabuOVBT(sxuI6^&rW=?W}YJ%8Jh9og8i ztzFsNl~%)Rw>{VL?S|)X_0g#Hf=(}34}4p;Nh$uscLGi~autOx>iktGZ9R~U-BsC% za1y!u#ft8T*Qtn*vHL}V0k7F)Y}u~B(cIj=q6AF)X-B$Sp$&xUI?sui3ajfnF#@~Y zVQt;(`3<}5D8cD3-u06{qr5dSiEAb%a3q22CT`%Si4_%oK*mH7B^7Q7Ed4KE61Z)m zj2}(>go=qfxND-Srmy0rz!AM%?zW#5UC1c|C-&LAK+?C{lDl`ebsr5LYN%~F!7~=P z4|OGq2l|Tm%cB_#mLR6 zy$}7;W!Rs47`;(OR~aU@Em7t5>t#vnUz_O@tMlM zhRDzVjo6>Mf0}Uw1B@qN;9Dd)`53dQu$$A<%=n(3$}5Q(WTKfZzNcnp^M4_>hXLbv z9L|Sl!+7`_adl4rwuix=71#h2*f?Q;Sq$Sl=HKHKFUl9d>}S9%z9A(+O6pB9t&vYg z$S*iVkBN{oeaM+l$n!IqJtV$@&J%i`(5I2bf`-mVls;*lC8;ct6wO(ZP#R}R_XLUb z(U~Co&|Hg8)vg8q!9z?-9n{4|jbVljk-V7tI#^Mgc@=3XM9#e&Pd@r3{RXLPGjA}w zwh)%h@Np=7j3{CGh5u}EiW^t|w_{l1_6k_UGEW!#6c#?4TR={mJ4RJ=THzFx<#}&t KHQiAPPX7yCu^gNL diff --git a/out/production/clients/org/apache/kafka/common/record/CompressionType.class b/out/production/clients/org/apache/kafka/common/record/CompressionType.class deleted file mode 100644 index e2aa0a11d84fe3a8249bfbd8f3dd39669226cbf7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2310 zcmbVNZC4X#5WSnXUBV)eDkv3+h>}DlQn6a4%DYXCKpUhDwxvs0;>u<>T@qXU+W*jV z`WN~Mu|Q8bZT;LI)zdqhwFIOeFr1wynP*<^%-mW2{^$2U0gRy9f(WMLn8AG&Srwne z(T|*nPvz!;h`fl0B4$O*iI^AhnTpR<6yj*df`&(OE{ga<#FufLKrxObJl61)h6@_L zj??KI4Hq>mNA5AdhD#d0)i9{RP_d#QGCMart>9YT^~zbJYLuRs*>z)W-N=^QO2u`u zo>_9e)$F8Ose0zdhUGd7&#UGgdfs2k%`1o%X2<8}iweT|r4a=oYgK`gPnPJz>4F%Uz>349COuDPY z(Vn*)bGBYtF}($2#imB}GMV6M45}*o%f|NrE6tY}9hB+Fb>iBa-BD zZzrBfFS2>HCl)JiDOhF4sMS5f?N0rladge~a-7KZRPG&6w-steY5k#5ebW|3oAt-4 zjh5^Hd^jbjCYsPrJN3%waeH)^yoeXvx>qu1ESbT{LzEj5BIy{CrOE0jfn^OHkF`4Y z&(v`fYdTU$>&W1;jw`sT;{pbBToiFh#2^L~^zTzQXWM4ku*bb}y<$4G=^sjFwZ>gw z@Ar)w{GO;=_NwVgnsEiakDPVK{oYA(o!%@^vc?TcTC2B{I?8w=!qV{+>*8apsOWIu zD(E@NSJvuwQufPr{911Dd-;V`Y0a%X zHA^*i|3)(lCv9V6<4%)Vu)NKl5|w^k&3=JdPB;-G8q?vJt{!309Oyvf81byIbszoV zYpv!sREJ9mk}EiMR6-)^I2@E9&F|`l&4oJP6Er2gG&amPXurHU$Ib{z0G{NDf08Qx zNw(0UQ#j?Hq?Lb?SwKgRaF)MgXu&lML%|39)z4Et@hvDj2)#yllfSsBlrjIVi}oY9 z5!jC)|+e`i~L@9e|eGaYYV`WKn<2)pOhW%g+eB{?G-=ylWualgOTd6#X+th|| zhjonN^GvvFK~cxWi%=NWw_o~*!CeGe4{xJf)sA z0mfxw_?7M1LD$bv13=9RA6>J;huW+-h`XD(87*-b_a&wWD`?}uI|<_)(f0FYkmc~M zv(;m4d2r UiGyspPZos&225a*x)7%R1y6nlasU7T diff --git a/out/production/clients/org/apache/kafka/common/record/Compressor$1.class b/out/production/clients/org/apache/kafka/common/record/Compressor$1.class deleted file mode 100644 index a684492291092e00604589e93ff07a0c320e95e2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 884 zcmb7CZEq4m5PlX84)7{itF85|+S7tn&X;OpG$sv#A;LjIQ)BySIaauG+>$-0@$dM> z&uU_fG5rDkQO4OrC4Os@o0n&HW}ls#IsWzI2*4Jc99D2Ci!4^Nf(MX225}s5&yCO$pC8Efe;m(J# zU`XuO88Xkkzz_Y&3x;H|w98;rWRDK>4L=m_bkr4Uhj#;tEHjPKI1ZPL(HWzAnoDWtFOYzz;b^s-YLi$PfEXF&W673ERXi3%9Xh;S4MbMU*Vu z!6w71E^z`L_8nJtrrx0Di=bCkN-7I?b=p0x$|x~xQv1%VeJ5_;nYHiC+IQv;u^%v4 z^)M8w67VPz5kp~4)ao7xZ^BSI@qG;C6XyQ=_31+%Of~VNVrjo|y6R^%Be&&N8SbCH z(xgrBLmtG_rturE_UdhhbjRIpx8G8)X0OvCq_Ba9cue*Q zw(*i|5(VN*$E?RVM}C52k>nyWZ%mv=^M6GZOL5i($^bDelfFrkqdT%>QONuTFf!KH diff --git a/out/production/clients/org/apache/kafka/common/record/Compressor.class b/out/production/clients/org/apache/kafka/common/record/Compressor.class deleted file mode 100644 index b194619ead852a217ea678501b73085f4a10dea7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8487 zcmb_i3s{ubnSRgAF#j<8Fo1%h7$BlSK=2lCs1Xo#l#2n>MZ48;7;t2mVTVCuYP;Je zF-_~HY13`9ZPPTSiEY!gY7}TpV%yzqcC*>8y(isXv%PG0x7+Prdb1n*e&_$^0-~5a zo9E&D=lXr``R-@Ti?2QNJb)EyqZjLNCL8%U>&4SJC%yC1j7l?>jd6U=z%vFe$nvuW zp7SCH&&%RPFPqOx?+XULD1*O}wZHaa8UDt=mkhk%MG^j14*s1qFM5%Uzn9)i()$N5 zzKnmA^OuCpS6sLC_4T>dzx2{2q3~4?z9z@NE`x8#p>Im_EooksxmTq5wh;O!X}%)> zzAMf5WcYp8ZO8liveAVfcuD++240o@|7_q#UaY{6J@|^(n(cwfOn%EVHjD!?!zUov}c87-JuaEIQEjKD{*v)td_LK;{El3p+N9p zsD2>OHxQ@~#s&vt(fW8O7>oDTH^&Bt;-TT;a4g!*6Ah#m7#a#id%F_xP+(ADPV1q- z;Xr*jR=+8b2y~1jhDH+Bn(RF=($^P?C-<(Jj$6~wM5sw0nMBA4N5hF7F^Y7?k$55xpmw|~(t3avTTOQHE% z!_iRt$l!rcygP6p!kNO>STGRT6^MsrY)`rp2g5{CbEB+caZ1c>Nlc+=O4th72O_cH zKv(!qvd!&E1cC!?fg#P2erl*JL%9{U{{LKB?mVk%(KV3xk_w=0bOsW{%PIccBLS9W8eRLqiE3;bomeqS5TbV)oXJoWXdI(M7?a8MVfu{S0reaO~x@JJ#1X93nku zBorkw91BpaDd|*T~7dU%0xSXnO>?OhF zT3)!@GI`!$SnItgG@J+zQsljIvEkG*J1G{28>p=;QUc-Q*VDAa??lbu=je!X8V^NK{F{SIeS?01?tfS`$w z;9(Q{5l~n(4XyOu<$Tyg3`b1diMKEzCG%fKY`e~tw@Z>uSuLq4)5JSeu8Dh8o~e8) zUwQ?mDpWHJHPci@s@TMv@W-Z_Md_;9xJzNRzrLe9l#Hb1M-GOf<&qb}(f;y8tUPfr zRBmPIa?6=&%OxgrRrr;dY7UN@YOX3Z)jX zO*LOtnreZ{X1q;adUDbi1`8E#((5CMaHRgu@KF7hJ^me&l%Q@=sHD{E?g%k!M(T#6 zM0~WaOMfQpRaI&c%}-NIM+$|RNer304OMNb8a!gETD6$|lRnjP;7}--FjbwZH`EeS zEmg~s%9|KTC|<#36-9NrD5_(sV^LIRh2uJ{!`3}3nj{rxa%b(9X6TPZLj8eAW4wQ4 zFceLsbk3Zb-bB}?TAD_}k={_8)zr@DKs0tFT26+^x~p7DwYBA@TCP?|bE~OVs#PNB z)k#uU#|~LuqcBf%nxKaa=U142Ee%ZAl{WEo#8FuFdZd&=&s1wwgQ3=$YQ5TEs@uSv zTtiwnQijIjl}nRan^@5))ZHKrJ2xvV-8)^#OuBtCohx`)ve3_BtPYrM{Y=U29qlbN zMph$MV$ra44a=aMtV^_obsvn!j!1#nAZ5sG)-@R?QkAbhHGx>>QkJ+mqjCGx1YtiDNw5GBqnc8gVu!=^JEf&->YMD04u|`rp-U_IulS|l}JjvgkLI4WHuUA+vB?Q`vPSoCqG|+dzVBr4`I7 zbug3~kVFG$BlYAVz(5z!@&SB1R(P(nYG<0HXDjPuK@_GT#CDjqXw#zTD(#(O?s z`L?zWk%M8^;0nblht&{le%xY(Og%XT4cbdIbi}XU>p@=amI*Md9Mh1A!=7SYYh`4G z#nVV*LP{W@3HztuFnPZx5woTwDrG{kW=XMVDg-(tSt6x6Gvc+@v|9^~Zo8ab^ZLVP zURtwlfy6-uPga_I)Jf2hAR6am1{M5$XB~gI@?s8lVi$k8*v*jxWgP!e#<-s29_&f2 z@8y^U689ELIjX;`>hza&p8m2jNgY9)qf`Jr759 z%_yAb;i}F)k1T22wP#^Gj>4Mib8zLG`}6bmkHWjV`T}x#YQ|umM{YHja%<;}Bkw$X zXGqS80k+k=7O;j4*76su1{7dDim?H6u@N_;8B4K+Gh4BZzcoelehmeKh;l?wEIEe= zRC;nrhwl9eujNFX{TWbKNW^7e*g(R-h-{VH9ky@RO4j^A2;S? zYbCHY&Ai;&nPTouG56Zcl~lks$W6osc#blrM>ho)&-du*Btci<>zMDBw` zCf6ym&$sgH&VK)gkmdWVDBxZhzbp1xZn0!psMXXB56z^2hB1TYAu=nr$k;4Am}SEj z`FNmyj~NsVnX+!TSgVKNhQZaZl7(*+GfpA5^#Y1Sm&F%3=g{gNWlt{FDrnYg-HXi+RH29BxYYmK2s9n&mDmKXl;sGg{mRt!}0* zwRXx0bQ6P59#LZst!O46LO0Px7V_yt?hcfM>EDuwP575RQq3j-~ z($7+LU!ep=;#)~Ahh}pOZzaElc+!@54W7W;XqygF7{H%uD}4Zs_%n`Oq)j1jAu#h9k|QV5_wsUIhz;um5rj}6nyQXrox*Sk752r>#iOmN#3;> zS)_LtwSJtlcY|*#TEo2>vlzF@S-~t0G22EUy}OA|X@4@{*iI}t9Nt4BmZb_1eMErF z9YrO9v{^uEuLtDa1ad!Fo*?5tCy@7C1w`zBA`pphB9sgu_u`#4kmELx`2vJixqyEQ z`FG1Gs;0y=gU&HgI1kbsKET=c6VG2T1br}tX_hS)1=&VC#|8v9(F(qLAQo{#XvWF#z?rle9zecn&LGWQ;B@X~uA6_zQf zHcBd&TlkK)0Y1zdXY)u!P3__$S5ekDs#m!ekE3R#r@Gi(m(@>O5v%%gEpQD8uR@+AXey@Ep}T$ZST=HsL)QCMS4ZrDcz*%qaBE) z3e*qSP}}LGa?)hhU@UYxMzJ`J%E}vOsJyc&rEzZxjhRAY+Grf|n}SXID0vBbUv=5M z5?4u9k-NwkN8M!d=aGj|OK=*b0ZH}jx#!-)t~-q;PYdF6G>~T~vG!-6tJYj58=c1 zHT!u^FxJu>C7(RO3tRbw;ryPb*mvKT5yB4@)*m|Y6!%-X&f@w!xn88Reo=F^QfaNt zRe}UL6w{>mgO&c8+cTg$(k@P8i!ZH$dnI!bxa~nnI_Rl z)kbIvhqdr6llYQ0iH(r2tjjmTb@@KZR=$}g@pGEQFK80KRJm|cW#LKX;lE;e@q)_HCh;c9eG3-QY#yP)f*gGepQ4%u zwW?2Y?pSK|q15Wh)anbm_f{RJF3}{Oq&DQiA};~OWMvtG#o_T3HR+`c!FNb(KsVYD zJj^6^8H{feN*0OibTo`(M{CXV>zy0sm7c`xnt99Cy5^L)SY?jm_J+Hi>eQ8=&2jC2 z*8Md2#L3uNs<$$Z>CMt~-mbU*SR$wu%NFt$piCVMJ4JjEmGQfPb?0K#Vk6sje!Eex zB_>r`5uHPB=;CUg-E*Lm=}#VWaq6kV8Knk)b5=eh>j^v1a7oF{h-=adI-~;$5 z#M&SzpyG<%o6*kA+`hfPyaG7EmWw59l+i3>lTeSPRU#Zp8!MyldKUz`CRDpd$BEF% zP=u*0vM@VDD4i>~rz@Is*{-XO6`Pud4VLZNHMl2G%NmN#iMlJ-`NLS`Di z3D`)m9pf*ukW5ufIP;C2cw#Pssq|*zekQ!YM3K>+m4UHiucsFxRO5b*E*}FqPdILy z!?L>ZI|;!ZnA8UHN@d==f9sF4A5^dmmvH*81ccVl7W%19R3yLSVGj=Pn)ikl3Mg_f zF>+Am8)dd!SD5KA=Ws6_9zBDe@|g-(H)pq5xq~X>&Ce;UqQ;2lQO8>TR)dj)^$#0D BcRT<9 diff --git a/out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.class b/out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.class deleted file mode 100644 index 3123be34daa95c2d889b6cc357a85f01f8c7043e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5162 zcmbtY`+rmQ8GcTa^dyH<+6DxnR04>$0nLP>B9}o+kwQvYQ&31G`A%u+OiFG zUN?2h=2TGUoH*3MHq@3*Q0Cm`W$G?&(_P%fUEK9Q;4}8TCnrs)WuFi3hkWnf_dM_O z-oEW?|9S2O04?~piqC4e95-pW8MjDts}JYkHVwDqbJBcXrgvy;?v&A88aCn!GP+xu zd!*Sb%@?Klk~H^9^JNY9;eHhlsMzPlehpLcpzGYuxo!;);bAYnqM-wi$jPsY#78xJ z4Ufs41718XXTI*mK^0GEcoI)(_y!KSZXjUS^`}mqlB-;8jfOA!*}qUjE;%q<0_u_;suRbKj_5?72j1*v#Gnib9K+=?)Dw+T@eNT@Gj$8 zBa}4KeW8ezOQidj@labPowtm%wZ%vlOwKoU_3Vgj-n6N!yQh7Pf+?-xuC@(3)~$}L z+tJw(p}e+r3eH{A9%<|D*woY2J$_4%nbAxtn=|wIOisc4aN4v&yRw-=+S(mT?rsT% zx3xr!xVfg{7;$)`{bqC^Uq~rvJnhQhV84;?4_$rrI(CZ}@o05#As#m=yJ~x@g1XYz z%vh_luBVK_)?_9+5J~K&N=kA#qYa6Lp&Z)JM25Bl9H<1Z-q{q-LYu=*3-Ij@w zh1zf;ZEh^2dd*yq(VOIoKb(mg$t^}MA#-QZZS^O}$kMaYq4JX?A(2*a&V+cVP5WTf z%vuS09VetPwvIosrmIQNR`)wox0sz;(!@0p%ZLti8d)2o3Ucg|o3=H^`0B}YbB#Bs z+IgqBkmGe@Una-C-m-saWe;M&kuZ^M*SfX3O}XN3lx9 zE3)#cj@R%b9jno*V+VHX=)@)+uOqEvEw0qjhBZ3crTLglyQPVsN5{wU2^DYXNP#R( zu~U$EY1f!J6kK*Tm@8DK-xtF$V5VbcEWoUjGOR+*Ob&@QZ>spQj-TMC^mXT4A)PO= ztK(;QOUF8dW%P3uztGW(UkXs#aE@~!o-e zR6@S(7KK`cS?>hd0f6C-v zI)<=Q!OSwUZl>o%${dqM#>5o^>5LT^V~pmr{dD~5jFM65p_Z9D%|Oe~#%3hnr}UYY?N^<~U?NpW6{|=&3d^;EDch+bl@A0- zRP)YcgMyVv=0lsMRZuTkb8|YM=u4ZijU5S*75j49fK<^0ukloR?HYD$2Y{Jh_JlV!XHE2(jK==dNtl*LbZn%1)%;}X7^FXz*GGG<^Yt^gAX zXH}TS`SObSOwLzS%$qr1Suqd5FSU?&{Zaw#o(0eDS@Z0kRgis^6coE>eYAU)L%U~1 zw0jms_LGQ(^_J(a;_M0bRqVsT1xHa89EEEH?qPU1svdzl3~%r#e8J$e&_*z67`hDg zW_vNe2G(W|D|IbV%pk@_{+-c;^;m=rWFkzhI=Q=% zvo37q>1jng*oN(#5t-C+V!noBH~XpF=QDsU@hkHw+r^y&byJRe!@;_0LGmp83tVh# zM=(XYM^JMNQ@8r39mVt${&N?|T3vN)6f;UYGi4`eZ+K#hN6|2hS;eVarj5hUc*$Fo zN{Y^_^X{(4sXcpFEC_C;v~BL4&WA@ZdvDJyTN1C9Gc{560Vmws? z*GV+1FlU^^j?>9wjdsYRbqN7@hr0PkF=u(he%hMmuRqejCH(;A+q}*7Jdb%>T@9l+ zKeE*=J2D!@e9@}Gf5D#pP~8XNcCDDT50kh*dE-I&8~hjUS2s(D79-n4leFihJht7B-UGpZo^8I)P5AYfH5Z=bawt-fnng(>Ck7jXkSEC^l zbY1f77|qKE+%ATL!tdQp)AradJ>=XF^EE&_3BS^;tx`@x=WjAiIlYOGtK_7LNKoX- zSTrUDc{t*KAIM4lK6t7Q1xIiZEyDo5xV%QzDm-?vIPvm^L%dh?%(Nk%!UBXq7so+FnXR8*{XoMILVj$-$U-n_8HQOS4sgZ%pr z&z8koMzLs>t;H>~joYcm?8CfRG{6O<9rP`gfBN|7Uv+qy26&YQc#Q^l9T(yaEXJD^ zMp!b2e+fPNI(pR^jKCMVgSCad!p}%jlfbNv36WKHZmPC--m*WS7f(4KC}&_rp7D;X zOL<0Qc`3&~g2mL1lp*zKC25db+A*|j^)ES!OHXiC ziR|49)wMCzHHN1I)EowL30Vte*;?ANE}KBsdlk4!vMROQgX0Bp4L0fRYKIVIz;p(TFWnc>PvN>sB0;YjU*%W2-IZiUHeX1~>0z+BS)m-j(YWHG4ay)UX zZ;Rc?34Df`d+)EAwe1MEZFzAovRcBrBzeANs3cZdJ#y6|Ph)YE(Gt-UA$)7a;(pt@ zGNQHop%xv{F~r^0V_`=O)0O6NkZz6Rch;7+@EJXe0==Fr89-*xs3`qNda{Ii4NQ7P z?KI6+q0k7>t6433h|KRa&fpZadFsJ_gvNftX__0uJd`j?GpR9`YFsA`%9R?k|E2MW zGTW4vl3T4vFnMCph&ZT6lDVd>+NrW9!f7E|X zxkDdRNzI;*=ozUV!+BIvjYg`mD0fE&XZyq547C1p8mY>_q|whG%DxzpT})*&SW1dl OQKmjm1$6pmkb4JW5;HXb diff --git a/out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream$FLG.class b/out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream$FLG.class deleted file mode 100644 index f67b8e1986cf3966138f04fe1ea3e261cb411460..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2914 zcmbtVU2_v<6n@@hv%5*tHcgvCQ=qX>NduAsrF^th`UOS;g$AnyMYqYeEKPRf?xq6_ zlkqQj$(7!D9lpGm?vQ&U^NKpZ9z`=gn{b{`fP1 zaTFs6U`D|l>-h*0SO}AKmBlp{@3F|Sc%Q{}7J1%$L%~fIixG6ZUvQKre1MwTeK8?{9kR`|@|kXF1%( znp>^81=rTK$}xhQq)^g#IU+$t>nylhapQ(o_3w?ObIl)bP}gj$GW*!2AZOCEqi=d` zaIL&fx6V>&T&oF7rJ<#|sg;eA#`h!)=hiDdD(J3PLyES~aHy^AE%H9bw%yz(&lNWu zFZ))rJZHb}Z10%&A+oN!%dNgqOvIl^!K&Fs{i?w&I)EB6=^7P%;ZaeqGQ9;-%~GaH z+k8(MPO4@)wQAL}UA@GYeJn7r)S7|RnxU6VsY=apQ>%JvOkl__c#6p84aF-E0)wp` z4i*2kuOsvRa1bEVG39tnd~M%=U;ord>9udMog^1mAL>O{!6#AN$EONDi$cSyz=i)? zc%oc7zk^S6Vwmfb%>JGoZ;m*(uG>7_{F*GT3G}CPqiyH>9W-2R!8XB&?rWA^(TK$t zce(0b8%i^~$#!5rAnJR7UQIwn5(hB=YCWw27(|@OLUx?_;%TCcr-?P5CgOOSxZ`P} z59ugj5Z6eK&}y8p1Zg=F+=BR?tN`97$!{?5CqnoO$I0STva5;RcjULTW(GBti{@GYPqYNG72)&_P?H zJE>+US%?rMAtNjlbO{x`B8&kM!I0>{h=}5Z=tNd@d7&>g@7oT2%xeUfXUuB_7h{a) zmr`eo+aY-tHzed)+!EQdxG4(4wC}W^(eJe4jAoO;r&4_?DUrOK43V5p$|NVGq{1qc zR9VT%nEX_(Pbe=CSQ$KYI3XoM2|1xq`M)ESW;=%&QS?9-aYRKg;-Zi8OCTlsaafmo5dNNrL!`o{M`82VVVVqZmrw~FzV-)qL-U3&Bd8H!X7d4?+0WzPspb;UCxQeE|o4!ItqZ(n}9otn&m;;0vX zlch6$YXbaM`WOE}Mdpj2@Ws#2`2w9Q@vbd&|H2Wp95LUav>ZLYLv1h`1MUOavgW@EH#VL%6EY6D4xFpVC zUW_9z&S6=c$9-`DkHw|744?QJ^0cUsdt$Lz&oJe89vA$4=crYjZx`n)eeDk=SFJgZ z&8l5)!|ZCq3?UG`LjT#gxGSKtC*aI}fR}a!RQCj2*bi`WSHSR|fVuqur*;L5>RD#_;wpkXq&blBts@MapsN$ ztdu^qwYDm4wFRryYD=}PN+k&e3sx;Yy88ZASAX=e+OGcW|1Ryf&rD`+5c-3am2=KM zXP>?Iw_oSZ`yZaW0H6!M^1_cpUQFPy3`c5k9q#wyGk8FTqe6aG$1yLe@i`$ruj30| z%*7Xlcuha?1_=X7jrd;Rn z1fCTBZ+Y=;oDdl&-N$I5@I57b-;ujd%kYdB-^H_DJcsWI^Yc1hsK)obSdAaZ@hKUe zmy;jrcu|8ZuvLQ*%4DouZ`#b~t-J>Jw(xchRgs?Up60NY9?39l@Th!TzCeaM$*M9d26Gq@hg?xApf#fSN9Urmcqxr&^pmH)#OpKWM5&xb&J2xo8u-g`bW*=sOhK1 zv24blv*OuY!oO4Ys3wrk#z*@K_ISZ2X3HE~PN!3hE0)l(>Dr|Pwo=HPSRt9Th|sk^ zprL*Svy}*x_Kh)fLRk<^9bq_)B7a{pnYV1BPNp)c{D?(Mw0)Gek)*v@LrZ5dQmL%p zY59iGzSV;oJiXZjWz7qxGS;rbSj@@|n6Wf>8pGMRnI1HADWOZ79(yFkRJ-k(k&4T+ z1zNk{%0_9ZiHVPk2y4~4w;WAxa3XGv+bO2Cj+Znnx*9Y=w(u&4ayIu+E@fLgEHfeg zc6TaINNzCK)RA)YtRRG~%&^V#+M?G7vUiCHruL;U zk{Qe-B4%!sHL6JZzB43>wU`FIbAKSDD0eoQq)BNj!-6g9>bJ&B1~W4}V?$LkT@YIv ztW3U;vvy_eZROZk#nah5?ea#mgWZ=j6iGi2#ih*8i z)$ywAoHp=Nyk;PXZ5lRQbEO2fikh^65bn^h@tWnZ;zZCH1KY8~Kp(mcOk&EwE?MnF zSjV)1v-p{T2*L*L#HS3Hh{>g>hFh;yvD_chan8VbWDLBHpX<0_AdZU$_F|uoHw=u* z$uIDx@C+Gv3vV09i|PboI(}*35;8j8G4L+lGmrx#zqryyL+P|NY^Iq%wpGq_4Xpzs zR!3g7n~qez!%VaBBo23|I48ua_YE8rCAP?0q~x*K8F|~vIZIwc)68M0&!J$EvqzR1 z>NYcOXLE2)dZV{En8K5#GFWt6~>C)hi!q6Xq(fnkG(c@DVy>P z5GyfLp;FqsyzFYpR;W(~Mcb z1!=WvA4=u?nGny8eDzG5VrNjy>^EXzV=AwZ&otSnJS9}g7i57`r5HS8uUEaV{e1ap zUVSBhqlcL(d~|EfT+Tck&yF*Fe5Ed|z7zUl2d%ihA>{aUnAuXjui435`lwhvh_(Fq z@$*xn0SmF7-**~rCgmcv5Zw;?7Sgvm=q02-iQCHjKGGW;=Nn0Ha?qPe-|nEdknVBN z0W?Zmr0zy(hH7RHQZt*Hn%RHU%+{l3HYLt3X;W%uH{v{(HnAIVe}L2n@bC{v`qoTC z^PPoj3hqgGxT>1MoJr`u^Qhk2I*po1c)6atw{056Bx-$RtMmEJz&nL`lc<+v{p;0} zXpoghs3|ncMw774_f4buw6bG2$8|6`Zbua{%wt<#NDM33LD$muZZ^*zw$z=p{BHJ8 zlXhFQd5|}JMQd>v_K>38(*KA^R{M$5rB-+Ih8y>A?cv#{72=^1;ybt}ABWYxHLdPx zEU4JzRk!bf&KshuqsUj*RbAFqUD7pM!-0~9+j&py(0yz4X?Th*Gei$W~UM zmlRaQy5#jJiTwkeC1Q`#wGHmZg=ewIr%Mb-`u3hd-9FEOB3V0ymVu9NGf=Ivtt$Z> z>fBRktFJ!NfRB!Dyo|r)xUjaDS*NgAMZAJ$C2DBRIDb;y%Mj+UlF!X193rv_;yc3e zemuY*J>tqD(xWA$gXrYjCB@`K%6F=WRGUu^Rq>WBHlBp%G-pYu2DfbRzlnx-h{a72 z_vRMae2xi7hKQ^;HZIxBxxJg0>4`Xw>Nu)Q!-lc zxgjYUKt0C?nGO%}C)LAf=Z~mmc!X*CC~m}KOyS3I3y!nMpI~7=$%H+@q&&$~eF_;o zO>aD-uy+xQMPCVGDK)#~g?64=$_vYwu1O}oi?aJL%sn?f2$qPu`a`VMbfmQPvwX!j zx0DrII7|AATiV@%!oy;#M5#_C@hKvB5w&V|S;NM`6dXxS9Eo{Nt>|XB> z(z`6x_sRjMwPlzw!VD{y$B-ppDdHFlN5b69UUx579u;sMuK%KWR;)GoR4X|`8kxj; z5rND;e+r!y0L3``n%Vmsw*24m+4w!1>>nNRUsN(syzhZ)Zmq-pF8Oq% zE_os%5*yBqqV)u7&tvu8Mml!Q#fr9Kmi>v^KHyXNXNQ(XhZf3dpeTMTmH?WT!iS~-ZrA%nxFI4zf!uuOj?eCb6e=tV>#8UhVtMG4! k5uFYr2z7qrET{!aa^XI{J@UG`DET^kf(1L5^E%Z04|_TTAOHXW diff --git a/out/production/clients/org/apache/kafka/common/record/LogEntry.class b/out/production/clients/org/apache/kafka/common/record/LogEntry.class deleted file mode 100644 index 40c5acd8d8f888c4b03ee35e443cf506509e1c4c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1218 zcma)5ZBNrs6n^fyt$QhMR1_7E@iNvHwu*vp7(ZwA+U%$Wq1h9q28RW5=Mh-`y zEroTiH&Q5bUCBVl9TA%%?uxj_kaU~Pp5-&JDjht_biD>cspfhu-RK(Tp`{-i&0|A1 z-A>1K^e{r-4FY!v^2By*|0%<|QoSTd-DikDcN-Q%u4X&d>wc$hd2fw+n>=#OHI4SZ z;o1BgIphAJ-D6NEC8)Wr7mn|Jpt=M_rG`{xB4*S{%6IpC&vsf2GXy&_-Wz(`a9Vok zCC+%y{zxSf^(%(Vp4<0K>!r=Bo*HJZaF~Q^*pe^}S;AG!FwBmVJ?q=;hUGD&hd@Qb zHtuurfFZU~;MPOhiYnnY7DYUgP{0zy+NBml!zxV(942PS|E=y#{m3%uF;I)Nq@>a9 zS`IyjvT}YL=O+|3MT44Gsux}VLl>spxhTFkp_Ooj*rZR8r=Jmn%x6!Q%*RhU0Y)pi zPI8t`2N38d06FaxOj|rbOe>#2_)4xAZjh9N>p7D1NMiw4XazUPUmzgCa0_#!xy944 zz~YGnvZo0XrqPLw-zBtfh*{wEGAwd>I25{|lyP5`lMv zN&w{t;s>Wl$jJdjUc~?@IUU-<02w(;_5jj2zjlUS85Kv*FXG>hI?EC#@l55Vcb~>e Otem09jwqIcsFgpVQ~gH( diff --git a/out/production/clients/org/apache/kafka/common/record/MemoryRecords$RecordsIterator.class b/out/production/clients/org/apache/kafka/common/record/MemoryRecords$RecordsIterator.class deleted file mode 100644 index b4dda1dc74b6c1992a06143ceeac380b53668328..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3595 zcmb7HYj7J^75=XESWDS7jq`Ao#z_i!*m4^65dzr3O`X_{QpYZ}NpQ>~Yk4hiEbofd zDvm>WgaR$J1xk6Pgm;thDBzY_?qG%j!|(&j<0msP4Dh2V@0Jt>j>`IR8+GzN5 z?%i|md4K0z>BldA`?~;kV>XUmI3B}R7(#}HWHgM#5rG-UD6+yCi<4M!oWQt{oQ4Sv zb{s2lydI9Mc_D8T;>Mw)5QB$e949d;E zGeX`YpLYv+uZH)F=sn`ey|Vj&h7W4EPs9CjB=LZ-J|yJB8a|>B9WIWJny$jSo)gAN zBV}7os&mRSJAHOLFBCl2G$s@h!3CZlFgzn|=Zjvyf5b~+Zz^vp+|c8=*_4qtGGk_H z+!!4p7&hGjV>m}nLywa&az_l;lJ7Dz?2TCrrGCN~H+#)foQsoFG;&4y5OqdJ3ns0oL-e#@ohC10 ziY|r!$LtE0wNgkbqn7pa=;-R}?mCq*^B#*#56ieBY|?$z{FVKlkr_X139@hSJ+lVI`C0E`G$^fVp_*zcwEOpoY(Lz zg}pVz7nEr62oSA%b-WsFI-ZbF*7~_3RI4$pu%^PULa@U2ZzIXvO@(!TdkelySRgIQcEa@pR47qfRDQ%W{q_(&&%kN!Q4CJFX8{g)_~y z+Gt6Px7U(d6hwQu{g-@$0-(SAh|`r1*hr6KXZ<8_@@c|I+nj@(D;%6rBcC_z5ryl^ z+41K~&@L65RjhIcaUN`JPFGK)1r5gIdd~@1i$_;kHK*fJ2IUp$?d$F0Y%b)i45wOS zW$-OTftbiU1xrRPFXyZYi!o_@P=l$tcHTW-uJWYEB>e7TF2|{_H4k;B)BeLHCFL_u zMmeUJ8Eb7mRI6e^zqX6QXn-36tM-#t7UR2F^|HuM1cjSwOD~3r!p<64UN9G!Zrgu* zE6w58dSJ|TCgrL2gVDx}f;`Z;jz67t@z-Dk4Km`nYmnjWublh-%Bjz_P9_W2Fq8}$ z(%13pZhnWk?nur;C3nuCF4;PRP_k(T;p7}5gUKflnniTylh9^R|Cq0+gLDI;q(7nX z(}>}xxC%eR8vGpF@eAC-lVqS4`+TD`;Jc@IvO^vux+6PrEySA~MS(+gE;Q5wDnf^1 zXYm5<*Ow6A7pyBfn$BU<9Q46CtQ?%hRSm03xVou?)g?5{V$D;$R`)p8UcfElcnRy; z!og;J)A2o#b&=qs@e!;@mT=AYkeC+Ex=0BdjwZrXbnVfL*wFNy`Zl>*pNNFW+n9(v ziKe#j91??@=dfuo5uU~7S!`*GBq(`Z39o32CZe=^9&w6=&mx+LOqb7I*`_g*TL%+b zB1*Y!GuX~M+e_FX=6foB^voeS*gA_=iJ@}JBx}gx5}GlF@B8Zy@Ql9-|2eeyd|8Dr zYY0*v7aK``Md{yQ6@H6G{0`UR_e}RQEXgIL@hk@L2OP&Aso_t&|7T3$FSrYT#Ton! z_fywH_&XoTe_$Ff;3@o*7A~{cFXGpD373?@v#JiysSuu5VfM4!X9K!%8+lLDkAviC^y`(0{qb+e?d}Fqg3HXSQ!*!=flv#7ZDCU%A+8|QQx-iN391URenSf zS{>S_u0FshhIlHymf|X^BP{v#)7;egrScWwhgFIy0fq1;|JIxF7Jq#!UduI3C0p@6 IyauuV0!YcqhyVZp diff --git a/out/production/clients/org/apache/kafka/common/record/MemoryRecords.class b/out/production/clients/org/apache/kafka/common/record/MemoryRecords.class deleted file mode 100644 index e828471106c9d22dfb84428dbd379d3cb7a35647..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5566 zcmb_giF*`h9e!uCo7rq8gapE&l!Q`Xvq@MAlww0Ul5!-5U?BnltDS5n$-ri3-JNi> zYQ3nf+FBK@w1Q_#)wc8?8z3#vdQ|IC>wVw)Pxxqkf8We*HZkqO^AsNDo9}vm@9&=F z<^Nu|2w*k-6~Rrorvee&8^Kb1T;4t*k59_uJ{|W*pyN|Ac|gafBOE^?kIz=%I6kN2 z^I<#~!3=yMf)jX1&OR)UFUs_!yq(hVB^kdg@L!STugdgm^7wj)9ORE7vq$7{vJ#Kt zw48lRrr(gaZ-((L86Vg2gpMa8ScWsg;VB(YM-an3@_42a&*H2+Ms?)E7?Z_w5yWv` z-Y$eO9>#Mre_rMn!?+~lWgRbQsNQKMoJ@bV!?iM|>tr+-9jTO+X-}Hjtd-ReNjSsl zjFrvGUQ?Ho8H}50Gcjbv51Io9%{aG*om4!dVB+nC!>t;^2{~!IM>S|28Y;5(9afh; zY`YrD`$h%^EF#u*9WW1>@s#bv+m5iD*XO3|t#CD=?H5+ZPQzMk;ZPSf>>-lAb{v~hypq*4$Z zNZM)IhI;rVYEJ&_h-)X~`O+n+mT=OvTE#AVFlD+U8KQ5R_UyV+p6PN1H>cdp(bn}+ zj9h1zD^}7lI}evMQ-g7Dl_x@$9M(`W-U_%RD+&?OQ+ui^df$kh?3V-cy#qx8+q^Bx zoin*mvC#O%YQr?BSEIC!@36f`6avRu!Ap^Hyvw;mr%VVQ~ zP4Z|%I}LKh4MUK=Yv5+E$~Tv|HHn8@+-hJKdUU*K;CtxSP_Md*GFeA5X$_i5mQvR$ z#HWUZ9!7(gtJwxS+mLeHhJ4!*(3f<4-@v;?s~_NH13$!%M60ZUALAz)mX;vt*Cqq6 z$i1K9RRcf6&kg(nzclbG{Mx`EhIIVKz;E$81HTtJ??uMIAHZzc)-z-^469aB;Fz+k z{sz}+=(8GBlJ_?ZI2lFR;u_yA9zATQ`WdQfOUk4H7q4oVHBFz@u&#s-lMH>GuTehX z#72KKFoHu&-wA_zN#Vz+fxY+>gUP_3F<{^?_$$j|38DEMVHNVq>dk`kHv{{`XnXN@ zmf2=*damQ~FTs+9!0cXV&&hmnt1l=f zjH-{HGR`&3o1VB_sPgQ)hYiHFQmU;C2O-o-#pVy0TB-bMWbx-1kovDi_6CkMTw$H&t!B9z%E(6{k@ZYm!CoR6I?DGIa16 zBO_o5g8Z>m!Cy%=h+-}pv5-59(S%iK!CEXQ*A?jG$<>~8yd7Pfk%wZdskiOuCRbS# z=4I+lbP8ez-|G-~jc_3y@6d6pj-48BRq!wSkFQ)O%GEVv9Fg8wQ*gl;Dxc!jfTCK? zQy_xSXzGH9>dH_EjiSz z339ojK8IQv>w-DVk{52x&Y{j*n`zvh!(4A^-ku4ciYGydI^(E@o|wz{sb>U589}RP z$_-?;l^zsB#^@b#DEM<5d&Pdmc^kXKZR9u?*Rz%9y;ugVhTD7b9$$@fJW@`GP2@C1 zjrqZIn7>9sazc?%UXl4!=`Bi+kf+B4|7#H@;H~7pll*%ypN?Ea7sTj<)fByvHk^pX zO$F|o3fwp0c9uSEj=z83fTk+)W~IDwTMg2#ki4sV@}1!hw- zRXJuJe;9MQ8p>~Rt_M#eSoSzYc?mT`i5;Rqo9S~v5%r>VZGq@oMNgt=t)drDq(TU+ z*~m@|3j;o_K{A(PEgr4K#k7u}QXw(H<>dcyiKuu^p#hHK;RR!h&qI=u`L?V~3FRl1 zFSL)hL}Y!g)tVHK#Xg&Nj(e6gukSk0f*?X!~AcZBbbGwtRHt4iL5UYN!u?Z zzD?T%R7u;wQL?kLiQwhu(GY-`vpDJRQdnMkRu;@yiGxTIYM7_RPAUF#kmge=Dn7Edi^r;6@}3-{f^{XOI^!WZy9 zgWan<*Gc8^0GC|GQo$_N@DXbGD6{dgBJ2f|HI%x*Pg>ec%)gyMAzqbQB_zAXv8=aA z>g5=s6O6qOeS#VPq(b(5T<0T)QN@^QC-4zM1sGSFe5LY95iCSys(Kw{1#TpMtal7e z+nZ-RLXaIVZ@73Ztfy!M9t@|qlO zq#Dl6Etw%NApQV;(Npcjkk;63@V+%dUc6pxRcY%#?oaOg9 zw#w&O!Y;C)USi$6%=CUiDJ#Wy8Be|s@24tYID2h1#1kLDT~xD*MeBq35LN#$_a%S5 ny^rus@^=OT|G{h>AJuW}-^2{yZmtIKF|6Yl;i+YOwxZ&H@@P9_ diff --git a/out/production/clients/org/apache/kafka/common/record/Record.class b/out/production/clients/org/apache/kafka/common/record/Record.class deleted file mode 100644 index 178f2ead3471fd470647edd6dfe764716e41a392..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7678 zcmbtZ2|ygx8U7!;v$H!4VL1~*Amkvz0>K;-b7|PkCLtl9KoT*wuq+F#z%K4C(b&dV zTdVbGwbj%{O;4qVN4yp^6_485T6@~lo?2@!d)iB@rr-Z&*qy+pNo)dd-v8eJ{`+_{ zJoDBglSH&qEY)ZY-Q%Ksx>v(+Sfl&sex5zR!-EVy#Iqz1<1U(@hgEunmyc+aO<&~U zQ3a&vDE=`tsnTQoe4K|T_{5VceMzHYx`&TF#lx2w{0a|G^YB$3zQ!BRFq^My^elaY zXW!)ETRePQrSGWpU5zT~9w$A=&+qZ@eJB0ENk8Q2k2G3LKV~Y=^NF8m^i%p7&wkEp zztAAbFByD+!54Y>6%Q}*@M|Z%tkQ3I`?nhXj(*RxKk)F1Mt`I~@$AnU{e}L@v%hKd zD*c@;_YWTa$rR-9FF~5G&DYhsX;ZtuLy%|^l;ev=;)y^cu`@6<8nUseOsr;q%jS+P zykZxmZ*Sb(ls;*KX%rzSud$<}t;xHi!{2Ul;IoP#cWR@{I3Or@o4>QGz3F_v1=C?& zoiDwX|X}g&@e$Guqb|iU}%e zzA&&SP#+0L>%DsuA+KCs3xsvyNH}qppgC0;F{*b8viqXFA;{Vsj)Yo9hkHV?jzG^4 zR`Q#p!NAbYKrGC#xoA%egyVuvYmUbH>jNW!;6SK;Fwi#`s1HVmhoh1DSST2c_13q^ z2Siw1wcESfdm?mSba*5dipRszNXOoh(AuUZ7R@#o+AAn)H!P#<;i!PL32X~G{Zt7G z%9a#b`}*RcgdmrM+QXNESWbH)5FFed7?C=vG_2CU1+98pg0t9X;N+>w$LCbZk!K2K z@_BJAoPdNancesHEvvy>(%?}M)rhuS zlag;B6da6?4hx!_*#i&`ia1B2@o)kJ1XUi_IHQU1P`ut33$9rH|D?rMGznxzmE?=; zGv!mman#LluiQEx(H@G#M`NM%8mChgThcKQh;QS(%1x!fR8lCzf&MU@1H>sKSdvoE za+v-Ec|SdoQMKYj;b3TEXed04ScZUes+x}HLD2(`qY1>2q0U6K9pMq_7gRXq@}WSa zzus6~i}tC~Ybw1Cl|mPd28IwsMW&Y|UTe>Vpei=Ymbh`f}u@eE`-@eFhxZ^Ql&zzT*B(??M8j>p6>ZGBS& zu@>$%MD$-042>AsGF!GSi&wp);h|pkRE2?C5sp`&pTtH-5*1bPL^Kxatw8;#SYM%w zLQ%v+F%rKPNHX=Oc2Queb^0WIip@A%7js03PVc7!s+g;bQo2&7TUgn7qD&X%V!lqd z!-t}RuF~l`u|O9KF%xdFKv2LCq)Mg+_*VATR;25exxn1bOyQL{0+^{)aeR3sMD=>~Nr~>9| z5--R$Yr}%kevM9*#dTB(#9{$RWv{B{3rMyErz)jmbfi}*>Ff^~7gk&;j5Wzgk=5>m z*|&px0xEGV6zJU%iHG|mpetva1{;jNwvM=2vSUhn?4|E^mXjMf&C_H-9!CclZsQK zNb{QOKz2ae;Q+K>oxvj+lE*5RX=o$@?B^Q_#1RjvgpqvY;#yjU7pgUQ5pYupZJ{Q- z9Bl=ZMcV*3Tj1@0Tc*LS7Wf?OpKF2J0JmFU-tVx$I{@#rz)XLa1wIdOrv>Ky^DQu+ z-)(^}0Q?Sm=P~kkS@yfJAF#mNpf}nrbgjS{v{{UOfHpz*EIc*OI0;Y9LzH!&v`0V2 zS;VhQTgVPNF4$=RP_cncVY(0y&Pl3h6x^!~n&_Ag{3r5yYMv(5HbzR#WUVpFFH&wL zXOS{S&f3W$d+!9MoYWMl4G{q#!zvYG9SsutXzW zAmusB;FOZbi=sXZIlvtieOOBCr6`R^UA%M=bYV(fieaWu9P>_6njA-Q$l*{aA%8~k z0-WU#M@h}J2nA+mccqtA}NtX@hlytMC?3b8yn@zeb z`~9S4pxh7H2P~Ae6y=$+v&4H%%3e@50#MXsraTDBLz42$;A0n~((#qiq<0;xtU`zAj3syr1` z@q_#k%w?|L9!mt{6fmze|3`bJhmhic*-!8*q^3s4}v~#j0!y^ z_DS9?vD@UVutZ7kRu-w8Lh2Z~N|ebWM{k4EQ{)a~3iq57FpqyQUl5{P~#`V$)ZHR<(Q7HE#QJgHd)Cm*JE~ocE zE1q3}BE%8bBW=W~)gx`hU+H?JWw;ddNXw`)-H<&C+`7vTgSa5V`%K3TAPhJV=6KAa z;h9XA57ddfuxHc)hh%+&B_!rZ=0Ko1W*Bo`7n9xDC^jRDk<3PBQx=kvxn5~@rjJ9*WqboZjoekpa8*4 z4HsPvvm1wdpc`vnY+HxpMTclssaxgQWtZQ$t~9N?Q3r1W?(Gob5FGk(i-5&4C|Q9* z=@=mm4n^LeLe?8^A11#cq#ZkJ-VrM99H-eWb;y%B_$?{zo|mM#bxA6{JEen>BzGbV z#%MNuR&q2-)I1m<2s}`t>!e9|#vzjh>_P;}_0Wttf6(A8&IVr&r+p^pHA;zsG?-^= zaMVi8-9-+!%2TM`;3z>*V;4c8mng`OvN0r!;z53t;bG&+T*OIgb5W6<)5AVSYRzO> zZ-WW~szk0S#}`YEPo_J|)e;+0t)j3)i20sgT@fh9s|XBVsr2lbOMJvrk5n9sCp0 zSPCQIzR0^BaWZ7 z1Yo`yfP&vZO-d^u7gKUE5j5qB=>(p$5OAdk;KkJ>usJ14k`^G47T$D_RJg%DBXgg( zP?{2j^HrfN_Xf_Xo9Jef=pHPxRI5O6(Lq{%gcf%mp-M#TlKj(>RE6aQNvh7rljjS0 zUW0j((EaKjOBX^TZ(cxQe-Yp9U!n$j*&@X%Qy-SXMxp^c_tODWhUIv&3{KO$(<{U) z7DCRHVep!4o?mHNR^#(-1oP7A6kfGZnChE?KAJME*OVX!D%Rh69IL)=A(CU!2UgAA zfC9+x>ZU{utRytj-I|R_PSEMo>mJG^U-yiHrKhxtS!fvXqO(@GJT9B5mvCmJftPYq zj$>jDxw(9rVMMNlKsx5|BAI^PTe#Tg0UBY;U@wdtE&)`AJR)ypPm#kA@z$}D7Z zQ^vp-gBfzLCB|vxbOMVl1k#G&`-4efnJI_raXT^`udy6gP5#^g4w=XAfJ3>OFJm=I z;feM0Zo{8y4l|gfGg|6O)$SxU>_Syv)l!H0ekM=VU3Dy{Q3scx0+CowF0q1g#7fFV z7R(l_Xudd;7K_#75o@Sktd)Xr>7Pw=(RzY74YsC8Jo|)f0xsMxj3&S-B^yvKXA!<7 z86qS6jJnjFzuN71l-6`|B(9yHb?u#Y%+|}9g4tQ^oen$}OwifwohoJ^7j< zO;BTdrxSoTz3w&FC&-u9a3QxHu>pT{YD72mQn~O^rPv6q{IpqYq6@H>j`?m=Czij1 z>__p|AyoP_*;V?CN_XG~_X?=MYf5?zi|&&{gdOmK$#me2@IPC!)>CsQDfYY2(G=TV zsAL=S^ONL1D51wGe>-~=y%ApBl)e-gnr(&0x8W~b&6F*+!-p+YB3h|boI@4jT&bCH z=dVq9s}`;NbC93{dH#94&}GSvqmWHiaW!HYJd9|Y z=`mM<@NbryH&>{6id75NtHzjUPXDVsBI%;}UpO&Rs%Y$gR&+>`2&I?}gfg6Gn6%H$mX!pG0dmflAW!hAsltKj=pofm#CMY;5X3mg{AY2n pDZrsXZo|)`U)b=Akis>Upo|gQ@y^ihAX0(dB)*!wHP}n+{x>%Ye|i7_ diff --git a/out/production/clients/org/apache/kafka/common/requests/AbstractRequest$1.class b/out/production/clients/org/apache/kafka/common/requests/AbstractRequest$1.class deleted file mode 100644 index f67a99440f432b4f35a6361532da703046fa909b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1164 zcmb7@ZBNrs6vzLUjdhy?2BIK7p;Q0p# zrLW+bhO=q-Hs_pGO|zb}EvKgVzUQZ~ zz{Bovxy$7qm-}26xh!#cfH{U)YKwH(7OB-1>98%5f4HweINnvCfcKRphe%P?7cYF#QER^jEO-AU^?ohB*3gmb{-}77>~O z@4tc_d{2Hx$0vSAz91S^KOnl+!Wdy^8wJ8x8{>pR8xw@_HYN$v$D+@0J{F$hd@R0; zbE1|15ed5bF@R1CVg#e)V|0e+$rm7FgnOq4jKbAB=kKIsDAM;QsQeTgi2j#r4QjEc@&pOZ_!9n@osXYw1>as zC-j5%_`x6GZ}B&Hyt8>JC$gqZlXLH#x#!M(p1E^pcmMk5k3Rw2!LE(-xG{<`w=ijA3a*XYn6@#4S(}3I+V~vzY~05K8(*MkV-623%v<= zh+$+d2m?LOFq1D!wde9C_xFW+!0QLx_2uEA4Be)ZTKcl#>Z7KJ+=^B$Ul$8I44IOw z35N6KAQT&|!>Ul*yxJhog|hT{V~4B2%sbAE-VX?JCJ{)9pIRc)k-JchwBo+rYCDR= zlN^M~;3lpd_9oS7Trv&7TPs1u``Ak!qZn;c|OReRvG# zDw_KT>%7@H2D>6#$`?z4IjBp0rJgcn$>9Pn=CFW83#A;s#-kjTuxw!^2M?rExMHu^9ZkDABv1btqI{CC%Gm=qr(sllV*z ztjekk!eu2}&0c$~4Ns&$*()(v5nSo2;4how4H)z%D`u|aE&7l#zclC*0T)SMqS-uB z^an<6PW%q`3yo5^OxljU2lyYkS=HR2(G}Vbu#5 zmbw=5#mlA$!oXQAks|Jf+pWa52pX~;2l)@ zUFAWwpD+AAHqC*BlhE?G+^z4M(J=o-p$k@z| zZ2GR)@RarAUCf(Pgcf?RVBrER6L&2PW5mKJ#w?8Es)cJRT*N(w2i@M$zeZ$X0@oS( z+q>6}lbN&Kh&xVpi{TZ>36vfa_YcJxojvzr6q%nqz6b-j-V*rB`5l*u*%|8@zI`q%z%*cO2>;r9D zbV;2Y*ECk^LPW!q(UegQEzrLLrxf@E@#8l9^f`R$6i+jX@T*i!2}&!xeS*Z9o{T=o zlOy&yE$LG&Xw99ciwnGo4Gd|faXR#xm-bjRhOFgS8*m?UsjKs11 z14jHfe*%-VneS9!e@ALlsh!b|rB#Wz9^e#71s=k{5{V<2uCYH&9xh~d_6FzOt0 diff --git a/out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataRequest.class b/out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataRequest.class deleted file mode 100644 index 750f7873844c90ec4d479d18285b931381659842..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3355 zcmbtW`F9dm6#ga)L&ix$NK9*Dm8M1ssbcNol2l6wDwRcnV5x2C3rxtEfx%&jru)9{ z{WE$_kF_?*si(`&{iAw%-^_#{LbUA-z-(B8cfB*VBfT!@ih`Bzv5SAcd zUV;x1zC0`9oEzu)W|Xg==d%kUVtt4sA<>YOa1oaz>_SRJ+Kr5e1qoSPmf*z;5^{L4 z4_ELKA6}O53i1+O1(UFdf&>qWd{2=GJ0f8TCB9r1p$ZrXFQihDL?)jOpO3^t0)jE! zCy=>D&`rIdYe93Zs?>sM5-7770Ym3f$%Tvg=zRV{B$rQw z;t>Jf*j2X5f*Pxof@#xGD~<_STgi_HChzlnpG>Rd0N13|GfvN zvI2U-dXY{H#?*?EsFxQNBf}Oo+VRHp0@Je0Q2E-~>@kfb){z3f}Byk ztQc{{WJPAOl>LFo9`koR-RcO_s29u`PSaJh=Kscv_F(~@l43>-LpM@NtxBxOwv<2G zqbF2tMZuJ+kx#79`gbYKWGA3S1;pHOvpHdt9&>jBZ#8{PISU( z+;9gD0^C+s<<+8$6=*UJ;(&}YDgsXa2T(f}Md&iBxF*9uO@t|67I)cGchKDFkn=%#O#X=?F^k7|Kh^f^IU^DXIyZ?fiN zhT=QwGj)=u)Y#ULT)e6jOc6I^tl>2ouj367Z_0QJCq%q0;~l&!<2}4D;sY5U;v*3s zbIDIcd@AEJe9k%hWqg4zMSLaWYkVW)TYM+r=zV?Dl2ZSwLfEB}2U2%UF$FBNDRWJk zWm~yJp0}?+l-yKA?6eecu#Jwk>3gV?=?yJbU7~_I5l$x33-L%QA5JDy^U*{olT76k z$xJ?!4Mk(2xmZL%=frElqhi!ku*y2+SenwR6OV)?+g37@M+}9LYo|K3pKbDvJ>;uG-7u&y zI`r)q6fYz@w8&Pvb}6Bz;iBgw(2X(PKxj6G$8dlij*~R+f=CZTs{Aw;X^)B?jZ?JB zqldl%`vZ3%30_%#Qy+BP>G&YX%J8!7j2)UPu2X1~xEw^A0>Y?~v;^cyrz3n;1^#dIIYh zS;y{luJ;y3Z{wj&JiLiL=}nAnVEhk4>IS!dl$e1m93c*qmi}W%(m&_vvl%paX7MDw z@i~hup2jm|`Ei<)yGJ~rqJkkm^)8x%z%wU#|(LskL2YuE7oh6o>k=F+8BIoS;6G9uz pGfnio4yGHYamGQKv~Ym_EwAqX8C^V>sA|z@+c-w6=V;Z1&_6qDi3k7y diff --git a/out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataResponse.class b/out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataResponse.class deleted file mode 100644 index cdef59c526d7e4219746e8ab745b1529921752ee..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3068 zcmb7GYf}?f7=D&Z65=AVf>uGRR51x)thFkFXbiDNNn(?L($=P%u)s>l#?6M7-t7I} z{*r!Ztp=ygbUOXmFa1ZIK4*7@6-m*_%sKCS@?M_za^ACl|NHwN08Zg2KdyGbhigGd zm_+)i(=7(xQ!)0$`W?Qr?c5aI$y}euOw123DGIT zT!^YwwX~>3m(=+sHCi&3myJraY8sYNGRjeFwW`&kISMSRV-j{Jve`_w5YJ2|3fB^| zg>)>Hkf2Q6R#(($S*oH!3wttynW^xvoj*iDxp|$z(c~r-2RKU1?z} zIq5m-ypqY~z5Mk|*2{NjrfHaQV~$p(oP@TDO&)9-O`87`k#AhD=+=aUfl%)MWy3SH z>Nx0{(koiJzFgGIyjm>Nr7~rd)bfmK>VkI<+pI;sCSk0pT~oVL*J@TRD$-J4*36V< zsdK8OX0=+CM*`rEl3|#0dPTJi^LeUcne~!I{ga`l?l%|IQ4w+aYBo)xXc|iz9s5K+ zlXFaNl4)2pY8JPuVRCB?g_-a}p>R`2Zm!#_s##;QvLUB8D!LK9v}$RW>htrOIo4eN zS@Ps7+Eyg=ZfRUXV7ydzYfZ!qGMF>!W=XrOivk{KR)XWAjU|jc>#eX8C*vg?k#Pdc zVi}cj4re8dHnXv9d<1LvccH{8_)(Ezpeo}IOc^z>27XbVf<7msjuk)N6$0iK9KPtF3R`_AIrFlPh@;5L_Wjk68f8A6g3pVeBsBJGQPssetaY2 zTYM*6j`;Dtj34l$gtN^ItOpq@vb)rhm36xRaj!pDt2(nsRI$r5W)|&!^sk8Fl9}gt z3FX4xF(m%ym1wPLOxd*Ch29C}5zZ8EYrH_sn{w;h!#>mP5m#bU#kX+r{03WeUOTod zFJAXXu*r(0Er^@BCzSLC>7Fyp9^S|e`SlqbSkSDTlc}Ry;`8XZ2@BQ);m+(<>Pr~h zM%7Ip2M5K8Ny4#hG&X=UgO8nc2(b3+vAxUn2=|7PPY0e9&GO8**zPY*I8#%Z{ICoC zse!eTs`Yq7&H8`6Ut(4LnzqV^p*x;Q=cZGMY$27%$0lQWKEIKsLv*t~D?GHVuj}H? zp0@Nd-Sw90rm0nU@0`cQdBDZBV*_n#>4`@lhxt7*z;BxX_KQ(RYCn!*kl!80NVmXG zKIGwp>YQ;WNgwol+?t|!U;gEszvQsfb&*?VYNL+cva zBKOg*`1>9r&_Krxn+rCuBR!(X!wqyk#7=QOqI8kl<&fPZ755KEVZ_$$dw|`M;RopX zm2O(_3fBmNTnDj}LH66>h52z8;eZ`Nig0fd@GZhE67VTo-iC-*ahe>#+963cC4y?9 zt@Aj8vuL%08+C*Gi^2LRbu}_1Vj5EX4fHm!hlqw0#aoY(zFYxV9+>J$~BD*1!Gr0Id z>%Y3y7&4c^ICMlq>NyHOQ@4ivo?FOZ*+LSgg%r{zRv2pkU}VVawzkXtzJ1mm3Ws)Q zRP=T>|I`-SbcO`oCWe9@kF0{8kTh+drk)~c5xRy!nLw`k3|3v8AXY6;U_6m4hAheA uXLy(79+mH78a4XB8WmC*y&`FCX&UxvX(EH_X9&eGOVl7rx-b@zN9hBphJk$m diff --git a/out/production/clients/org/apache/kafka/common/requests/FetchRequest.class b/out/production/clients/org/apache/kafka/common/requests/FetchRequest.class deleted file mode 100644 index f7c9b5011d238b513672d1b19dc0b3ea3c3719ce..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8557 zcmcgy3w&GWRsVmsq$|nSPOMl-6(_CJ=Ak$br&*gOw(Hh$oLZ@0wVb4Gw>j5}Y^#+e z^-4|?R#u={$GWkO6*gdljuukZ73jLgc4pQV7z{QT1;%Rw#`_K9%_!Zx|GD={zEY%R z`K`>q_}=e)_dAd8oc}rBx!3vZ*FX6ZfSXiY!_Nny@VthP2Y7!WhK~ z+@c>#L0rNo<=L_yzu?EGG`ti<6JBn>M!eF1U&Jpp;Fs}fx%`TTU-jcNviR5hc%lI< z_;tB_R>S8s{DvRD>BoEg_1cbzATqN6Ip*Aq@cgh@RvdC$6twcf34wfg7{nfom~E2!#{Xe zoSy&=;va*!8UGaI=$|$GiyvRn@UKDKj(-c{-|-&-{3rfPF8>|G|KO`Z{4ZV$;{Wh^ z5Iy*s%za&)M^~*VkW@-jbqbB6hFM4zQrXNAqhKiLu}mgu_N9${KABfo*Ecd8KR(ny zHaXTmIvDHgos1n(P%$$0Wi$DLktv)o((}oBu~VU`@A%kQ|M2)^ylp$YTrQdKh;zZ5(M64`iYD*upO_r(9qLyI z4c=#*F*?#lW~L)vFjJWsE^Qv_y?gRxZ*1JXa40rBd3a*HKkk}bGd?mJ>vPXESu?IB z>qdLW#$)3#Zr3#%D$Tkk!$i6YF%VD9WQ@YRnN&Da{RH78u zuA1x2nOWZ{-^lL6 z)LzXiAuF(D-c-2t+X@D=E?1tewY^xGUj=hMQJ@!_N)q}r1@oMO*SUwwBfHg_76=vu zPBuVSnR`vsAoE%m(_ktu3WC;pN2pEYZEl-zIA>g(#Ja7n+n%)2JeDaWXOiZ(58qyw zCb6+LII89XHN8 zb-bmtrpPxkbzk|x#fhYqgN;^-6xT5YEt=U>HgEC!@>b$%%FEMUQ$8-5Nf!D|Gi$E$ zk42pVWF^k{teJgK3U1fVn)@%+*xo5BFcJl)`WIyqWj$ABcFqmkO%(?Rca^jYOyEq? z%(KCeBT&(sPNAu=m92;AINYul50#_5SRiWk6K^bh6do*B|Hk9Cs@EjF>!Ul_Uo}T} zmRrClXDjF4c&n_ApdyrFl$Y?9RJbE*Ta`&=I|TWM=clJxq+{O#)ZASLCHkf;))F2n zBq|)OuIlPF`MVQor-Pwynd8BDcHT@RkEWzaX?EMSSK3U4J+)9)${RP6jv3^>2~*W# zqhlP$$*DA9e@s(;T?JH7#}9+WCFrz`ld031YS5Lg8gomC-Ak^snq;eev`$$$?L_)>x>~E6H5Jm;I@Q7gt*fwFud5B}8aCXkOr?{J(XmQV zURM#dQCF?%8eMIY)s1DV6}qmNd-m9t2vh{0am)y0r zOC68nE*>+wx=yv}YKyvF$0^*U!;s6AToO2~s~gl-U2VhD4DhPS14e$9o7B~IwL@1? zwNqE^YL~8dt348my$aiF?WerH$m$MV?Nc`@bk}031WUDy*VKMpy-D4qtDDsUg}qlN zOINqxqhiZJxg1is-rB3(Kc&;j86(|m&dkrT68E1?By;kfr_g!AF34@$PPe7pc>NDu&zebsIKl(V+#AOxciFZ zitX%ba0Tr=a91Z{g`{Z|vL-QpuU)k)$CQp?*H-Ml>8e@=`-N4?NzG~@nV9eB%Q6xP zas6?j}5@&+aOYvYf zGebN@kGZ`qleE0vIL!nJyUlR0Ci~C|S?AR!9Nk$JkD{7^Q`q4a;Y7==Z#BBye$hy= z*HKWFzpBarnTpuoeg&0X>Y(H$h0*m9AK$;G6e954)arQ=!#UypsR}o-6sL##Q z5eMCm9bdXoUi9weQ}ki2T2g|t)kIt8PS$i)$J^aHzm;CPLLh*i0VgH6)<-$Po*N3*Bvr0i&3$KGql7j! zmrEyy4QUk06Q~r-=oS6cn@cev=qRuC;Y*>t=9@vu6%G##t4m4SPe1IsqM}jxB~?@yKCyt1R5ub;)QGc}Krl4sqYOX(IfY0$qrolRb*e-#y_Q?Dg4 zZ<;LIj`{Y7S2_8Eyo*i0w+uJ0n&H7 zXr1(gi*6)+kBbhIzSl)>ApK?+y@~W&T=ZtrZ*|evlAd(YTM&|7k#>coU9|4(7_B>- zM%tO=xDL~3;CKeJ9H%%l11~(h2iogeH|<$O-N#AP<33)4Tml@1AAPu=qjq~C(wHMf z7E6(`Mxshq2c$zKe}-e7itL$b&e-a+mhzmfyzX^MTmz_)_GQ#hya3N4yo>NfJWJ3z zz3nf;9}2WCA-I5slNP0Ka#M|)9cmd(6N^~W>5F(TVXfjHCz{y`Y7y-cLJB{M&&g<= z!YgR$^jm_$3s~P7hy+3#BK}KY^DiojVuCUkk>wj*;u<0ij<{1Ya>VHo-C5h(*+_Yt zL~v82DYRK0ID$!D8)jtQLEJ!0--uz{ zh7k-9*J+0BK_dH6;`(vodYQ=n3~~J`k@aQb`76Zm3hq$>+^clFS+(LVeDQdziege7 z!YS1QL-k=w4IrV08F+!oc{EbSVLSknqgQbVc@${DGWE_=o6g^#!&%M+sPh9jhwmcP z(hS<$_{1-8C_;=Q!8BpSk)WCo<4C|(Z;b?O9)wjV9^%N00j1%)DbHsCu!nbB>)WZ- z*7a~n*Y`NO63mIBjPEVUc$mE3$K6c8vqFdXH9VsDcO{&+{=Gw^%jlV5=$Ajx@JCUZ(*eKKoMpE+#_zu25hBI~UL{QQj5Oi^O&k zjYVPyvu79aznh`o6(X|t4DEiYr~ZH^?AiPjp4c7s?C6J4ZmP%0^Fj5@%16$7c%543ASo%DleSvGgNHhuZa;$BS64&qM zbDgWZ@g5?jkv#k4m75rR7F*>V8*9nGfs#x#Q~C(rhaV*GNn+>yW&0Z)+DXvcZ{ z5TAM_C`DXy#7`LvnII%42sr|N#C*gG`v(;9y26b#xMnnb(0ceI8l2}-6&zos3aM2; z<{;s2D%U9ff)m_fajmNyc_JRC937OAQ;xjLxOrmt5)LfjmKGd(0bPse?)2@E!pLGI zl|$n3%*v-*KStsF+00+5)AG}@jDr(A1a5tP89fuB+kBtG?GyD33n`F;AL{#C8wPU2BxDQZ^vFb_XwRdjz{RschG(BWZ2%tnaA)X z?-Ddg(jyGWyYV58_F_9e%rYSEzEE=a1^hUJCRScxFrQ=@@bmtW(n+(PGdAZ>lGA?n z6ibAJXDvKm!!=gcKKdGZTKtrA3QyZ!MhEXAR}PzP&v8&^)lHu@`M71EeT;b>V-)L9 zHRkP=%uiHho~p{cqmuc9Rhbj`DcUEew%hFJ^A%=)s0y=PYkry~_X3`A_I?L_#e#}; z%TPRgC8n+IONcGvSf$7(Es^48F2~Q{qrA(=4vPqV))CrG7>GkdwopOMoh&6)WIpXO zGvvth<70Tvk+jW{12h)lHuwp6q9-QMJ#xPha#vNMAy!6lW~%Q#DAxN=G)6cEq&{p0Sh|d)p-^(CiH5gN(H@@!>Z|hm(Gv>?eWBzS7Da2&`>) zFJ7G4xe*Bd5G{0+rprmzBYQ1-W2Ro}m_&|mq*XruTMS&Pa+dU2TBn>c`dB(C5T0vI z_Gm2goVb0^F?JBkJDH>uk7R!&{LqV*|C(dHE8s1-~ReCI|94Yte0 z&@lp>FkG9DpW+S97#xVW$^cm6-+#@5SQL?`E)|g{4>(@ttb`Rl4eklX-N0+Ko&bU} BoNWLA diff --git a/out/production/clients/org/apache/kafka/common/requests/FetchResponse.class b/out/production/clients/org/apache/kafka/common/requests/FetchResponse.class deleted file mode 100644 index 850d63ed0844014f13477a4a8f1c5ef58f6834e1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6236 zcmcgw349z?8UMdzv$H#yB+aHtyKRxHl%xk+D;7v%HBGnemR#(nN4S>Bc9Lwn*$uN3 zN&)czJh)X*5k;*B-a;uks1{K1Kt=Jw6BIAJZ$(t%|IN&9WuZD(qN#fmdbU?v-6udWp_XRK>!25$}!tE+PfDZ=nAxV5# z#Ygbb06r#(k5}Uk+$kRirRKX798$3yH%WDO%g2O#Osbf|v>e?dH650tPe?VNl%pdm z72T`gJ{4@)1*3YKKqwwf^!4^6qKV|0(Sc-7M|YIWu08ridL*M~ha(Bg zOlOC=vZgba&0Bib+M#F04I1z7?TdAKq#F7<;{CDy7`1u0p(5AAY>39=z42sc@204S zyJ%}{%hu%1j{a!8yCZ(4XQm+@?d*+jN+zQH9&Tf-XGcd@Y*SLo6~zR^wyGKTM_@s; zyRUyBIjcaR(b--$or`STZy6iMhlUK3JJd`gKgLZOPO$5oW#d*l6Y18+xFDEF4`+32 z+~oH6mCm-8jeLJ@ES)Omv90Vn)3|Wl$XofyX2VL2#9ddG7bT@rZ5!x?zdfBzTN{|B z=29QpT6PHdI&-_}UR_r@YxInd4jN{^KA0gV)Rjx=nH{>BmUMwRB#o8W5ZLhF`ADdY zmBRM9!_qRxQx^AFThtKETIPO%D)+yeyFyZy+iaN{ONQA=+^leiY3dYS;E7XLIxh{V z_Ig*T&l*XyoYo%YzKd$|j$KEmYqO~|+^jk`abEeZ`M1qw* zk{%w}sau9Qs+)T`ZyKqbxjSK40t<_>1%17PdrA)%G7P)kYwU6?tYf*`G%L4+Z_E4h zHaHc0Rl&mwzDCEJTS{R;pzNQ^d*0$t=ehAbJMI=20&@|1=c**D}fzW4KKk44RL&3KJp5_q2ZhOmOz7j zC9wOh-Qi?EkxQ=D@Cd%G;5!<=i|?@jG<+XF(C{c8D-kO84h^rw*#f7Ob8G$}*wAuL z=e&j=;zt@D$733PjGt)uskBuy>mY3jv>%;;()LN$i|9?7?pe$uLbdUqO!17>un!k2 z_?d>E;|ZQvvJ#T}3Z@X1OBMtPx!Nfzi9X?{-)tc{9VIS_=kpn;$Iq`#*D`L6wHUH5Li)e z=cS!SuCCOCPgE(wuL(s26cN;fDylVVx>yl4ny3|ZiddkDdJ)plg>He<%PBABtz(cC zs;8{DbD^0n>e|^2O7yT5%ratZA-a__8TfOYVz$r1VuM&@7~Q91Me1G=*hq;R?fZEH8!`*3s32!yX0iA zl0tCj6$( zzPPbf^K#(khFW+ZP^iU;^YUj=;s!pLVFG4VY;zouuVt*Xmhw90oT&m!niJcc-Orhl zCEhfTqL`eh92;jSPxnr{dG|!JfP`nV{zLPaTQzpSVU3!GPNWrfHrLnrOf;A4xJ45k zV`&y516H*;+TXcVU}f1W##|XNB8bsvU*U=m8t>S+W7rV$iq^@V-YPz7$m zBXU$Na38eRs+EUO6RMlQf>wXnAF2;mO(VolQfS!eDrr?DZ=qb?=usC42V8Zda^$KD zhJ)_frdE~u7D>g`;p)(0IpHcM84jz5uw)8Lr6JPGBz?>jUf|NlO8PjPP{P&IIKBcq zZ=bnAY78sWIKlC7P1s+UI?36`n&RBbRd?gWI~h42w)1&5zn_6|tz~2z7}ceW>IR&} zxW}=JG1VE<>$vh3?7#u+#GT~cj{!VP$wzSxp1`@hYn{gn)A^zjN#Vx@q6xZKia|;M zLr^K(OUYdv9bm=>C|OON=kWJD1Zn9WoKK#gwhkbP3!u=02S}aA%rcMEl%gg%a`N6F zg4j)+6<8_+3`$he4k+b!^$t_7qiv+9E$wRS;qG`;**f+Vb&OEQUh3$BZwAe%Qjk$F zYX4;wm`4VCMCjlN?J;I`F`zIClt;i~%}94JoUWMa?8(27>V?a2CI<$ASI zZ44g3o^YU1T_=$xmuq1yw0Hu|6KGjq!*O^5D<`mOeJ#h!Ca`(}Yfh`<_}B@Y(prBQ zO#_X!O?Bb=Nkm#h1kQUHXm8!UX=H@GXd=5$&RoXDxhy@%Q z4Dk?0!)U<>KggzW2KF%Vdzsb@pQF6iXK@9?c`e3pD=x%A&QEdu5m+`{E@9~=2|J5` z&?vD5MBRAqbc&ot{LO`3gbxva6hU2L}~))ciJcpS^RSr*w6?1V387_ onNwak`Bdp8a02tt#?dt#S1{)7{2$_|iMeQFOV;?!{akIZB$Yzs(O>IkuFocmL6DJdt?)$#) z`}Uz<>IZ8TIQ4Y-+&`+P_stAp2tlxL&dYmm-o5vCe|LHBpMQV<6Tp4=Nx-lhPKxP1he3fY*x($^O~xwDK!(+7xHo;7^eeSsh45L zKx}OM++_6hSzp)jmF1EBC)mGLRwRc`RFtmHf95IPSzw{QFDBcMHxIPvQx~;8sBjC zDLF-dh{5k)k14u-t7C$pI;>6;zWQM$Cyy4hQ?izjrZTkS9ad9PWd9m|=Tb)+3sx zYB9NxCz{BqnBPuAVpdZxbMJaPHbho(C^SV#q|}N(D>}(9_q?nXR$@X%nruw3nxLGB z>r!fNM9N!aHu^hkCrhbQ}7iVTYVs^)7e_#{pC_4ss^&Qry# zS{R$UD5rE0Z{jThZ;N;bAp!4-cn|N3_y8XY_(;UZ_(Z^`oRQB2d@kY(e95tXCE{y* zBj8&R-{E@^Kj24(o=r(8tFSw27-oa!IaXDWb%yZ@9ne;6rgyg~`^pW8`tT}4(VQ-) zWsN}~f{XIl3`4V@>T=x)-Xf?KnFY>ppaL|zRH+d-Nr;Rvv~MC|At5_1rF7%8Z^U=W zHKR9DHy0@tY#DUQZf&)HZQKIN6cUW;lyRw0(Z^}wkMp>=$eVCRDv-g3HI2{MB%%6M z;;eJEoHZqz&&Z=vmO6%iQmD& zSlMyGrlXY1s|1#EsHuW1kA`SmFV@}6FzAm)O^szY!WN=#J&{&!-7U9oqhyIGO_qrA z4h!fy`>Re_eiDbNqNY*Xv*4SrGu|$4b1RlYz2-%SJ@ig&qgSke-TWe<*>3pJLGRH3 z%^k*mr)}+~?+0z`ZS+K?=1*t6y#E`&bOq3_n{*t)VH%In>St8bKhPhz1(v)H#}cZR zP#w5|8n4i~2-kUI>TX?zr?lGEWlTG7LJYLtL~Wq!ChC4AEe?8h1AF13aUbfiAG^ql zcJkmbjSr9qeS{%IJ{+YJ0n;+>r!V}k-$2Za8OQ7}W*l{uf%%w)d5+fnTYI3BL+$hm zi>T*VI=${iY+u9|(L7+OW<*gx?eHA23)fyP z6Ao7t?x|?GuH5oK1z|VBIBi)TpiuEh6S+=GPEJ}&U>SRsu(w>*kfF+qVgn*LLt-Ro z@xzr4SW5edAI^o>RO+Cx?E4$6g3d7;Jg=qGg+W9uO>KrApmrsm`+tLj*8wvEX|lVE LR%dC?fua8ZVur(B diff --git a/out/production/clients/org/apache/kafka/common/requests/HeartbeatResponse.class b/out/production/clients/org/apache/kafka/common/requests/HeartbeatResponse.class deleted file mode 100644 index 3e41ab94a3ebb839b0ae52ca28f87942601a7e92..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1934 zcmb7ET~iZD6g?dvL&gDiCxQy==gOJ{Vb-r*cR|+>$HgTOO@J+*Qk}7Z(J&KxCRXvE zd{7sG(kjbmf7F(@XGV-l1Yc&lZ{I%m-qYva_y50s3*ai2HB4rZ#*_{QONWVH)%KEx z%l){blgBl6@|!aLu2$ET?GGJ)Vp=B~dC%a6`rOp<7j9`NG7OfMt5th(sa`A1*$ZT_ z%D!B)c#Avh!g|bCA9Ks`H#U6FYDquz9lvRXn=KJoHFDVC(+pX=TCG&;rOIu)K5sv$ zFBTVU2DAK#Kjl`FduvuLl&-f%t^+0C3qtONE4hJ%FH8PITC*BMddBtK@FqiV zu9jb+UwkxFc0IA!-e?H9#2Za=Hp{-ln=4$pYM&T;!*w@cxZ1S>DW0@N5C+zq;4*9o z9#%!rqLc*Xb~)Ge-j|h!vh9S^D#3IRt-pu1_%cIJE>9KrwYUtZp!r}Dut z+pDWWPUpMk^vzn*jp#bV@V=-F{WDH8p_=lbLiC#7mX5gNDt1S{;%7=dCd1@mwGkC* zAd900Mlq{CXAPXg&%|>#jP4atdXZ4-4k<-L$-r&c2JYZ*1IKZ~z#Q%x_y^}T%o`|U zLBpbf3ho)Wj}scI25MNMhBz>Eu|W^Y9nwXll#lZGE_K9xp-3I-Xn}QKh78(MCH%@N z!%XkI^~> zjZR6{wC*F@D8}gf6YU!4rBmQs;SJb>mq_iPX9vB7E%ce%*e?3Fk@>eU_6quo2eM7&g)&s}vV{p1X2R6Aabz3DX8{Ju z3Ajk9PExR`4)E+Acvf{jfienmJ^{C)J(Wo5>(2-x1(k1_%)q4x#)#H^2}}m#R7cFj z3{|E4%r{6YA`06Wcn7wF!RKEfUF$$HiGv)S_Y((&1Zh0N0R~=TXbV5=;>Z7xQuMKo MbdJ+*A_Bhn0kO#Nx&QzG diff --git a/out/production/clients/org/apache/kafka/common/requests/JoinGroupRequest.class b/out/production/clients/org/apache/kafka/common/requests/JoinGroupRequest.class deleted file mode 100644 index 9994fe442bdd2734c29c0ee2bf30e1bf6149c389..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4986 zcmbVPd0!jH8GcqstPqQ_WpItJC_aTS$aWgHG1LymiY*(2ivSnWT3N&bYfD;XwQ@r5 zBfanYzS1LYleUgy4%?06-fencAE4i&eAo|GfnEvme#4xOt3@?Cqs1?)QV;_@Z0a)#B}*+>qiU|K zN!XfBq|?d4RAx9ikQf{p_Ru#E51voeS@j=$UtIfrJzme;i2B)#OX25(aj^N zbE&}#sf>q2f;?>+vP{*|re`UpM=$8saS3Oms|k};yt`K) zS-sOfD#71pOj5Vze!ZZj%J~V+99AcC+zIy^Sv5DRnz~rKn|^CXr=bt5>1k7YysVY1 zQv9r;7fuVc~vz)cM1yR6u>&k!} zoVcX%HJw4f@Zh}!y~9~TqZ2N@rm51an;-S4Tw&jow$$w90kvo F~BB4d~#Hmz9+ z(=^N>tyE;3&{0EC*ZFqJ49}Rx6T)NN?Q7axaZ>Mu2(PT=`e=!#h}$cgDFR)@phO|9 z9xazeNN9<+d)`#?wJzGu#0W3F^k&5VRhu#(mz5Y|uF5t=0N!c z4k-mk@PL8@z9WW|f;VDF!r`@eRGeQeoW;hja~XY7!FTaJ76rH3Wh|%M$@sp4XYd0V zKUDA|Jj+;Ks#PSCu#+CMbcD0m*%m>6~&up`G#6;V)yfip~k>JFjc1zcyE-;2#o`pB45FpmWVi?|^H zGHxokg~x=XpGnxgEGe1GY13-1*PLc;E?9{tvszJ@D&g3ulPi&ZlM%fXDHvAdikj0W zyCUpX`eejSrBY4!7EBso< zZxs9%zmxHM;f9wL`~iOyivCH#pYazNe^u}|{9VC6@J|Vc?g~a#_TN*PbUL_*=(>`| zGB{GBgnJ5uTiDi4qS|GcHw2FZ3haZL$#zV1%i7=+J4%#QVATnhuMPHQrRLkOa z)hlla-Er!%me(r)+uegbTwyeNg7{%C!{rLU4v#quX_-H3UnN-R(o}HS1APWlCo6J= zVjHx4(V7*>EI>|QL8>dUdHur4h154+vjo0Q$W90l&VCgD##U(9JK zHP2!jy(==kMg5#MOA(>7gUM9JzMD#jt=UE^kh+9a!zaq1MDsiDD($?u6OZt!NP$f` z-tkJ7TStf_{4t!eVJGJ{vodyT|8?<(Fz_*ckC4*<Kjo)_<^7HE`=RUTMZFrq$-N5zhJ?loU zAM&i5xIX4tZ$KNmJ?r}r79Eqk!=h>0Pxej9>$T;b$jHTV`#w*Y_aIs#$2V;;c^_OxNgB0`l_W9JPh5<8;%wyDiunQv$d*z=A# zG{<(#VPmXw4x3_a{yBtWZGky#exBm$aE9X%!US|;1G;Ed90zcS;2fsHM=(h6h6&vm zmA**uCi&(Pp-PhCxT7-8;v830-iCmUnKJ=`&z=bw>g<^Sz;Dk41OfDOt`G)s{sd0) zlmpg)4NZz8p}{NMm-#HjIs_seVR;@c!m5sN!#rB&v4z%kgd6AangOD>b!-7`H1%Mr z^Y&4H-O)g6pw<60c6GJ}jvT8eiksT%?_k>vZ1>^9QvvCjSN}~;O*YRkXB+I92c0gu z3UqdW)-}_v9kgr@_3ffvr>Jq7&oOGBC0=>%2xMZk^At}H;=GMfp@LCC-1unO9%7az zvObLP{5JSrC9w4}MrB-(@h1Kx0`ngRvPACVE*3Y5gurrq`?DN@DE|e1;&1aU>=-+^ zh@JD;`Fq z&INQ;v$|xn+Tt8%-fAQxq3a>RbjoGg!)O)W2s=!D43PL8NHusCJnDsAoMb@Dg;1h+0t%#%J-<@3PT={V6zu>0`wg3PC diff --git a/out/production/clients/org/apache/kafka/common/requests/JoinGroupResponse.class b/out/production/clients/org/apache/kafka/common/requests/JoinGroupResponse.class deleted file mode 100644 index ed7722fc7d867c79a92fbf509634e1e7416044b0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5149 zcmb7I`Fj)B6+L6i8ZCnXfuO=9friBkAWA7sjN@Qr1q8P2NH)YIsXCU&WMDTU__vHnFJ!#c2MKTUVJGhM;q7>b*gGxbogTc)hj-(CaeT&u_xP|G@Acq) zGT!e)BOdVM1NfjH58^|9d>9`Q!$)O&%!7{$j!$?n>c?t)(t}Tl-%rcV%p;u7yvkqbJpHIyXL^%Z4Y6oS94K zGGTLaLd%DfBp6rsN>~+5B>EDmNMCm}bvQbdig)!!B?Nj-sHfC$M$L|elcu3($4I&? zlFQ~zHESMGGX+h;ngh{zG|@E>>x-vi-45>6k-m6xus510-(KI9OvVnxqur_guEao0 z@Fg8wjRSrCv4~@1RaLU!VEk~r?`S+#rd@&*tDoaP-e{h{VTA{eN$_fhkuxH>5!x;# zB`9NBRx?yn&t+pH5`1Z1uQ09|Vi8dDd3`Lajr6OAsSA?4gcY_|3Z|Y3_vm?(KJg|= zrJ4nUGTSSTJ8FV4kekrcCCPg`DahTSXLYkv!q(ccG#QFKzEP{kew8y2%vU{Xq$_6?sXFE!lItZ-|v5G8FXXf0ZI-V(!NF5^iVmn76Tw*zVsz?1tYE9ri-*+vnYdhh$=2&`;vl5!RxR~!9iRWLt4g^f@#bs zIF2|WuO+MY&1j}oc~fbYUKz6rp28Ic-@(%?)?*^$QwxK8H zp_S+&pM-s<7~cxMi)%8zr{EcUpTVx+2l%0aAK}?DzEWA@?tFPqm4Y#^;K%rhg69O6 z&$<0mvD3Jq-&XxIJg?vd{9M64bSc=64h0eHR?v;8f?wd53VwxOOW0D&VOBZopRaGM zbqg)GSZ5Txh;ebr-zfMkey8B~$SC*&{wU*53jU10$oQ**zv1r+{(&wTFDdvZ{w3k2 zS}IEzbqzC<)U=tfi|>5JmB2U0iUm9DGRD%faCthBDYdYUnWBZ}4Epp=NA{I6*&)r| zTT<1qUnwHeYsRn$iF`Pc%VbzO*rx}@nzyh5OSHREFUBIe+;REi zIm2Y(zlG~g$N8-Yh`vMJ z0nvJ_Uv?ksmko$~w^?=_pk;iiB1RhzvAVwm7koTYa-%)GjUq1t-v|6@ZOs}AEXd27g-JynAtdKB^mHhgVL-H&F63(Jm%vMRb zhSlvJOGWllT{>-T zx{T{v*I&jBt=lhSLu<%&85^JAja)d)@eut1db*L`4AL(*;uhl7$=R*+%0B4ynMse` z&tClyTlIPFUZAgL=%c6UrDrJNSxR~yJ^T;c1uyA$q8D+_bYjwn1bs>mB8dV1ikU!y zy^}M6MV&PhXw+LXfrpFM%wiBn(7@FLILgtDJ9ys8Gh6OZQEoR5l<1F@=fff%|F7QOJG1wT*BSd<#Lkt}Q@pUfa9v zYG?f(SI~9cIi$7)UAx*FLJij-5n^{>?G$b#y6!-Tl!X(%Lz{MWE~3R|YJ)rRqmPr5Mo;u#mmhg;hMo+&g>p+&&h7VuA@m9zrR zw!ktfBBlb(dJD}x%s0^E4d}-T+>RFdekX5uGd+I?y_x6lB==9#pAT~9EQat1j#o5{csZjtY+dP=!Rb85<0Q`76Rj(h2t1#PvIKjlZ2S(9?4u-b4d*BLW zQGbYj8v|_`+bSiFJ0u1OwMi}y3OH4iwAqrwl6(={FJZ?F!cU@3#122pINIK9qU~?t Ust)%uPQcp7cO3ti*YEcJAMArpDgXcg diff --git a/out/production/clients/org/apache/kafka/common/requests/ListOffsetRequest$PartitionData.class b/out/production/clients/org/apache/kafka/common/requests/ListOffsetRequest$PartitionData.class deleted file mode 100644 index cad10ede262af6139bb6c3cac7123d46271d4409..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 595 zcmbtRTT22#6#mBblD6ihvaF;YdN3bail8eI8G*87R0ute+o+@NuI-M|-|9hv9{K_O zsOXIL5bLO1yT)vMvAD?gU0QOKyA%?{yq9|yylt3|o6^5jet_T%(hYYOFkaqd= z)yQqM+o4b)VKs*MiS(p8Wr&sQwXG&YwCXX>}CpW{up`8kh=9pfg>)Z7E}0Dzun_c+(ZhiCgMn#m_aguHHNeQax-N0Y-_;1 zj@4-OgrgXYvi_{yzt6@Fttdu&iy)_kl9$t3>7|X3{#mjnMfWhM63A3u!7A%xL@K2* zj2D6;NR!P^()(l&sQnOgIHC(|P$RX`$9J`_x8(?sLXsXS1qw||Aq^=kBx&iUN4Uatvq={=yJ2=x zTA&}EFIwZPAkUv~DrH}5;% z_x-GU~!dQX3!nhmv$l510d@6{0!?+KhmfQDh z_)Hi&wpU>Wc2wa3JXnRDct|cI8b*WICEIofaZME>cvvoD8a}Jxks$U2ab*x!g%HJ~ zVLXP%gZP|Go~Xj-@ddejQDlBeE?*WlUlF-aYIrJ)qp@99zABfm$>r<9;TvI+Kds@L zVa&(3M5k|S_)Zw##rNd${VbHRjN|c)h93p-V+}tEV=;aj#?SC<2tUU!HWbF@PA==}QcwVM|((q@6>Q#nmrL0sobE;t(3VJM)Nt$hG zBbQ6&6ehK;S-rZwtGhSec6xhft3q=})*NUyhK$6B?>jG+gEkO+FE;Kr}m!N-qYLF+S#rU>A29?WHhIZ%s_M8GEjlsI@{yj zt(~hpi!(Y~*Y|d<>Fiy(Vnw{Y+shZE4cJv>%Qt9SS zV~EO!;;Dg*VGWy9{IcS^OUoLqJ3Ew06kM>R>^?Jj@o+L{<(fNEIcue@$7*M(zCbOC zEm=lQ0!vewl(mdTZYb7cN#j}tUt6|MY}=8_B)f(OHzdt&V?&xNk&bM_NUt@_l+4{l zpS3YX931xsG@~Zdi&+vFF54?6X!fRBNUTuP5G!Gu8pp*&CRJGe1`a>LEyVUvaiG2S zK(fr?ghgXcDsX7eSmqW5zx!QVe3E*5SrBOnx*WWY6StbCLEZUcftY4XcDgr8_EV@X}1f} z)I!9pPUePKTbSiSg*6F~)F88}F*w8we+FZ8(AdnB=;T*sy=jHLdHR(bE-bEHTu9o{ zS#1YTejkI=o-@vh)@}Mg&PJw&zp{dB_!~D3B&~MS%$no$tMeweL#lhDnY~2P@{-1~ zb|}<{)(sS3BrLZ=BHtA>B*_$U6Q<5~b*)RDrvUg6j`z*W3^j=~Z4ft}_molXuI=~i=Kc#wIxeRCo?B#&lV~mS?*^A8kgXdG+oVBM``M4 zT{WmiO*QFij+(2hd1}6TlI!E|+;I$n;sNLMGQ6E(G1S1oFZu9m8k z6c+6F;1enqd);vl3&>e>#}{H&(lo5BNob$riqlHHuI|BkY?Yi*HW5ew z=9=5G^h82TzeeU##LevRkkq*4TWoA8)J=HIVf7wA2Vu+1=wR7#t1P!{B820zR_!v@=3I0N~2OiyTZpsL~)Vg5^sBwguN%2D!62N zQT@uhiZt3rBiEJOOpeURXY(;MV^5~#_j?tl0xA1NK!Vx2t;88$|A_+Zu+B{KxhKeh z`+C6vnJu^@n_;@HX*k1cZCRw|l|~-}q}Hp3i#0iiW@X*yqS}VWl6Vx+bezJ`UKXym zyz<7q%j4&j6n$L*CFx6w43Q|0{iF6H)m4NiA(3N;`u&i&h6$x*pS_Ds;^b*{ZgCX>gb+1*zy&PojqrITI8AOT+-5(W1( z?DyoYLn%UBj@}WAcjw<|DKwSs`U_`tNvU5&N?v#IfOGaMZ>e31vb-Sr`ABPl9 zZ-~Vl-%V&kUw>LDTO$gV^}Y?IS3V+(-U6I11%#T4LBVnU7udo%gCz(*8#; zUQ+rJ9L^WOEBLlNjG5B5ax@cXq5}$@99JO3d6#Ei!}&_jyq5D-p7~79&+^P?alYC! zKbZ5lXMPx8p|hW(tcbL9l)K)R+e67pIvwh8E?3S%B}ahPR5521Y6pjvIFDC(TMjHk z04L*oj;1@?(2EN=BZ14-;8-V1XQihgXoD>#;fSfY2R>8-YIf5YDtaD35e{_~z}#B!%6W%y!Ek_3y?MgPLf7eK!uTz`DEl-YQI@!QD8*Q7Y<-YWaJ|IJ!rDg}Y6& z9>S4Lb00#zIKgK-;d~s#@p4Q;8*0!_RINZ0P9ug+r$1vDAhL#tt6S;$d$_uTn0kyj zdYagImcD(C7@mfHf+FwQ45Tsl%{d)uYFD>=3F*yBvK;a%P?0yTV$$Z}Lkyc;d2# zOEp}k!P)W>`uAcM{6)LS)CrTzZS-#Cl|iDOfuCUnJl@0@@i)zhghnuX1V@R(#&9(8 z+CaC>i4cm7opbjrt6b!(^&N62Zkk)`TPWat3>7VbXdqHMf+k`$AlPbb2}Xl^7KJJo zg=@pLp*ztV4c3Mi5_(m#-3Fc`W>K}EZUl2iF!zK>f;e}5${3sD zy*z7D-fcv4Klx4NXC>asJyn#q6>lSLfKn}7LmX?=V;kPik6%ol#}-F?&Zk4f^D*&! zlkGg)rPnE9ZZGDditfy>Xn2P`e5VHY{)Pz@FH(fU^>~--=k*k>k^Nn+QzbRkd-8@) z!qrYgi)G~I4gVO9>0u^YID%u{>@|wxS_1PVRWWTy3KB=$z4wV(_mVqsFh7zp<*#N8 z$M?(|#iH$FIH4zUV&D-h?x~b)81L~lji6-&OX5BL2e5P}PTGZKyKr)R7fu;P>!YN0 z(nJ$O2n@3zF6J+qCKE-D2DgaiO?1v?n(R`#;4<9KZ*iIgUppOg1>VikeEv*&14CKN zeOJNUchPZnD0A&ScrU{+h>zeVJCr4Wr5-qx?<1vi@BIu>anBU^Ud0SM9=E)NlWT%J zZ8uYg9ZhKFx18Qj>mGR@2Nia%3)qWS*$O%Z#qTPkm`gsuT~e>S#P!=J5N|6%?8Ng2 zS@v$hhg=O#qsCI*X!{_Gkyv(C(=IF@McYJ{*V-(d;#Y$YGiS+PjcXiXdYj91DV-t5 zia1Ow>AO=I$R&7Q?@=@2@)QH!?sA%K^8u>!PH%q@l@hwjW2hr}f;+fYfsZo$q`j!* Wui8NthbRMj22z0#f5tvu=Klp)2;9{G diff --git a/out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse$PartitionData.class b/out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse$PartitionData.class deleted file mode 100644 index 4fd667be019fb5d1b3e35dde835510544a1645d4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 772 zcmbtSO>Yx16dWgMw@J68Ev2QDkAe^?5n_elmJ&cgNJuM+NY%TuIH{YB9lTy7einxc zi32}?AB7li;E=)zF81^Iy?HZ!{{G|ZHvms?yM-m}HqpSfusI*lYKL_naoe5)8YIJZ1_`6VQX)t^V(}`UP>td>&A(k$?Q}m6FHj5B(u}0HHlLnN>%uxvU61Ve5Z<;HHEsnz)9uX zivm8)ev7#Ghm*fJ58he>Te?iW(!tM8c)`V%55x1uh;b{%GNKqOXhzr)L(6@`X1TwxruS-4xlDv`N_ VZALppYq*Nb+*{0P;5s|){sgcEzK8$- diff --git a/out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse.class b/out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse.class deleted file mode 100644 index 3ae4fd4c234f8a8b9720be24f74e44c3c8bf59ba..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6013 zcmcgw3v?7`75*mK?CfLm+GGR6Ln#HC2XqnINE5_{5G1g9m<`a-mUOaAl7(b9ot;3j zwV+}Zv|8(0Yz%HD%NLxYqeHweLv6faP*wU@1L37%%)jNd)l7l%>Ix2 zfA{|P|L(p2%(9Wve}mfR`h-9EfWEPSsIMi2_#CpveD*864LE#n>^KHx*bhY$Kug%1VrVSL1g zVbS?$5Ff)~@wiujeO$&T0=NW+1i~l9|p zS654WcWGm=H3xK9f?t3 zSZkCJ>S~E~cC^P@V!fMMdV1TN+FCdm-Kp)?B12koAQCf;L~?+V#m%W?+SHQf7Hw!$ zr}FNO&emqvs*290uI|?ER-(B^L)lT+U`0z;S4UTGbH}B= zW=DU2S~t5KdlqGv3(FeU&=l|LL^5Hnq5GqZxum*y(wN3*+~iJ6vIs3n)7UGa$oUQD zZhVn7EKbQ39=FeMYH^ccXk1+8S|l7z`G6(wfO^cq1j&Bo^VxhBD{U*cfgQzFxt?cW zINl$2Swx!I+gNQ6T0zm8H1z@9c%hinj#yEhm*X=ZyjbM5 zQ$=pp?&#a8TcKK>4b@EMVl-yQBuK67>xPjs?EECfB*-=*W3J*_`v@)4XEB4$yr8Ot z$}Cz&0|l97tPsu&F-2H?MtjbQASe67iD}&LV0zK zOWrOAw@x2>X%CUxq&L#c>QRSG$wVr$cCV?g9qsRD!l=o6z!#`B3?*-$wO)dgVW+`L z@U4yyIpwFYTF@d8ON|%Wydc0D>I-C&?UB;Ic zoW+AAigi#{UAB^|&?iLpEBFc?lJQjqU&F%;HU(eDHxzsm=W-ThGnN7qQ3)&anRNj- zk;sNqozn`wg+~;88|M^!2j5ljJ)y98%1SDf(DXdCWYrcYA z7?SaQ1wX)}3Wm`oVTtaQN>fEXlX4+_WX6pH%P^{vhLz3jTyY%lM0ezv6ES z{w^&3hk}3NUlLa3!^@sUQy(K(i}P}1-Iu0<-Zs#pg`F%yQ`>Acbwe{#22;Y+HtZae zGh~;F=ad}n%dLolyz~|}r6bL$p&{l;-mf={KJ^Y5snHQ}S6I8(a+ib!)5-#4cxn@N z*751$8Whu+4RezchZwvr+F3HbM0V>{Ir%E%60VU4wRF3F6A>BC+jxCUTAj*_d~Qbe zC9JD?oT1+Am}?0*Wj24-nYpYm-wf#HCY_hFLf&~LY@Px1nU&$WtMO5d^Ey#FMS|F1 z%Aittw|LR$=x2$kjyX=ZcO~Y9kIa;FlM7eZ%vh-kwH-T{Gu_rZRZRF)Xmf<; zc>8uzOgKJj7)%(MENQ=ri~QDl@d0FKI9$RH2P^q4#D_VeGP5-Yt8f)RIW(|cfKv7w zUHx+QSG)SN*ngF)Kb!qEuD;6tt6lwh2#In<{IwQ-Gm6+=hZgEuPlfy7;rA0r<+Z0F z^&CgRI10z%sXK|Hw$t$Tj6-f;Isu=A3HWKGNA*k~AmMI2B3eO-w+W@8sP+VkL#1OV zYw)VxP`O$(fslkTRBUxX8e}n6Da6ArIG^fs;QB<%f%B_=N4BaVK)hK3IH(3gv&Do1 z%&w{iPGHVS%oU2*pC|hBC-5>`wJNK@3B0@jTaQfMBVbfHJGV^rW~P?g#xKjx)z&?L z1;^+v4_?FP7Jl^sT~GA%kyTOC({40RSM!7sSG_t0s0Z(x^)&-fU6sc7e$bGokZ+_pELKd9p%?C4R;Z> z3o^BzB5ntdk_Sih%|sF{`=BfEG&MExfal^r*vb`!v=dzAb#Qx#Yol$;qFw8tU5#cE z#KyQTi?NLu*At@?o=H@rNXB*Dha3K$+PH^nfwxws*6xMsfp81FI@Y1knMMIHVbQY4$ zy)i5z*(}mr+CYAtsrMJw2f~4{|1j=Uec`|gK_wxss7h$|7#5FVNqsTf>KHB?Lv?)# z+w-WWp)^C@GPSg!TrEFSA1bV`2v>wdhcRC*4_B;cs8q|;%Co4k$#kGXIx_jhX;k%u zOVqM))HalpTnkR4u1A%_#p772mQP?=A+{a}P;&TyhtGy7sJ1ag@JfNZ=vtb{<+kHVUR(vm4xpGa#graE zB@u3>e3UW23n{MH$(cPC>JSOOAKSPp$t-k`R7;UMWyG9CQf@>!EqEhFh!UhtM{%P? z%Bye$2JMq+>p0S!B`JADbx6q;-wt@Jf`AHtZQN1f3OenBcM|g*I>lw|mLx%?r3hLq zu}5U=v1oKtc77u&^UmASl0;#Pc^2e(ESRK$Hf=L{&eK$hF|*fBg@~|6^$^Ky!BI8} ztnl|*gCmwN#SSRo?9Qg{>ljQo;}*`4y3ExcOD1$0B_f^Hj^T20Z5)y5h^7lMgxBL% zuJ+L*`yHgk77kE+9LrDO6(_OceiR6*NG`J$(#SV(tbj*zC4WO~Rq-INfJ%3V(eCyC E9~PUE_W%F@ diff --git a/out/production/clients/org/apache/kafka/common/requests/MetadataRequest.class b/out/production/clients/org/apache/kafka/common/requests/MetadataRequest.class deleted file mode 100644 index 15ab286616f3fd77806888de2583857264a8a332..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4481 zcmbVQdtVgS8GZ(Kca~uQ!$pfUx+bE_BCZ-sBTyP#F|Gny7O^C4&9DqGy6kLbXNk1w zJ#Etay-j+h>BY9`#TdgSG{!c)<~#I{evAJ4Yipk~GrR1ryA>-kGv~}X@B6&ZdppBh z|9R~ufTMUzMzRh8%qWncE9l0o7|zL<58`k=3|ta#E{n;Ocv_H=R-nR^krA(-lwm2@ z22%{O^{|l>mqi6n;YvN0@Ie_LQm_M0%lNQ@c6>y}M}zp7jE^h$1U{+YemtY#Q}}cp zp2c(G@)-r6#pe`!9$!$ z;+f&8i(^Bh!xEZDE@_KeG^M5Iq6ynF(sNwa5Hr(RTT9!MS}LbY2-s%ENM!7(wbZ0$8Di{B`s{gw$c|Ly!qT72=~+7)9o21ZMzghXcY;8+mkK0T zp3wc}z$#iIz#a)#Rb1`_+sY+v$HJh)=h^8?rTlY3=L!;(b=KaChh54npNvL=BIOyo@rlPo^A(Sa6 zQ@O0ITN3tEg%)#{aP>&({tDi$Dx~057fb(Ozl?9l_$HH|-VApYN%k^i-yM?`^ohLi zXEaNYe%gz^v|&b%FWLI>-0ZAw^~bkB`kgTzn09s-m@N}3AzoSA|D`NAkW6{GT^knZ zp(Ma+|M?%k5tC$nxDh{AW#XImVaT!!GgyRwpRg+N&$1>eAo>%b# zzOCXr_^yh>IHKZv_&#&SsfSL5bFxS{QD}OC`B7dfRr~oRVNm2arH zh2IIlzgO`l{vhLzD*l8&tN084x)raM6&bd!O<&Lt%*f4goaI5YO_qpzDB5o46rtR> zDM`cxOO9=Nb+#?sRk706W_7MR?I_53*c)0P_8+SVa{` z+6+C{RtzZ*;4uk7qrlPjlB$*GPE^`?Ejy-PprsN zt}?`mBJ+`Ah+AnqWdez`Hj^XDhl>SyxGK@VnJxwJQE#hL4PqvyXrb8PP6BE=A^oYlxFiZ=anr+AIYJRB1;qVrbV~6h7-lCIkQw7V3aLr ze1EJsEIh+2Byow1{Q0;feZ0NGkE+way#tSgQ{i~rbzE7~z4u(-a*uTC6=|__jr?|b zM&6-+GcXFl-kQu=7Q2H7-~DC~weo+QI|?~h@|}S_{Cer;S63}|if+xPoj8anKdKJ# zxdt+tMaSj%A!mG)GZLFXFW`rE_G1^qco1DU#+mJ|EcBtDBTDsj5Aa!~UU%kb0nNJ{EQ7oX zEN@b}%zG@-O$go1@(SuliA$YYMFaI880)#&>#ObcxB1(AFJf0uoBv2(fKZk8z%4Yc zqp1cLp7BdpZ~uK0>Mlp*0r*LdN@{kHoF0-h!259$BVb2pP7j*!ZahLd37}v=Oq_WL zPbI(@PSCtzzTb(O+Xx{b(=8+}P*uY4;ykL)$vG*W$v}+ke}@jwz1-@3xA!_4ztpqEwK2 zP?P(wWdrR~?Y^E>?7X&tj;ZFi2VTehQ?(sy*p-;_MOLtT1);>0|2p>EzyoV|a1DDC zYv^3XzBees=QuV3h0nbR(%vr4^pc-rbmlqoHG@HzWaw#f@&fOIxk1|3g$A4pd^o$fjKi~-+pjiT0vn#ZQ%y->_R6*ug5zJ;! kW)K(ggeR%fkpnd3T^4x-H6pUvX?b;R4RNG#RD?(`j%``8ti*O?TXx*al2Qxn;b`R;k%CyQR@U0` zuGn2Aabv(KrjU?e63ja!!GyXgrE#2CwlEmeQfLE|mJ$jr4TZKqTlz>#+mJR^|2s3g znpL!R@_qKqJ@?*o&VSE6=f8JH&t3lfMF5-R77eEZ2;=D>1ZRTSjb}7G8-y2UgXq9? z2@Z})>0Dq`@zT(FNs^^dVcu>8+s%(5s zJ^omk{F4x#!+8x~_v0IWOoi}G{HY(`Qtv<0@aI8Xk8cO@7x+$qt>=}-cQyQ_AK&xi z`+odY5Nq()emtz+KTwaqQDDF3$07Bmhc$S?kBmb3Lk)i$#LYOXgdZtrKh|(Uz5h;u z{d?8=2Zit_%K1-~_%r4CAC-}RQb7J$!_U>&zbF|08pOZh--EREOaL$9Kh)ztgZKsh zOT&K)A{~1N2D z$fR@mcshT3d}1moSTxwbr@Ldgw`;I{u(N%z-KAgM*WcMS+}-Kw3h(NV4Z7t${R3{< zw5@dw4D=5ScPRBaI1zhnuH)K0?E{0|gWdgomiu|$ubUfP&|GwGe9mN5Pgi?q*MQ4w z%|O?lp6-rzcVP7Hn0tzrn3&4tlUc!vvSIJAr!auNWHy`0<^)xCc&GBIiI(2@B!>dA z)c$llKb2)%drC*Qzj`mq`YLX5m*r{O4&`-wDxJ#j5NvEHt5?8S>UrDR+Xa;!nNfnO z>Pe-OeN%@OQekxIlFSHp$BJ)4am?n&j8>55d2kfLMBd}4f0srH(( z?urw;(_rdc;GZ;qotJ~Nq0*ton_o9oJtr-*H1Sx4FA+oqDae6FQ8nU9Zh zO#{i?Buh;#He|=s@ffsBIzoME{8`h^9yGde6{FzJqV9A)xj&h;vZLKKSuJ9FS(wG# za_Ga}H>d?JH#c!SlgslpnaT1cDgHm`PDPa?tm0ydP<PgW z1+{nB@wS+?f8;>YN`2i)VKs-Tvh0qs>vL+r&xCd+%K>+4Djc+6=b;VCP8us7L>(it zoV9fV%}y@P58Ew9m$gVaSX_t_tgLe^?{wu0cig=BUnj;f;FI{2faEH zcv(H}W9nL00_)5wvWgn|oR0s+|FPJ0yn@TDcpb9>T|_*BCFOjo?A7r}u##>nH>Yfj zbg7U^=5LxthEt=uc*UoQri)(!x&$Sp;{nliJSaY0!V=M?N~(3Kk*FrMB<}f_8&;5& z1}1yyQYR74wFj(|dpMy$iy`%bHRV7TgK0C6F4xE+O-zMtu`buj5?!v7rGi!E z43wliPpx7p7SRdn<~rw)G3ayMj0zs&^6{`N({V_a>#~9mycus{0nKgYu6S;o6{KSr zZ&gdXUYC_}gD$I7=vK=b?yxlIvQ`>(X_9riG|M8@w_cYPxshB|ZW&IE=yH>6PzoD0 z*`&+O(yGa3UAD+p?iwD^q)nG?vR$yHoS|YCw2zRu;)(o#eO?w$NwU}6a<~+3=A5k9 zg)yA#=;m!Lp3jh%U&o` ze@sx{(Cv_{dj=O!kxLT%-UZOu4Cr3K>&I6b2Zy)mG(Jk)hTExgc&Zp``HXWXG8)z{ zc)Gxx*S$y@XKN*WcGAcn$HgS%oTdypX#!lO0s|PQeXsm`?QEXQh*<%dc>@ zDPpGi6)KYDO46AVNu^_l(}{uN{ktSerS9En!7B)Z%A_NWmIpj9?^p_jAD#hf+t%l8 z%mOZ*F2OYo-LCy|y1P|PS4>V3$rd*fC%Ie-g>-v$7ms6w$KKynZYvAHaW*`dQw`^b zbanxy?kflXDtmL4VYO;)ER(f=+O1PLS#GO~L{dyVpT!ZZr9}nKvhD;OnTZMBX8E&> z+0){zX#R*Dz4^Tg&TGD$SX|7C$?|&5ig6sxyCS)3%-K0X!&TSVKAB?8b7_9{QA2%I z5HwyrG>#hA-E>tKd(=mmz5L-%FGgZ2n@y&9pxM9o>@PCvWFq%o)l$h<9IW66qTT$q zR*l8#;>Olu^r4?0m-evjL4@*cF8Mmj11?$V#awb7EsXZIx!_3<83WA*NbE!6k&+yp7L7k4C0Z z)#fvO(Q4B>jhgd_PNQ}o_JYn;Jm(>H@f9_%|c`=o87c?PCH&2^l&Rnq(7 z8C*;6UejxPmm{iWd0*1zvox<$-ZfK;E;W5heVsDB%tNHh*ty&iS9l2b`f02@j~mpS zQ?f>lox!TPzJgnSld`kgUUt9?ILrR2YFUogv<0c$pxlPcP;_mH6?Qb5!D%!Vq7aI% zQ*!fMUk!deAEuM+I(l98k^2y6Z7U~rb_EsY<`*?q(JBqMmiNnVGqh zl{LVc8s;;@+PRzcat~|deXN5M%>A>>_e;#_=a|ROGk0Iauz2uR2_Y`)F(R9JDc*%q zc_Wf?5Mz?Zet8Jv@)+Nr;qRg|I3O2rP`-x=`3c`Y=Pmx1NP8-f@%S+5F_7~tL*8>e zraT*QmuCy^_H^Jd_7mwz^1x2CVjL;f$7ZzS01m=S`w^>EO{;dRhB&@~3C;@DbvMwm#S$0XjyL(Vk?=p-$H8h-+1DIkge5&GbC+?ze)l!uzR!h~Y z2lbxyj86^~I3cHrF~_+62B+?T&+&Rs(d#|*YWq4uU&`7rMv69$6m1-#jknXrH=)mx z{=6FAq2Zkxj%v79!*2pFW9SvEw53;Yn}&C3IQ9xw+dZ=kM0Kf-QnP2-Q#8Dr8XA5} z!+V6wc?m0KaRX{zvsQ_siNF;U72L>6&>dQPuSIJ2@u^7d83v*rkRLWSSzKl%Xs?r~ zeNBq5wp_wi{%fnOX!ROiv-09e^qQ51cf%!!O=Z()o5r?QP0`piwzv7&sCPO8&4$MM zzJSn>p_zUoa1OW3U`GY^9rLolo-S0lw<$_?D){nMv}&C0yLeJsOrN1yzWk;?x_lbH zGL2tt4Y0jb`3f2Vn^Ydhi|ixS6l{blT6IG=L&W-vlxm$}G@G@C;Zij$Rqq9?88U-L#H==>X5Bfo+w^iw=Wru& zk}HkCdF*teN$i)u)wqu2LZ0au;u+B?Fumu{Me3Q)H%IuVrML53e+LY_6Dx5P>v1o( zFtTM~a2i|uR>g7*%eSrjhX;l{heye76<8@>z+<#riF@TOc%1C!WxV#_Q|$9uJhz;) zY)!>vw&qFp+4Dbb?K8;3ky(bvXVC7AtL)S^iZpJa3LOM7rWS z$a6~RGBzq%dNn*Ti)&GLRUKa%{TJd1&jpnJIowg zc{_JUIUvV{)wPiHax1Cv9DZ%SrKen$qCELOOD#Ws51(@^t+MO@;WOCvEV|F*4bQ-% RN>^NKT2ATr+2_Gi{|{)gD0~0_ diff --git a/out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest$PartitionData.class b/out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest$PartitionData.class deleted file mode 100644 index 68e9bcde6b2dfd99d50ff4134ccb61f40597300b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 916 zcmbV~-%b-j6vn@4yDc4y0z#qwD=2E&L^mefX-u?@iA|*l*!$^rS_Znab$9A}dC`z~ z;RENzW+67hyj_MDkBXTI;ZGyCJ`x9Wd1-%8r8h zo|gi}csNY86Ih6}Qv>Ccz{+3Ehb}R8#Eq|vHSV>*dZXR`w_&FzP-w;jEpV%2tlpbO zeVrUB0%5^8RO6mX4A0Mt1$S(SY;j*D&M;;xGq1e2RwvD|N>feCtxlYb0yRXK=LI3~& diff --git a/out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest.class b/out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest.class deleted file mode 100644 index e09a391500291df330e407bdff978a1124d0de39..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 10053 zcmc&)4SbY$egFQF%RQIN6A}&*IG`FR(2xK@N}&xeZAbzHNJ4V~OlytD2?$Wyp zEpyIIu}-VbcGFthGK;p%+HSQ%0@BtxZELHp+BuzW^QG%_b(^|%o31L^_xC){-94Ay z734G5@Zq`t>+k=2`Tky>|J^fZKl>PfH8O1A3zhgHp3ujWK|F;o1tEAkh^z5T5S93H zB_Cfg@Kyc%s{o$W&#zTXxqsUGx;~!M$6p8VjUfI8-&7jk3gB;b@Y?}=Cy2kp-v{vz z_-+8-Gw}T&=HmxdxCGBv;UDor75)i7)W?qu{Br;=>h>Q8a9GaI6xI_*no)b>Y8j=RX4YxPHD=h5y8V>Epk3?&tdWg|hTN+Im^}_@#kg1#ty_ zpd-Hya?@FTOc{b8Hi?ieDGN%u`1DZ`6u%gT1cK<0N<)GHsWQY2Vw+S4WroyHXHpxK znGy-sKD$>B&<~XNHm?c(t;W7yZib& zd-?|>?b|xL+XO8;ld0jB=vXv%z-l=d-G4CJ5=)MbCKD}VsbnS@OOCW;4v$&smIxP& zM%N2wZtd&cy=$ycKbcno{axLio|zf; zTF=0&U2T2+UAojuAe0;REUjo8IEDDiK^ivS7P43^HwlWT)Y8)BHGx3p@?&uhquZ+Zp6Vc3gic8;CIJ==Fy!w-4 z@mLP5>r1XnSvQPZ=}fw%*XE&};EZSboWaGp{mR<lDqmsqI4{m z@NMVh_WxOvs+)C1nCdz&t=@YfJ6{6{FF!v%w-*7=-LdK{p(&*SkcWPTJY+bljZ@&rSw9=eP~dpq=Tu5*cgQO8uY3 zt~Sqs%4%t_OwY}Z>NNh{>Y-7T>7t?Vwe!i};M@Z6IdW%_b5 ztspRJWuilN30Iq&*Im8)t}m>&IAGa+Q(?oJ$vo!{_F8}1KF5&xtR?9W!&asy*onOkMG~l@Yi)7 zI4N=vG2-86Q^^zGu+H@V;B2_`PzOkK4i!eQ{lvlA*~V8>d6hvT<&M)lQn4wN$dlO=*!;hP=U))p9ALt0`;bGE>&d zx`Iu)3`JAhQ7diA<#L57t+LLP^}4z`e*{+xHeZOI3ab=aI~P*ZCXP{SvOzYQII8;l zIQB9ln6gQ(H03I3Gi9^1o6;ejrfk7UQ?|-BQ@Sugcg^#*Et)<+{!G4j?N9^n)JL~q zX(@IKBZjW-F{M{_8FIBLeG)OHUv?X^N0m=_Tvm!s33xcMUCF*tu*l}qsaZxwtl{WL zTWXl^eu+%yO)+asU-Sg6dz?Vsuw}txOyhVA68nVxnm<*Xr3Ht@B z&MDorL_?V7b+4#nPRtKnJVDRDK!YcYdujompmIwagg)twz2pQ>oBY5YWJeR zxi%0ql}wY<_cG*_soO|I(0tAVcj*PpZoX#gnXZ|gTNuHmja{!2c5~-WySn)eN*d?3 zMKn;N-uT&)mU4r9DY+h$;MCHP_mqO)HSihiwNHG`JGjL5?2IswfXM?K2?S zrA|cUEll7X;`q|`kppiaZeO&; zn7Z1FDEv29ub)LC!LG61%3j|E$ddg6$#h-p5OP@_9U{65X0sONtdU8&Z&r1U%ZfT- zB~Qz`>C$3nc%b8~qpFlw!cK6>Idp=Ci`o@wIuN1sXca)EfQ}BXY3|HZv4OtVsa)>%RXB4vE6&Qg`hJ8=A zxXP>Sd>wxtP;&^JmlSV)!Y(^kB*I4JyE^mtRCU@Z93y7L2}2yM!x(Mdo)URuJ8)%g0F~uy@%%@ zTHq1xsG6^2c$yye+$z|YvZ8dbWiBhz%4P4-xzI2>Hl9i`KJ_^_rrG?lTEq4^ZC{W+ zu#n$0T+MHC>M&0$MYiT)5BBn#jW@Dg#@;*(c={psukrMC-?g6pBKF_p>Ca*RI!}Kw z`)~I2m$5(S={K?e7EiyK{ivtEiv4};m%IF}K#Y6>L;UY2*A_p+*pK0y1P9oUdni<~ ze?1Q7=BwEsVLwZ;x(LOPmh+?>(vsZ%WQlHnvO2dvS)h|*n(Z=VFoW%J9D)xwa^w@J zAa&sKrZbR%hf#I{<>jZ~n?%JV{7ucvCt)--pG2Uk>?DFsp{kQGn`Wc>Bx;(LpG57a zxwjlQ@wo^AJ_l*7D8B&PhX(Sxgt9kN?$wlf9l1Kp6%EdL_#M2JJ)Ugy&vy#8l+!Z& zF54!#bVf29arEd&@U{Ydsw}_f%JLn`Qo&t)9$9=IS;CacXgYzJq0oX;m^Fdfd+lD` z3cG1bxj>~{;Sn|0!jr`h{Q!Dh@8t{2Epd%T|=heX5U!j5DCmprsPr1_I=YXq$`m7nl{#O-U zbuy=(x|&-|6ZKRI;z92woZdraycZYa7Ocekuo3UaW_$o!@kiK+4^pIC z=?=Hq%Gg9VAaglo4CItCkWr z0a>{G6v92Wh37S^TxT$UU=j;jeJf7mBEf0AjxQk<;fm9Ez2FW!udN2b<5<}0x3?{t zz{Rab*a$5S`%hzuz)K~dV@q}UB_48>;Yyd>joNa_1;asiZDVT{=`B;jX4nih>4Zy| z-R5xB36O2u4D1t_EAPN8ZRt!)erA>S-=O@217Y(tR+nM#@u_zy?Qq~UE_HYf`@%+c zZk{7j*q57C-}TwPdkyN@{t&TxJ8JO}MxBph1MZ{|?_$Kgn_hjC=KWL7evJR_!C&wl z_H&FKPvc&E8~5RdIF6t34&WtxQp#|@nD~^`-~nmCgSO$v2$tJv#&_ZfTOY$NybCwe z$8TY5csBtQD4gV5b4@Zi-9yF8dbM zGn;~MY-+ZHb7_XAE>>{25$9pJ}D*H;3pB>${gfwyAurug-VTF`Qan=Uc4~^f;u|ANGe9 zlREvyulQcy8VCm-TU%MaHdq&|t2~Ay;Xqw*HQ`^S%Wd#aKv`W|t;lx{UpIs8RZ6Bd z2mggmOL$WjrVR%?Lf=8xqDhWfh*5wRloC z&|fq!9ASuhnWypw2qCU4r*|E}Owt;qf2eOf%RG7yGekA%?$$dahT0bxQh8m3TGE{3 z?s$YeeS&^iMV^k?KIA87chD>M5-#t@C+VB_Y8H}>IUm}n_jLr7BK#dNrzm!r4Z%AZ z5$Y=D`p;sHL0^_BEJu|yWA_c*Zv&m^=d6PFI@Fe&cUFs!-U9pzrh&Vrm^B@=(_L&5 zzaVddPvHRvSyJPy} zQ_s!6pX8XJoIlOD^RoZXpkrXgBsw2DgDnH0t^P-`ZJ>NUi-X93uW15Z6WAUZsBp@n zQ`kYOkyF??iSEylf)hKJvPuPi{S&}|s0}8P*cJ#$j7AVQ zO}Egbdy_7qX&TznrgTf2ki^EJrAhauTe_x8+WdijX+QPphtl`XjK(8j1@h#9XEgWR zd){;IIqy05j$i!Wa~A;Ih*xEpLA0TwK*BKv8*p5~-I$W`P6dA4(~1b*)rx!Z?pC}9 z?-h^t$#}m|xlhK406riT?+@T$D^xrn1|Jk}ACmFm06r4HodJBb6(7UL#p4q~??D+4 zDYy#vh{c~2k57rmrxkn#4~w^x3O`Fbt_T=zzdU!OE-j?)8RYNbAXhh$p znVN(JwigvMmyZuve1c_laXqg?OpVG(v&Kg`h;%GpSixe?KI{=eFfOj7TG`G!64Bt^}wlt+GSVh zqA;+xvF00v4rNR&dzfBZ(PVpK6O!>&8E4qC*ytn-)KR)|gf3qxq8c{F8nR>l5DTS6 zE~TqvAx&MG1S!R_ZWu+wt~DtmVX-@3)k%*Xs%=gb=vK*uEZ$Yuv0|(-*MPPGCodfcQg#?~}Ccvi(UW@?n&ic~QUo^dxd za_coiMS2-dn<}d~hglWRVMfJyd{@QygvrpHpEOAqT>6c=nv9-B($OqW1eTeK3;4c@ z=kbDyP1sB-&4KLD%98|A@dNx&#gFh~6+gjGW&BLV&+(#)U*MNAex>5q_>F|MjW$(V zlOk>}srW5^r{eecgN#3__!C~1@n;o(!CzJU4R^_SMaAFo4+)zafx40E8)GqOS$1%% zt}NeNfiwL{k4+}4;c)_y>ohf^7y+pcX56mi-^ z6l+Z&=LTd+P1mRfiaPS7Rvy&vCL)XQb{@0^Yp}Sw-V-h1K+d{pWm%vJ$6fDTtZ}!P zzX9`pGohP1b)NpsJhUb3ng{gKOul|I0h)1cht1l-gKSD(Lk>a2_GPo^Tni$xj(JHW zpqwSwdO@7?a(<;Cm8$$&JD-+klMav_8(uLa%xlM^QN4#$tK3S7BeZhn?mgDQ<4o)z zBW{=y)?CgP>mte3q6)`Z78j;Z1@DQ!nWr;QA=-9f;8XFRyO-?h3WGjZj)==$|_BWL1WT`7J6}y+= z8h+Ak;Mb^(<>K7r+j3lw8~7!-k?&1d!0}r><3$|b=ov5J_$JSIDaT!&@iK(PfkeEU zExc~D@Vy7U#C{tM48n(2&iuRcJfwYR(KLhR8Ti(nL(9N<`1j2~9$Y_*fP`5DC3O3u zzF8;|9>EjhtySWgr*^k=K7&wr!88_j`=kEwqG-!3!V*vL_B{?rw=86f#qx*;E)Wek za0&6|zy*o1M1Uz#CA>5mbip`WCWgcj*ikvE%p%%^J!dY?3Z1Ati{kEqDn#c;Ke+3U0+eu@nD7Ked265ajw8wfmq@`v&SKDNEr&YWDHZ3mX5_a0|6% zT6&c7Bs0T25N9WU#habCcDzc=&CJ&dyo>>AG}B{H%kSW(iEE<`xoEdJXq&hvp3fG> zu!}K7j0`b`;JZjCTV#yL*k%3SEn^RGktTw5QW<+??2}=y_%Biy$u*yM5fN>c-EMJg zKcD27bdtU-;M}Rsb)*)Fb`@#0k||gxD6?u{{e>;fUA~BKI*6>RQOB1$B98mVX?qHOGLjkb#AYE%s`oyOJExTY(__hr*~^EBSl z)yDL%IFHr)qH-iOgEf)1b67izYn!p>gd&}}IDNv$=Y4_?&5SBoKOsKT^m+&jaT_`? z%$m$_ZzJTxE{tL~4l_!}IeLh)lU(x{_Pco>W$oU<9IITl7I)$R@s2ZAqqvJ4+e;h` zW30OCF@dbbg`2tNAUPt_RslL^1s6`aTsS3eyo{#ESxV$=rIRzJV?vTdzSg7FN`)I;k^!O_VgeXLkeD$9C<9T{N?(Qtj2ULgn@3FA z+D&Ws+Qp@|HLZ3@Tf4YK2vVc1wYIieyVve^v9_lBW^I-J_s*N)4a_vg@_lo8cRBw( z_niNnd)|BGg?o1cuy%byw?wb>-}iK4Sw8+o8;L0G~DdN`{m3BG?nAo|DcSq9Y(695 zKP#KhNwGU*bEjLgAXW$+U<9;l|EgB{?94VQ0{(PA9<6cb3=JS3m$9+<( zqkh!kehm-!@C6NDlt2&qaW=l>$3ysXDIUgGWb;)&9>LfA_&Of-<1u{0kF)T&9Q&r9 z>hSR2!nbAjI~u+#P|@1k-QCvF(;sQw-nO$vAk?0;hC;@u5gj%|dyK(7Mktyb8A&EW zqgK*RMw9W7ePGl~g(6%qVl)XXxS+eUx2r#Ve*ccPzW$Duooxbv_A88iMksD1hC&hB ziY11)u)H;yNZCfh-ettcOo4elon7J9h)b+OjkpFXyIQ(?!ad>64)?M^cGR_08?usP zqu~Jo|EOWvF+0ZH1m>qT8M9;YPURlvT9X6v;O()5*)cY<+q8O&-Eoct+LKWuzRR#;vhNI* zXU}oIVH+pn8rvF+%9EBUgqWR7NV`z33|_@ z9ZTBy#1TE5NSIb@+(@P5aJB20JB`t$Y86pE_Hc~T>kFFGGWU*|DLWPF92`uU_BPXw z4tJ-=DDGEErP5l>%iSY8?dZ5L*LEbQW4{)coOFF*-_E9X+J4f-9WIRQ`I^&%Zg#tt zrMH^lf5z0C6=$})Rub_4fhv#KMsyEB)Onf9G<=UKNyGOw{D3)&nNOhe_|zKRE;R$9}&H=?#f-3%s~ zQ`kzgtV%=dGp!WUl?+lr!#F{u_%)*qW6yGImyV0_u1q#c_ph2wtwNc6mIB8=WloHy z-M2Jnb4Bf{x_XxxvuQt&8FuI3<~f5Wz(Ts4RFM?g>99KyONO={u+6PwgM-X!;TPe# zoHMChM5lRoD}byEX5k97=il(9?&oWc#+^H)xK752Kaw1?qUN@kycJHGtK({!R|HlS za&6uS)Ug$>9vCpG*+$1m|K<|ZA###1_egQs)kviX&MP+^?X@mu^($1`|Z$1%CO zVrCb)0rSe7JgejP_=Ao=;!iqupiM_RcIx;u{-Wcr_?wQu;~zTyiDw1QEOhYP_>iag zS7s-5!0kqAn92%lErgKYbonAjVBJdr5nwU*oQ{9vKRTYr3sT8xo#{jHNfZg3?lMd~ zZVnmo7Hep1g!g&d{-`-Bt0PuFyV8lLcG*B}ELEFG+O_+Pcx+&GElaf6K&_LPQniCg zt9Dtc*09XlHMMq<>zPPrc1WjXT@(wCE=q(~6Phl3xK4$C8 zGVG*97GCPgT-i(J1a7)?F)SmvU}#F}q(ZGpdLt^$-z)p_K9Y5-%-~xODB=m6<|G+K zHpz3|L^E3R6wFk`ESdy*3*PSP@^DU4I6LJ{7ex2Ty6-?lZm}RIQ*(&ZE}N<~zF5|h z3v-rdEKC_~*hqDl`-#X*cL^)RgzDs88d=l(V(L{P$}HHL=Zq(QA}_D~9VQD{A0M0- zkYZziJg#=*XDmhqH9P54nALUldHErsoF&WMAkNj7U#OHq8E&4LPs+u_>>Vn#XB{b! zcORETrr6{;W7!ui8l2_T!aSRWC1aNtinA;`RJ}y5>dXVK^s&W_I8&&+2^Jb)+b>-5*v60cZ*|6Lb%ZEj>l;hJPoR2nsPuj-kA~k-2tG|Hr+g<%i_QS4zko^l?{Y41KT8UTz zSuPQ`L*aH3S+KHr8TaVsM!oPr=ghH&DTuziP&9$!33yf>K}ko$VR!?+72_zqXbS$m z36wQ@R!xF6$t22IYLo;^CQ%`9Gj5Pi^8_Bi{6?=5TQH75qZZTxl|k<$ss!99J~>t` zmoJ>eNkl3QmO4lw`J|9eZuC<;C~?YyWr0OOzk~?-f@PDaDZ)jErXQ8_LEj`;S*STr zP|M5>rL_opva<~#+nI_-*bST5=1m`2oB;1c@9$Gxt`rMMhk${j;L$K~HKeOR9-$sb~e~ZfaOThLLZIQuN1)0NHZt&Xq?u7jN)X zdluh@_{wU}TFI{mAR4_vZ=iAFArrB|gUmTi#_!l;=Ce8eU-BrA#)D>k5< z&ta;2A-@UlKs`OMhW6gXV_i&(_A`F=FoIsk{BXG^4cvK+WczRl5i(pO>WtHq9~>3~-W za$_1RQKpQa9?|e>X7<_OkB`aBKeSU-*pk-n0B4RglTh(m9tNc_XPI-7p`X=TWKRWaK%ltC7ic2T7n9ju8LLla|nm zi|^n=k>ZnA4UQ;fBqSF&D)8$A-o#Z5j~wA8vxWEO5l+Y0n^B6l;H`v}AQuo?+P!iL z8nd~yWKF{noI8QdvmwV6y%C$4jlmMI*dJmcgoccVxKo^ToJbgJ|&juqkE3=4UI718boahbQ2sjm*#AoyR_ C?J3a! diff --git a/out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse$PartitionData.class b/out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse$PartitionData.class deleted file mode 100644 index 692588d5234d90d1e547fff4e465ca245735654b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 932 zcmbtS%W@Jy6g>?C12{1Oj7faOM>H^+Oj(tkmK6hunj&C{m4&-z(!d0m#-7ICap}g5 zmS~j=KfsT&yxme|`M`~v`{=&+oO@6A*KeP{0NBBH0XbaG!^IV2){VKE$2D9pkh$UF zri)t)S=sF-LNl;KhTKSKe!?|pC^b&`D;^Aa+zZ-TMRD&QLtZE)RZX4M?`!qk6Fs#iBVGb%c7(J45O(HDt(#$6Q4wpB*~7A0-S6Pr1^Oj$~Z7 z!+1~PSg6{NCyAizdPAySz{fo7i(tUJ10IBOG?H%Qz7m zzkt#&S|!q`wZ2& z68_dfIys?zaAKk--iuC9>yj(JmZ3B-lLys^J{!DAS!!rvyOx3V9P5VrC;lowku7 zcj$B#YcuM{b`+rOe}MBInUBc6hU>h6lYUDrX&BK5bgH&;s5za+Q QQn*BO!g9OVpgGpQ0~-9)a{vGU diff --git a/out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse.class b/out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse.class deleted file mode 100644 index 9d205529d0d129ae766fe3ae822a3137f2a387de..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6111 zcmc&&33yyp75;BBnK#Qz+Dw|XL(66tk}eEZ43L&;noeNaBoi}fYYG}(rZ35~lbPYo z8=wMGwJOR|K~WJ!txMgdr6dj20xF`YxC`!zyST52O8D=4ZzgX-rUgFGe*IqVJ@=mf zo_o%J&pmVRfBep)0L~POf@}P!!?ghjTo=GuxITcrc$0z~0`TI-AR6)JAl`zTf_N+5 zCO>aiaI;i-hk{#tc&Ai+mk*Z*q2gBAdADr6N5Oj)?DOG$J{Ufjel+3z0ek=-^r0X- z`+fM3{QKbmK7x-1@iBZ{em)_QKdIoh0G8lJ33ot#4$9B%0UW{|vNbM|CS>cdRG$nG z=uQPk0*K&K(&SGEa1?h1h;_XmcjF%U`Ah(x#pe{@y^afwBI#Q-yQ95 z-_YLQ?vgb``}*R2gB|e=QJ1tX*0ZI(E4IPG5D*({rksBQ!JhbF5z2?p({qNN&v65~ z?Z`Ry4hv=~9qHC`+@3#?8p&v8!Qj@eEFWE4(X0L0Tq;?L(VB|$4E^GQo;UN6`0#LE zH#>DRIojtKw4|gjO)lHm8~JL;n}d1l*pCS^QqI%Gzv+RJFP1rL^GzbOQ6R2+gQHw zC6-u*$&@a)gE+0+ZWtPs=em@-QhB+6q7yeI4~xefP4Q3bGuk?l0@liTduhAlEZbD> zcOEdu_ZgQxWSPB*wYJbYT(OL)kLbp;Mx1ung3L>J zmrC`e<5qG6%^iSF89MzOLXK zD!z$t@t~;qHol|cK|EBZRLVChuEbV>Gb@>OEJu;DhI6;(ReTrUQ}KN~q~Ztop^6{L z6>F!>_#J+);14SPh(D?LGybCDulSpa zzvCY&{)tDWXI=@OpEM` z;Atg3y9k!&q$2+%m-CU1Y&y+s#X5Ym>~q}_BU{MHT44Py%Z~!5%*gpX=F?kai#J%86b9$8^NIvY8Qj z;1cZqrj}|xHO9OWigXjesP2r8;V{f zd3d!eG{$!7lH-`-_|~7MkDb5}Vk0r-erz`{Jq||E?}5CWKaIlrVSe*Q*DjoWC*(nG zngui_2dZ-gI^=BkuoFEOQZb%S79Ar6%;dh6HF(ktT_aywoE2x>#cqO?5?FJk)LB}h zJ(uEfVuDefpT?ch{*LnnmRBqpW~!3NRVmuF;<#(|$|nIQpEV{6hQaJq%+>b$fJ~;b zr+!-Pbc$2?nsF8%ZGt#K)(LE#fVDW6$$uT&RhYy6i(LJB_FwGk&u72g)nCB=dRITp zeut~Si2V(&{z(WUiq6t{=dmAijc-Iq-mV<)w(RLa4cl?_(zch-!0X}R!w|%xhp|vD#Qq}LKhf!&Bs(V`#wk``2?r-}Y8AHbpV})C!^$L{XIr=`>@5y0 zwfC{AG`4)j9awyTp7WrO-)(%-0=-#>9yHL03+cmk9El-;m!cmUHsd;Mp?9`oA0-Dc zfP1L*04`uLb|K4|m*Ek-9FJj;b;d=)3r#d(NGwE>2*5A`)QuBqGh6!@*DXW}67vG0 zZG)dH-O2V=z5;3-zmRweEx4DM+Zb8K5rephNU~+ey+Qanw~O&!Cw6uX^cP(E*YLci+NXw-R%nC`lW!)=}uB&eKGefJGpwYh%JHo!kz-q}XIb98_p#|e; z9!E=SE!*L7EFZ^;v+CGhBw^;Vand+ej$>78{Sh<`G}VRYHr7wz^tKR*v-k)i17W4H zb^>RFLz8%ZHMZ^vi2YMH?eXwCBxzJ_(P%y9vMpUa!qzBSkir?*fpf5vVNNrIV{Bz` zDYDp&9InO1xCI6dQhJ#4kFvOyba@4j?jTzilMpJ$&f}asYN@vdti?&w-B>LPajtSL zmdOGf^+cLaT5V=xkd(R%yGXDza5d2+tZx8Jjx{m+4?FwC-J$MKfVbsv&!Pz*r>9+ z=Cy>ESvyzl3651ZXgM!FDmZWVJy9nNMxR{8cl;PpQN D7@Dyx diff --git a/out/production/clients/org/apache/kafka/common/requests/ProduceRequest.class b/out/production/clients/org/apache/kafka/common/requests/ProduceRequest.class deleted file mode 100644 index c63fe22c77164285a97583c7e1755299e702c4d4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7363 zcmb_g349dQ9sXXjo7vq=0xSt(Ib=lwNgyouksumEfCxF74S^_a95zd`knG0USrl5W zwY1jWt+rNetF;!|YFha&dp#-0hy-yb7Q@Fnv58$|LPAGWLgOhU4um?wqQHBw@c2xFGDR@Z1X&>g` zVGkbh;Cc_v6ys4mCYw)7!H>)4GqU-tg3tM|5Tg?7^RoGZY`!Sf_>$~Bq2Ng$YVl>M z##3_UD^lODD)^cYU&l9O^GyZca*x@6KCHmEeOQg}DEO`i-&63k4;%1(AAW!zdhsJX zBby)l@Du#hho9l+KGfkCa`;Oh6{2^3gK;9w#i8qyO+B9a&g8Ha|V$xxUJ2DN&Dl7`0Z;hybH-94=h zElmRc9s9L|T4+Fv_l3fS9*g(MMU9Dg($L~Ymo|`!3QX;6ZfR=U+3DCfy|b;oxv^(+ zLuZ3yw$z#(hp%XF=;&NTY4M$CvM;vLaQLht3GDV9AcBQ_9f(S~S@HyrDWYeq_^A+6a@H{^9+XJRN8 zX-}iqTifEXL}=3?Bf2TIZ(mfe-$?MH4Y7F4*eFm|6>iQ3udnVBa5W}+DR;__SUlRA z8r&PzJGH$7obm5SM6`h}O^?a3Jz2EZL>4H|DUfi|;PVN)p{F7S^(@VJswr;hhXmaA zui6}sNA<=5Et!nceYW7V`xcn1q!SJ4kjd8jFc?8(3T{{deTHkf$S z-t;WXvsp`@Q}8@1E-Se}coOOJ`|8{>O6JUps=RE;?QpqiT87{q(c}=z7jw*;nPcIw z*~CoE%%)ei?cJYU2ssfo)5dL5PC(62&sd@*{>(AmZ@G}o!7cdE;Q_=04-wJ$&kim$h`-ndXn|7xtA4?CffNrL4-iDZhL{@ z6X?t)N;)UED*C)VT_DpM3S65l>fHH|PnnEUG6lj4?}ACf&4OH5RUH;!%j02M zRdu#DbYT0$xpUXhJScOV0d5jX(yh75k~U#U^Em`OW1CqFsphuaSpN%@XG3JbKBVXt zriw3|Na>O2mY6(;re}*@DvcDV%@>WF38PTS6suICL`5fds_4Ql6}!<*Vva*^(~|v!S4F9qqKc_vnkxKax+=Oby1;;nIfQySz@*-=HO<5P5CxYc7DFJQp8+UTp;GDA}A4;oiCzP>_XYh z7npAvZXLq|1JOQhph53T4f3+kbWJ2WByVy8bzN4C3s&|9W65AVVFV9q1F_zvL0(5= zy+OMKCWHGDdayDX)bwa@SANAN)^?jMiq;(QxQv4 zQ7e|RfU6=TmPs9#t73&%sfbmoSS{A5Vy#$rE+$VX*u1=qM_3Y-$IJLcvk}!bBcYR- zS6CIpbV%mVahPBoV(U#VS_eaYfTtKxS?-c9>Pr#pm>x$Go+MrFPk82d9Yas0ij7Pb?G3 z8Cl0lI>Owe^lQo1=ru%S@x7dq;^v?%y*@kio|yUG6`@LvIoxyp&M4x&WP6l%bPqfB z+nl#IH8c>5Xa=QJr1i+IYcfb$FVm^qsfEo@X@+7X?8|?7Rdr4lOM&*1MNSa=W}17G zmz&a@Uzlr`BZxK--@&+v5f#WO$FXBP?DTWm8&XgCQpW_S&M$G2duHTVYjUV?NWPt} zb3V5H=FlYV$;nezr0h(pbfTmk=h%}@C|^=$o*mvr`3iIv+aIdlywJh%qM?zpR*c`SHcBswD$V zw4`C5X;!HTleF~xphl*wcY576B}YOz%U8zWtn5tlgDzIh8>#*FATN{6%@%7W(D?l8 zN!~Lh<1FX*CR(dUHM%UFS(D_C*W}FcR~AX>It#I_zV+ow9)bVmBAruuJuc)ov~B$I zSd3Znipt(BY{w3Mw`*a)03ME89b>t!%`qNVa%xW1U&wVfN}~m`3coV8$qDXE9Yj(<+B}dzJSk$+bVlDTydb-UOQ(PbL&(> zUO;UBJUMUS`hx+fY+Xsf8z?!23k$I8*w{(AFyK7}-VUYgfIHwxKfBCQG2qTTt(koi z7uC!;i3K&aCsFwTz34&@+r5~{b_Kn+l3#sSVLrdPF2fpJinZv)IwY_j*YVrxji|>7 zet~@)7vm{x#M8J0&v4bVXu$K>gtKTAE^HPGn#4?O5p%Is%*Qsdi2S*d5)MP<-i6R8 zQRPk_BE&Bt=4wRIPo^dC9>h53p{#4M9|suYUfhKNK2>CFF^D*kWKVKKm_5mk0<$Oi zQHV0W03_IRnYs-T#f_`DX(NhkSvqA|5|SAShAm+?uE=1dG8jo>93(~;Tw|oYqTp&l zL$5La4=J<_a=v)dynXmNrvr|{YFn#CkC zX}z#}1dABD#ePz+s%6QUjfHDm<*vDR7}zH;x~XbgDDeJkonJGop9sQ!7ws2RrMH6`o^hq0utl=P`QjoR*jQeHBGrGe5@ z2o+-25uZ3V_V5uG+nZ_@`#r;0HjL$xSd0C|76=I@xm9R#Yolqw4qDQJsdP{^Ys50v zh4rijyP5wz%=rV1rNL;vg>&!Xnp^qg9>(thdgV$Y?O}|1umrCq7gVke;(A8BgnKIR zI&!I$>sR6iyq-~AMs`UC6;Z-&9AQ0CD6=1L;8VBsWtu_k@eP}vmQih%QJo7n!(s;O zMnPtcQKXD#dPc#U%CB66hg;ll$FNm;*HIQ+ zvgVjoX1VOyWx-u5vzPgQ8Y{Y&jAG?5R+VAR2-aD#I5zg=?0X4MLXj=9!$1FNtnRKI z!FtN7>-N_dJ%SD0g{*vPxZA}TUp$PB;coYFT=F0;9Yq66Vt5prM$q^ucUuKvDIE#A zT%r5DbnrAfYAsW=3;P(Bex~Doq}bj-XWq%*E4g|VZC{IdW~xd@AI&)WDBj9&N&SxE zZFoCVbsE$B9cHS^w4A{iYv((OVLf{{=_3O(4Q}s?s4%na@L8<&dkEBtn=J<;#9s;Q zmJYs;odT0|MdqY#>ZMu>Y;6Q>xP>!JudK}rCT>2MbMtO0Zk{$-n=9|1Ao*~P%~pcH zm;SsJ@3S@7N_$zjF#R+Xo>(#>vuj4Nc?3-pA+IwbrK`Bytf1_$XSr5_-e!YtU=XEG zNCXL5z?g4gn&-g0!J($##`NHJ++ibCnm9m7nk-vSpg^X!aCkXLmEO;}0(<~{QY?FA QWaPdgFOQVfOz6OW0O4cJdH?_b diff --git a/out/production/clients/org/apache/kafka/common/requests/ProduceResponse$PartitionResponse.class b/out/production/clients/org/apache/kafka/common/requests/ProduceResponse$PartitionResponse.class deleted file mode 100644 index 786cb1307fa43b9fad216fb6872aa1f516027883..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1077 zcmbtSU2hUW6g>m9e7Fdu*ov)+SnC39x7u10F+LR2q=eW;Oq)IqFu=C3LuVIXjK9D? zqlu-7CjJ0_l=03&)1)LM#+P&Fo_p@c+?n5he*6Nkjh8XpMlOmt9&1<$<4Fu5Ft8d!tj=9=&}zBDBcj4E``UJF?+wF@S*zra z7(yd)qG~(h-Jn+&>I1KLDLqw{4cfWtcnU%CkLNI`4(smySz%W8ZOv zDtEc-3X;67O4YV_pEo+fI^(S~ZZ%}DCml=(W5=a`D zLyBSXW}9&}1JCfBA?z>tRaU!O8n$WiYtrBHh<#yT8TZ53Vc7luH83Qv2Rf*qiUvJq z(A2Lt%lm!dG#Qr7^5k}|m#}V5Y^XF%zijSL4_BszH~;m{w8J?4dmsR&-+@l_$&eZ| z^GK6_hrAO6>AM4o!X;Q?=>ma5_5#6gzHoss<%b1f3sJPGXa|eLXDN}=a2HEt^+~s3 z{z*5|jp@n8ZN88?K(W$KgibCoQ!OSx4KaIMOofMt92X1U5f~y$e(cH*4sq+anEdR^ zhA_TTdH{N)93q5W!X7nsh?97TK2~uK3ko}M{Sg;NEyp89u!RStW5j0tT90s#Y>b#V I){#NxFDyU(yZ`_I diff --git a/out/production/clients/org/apache/kafka/common/requests/ProduceResponse.class b/out/production/clients/org/apache/kafka/common/requests/ProduceResponse.class deleted file mode 100644 index f1e1efeb39c1d22903e436fdc4cab6ecd3116e01..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6010 zcmc&&3wRt=75;CsyR$nxN!o4GvMp39r6hf@wPH$>T1}I!-8Rj}Y}!%~*2#8~Zkz0e z*;%A$1*PHxR1jbI07S5gBG8tkp;|!&1r^^4;`^=mMiEen|DBoLOtx7H-v`>Sy}hq< z?z!jx&pCJJ{-^Fd3g8S8RPfdi8t^t10+*{;g)3CN9q&-^&M@ZUN)-WIC427*<7&J+ zjQ8NZvU#6^_e;5J6nr3v4O04pLF@^m4j+<(4@<#ov0n~8uHqB8IgC%@Q?j{5s{OQrTU9K=wNlq@vbkM02UHxyA=$e_Y8aKhF)4mn z!MKXkai>a!N2I-yWOKKQ&*KXUz9=xWM>nmkmCfgRjlxJiS1<&$ zL@sBT9YcDdU=#%EJGS)p#=HB{sg4ct&FuoQuDm%I(?|5oP9wHkAK0zOGWp@*d@eR( z=B<1tKNPd}j2MMjiW`RYHi1ZQJk^uzPQ_E{P4VsN?)J@bPIm3mFVJH{dTub5vdnC5 zkUQ%-XoIEatgZS`k+#(LC3_Mb9;p+1+I#yFeTigudRnEE%abR~iud*=d($1s&bUWY zeWH77dsm_}om{^@74H)e8~qdRAA#BH+Ea0N$s-gpT{qD&4bBCMR(2@1Ss$VGp;UG- zr&~pnmhYXqx~8hr`tl>$Oc{o@s{2gi{Gw5?3b7tDzoVEjN`NgY%a&%^*3y>1nrtp> zt)-uvrdrb0yj8&0k>5ch8@jSNqq{iVZ8SgK65hJ0=!tg6t^iG&spXk0l)iy8L1K5J)|Ft2>px2BGU5>M9cvGL`t%T`e5oT*-Bw78zmdCVcFoI9*>ChbT` z=%HVeJkW{<}~v6n)2G|I$N3g}QdJeg4wj6TCO z^QMz-M2dMSKQK@*82L5Uh(PrtE3V11Bo&uZ+|=@s?B8W%Wau{7LFP5bZ_lu>lvkW0 z7bw{LRq!PRUsiA*OJP&9?Mu)92j64Xs!xb zwQ|8*1^R7a0XJh-=oAR9$qc#Gqqa$+tfulsGh?jJ$`UcVq5z#P>$t%3XZk9z9j~Dk zZ5q08zifsTd_}`o@il>2_N%}ys&YH%|IqLl{;A<{JRz{S z+6AYEsPxg3viX;Wf8#$2p3*QOpa`Lf8d0kWA1@&Mng|F*;LK`D%ZRl1Geq?aL*9P( zOh$hFKnd1v!a>}U;&ngo-Ph1%zRrxQu zT!?k#hlZG>_|o1Y$2@n?%oj)Gv!wh+IJu$-FuW(PM%Wm;zC!jQ#vsFz!1|gJ^5mfX zU`ffOIyH%LiXtAu+Ixc8vXkaa$!{g+QAXLR7rKoLsgWn0&69HWV0LN0$AjD*%-WCk z3=QjWuV8O*{P{FVYWlju3^iz2n+!hBYVA_7Wg4Z&qFX%|Ry!uhW85Da4IUQk%*)rV za)JB_k{n>dYf8C5IJc7PytOK0FzHq^hm`huS3WmL{xxm%3du>z)apBSFz>sagUpAJION zWehRXC_TN&&Mh5rfhAR+3zbsjYE70VEj{irJ@QAto6{OIMbl(9DkW>@3xG_hiDw?2 zPD;f>ezRD`k24|6lWzs~=3xz9%1<3@*{{J2j$h^(M>u}DXB_3Y-7}uY@jB0VKBDMA zXIWpI>#y*vFF-_=Pp)sY&)Wn)`(4<~v%6`)Mesobgtp`e#P$QI8AI(Dd`l0*zxfCP z+n0^A_F)XcZuUX~<4^@!eNo>y!UEUeZrRfW?m}Iwf5{=#M`ny-W@{iCh(x0Pam*4J z#q4cT)fgIE6*)IYZlCK>7mNm7b=zdmRTqke+`T8Xs?-;y;>bKX=_rlNkE(}o;$fV` z6Cw-b;01?qG6${Us1gm2V_^-p?Vq?w%0`uOywI^_c{EU3YI7`JUS3PBvTZ>os9%>zcO1af|C3@L=fd5kHgZwZT#};fOW0JTLXK^mTbFRR4oDGFK zZ^b!W^-Ij~Dx6DAvgcqt3s2Cl1hHL+N06pOEndwHYpKK0t5dI|t-q`-<7(T0P2`57 zV@Fv>KXn+?(F5NEKg#(P3@8}1|L#3@gU*qFQX*r;7a(*YESO z9hSVQwRy9aQ4hwLpLd$UnPz%#1(9A&tj{9OX`;HDD8GgbyOd~N$=Ur}yM=oX5?{%W z5pp3-Tyxy3arYYJIaf!S74j8LPF~7yss%Elf%0=YOuqV&6@2fc*T&o;zTtc?K0dM3Axj_&3>^+eqXppIj zh^-`;V|a0;#tS_fBlOcgN(b>KT;^&z)z$;l9l#lf@RGx5IfxnwpYWKqfJVOAHuWt8 V4y0ubzehJP4MYjUTx<=5{tKahpK|~J diff --git a/out/production/clients/org/apache/kafka/common/requests/RequestHeader.class b/out/production/clients/org/apache/kafka/common/requests/RequestHeader.class deleted file mode 100644 index 81e838af4724bbc9a6ac2ba5b58f5335496a6e5a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3097 zcmbtW>r)d~6#uOW*(5AdQb7u$_N5^Ru~u6xXyp+kH3BAx+E(Zi7FY?{xY^Ls_xo%A zk$z}x2~M5q^m9M=@9Fe+?uIv+;y9VP_ndq0d7tyk{{8PCe*(COAEP*p(FodcLzb~9 zVn|2v9&QSok>!>w9#Q3I?=HW~_H9j?)9@#uO zHawBdrZR(5@@o4yWR~ok)|v(_ry)va+H%XbQ`8X77i_EOrRO!oG7rsFGg&Z;i^-hl z+QmhhZOS{YYZXk7ujP@J)({%7i?(-7!-e=^qoBS@-s|t4)({+a<}D4)8M|nWS61dM zcgmbA@FbRT@@8S$bZy~lhe2=2E^9bn*N|&Hs#s;OoXq;fm}SmeEw8wMxB=ptIR0rbS%>J=>_kP_Pc*)E*luWnGRNRRB zZ;G~)99r|Np~}JngX*puN&PI$DNzlr2Wn&k3@{|Wob-VqMsrTZ&0C|ks>!ab_DO>n z=tZA_hqz?mWxQfw83i3H28!r6a0X`$yeh|Wp=~O1&l)1VeHM)j3eMnDj@_nDiZJG%@$k z5|pkBGg~S3|C{^~WIS!m;toCAwLA1Mt1$WJ;1aZ1j(YgF-3m2diJa^<)wtVtem5UJ z4u5orX6^iP0K(`HWFyrfP^NYQnc9h9++%`sYA4F6o$#i1;u-e{h-2vEnk4rV0yx6` zMxti}S|af~jyy)7ieMED>j>2j^v#2C?I2P`bRCT~&ZwfPilggj{*|hZ;5^q3gt*4h zKy+=R17``Rm*~JKjlaR0Hs35R;7xLLfL}+K=#)73R?WG4ln(OFNOX2S!3*cQHqf$d zA*wW-MAw z1r4`W(RSBo^%J|;NbmNQ(HeY$$Zcz~-au8Y2RTXD^(0!v&oe0^5MHKQU7s56N$$k@(Mk6+{@cX{8y zAVNQr@Jkd@hu4)6!+%gSqOouX35WLzj~po6P4D+xZv17B<*@^Wdl>e9;pV-TGY1Gq za1HO)ga=d(I1W$IE=5gL@zP(2ZJ=YX8Rv89*sU_A~#fje# q5VA{=x{xOo84>c7A`NL%ej0L(skZE5o1SFTpW^hM=FN!8Me08nPwFKA diff --git a/out/production/clients/org/apache/kafka/common/requests/RequestSend.class b/out/production/clients/org/apache/kafka/common/requests/RequestSend.class deleted file mode 100644 index 61f8033796c59dfca9a26c73528ca799b6f6e204..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1900 zcma)6=~C1{5dMZe0!t8BPUR51*gJrDa;S?U=mM5dDCHl+?f{XHtl2CSK9yFkkc*)UN(|i?ZKB@wiLCj7Pq;w zqi*h~O3~ciH4Vk$AFI4#SCrT8X5R6bWC(AVr2_^f(6elrwplcF#XczWin3x`)uK&~ z(J9T)>=_0-QTU+lsfw-|TjE+Xwk8A70^$g?_h(P8QQ)&?i8HhoG=ncycQ?4Ts&43X z5-*rVRbNvrP3YcX*xuGE47os}`oLDWQKBrHOY~}*q1u{h2P8c(EI6nWTDB}ih%IHI0Mh|*rT*eg%vofCInS?nR z&oM7!0WV~{M1i5Fo;P3Bbea(Xu}BnWSQGB_Ov_lpvWyhc3{qxD%*8YVOEP3GI#nVG zhIEj*!R>w1+EJF=)wA{)pLf1R6kVSS!uGErHAaORb9s{?lt?ZxWP|*D%SqK#b)Ck^ zrb_iD7QAx)TN>&-`B+Zbeyz>gLBfs6d&amRm)}P5wkwea=x3q zs!n2t<*8WsF`~z$~l*t~WVdV&+ROS%j zRQ3>&FLd94L6ULDb^z_9b>k}G43Px&h=W1eFlmH_Yq(AfVkg)eoSongAtpN4W7p{> zM0bO!8lveMr1!^YTufzZXiAfa(bxPY^O-DM{%(pMgxo?HaYE_AEn*mPVi!J z6UH$ijvtBRNwJ!e@Hm7gm=4jMXd|AA%`*uz44n(9bUL1#TTi4`w^rhtTT6*}a*<&y zX*#?y?B6b@ z^SO=*Su(oNEn^mUWrWc#<2hbPcq!u*dS!H=lVP;R+Iu!gn3FM&1p&M$BaS5*udz&a zR)eXMuY`n*H&|h4sc5DxJtrfHw+z$&#hc68j-_f&+N<@S?5V_bPJKm-Ja-ZXEf#8S zNw34BUI7ZKkyi+S=iSz6a|>O0NF~t?XG1NN$p2_H#b^Dr=1&v#c{-k1ORZ+&Tg&me z#d!KE?MOqquTm#pVa~h|(J|7Y5S!X&=%Q}EG|hm`$+CkhT2=v-%ITK~%n6qu+ELZi&&N zRaguZy&FL8qMM#KXqQ2N?f~7f;S;b}>@(_)5h$a+jD{}=exy%zxJl0zx(Ga=@F5D1 z5G?5JhN1_zX-6QqgS!MQ2=4g=8tUC!1F=&`o5yH8L6czeh--_sJm-Y0%9YkO?K+TE_KH2naWs>!zos Kk9I>YZsa#)Oy>0f diff --git a/out/production/clients/org/apache/kafka/common/requests/ResponseSend.class b/out/production/clients/org/apache/kafka/common/requests/ResponseSend.class deleted file mode 100644 index ff672c2d9d4eaffc8d2998171a429f758984fbdb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1762 zcmcIkYflqF6g@)=3)@Ac2%;ztdG!S=K9M3SCPtEq2_+C?V%)Y_sHAaDuM>iJ z!387^JOB@cn6;f|oAywNgV(z=^UeHbcK`hS^&7x*JkBGFjXW~gw6SI5p^ZljxrPj+ z+GEI6w+|Syufm>SD7IuEK8*TZ5q;)epDg88=yLynM^c|tV^$r@A;XJS7x6&d&PYk$ zdCv#+Lycu85mc)<<7SwX!*^oLzz!Lf&Qp#D6Vk2ovPN?BdEhy170JM>*Y-3?TY3Rk zF-diX08#*mv4kn5Q)t8_USkzK{f9fp(z% zb=;sRE&fDbM%%P7V+faH;dF`g2H b#;|KUyFXyfvY5lNQx+@j=Kfgc5Pj<;H8FvN7TQ25v``>TE6kx6d1}Xe1 zxPZg~app%MW^AW%(;g}g-t~OEH*aS4&)=WF0BoaXp@7FGRxISPZsLiFrzV~W%r#V? z^o~HTwy`gee;por0_Bzpybq&(*NZ;Mu1}u%R_IE9Uq*`U)S1`EYAE2e!l-Az@p3~gDRD0!U6>~eCLe?#DT!#d5z=2MCbZ>SR*lg8T9P7j#SWd>N~8Yt$Kmf zv69-1lFKo%yyB!wYM`3JvfQ2KMyE+ibvRHiPcm*Xn?Z9wlv)ZHaiZ&oL*kl9$Yy4+ zd)*Vy)lSU)_M-oaJDIMc(P{D!tadnxTjd(f{AM-RB>0}<$rHqpfa6B z(vQH4tInvdgkh%pSfNwk!CA+f{Qcs3w1y&ZTfMgOlGGThuhA zUQ1}RCc7Xilf~ooa@S$9TdFF`@as~JQ!UA>o0!51d8;%T4sQO_+OzTs49<mn4F59#e$1z>friRwR6@?-t1S(`yiRNj%nYi0HyViOg zHGM!l@IR1v;kCR_1X6fJ;$I;C1W$xB>vgj!j?{Lk_$rrMEmcPUcbjJTyA zNG#IH?F9omjY~*}JIUl;=aF_J64^>6PK+c70;Q1J6j}8-{fzIKzI2h~0IR-gT+$g*`o9EdAEg8Jd z6}!W?N%J@36!_h!|AD-CT0fNEW9m0#zMjoKL+QkG|N^v$=e?=11-(O;EckblZ(Bfd3i0p`mwpT&$NRLJZixh!G@m(ajv z+~)c&uEaI1Qoj|hD!4#R$?17@8+Z$LT-^roi>zXvSMVj`vv`}H738exBPNg%S0vt% zs0#B?{eKAHUGlss%XBwQ`CmhpQw`z$Xc{u}5t-K#rB;~BG(^^x>p#OXWuDegnGnw; Lk}snkd%yc1!t`?G diff --git a/out/production/clients/org/apache/kafka/common/serialization/IntegerSerializer.class b/out/production/clients/org/apache/kafka/common/serialization/IntegerSerializer.class deleted file mode 100644 index a36441232ffd3fed07dd4b22736c04ee444cca19..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1355 zcmbVLYfscb6g{)ALM^g@toYz5=)R!1Au(nNiI|uW7mZm$0)Cn;Wf|zUo9?X18vg|U z1ixTnjPaX^KgxKg?Xm?lA%5ue&YgSCJ@-ue>-X310M<~^q2aNP6rQLhuc4qLgQA9# zhO&lbhRnKfh1_6B<_o(Fsh8e?%`jRMuKlLlYS{h`Z#a}0uXz@CcDXOqK1!zKk?1fy zuX%pc45p=fq}>M@b8 z9rF*|>`LL7+q_-fSCX`_g8GbJIc7+U&TG5J!1ft(mlgKfeU*!sVe8@6;cnBcOJBIn zYH>p;sf(t|WuPQ~MakbWfxO}Vj5I(Yg2lOG-Jok z5T9M`u{Q8QEY)h9Tj~ zUG8*g7WzZB=rd{bq&>)NVxIme4OB_LT$bQQDDcj8T;WzoRFqatfG?0Vf0CTzD7(6?7aE0U5&`OcF_sa3Q@qyGv(Nlu6(o zMPx0ls?8~BR&6TfpGdPa?c2~^vV4w2cIwmV&R!}Avnosn;gAY56ef4h2o>s@q4`1V zDdNr$vw?@yZyF1j!7^rJdMg)tD^xQb=&exIEa{pH_?U8M{=&i%C6$HO>|2W2$3oWZ m3z};onrn@;sk!uGsr&<}bEK8te_enc=}JT%J;Gv8z4Qn9k1aC* diff --git a/out/production/clients/org/apache/kafka/common/serialization/Serializer.class b/out/production/clients/org/apache/kafka/common/serialization/Serializer.class deleted file mode 100644 index 109d005a91bbf833a497559d7962eff258f50ce3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 460 zcmZWmyHdk25In_6Ou{1xsrUnj0#Q)mObVExf}ya70;-EfF-jCEqu3OD8U-J~M=_k` z7(Ap~xQU-9PsB)6 zW-#}R=pSQR2{f9FC%IE5i_d2}*;YD*TD_BTEo<@`%E$RqiaegY+%P!XO=wcnwCZ() zwp+3bqB2)JO|SP&X8Wb8qAb5HP;#Kk$ zE>_5eU9YX-^;VcGLVr32Lw`G;yIP3B$d{Uvy9^k#=mqo>L7PDvI)u7}S`ZRGfZi|e Q6F!6^->VqMaN^P62c*(`EC2ui diff --git a/out/production/clients/org/apache/kafka/common/serialization/StringDeserializer.class b/out/production/clients/org/apache/kafka/common/serialization/StringDeserializer.class deleted file mode 100644 index 7d354c2acb18731e0d0c261619f935d251122fea..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2147 zcma)7+fo}x5IrL#?IJ|L0&|INtT@CG5}-hEf)ODB8*;IXlL$zF%EKa!#TubityWIR zpZE*>N~#j5n5tY}>?%LxkCMun-IcWFQg%_y>~!~Z_vt=8dime8KLE^ND~5i|>-eM- zeOS=(sgBROu!toccQo9MLC3O&6%C(j_#%e5vi(x+U#aC@3|+{_c=f)9RSgd`d@Z2M zYSAj0)v`b$|Hyc3WGhCsoGmyuWjTT9yjeA!1%Zy#*oHv(j#ZKZJ$bV#R~!2~(q1!m zDwHJhR?(yk(cOM$IVh%Iu!;aL>pVt^IwgnypLPSInOc$K=Ti zdRI23(q;mkMXUPIEH`Wh=}UPjHXO5(y>HZVTPi2bvtXc}Q)+>TS-&R_1jLp=yk=W9 zX*&n2#y*{OH<51`m4=I;4gbN;BUyA9ec)AC2el@&(N{ywpRHYbf%m%$j#1oWs%}Nt zB-Wjn3TC-#I4-8tDNN^ka)EZ0+NBX_UJUkYdui-!B1BJkH$8=EW!J0qMy+Pqjw~(v z&0qehC~K+*OgZ9MHM7W8mkh@c2yHKsQxq2~R-JxcAGTGztw1im@C}&|YSemd+@!>p8~EwHkzZYF=+ zmDOZBXU=tU=fIKMKO`M1S$79LS!zgiYP6}5Y?dV%$2vCR_zvGIK$~%7@qxhXMd>!D z_oA5=2o%brg~0Tic?a(80z&br(Tq( zQLD*nNnj%At7&lNj$d$+Qgz{dCv2=fp_>U_k(KwhZIry;79mP`f`1n zr`wbZ?p^6)h;$EAE-%8b){$xQ-+(l$(FJN4V#%K7pPH zKSd-FS-Ze~!{NW7KS4Y_9R3TsA2#fU9f3wy*BEYunO|ps zBu0s43O6u^cimKGyl{Rh3{3qEH@zG!MwyNx)O$qSK_55y_da2y=+^_G?8j)IrsNXB zor@YW|6+)%hVjdopg4p{8jzJ-6~?D@5avET{sIwkcmjQMJpC&=j?sCzdV;Ra6T~-< z(Vckf2t7){?FhZcxb!D|slxZMqERH=y~Bk%;KE4J?-(^1YR1{H2^V6T(teg;3eyC^ zxGk{L_>jO<+ot*ZeT&`|3=N@ZUc*OJeBA19h6-gU+Fgyhf|)=^h3C0 z=kKrTdvw;@#J!={8bSWY;O}lfG!388hh7G??>2#rzZ`1rbmemMyx&{)bz_&Q2K8Wz zSao75TUA^415DWym^yn3fOa>F#cZ3`+Iw|6hl%@(#7^_eq0VxO>vp~2x{l|Ym8Gch zOFtN7DT+)mPsUkQv$SF-=Pk&psBLc@CUzGPvGW~yqWoa$Fwue zj5)%YdN9cu-^pz6`{w=cGL?oYPmN}CfU*hQ zbxpe>FxAPT&1-lmDmn?7^qo*7Fp!-%Wk`#`P98V#4&UJcP|a^l(v&ZJoWDt4u5a=5 zG3A1LPwogJmyZ30u3vbJ;cc!d?x3+k^Yi?rg{HWGF`fi2B0~%5<{$2H?xpjOkxVN; zBc6`0=BJ(@5yPx9s=PpNKCQNn&oMBnJVWv^wA`rj7m`s}B?vnPm98!@+!!;z$N(9P z6U$Xx!YtkmQkf3JMX4|_^_PP5O0*bfI+9lJ5pfrNT;}I}!pPEZ2twLV&^|}WSt#8L zD)RqggsY0lK}=B`LxBe5CRd5^Cpu8L7bagIE)FJhzoP39-3O~r(X;s!+U5~@(`OFR z7b^M=(SL-qf6|XEdLOHqz?oq03SbTe5VG_+K~0{TN%S+ataOgje!{)HPS)m;Tv diff --git a/out/production/clients/org/apache/kafka/common/utils/AbstractIterator$1.class b/out/production/clients/org/apache/kafka/common/utils/AbstractIterator$1.class deleted file mode 100644 index 70b9cd8ad0eac3ce2998c09e15776049008f8cfb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 897 zcmbVK+iuf95IvhXb`o3~LR&((73#JLml{w3ArKHXi4>8GL{Jgx)5cj6H;%VjZ^|F= zD?9@dLJ0K(_$b8KEmZ1TD|s$wJac9=JHP+@JOHqXdwElSWUxXJL+>3{UKH}3E~N2-y-_qaC}&V+xMaL1F=sSKT&_Jhc&4I-_$r(0SmuBCGN znrp$3+G#OlpL>BH`uYV!x>DU{Ft+57e$Kc3P;_V0flzPxAfQOGEj=D=bLGeQC^$!4 z-h^gIFb&F(@5`C;MAMIp7Edz9ihi^XyKNH6&tHqvysO+8|$dp zsNyz5IWBPn9**3u?9aS$(-*<8p_EiM?!;+#QDL}GTW}Az;3ive54YeZTX0Vf;yhrm zTVW{FR=}f3L=1)FM!mtV@HE55=|~wiPgn5oEiy-MV(#-`7SHmuQr&5vh3grE*{rqN zjXJ~Qvocsx4gHV@$x+ai81-Ja!H{{=sMX&WmgzimbgryWir*^sC}Lxd?0G_;kR}g! z@&oMPSN{SR%blQlz3RSroF=WC_lz@?Q}3;bZ^+ diff --git a/out/production/clients/org/apache/kafka/common/utils/AbstractIterator$State.class b/out/production/clients/org/apache/kafka/common/utils/AbstractIterator$State.class deleted file mode 100644 index 4d5e2942555f2eaede40b1804071d8c3d772d8f5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1323 zcmbVLZBG+H5Pr73_S)lktGoyzqN22*9AE*l4WWS&YZ{t(NYePhY|o<{T<^&B%J0&W zh|y^H?2j_eE{zuPi@i%`GBeN4JTp7{_juR}C{r{o0*Sokb`HPnN?r6?jeKwXH|bh6({BZqz3Yf6jsAW6i(*td zZi{W4(D57_Dqb*Td)Xl!)5z&~h*=%O7|}5*#F!A{7-1Orb1kAL+F;A|Jz_PI&UE!# zsU6!^NX!@ij|9VN_sHdg$aUrYy$aW9FxCdHr# zsUV3-+yTQBSr15yPiupHLgW(B4`eaiB^j5oDZ)?VUI(8=M0&N9GXEK|ODF;#xrU!1 zkA8*1eX1bhG+2_N;xeN21M2+3)CJ-nWw^WC40*1Ot9T@+{V1m=t f2s9!MYA#FEqa7X@%#$yI1>7R75mJGH)YjxL_|`U; diff --git a/out/production/clients/org/apache/kafka/common/utils/AbstractIterator.class b/out/production/clients/org/apache/kafka/common/utils/AbstractIterator.class deleted file mode 100644 index 594847603c6bf95f8b6e9acc9cb3967b805d61ce..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2342 zcmb7FZBr9h6n>U0B9%ycfoN1Pd{AU@^v(r5Ms!mS6Y7co2goPYogy!53kC89_FN9xO_r zhw_n=JM;3)BWXSk<4G8WFp6P3RiI@oOLrDcQLE{-Fjf?_7hK`$eAoQes=1CRyBSw^gljtq@BEAc)zWud1-;o9;-yHL!rDyb zOE2`Y%e831*tCROcNi#CEYj04YgmRmM{{(HEYwANoon4$!_ss0N=bK$qGZz3o3+cr zToH~T?ZBzJTSiU6^;1%1*n#74uc{5mo2SH@WfpS;3~z~A&P$w$tSRVZIm+7)MKz$> zH#+uz1!*50^UL7q75gRgRI9qaO*mniOSVNvq$0L;AGxO@UX}DkyHc&Yfvf)*?1F8Z zy0A!RezrK>8gH)E<$GjHQ;kkGQi>Jqx>MGd4O##Gll>l-?bUD@pJ@0BaSa#c<7=#H zSi>^~7mwPXF-?6_n4Un=J7vA<8n&h2LL*?@sKpH{z9ATCJzkUglfd6-_zbth_*TPr zNNE^HN27!`>GU5uxPj_(? z^c->VZwlpplxCw!)cNUv6B7!iPI<)Z5$|a(Uwr-~J-@W7;BKPugW;C9sO1FF68=a? zcq1X1!NsI;8F|5rQIRVZd6(f+d0*_Jj zHChx0QUUVM#E{6-Z=oiCM`#c2lHx`)^< zS5%M30NUj)BzSt9Os+HjB(61anr`7FJ(AOP{yLy0qKd+}moUMcB!`~@epi!wIP(k6 z96(#$N9X&X2&CNgH#$9H-IADeyyiKd(RpG5HxTB(xNh9U0B^}K@ADPhZen${g_ZO) zSgC01J{R<5lH6__c4ax>7CS)CYVX;7^uA{_zb74D_3ku9iM2$L9={(`hxv-2(|srN zJ(gFO*Oz(bwZOav?v zaimAa@fAF&s&XQ+3W3c4oTId5SAPt`B^j=d;gZa@{~fxd0!iRkL_&XH;5ApzaaEPy zJk{LO9Qov#`y;CJF-B0J?ut$Pk}dqCr>PDjOwXtGIi?BTh8cXwcQ3y#@Qt1br80<- Hd*}ZHV+ZRU diff --git a/out/production/clients/org/apache/kafka/common/utils/CollectionUtils.class b/out/production/clients/org/apache/kafka/common/utils/CollectionUtils.class deleted file mode 100644 index b60cbc9624c76d39b1131a7da524c8f483a6c989..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2901 zcmbtWU2hvj6g}hEUdP_1N%K+6N1J}eNt+r zXe1x`C&GJvk2QQI5S_7X%gqUNrAAi-!gG~%Q{Z^svdzW%_L}JwjI}aHPUI^^ zqr74`mb9CLVRyr-38eECr<69TMsdSTZy8UwjC8TGyb=NA_(sPw^*(|zN#a@T9UqNRI{V|Y=MaDK~H!yWy3C|mt4oPOQh_s z8jfqpO$5XOZ3UKF$Z&$xq%v*S*=2jvYxM=&Hl4Y$QLCAZXfU|oyh3r4@Veql$foW2 zP*9rPYD=DxAj^!;?FHL4OQw@86tX$;&IXa*elv&4swBJLinm45RD0gt-;cF+gE zw@r}`2L!^A0UcX%jp7!)P?^G<)bNBktAVG{y=0Ya!>v0^tM6wD8J{fo);5_a+1$Qd zI9e$C66gN~a>nmU-m19=rei9umneH8vB|SLbem{_dZ-p+Rcsk9r^fNSl z_74$WtrI?d0NlF>e)NF^UP38Y^L{X{9iv<~?>!Y5N-g+TwQdmAq?;=eMHh_qb-s`e zaA9;6XK<2V3i8(lAyb&o*J!2L3bv84=MX=!6T<6!M%CaD-e6z%062xyw7M}&vXERX zwn?UO?CakVdiWe&`LPDV0u4k2?p|&nD)1dlb~J%M5nCK@pj-ChzaWtb`z(Dbaw!>Z z;8=*WJ(*}S`WH?%Ni$JSC8G@-@4|y*bTINOdbzYu?jwhO*!%X;WAi`j>+u9@aPMDOGgWSU2i;vP|sulbJPgirlDE%oDz<4X08&i$E4GNN=I z;NcBsr}ddp^()0XP?Gn{DhPd#2Cq0x|yPSlbE7U*Qnk!7TJ4*ES_LS z={rF!CC1ZtlKqREjnK{#@vc&74LMp;)m6OLR&|(j9{zpe86{EI+o;*5mgl9G$2~1y oAg5uD0&2Lihd!y^4JBn$y_?kS78yc3?=iMV!*JR0r(7ti15kN^Mx diff --git a/out/production/clients/org/apache/kafka/common/utils/CopyOnWriteMap.class b/out/production/clients/org/apache/kafka/common/utils/CopyOnWriteMap.class deleted file mode 100644 index cf57ab5dd079055aebd1872834c6e726ecd133ae..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4034 zcmb7Fdvg<26hE7!Nt$daEu<~jM|cQrk`hqll~{pVp!9{drh>e->9*Zwvl})^vCgRD zFryCR2M|94f8dM))fqp4AIkA}_HH(t*G%~%H+Szn=XZYRdH?Jncqki0(z` zE4trJgY7O_Aik?#}jlZ44KxIoSOLS)5v^LX9 zNy%@i>4LIRP||s`STqfrPB}eomUfqnRZFX=g4#eM*lwu?c;TIpJZ|)lH+q<1XO1=< zy7GJ(&6`F=(Twt(x+_u45!DrAGrhF7t>!D4iF*=B1$NyIt@lczo|<}2(W{7$v?Ebi zH7eHbvRaX-Z+yZ-2uGO^J&~wuh9xADXAMKOrgf!URuRCn&C*$qGD3FJR&(id-u#NPyz5bSWtxv11+RbHEcRi*1mzq5TAwk4*L%uh8o1D zO7$M?$?x%DD^J$OQM6#^FX5Gb#t8}0AK4uZc}6Q)KBYOgNh8R+?Eh*o+da# zMfX;XqPebZ*z?$yiZ_*M#yf(9QcvBMtrAq9R298U$MB{M5xM}s>l|?W593wy(FpZp zk9=T_c;XHzXzAlJ8IQ|Bi1zFo+J|pP&+7ngZ#-o>hjS`91FjK5F)T-e*|dMH5jENrEp*OjPHbJDAW${!B+TG2lz7`;n4zM z9S5l!FvAoh%zlqT61~Egm;5;!T_hvokgrqb6Q(R8a~@deEC;(eGGSi}_=OJCJ4HsA zxd=1pU954KZgQbpYKeblUoJ>I3fhMy(i)bl`@$4_Ghm7d)1Gg%mG8{Huq7T??fJ^B zd~fdyQ}Dftp1kIE?k(8h5(&3-Z3H(&OB6zP*E=@cQY3uBMRJUXsKa;?g1Y( zV)QA7lOK9fn!#DXZo9gs=yTTOauS+1UCj|!^CmP!kZu7T&w=C4FOc97Htca{A%$^p z&_{_>^E?us+y}1*bch~;*(2MNxWI5d8K&FzNR{cJ&EIFUu1g8lEHww#LTUtz1jBhS z#9QSs{1w<7ztLufeLh;OQJid!b{?qfbQlA6m^N&V34!8r9Hm8j&d3<}9Ah7(BX*4a z9=4-S2?#0XD5P$Jm0)`jY?s{fc#d<9XDpF?P5r-NJ|$DH>Ch|wJj^5m5U;%(1qzc+ zy<~WT7;_26+;WIH<-p@h1d*jg5Lrs3o~1bTcOc~_sn?V51x$MhszlEq!2CvSelKXy zCI*VJ7Zm0Fz-r*7pC1B2RAjsiuPbM-F%{*-;N!F@0+>bchWj0-w_| zh{U1z5_CAdV-BGq42G5QvazC+f&$K(5hC%$%KJIqI%kmhU;Y7^l{2yjUI91m(KmzvZ46Vm-N(*4{M!9JI7 spUbz;6G4$~mJ%#kO0Z<9w&|`F<|=&&RzA8*AL2iP0yu|u{UF%=KdsoAoB#j- diff --git a/out/production/clients/org/apache/kafka/common/utils/Crc32.class b/out/production/clients/org/apache/kafka/common/utils/Crc32.class deleted file mode 100644 index a6658fb0a5ed8eb0f0f03140cc4986e6f2493d91..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 28501 zcmd4ZXIRu)*f#7*t^o$=xdjWLDA+(mPys7qL9D1Cii%2Aihva>Dq^o#v4Rz`AQlt_ zQL%SW6tRF<5Gxk!?OXTW``OPv-sAi6{(SYA>&#kAR#wVnl1yeU|303pQmI;MdRJ4) znoc^J&TJQst{fiKG+KWbyXlIerW}!xrSANU9*EUxz6Ng%1Dax;ZolKu8GW`E zV#2GI;YHAnR@fZ8dK(Py&v(M)2U*+k==O^`SbMX0K6H=tvP0Z~X(#YJ%U%b|Pg*9z z@~Ckz!ZWNs;N!6|o=C`bFG1~lh0`(m;nP>BD$MMM*|+9g07WMGA~<5#Df|ljHU$eq zF134aMf3_Y9){Z^+@E6W$T4Hl4o?fvQ(f2%Hx1_e z!j5X05ondU>pMCfOp3()>`QmBH|JYNINgnWg07`QC*Z*ygB#dW(B2OXi<|F7zY%W?;I1*~#9upt9FNbu0j5d;PTIHbfwy`b79GJ5J+D9_$BKhRg9jM4EY=KES zm$t%4^{pk==iZ2%G46rcY&;3lQ?M$yZ2}BNhj+k~@WH$A z)bNHLR_gxQglekvMwqU%`2gM-t?Yt%nr$~xNBS@dVcL1W@g;DeKW2uEyn4*L($R7C~J97q(gj=TeLwFu^?uokwcK5Kqnt3lcqsM2stEY@W>4?fR*fp|r z034&1R>C7JcQUR799oQ=(Al-oAo^ws`j2|-jH~*wtC6kO4X7`h9m4>vh7LG)ylpNH zp3rLyyTgN5VCbQ6HS&u7q#`r_h9jC5Z(f8UCFz#f5S*8R*Fhf|VnX<~jgUvLtcH!c z+G%)Y=0zVp+)MKlmD$9)^I3Keu2Od7~>*sy&{BB7>W6(MLVo2d!}^52+&?1|WZg*%x#V z>((0`qGInOy&${?ith}5fR3elp>Vm|_8ro4(!EiVz4;uv9lS9E%`^YJz=;zdEO7aF z-a>dETG<>nhqvv-@qA}3Tr1LE37?XIO<-L-@+1z0PHBd-0X?$dKgwK&ZM5Ag9M;Qq z!5RIfTQFFw)ExG5XD^Y0g*HqdeB%AAcevEl}w0R*Gwb4L4(T zg3VSePBwjl)Po*Y$W4n4!261_ov8Sj{~TvZX4~N2xq~6__fOxAE*{CR5FK%~F($_R z3PWqhh;&%l4z7frj&4&Z7OtaF>f!tm=lyjK;!TXt7*s`sG)K1W$1m9Gn0EvVED|SR zxlT$;)JV?x0mFnP$I&3IWHQV^E20JG`6hrG*!Pab1w- zWj_!1eJsvnmFYqWOANC+VyAV*Y@Bp0&Vy^pIRn(ouIhzqYf_himaMssp;uM9=vw6H ziKy>^ix6ApeHq8I{e7@4CDIi0m$=K`=74t_2&{n0sh%?Y;pazammGtz2xU1Fz^dK%acqU`UDS z+fZ|1@?*3}y;={pIlo3=pH10*9Cpipi)3ZCJ?82jjKn2Rn?1<)Hhqa7aUM?5QCRAlkB3^ZZH1oRIUms9bIBo04k?Mjh`35eJTL$F8^3?$oy6_@#A&#C zC8a%PFLeHn^@%#!IFREr3A<85TBGgkq@JjslvWM8ncpwLIQ`0e46pR}f=5-PAx0Ow zUBmc1!^QYGSm}(H?$$C+N5-7S-B|aTSnFKW4NL9IwXs8$c@8<|2?-eEzOXh%4bC2d z&ao9q7#vw#3{!irI?#8H8;(Zi_8ZYbW$_Tll2i&(W;=RgQF`EVtjP2(z{9F@W;k0} zH4v3~sjKk0c+G93>*Tz^QHv$Ju+gz30&{FDZE-nb@*CWW@!X5A{^rpr^RR7(&=Th| z%s8i$0-p+>AoTtiQXlfckI!J9mbV?v6B0+FNpea#tgD9MkYVa3>^5EV0MC3>=J@I5I2c8N zf$MO6viDtdFY~H_zTe|~F}28kH3F|%6vBSVLPE=~#fgQsjqu@0&Lh0a zU$ODz#ujMCmJrC?FP5RgK>z-w>gHeE2dx3sobM2hWv`1f(IcH znK&0WxC(E*bzShq({&=Ubkh%Di<0~f3*D}^z)G86VjAW{ zqgk=+$JLv62?Kc4%P;*ad9%_YUeuC2K!$r`Sq(3k@&~PV=$Ms5t zX}iJIQT=ywYnWv%oq`%up7w@oZ;wZ4y{u^w+Fn|*6m9gr9)nw#x9!kjZh{`#oodq` z?d859a2{4;5L`A+k4CF|XPnWpW|{=YfhQBteD&=^aJm)o5-m)mhp>rwwp{GZmZhRm zrQ>fjv>$Q_)`7iyp?*rEsjwiNsR5Sh~~1Eb@*%tN&6_Y$#B zT>b(PnR6o%9{YL-MmhGE5%$rw6$1Cq&>(2EsT%_9_Jm@@i_`rOva_fV!NDhbOm}RJ=_70v+I-4P5a$` zbnV#ZGrGjvoQC@~r!MGdG-w<;buYI?=S7Px;q!h}Ukqp=kHo+UIUV48^o}mPzn)r* z{%v+=qwlopkI^qT?luNLZe4*PR^FxX3w<*egSO8*0RLBsEitso4`5hy*TERR&tWvm zTA9zpqc3BQ;_-nSZ}DW@j$)LzJ2Vu}e-}sKS=LA=Je^{oh02CKT=Dj)l@VTV-Rz4u zBdP*W;qdq=Uc8xq3$OM#FT~5Rp(pT*vfklm!OaW!y*6?){tOTYK;LT{H^q;;Q_S&g z^Vv@LIy8L(KB``L;zQo-8F-h_z9HUw3^YNNuKOx{x?;BtUzV)?j?Z3yuAz96{|MYY z?lKVfel9S?{SHrBqh!pd9F!hPUWA9A>fOWxm!8j16rq0?chW*DQCNBR2yWXSoQ3?r z?P?UHTyKF}; zu^Lx;@85;X%g!a^;-#tQaY@hQJ+5}q8INmoM|Z}JQ@3j4x_nHWFYI5_5z9yajKvDO zPR+3LMdMmnv$MlStPS?rgVpt4e!!|nGtVJ8=fYQPj@x$w8(nD)HhrJG1xcCFHdq&{ zMcCk&-2>}C7EH&I1+AK6anIp8SZYu<49jjT3d6#gn_gp4r!V)hK;1na^UqsnV)pvr z`G_B6coH+~WRzppgL?%?Sb5b4bNcNJ!n_(a+hFePu?9$coS%YSR>#+3cc}V0_H2*( zhV)mHreS{*r6=}9?^UpO-}weeeLt%vcD8usfbA0oOvH|(Zth6=+T<*@wCVg2+ot{6 zjjg$BHzD)vxO^PeuJs&8I;Q60X#BMWIC$;8J`NcjXo~~gjRKLeC}aS#i|UWUaZ}IX z$nyTIjbn)$9g$PA@GwqTJQ7DEgNNNiZc@uP$jTWRg5A0o7eUF~{}#&v*A9eB$vY=B zh#39=n~GeI;oRZIm6)#aT8QWqkAhJo&6CjM&dDr%2*3Lfk4j@(U{H{;FZ$LRn2d@Y z&A;QjadLHZ+WU13N~fN>h>;gk_hG_VO=n!Y9^DYD_sxiby-7_|xNSfB1M_O;Z$|AK z6^*cU+_Fx{IB&HVnkn5bA;oEb2h?18aR{ol9hr%P^Hlk8=<6DWwXcWkV~*du$7ug- z?L36cohRP5+dMjq8_T=~Beq3jH@tjmJQw{3#XdpDH+K#2wcp86cxU%@3q02*pT*$% z&28~`^FS}m=pOwU7d~lL;&#VWacchaR86!>&hLXI)<^A-yr$+kG_swMg5$rs)q?D9 zH4Vm9%T{2QXT=w6s6MDYtarH$f}`bfaoD)!%Pmx&cq1HpFYhu!?hKtWXrCF8fMF>y zH&Jd;<0z`qvi#7?w6vW#h<-i?VR17bVd@nJL!62AjKCEGqpcWsVAKT!s4p}@aaKw% zbcs&-grB$mEXUi>zH#uq=VA`SV~u*_h*rnO*q%{&9v1rZw_(=(jM`{DvSuF=&j%pFEoCEa&A#vrr=3UDz~qHSWAW7ASnR;b=3+ zu?CQ&o7E$Fen0s1-%r-km<3z`@?a=x;_cUsBZ{mRcpZfHGQ~ik_v2e4=8YJ1} zPJ-3iVoPyOa;Yayd8RkR&0o?fgu93Cz?U_{EZ}9^dNTZz*Q~`e>-QhfJEE2rJ{84{ z!1KU@r5IjvgrSd37eonm{%rA9(iRMAR%TWK$w*agU->intiJd2K zQaXJX`wmAr!&IZ|4a>28)?>%@CVy~r>Rb~TT__um6~@KaV7oVW7hG$Zcwpv^iPpG# zK6C;`Op%N+_D1?|Tpo960|st?Umuk<*L1_LeXY~b)nwRJOzizu8`s}22*QId4VNJJ zt9u2$IZW;c_my>8!n@&tdw8|}jyM7QcK!wQx^7vFTJH=4v87j-4%%(#my3i(&iAoq z#ky8#*6ep*H1s*P4x1}Veqyoz7zHiM%wokUS+nafS=fCK;8xxh*|TR^;mM=Ki5N2U zGokwnzh8JiphFTOoL_W(H{x86Biu9*~@S5z|ZoI zI9A?rpc+0_uk#c=cTJuJx)N4|J+X!daV&4kP)Kp-TVrF|@0qYMUH1&mDb9LWYSAwW zb3@Hi(c#{hJT#3iX^K_1j&+Bwde%D}$SPlrZ3f+^!u&w9I_P|}gC~AO`Pt*e{m*C6 zZ)D<5T+K+Ui}Cu`ry%rLz$%nzX}?GN5lu2M<&^gbbW53%k54mqp2EdP%VMyseL+=AGAM-!ZT_eR3k0PRt`llo zoqdc(9pCiF>Jg5euuE0*11eJj>*MCBo-5FI`Gd|F{qEC8*!Ep!jT+5&uEY-GDYKCj zKp|SXFE|U`dMV@4XV$+3bOI^G`$ zK8LFXy(eJN7?&M5Vp7KxYR5b+wC?vR6LIeoL(n-f>o&I3%bJUg?upL0U-qgT<+Jj9 z;j^WV2f|Lfd`7c?-VH%g@ERoc``8V89q)a?tHhK>xbuF&S_Bsim##u(?_r)u3EKD-&(D-M#=UJflHf`=u0U&C`BZeewQ&z-9LhAuwUuFdc=xem zHja9?iNL}ZbRU|aglcGoeaA6&TTwW?&iuZIUr*{^#;MtQQ!vkZ%x)a-Jg7F-ImPzI z_Wr|u;mHRZdz7p&*o4S~1xwKH=GXQx8yo)ycFG=qq;+by7^~`=w!^LX=+}7r*kdS$ z=B@9AfUSqb$%TBWDa->OZN#^mk8a@X!AmnRTRF5JxntMYg?@hzBRD!mpTxu!rqSs6 zpZ04m-O#0p}y*F>B2aZ{&Qw`2;_FceTKI=UKBc-pqd`x=$Q$ z4Z}NzAJO7Sqs}PMZe)ztp@u2gbAEh`H~{iL1)I6E@=>dN*9Z*rxS4^VhCfs&n;2Pu z2WDLZu;GZ?0qneECZYQ0%O+^D=E)XBIjg}Ycc=mkVd#squv*M zh}Cq1@dZtF*r(Rpj&PkJ<1ujL*t2-nExizT8_f$w^1NS%vGsYO8dgU;--cyTcnAi^ z8fRi;jn-PY?efeN6$7vC!2T~sCSb+di;4M?}{d)v(~x z3_l!R?fU{BTQzNs>jShFVA_P@b#OO**96*`3%{Z5?QPw0Y);E0Bs{Zhj5A$Fe#P$w zeLT@iYsDap3CXU2^SRTm(A!ZlAB_tyi0jXX-$tW<^;IVkJ^q9d9{L2-#iN#<`;ofJ zX$IC;ncqOo?ThQ6!Py<@2n;Tb!*KQI>v&^hTY~&9h{D>ZlXI~tsUrnaf=@2t{R(;4T+u1(Dz#6A>8qMVhKyF=2>{{9X}e4 zjQU-~mJQaS2*2>|3}W+6O-0UTDnXlS0Wo;*d+;`JN8W&o3ky1+Pn~VcF{e{ZD1omxtsexv!iSj| zeW0^fOI!zjcV{Jf1aEIIE=9y9VPpGlRu~wtY7PoMmD{0qTC1ga^=IA?SZ(w9jh!wH z`(kv&r%#B9I@cT7PG%KozcbhapT8e{0`(M~eYnxEP=oFVckV>|+X+DOtFGDbIl86} zZdyOhK&=@rVhd|B=M=0b_=aFdnaveMo%j-kaakAcAnR%Map({e`VgO*W=?|9plXpg zt1iBTPRsX<#Dc4n^04ZX=M)TIy`dDv(u*-r{M!`biQaq(jRy|dj`TB4sv#u#`+iJX zf4K$@ov(Eet@=k;;J0DcG3YPVe8GjA_xr=$Bkd;^)}G>rwRL-xVQA-dU2w1H)k~Nx za`nQahYQ!kZkC@L_L;$mbNn>T@TLfAa~x_i7nG*w1}ac0+pKj_*n zy&sm2m|lgIA>ssoQ0vVvP_plJH<-Q`JDI1cOI+aaW7rC$MK^7Sz!rbjVTSXKdN>?z zkpQRPQMUND^~5sh)i*eeQ*kB1@OZiRGL{^f5sl@CyoxZSa^qM$oci_w%o??yjOSyP z?L|}jp}N?8tXV1op8b+B?a{U4IGR(ZE*u+29>DkUC#$1QM)kY6{rbXmcu&f_g(REM ziI`K*_d5DapA&(LFTR`uGHpiS!;>|cXnnG)F;3dK8!&2+S3b9}-s zRF>~FfMuvzXWVIXt{i>Sk9uOmhu}Au`^9GfdheO{4p-bh4TDPT`|!Tis%B^t)@>1T zezj_V*e&HV5x)J7E4F;q+Jr`t;wZg$M;37^7@XGoqO)DiJZyuYG* z!AyVL==$L!)V1pQ;*;0P+2}wW8zO6-(|nA(UuKV}l7iLP5id7~wOPz&RP@-^7PT+a z1Qe_}m5+h4xJf2vE;VjDtIhd=!jF6Uz4HdZr29i zYpkn;W6vHvaJ1C*DW=VS<&J>4_rGEHJa6@7!RjSo&`9p+hzu440v%5fqjbuk)~hm0vwVi zh2iPhJy)?aR_`#@CYv;YV@2&vsPeKQ^mlbgMNodLHaM5sVJTWv_8E!QNnLMYd(7}Y zkac{%<3dXKcmxy$oI+o>ss8xv8S?@oOEzA|>9n;|@Xme7bND*X^~TD?tcPe-ekcrT z!?Y@FoxHUpd>n74;e&@b^*ED$W+p<4FSbNV+~?!a*8OD&m$K(8#U=jN;{EbO>x~Gz z?9>>8>Nn2B?*Y~l+J3NHhD~$K+G6i${hg>*TSvjq{;@wWxWP0pL|hBKiEGD4heBiN zU4iuQK|_%o-{~}3f9gH~yMlJ~hhg>fa=86GJQeE}oxFjeO=r2|`;dj7aOKeQQ5b!8 z!vkzs{Hhh&|9CVD`t;obyMy2FLiCv%4RPu4*%jh)QHc?T+1<`Y7d>(D?D4d?Yq+&4 zxGyF=iGGTVF8#t`)?#o88r|>M8MF5G{DRo0ZR?<5W%J#5G}YDx9{TlWWA>i9259i0 z`Y}{*X=sCkowb(3`u2;Xn6vdUaJa|U%~0BZv_;YSYuhpA&3R*V)OxT4rY$HyHDF9QlTNH&67&(+RWBpp!g57UN#7c!}Ga)(yj=BfXx%{&cr~uxR3c6~~A9 zic5~)Mtw%{(x6VL2%TJtJ|<%$kmV=GqfRqjXE>avJy>v{W?l3$YnX_a0rm}We}PLD zrvGlIkJqE}?a-s{{bWr2a$3g2nU{~CW}_-sfLG^BFQWH>!fD91*CCkFw?!p-W;Gd$ zH#h2^N8?czo>*G9#!sC5Vi15@GqrD{+n!rdD0^`C9`ak}b;HEYmp3s|j8y zm@9Xwj-xM!Er7|U0qs%PCgKpr_8y^w&O4^8$FrMbnt|3!7m)d;+BhU=8NWsSvF3yE z{C0yvxNmJX0x>;W{=%L1VxM!>Qe1Kj4`1bmA@NJM;@77+W^nDForKK|GMZxlwcQ7x zdwh!){(LBS4Zk_}24U3bQ|EBK_LVWvJMrlc_Fewn6C3M4y@NIbDg&|S47$SLu;xA5 z4OJqs-mcbN3}4*H7vFv~eT<9bJOfd|?XMuIY47^zFvMdGj1CReL)zJaClMJG=7`JH z1Lxq!&q+1mzi8ZEAZ2F_>?qo|4XfOac7T&-&O&^tj8~&y(t-?(h*@cm^EyfE(A=v+ z9HYBFIg8Z%Z+?(cKfJ+N)zu&b#$G6bck;s?_*8M{H-h8Z6pJekj-&Ch%+DTDGGdGE5vZ(P6W*FFv$%zKE~y-3rihh?^IV7rO4n z#5h+c>>p;k89`OHD!iF${{;04?5D%7x=k>i=h}Qnn4|cZa>oYidH6Ol&=;*91YSh8 zxA-6^_FG6MrnigSivw#So#0k0q5vNcMR>t><>V(QZ#{WBMjeUUjO`Y2Dx??3)vl-Gn|9QA(?_Ud@w+{Y}Wtzdk@NMjQ z5$oeUeNo-OD-(B9#fKI*1I1@;buN1B#qw_MUKskuy#P%%<-Nf(m%MR^$U3(ismeJ+ z>?^uj0|C>oErBVjOyeRfr9w zywVVwUX{xsFDPGt>(=Ep&~a~B6&7okg(Lq}N;WLqQyO5!_0;ni7MwaroZzRsBS|m4 z6lMu&?Zi$dZ7X1yZHOD&vzNo&E@vE;%*uI#d0jHA(DQ9(IL-weTmaoG2Wwz&-^Aq@ z@jcNHZzrsIgZiaw#-VZT)iymjMaxICgLLR!oSQ-45Pu=iEbb;+!rpsS}@yhlk_EWr8t_tYChBQ7%^dEcS=r z@5N^@vt)BLdX3-w9(h03DKPA}ZZXuWmaoRe4$Jl6abRr)7Mib}2#1-6-{Xnx;b@HB zA#NktVsuEsp3BGeFmlB4)p${PVj>!JJyC(5A^W?a`HlVJYInz7tuglHu2f8N*piFG zbGBHCvzVP{@Hu6tKX&(iz8Im;pDU;s_2@lp@*YK_na+!ec)agL1tROe)kDhsx2rLE z*U#3-lz*n8?UEl|@TJiYaRK7-hcj^W{NRt17e3`;+~`kMxbyi=DryhC(^_1nD!7M1 zlMA|_f6iThY&N@l2FB}3t?-~lX)d(rnu1Hmt}RA4=gZMpwDIzL%rr`cr)i7!0o;51sXP8*rt$off*DY+8;5rcEcIZ>tW4*qGd*2NYv* z9kb+kYZnyGXxybY58xB`k$Is|JG>zQ{((^jrKn^F8|i(h#En6WzOWBihlSq2ofvi`K@4O%dZZ=oLM7q(eYVK+vcVhlqgChyaJ6=+UF2qZ}rT z4~rb<(3$(tR*cs!EGle#2eBX)I1yf&<2j z_Wzl(7)1_h&f&-*=1LA~!O@bV6^Ap2i^05wHMSZ%eN9s?>S#3jn%Ww3{zE%SWz3B_LzF#D^Dm!!j#uo#EVqe6lr#!VQlswZwa72AVGTg5k@igv_yS6ibilr+`Eu&mJ& zW&Qt@4Mf@SKV>6PHvUi9M3j~Plu4AU|EFv!%4Yv5*AV5J|0&lJWicya+PW%Tv0yAk zPwI%0lZv-(l}cj^+uSUbwC}Ip)fLHGuS(TWV1&3zd<$kJn0+Z2unbpEgAc8XN8}x20(y zhGg+OY>*gF1cqowF3x1nt-XR0*+ z+Y_7^K<^EPTS5x>;J6Nv9r+rKkGCHReB~mDpRcy&3H&#WvV@*Sv(JFsZ4eL zu2ic`HKH=!S#4*bGor;`)6})I(EKx4&CbGDwEI`Jo*4e0r|6rixgDrXb^mThR+(1& zr*TPTs`pnVO!c{G(RPBYNx;|CKlLnkizTB{Rfq>KIUVu1CLRaHW36~h6ORD#@DLBX zzh+uY1*W2J39{7UuO%W{bXNbr|0i2h^*`GL{@OZ2Rkg%C^%U#7w|GJIQPoy?shX+! zsyeCqseD!aRkKw-s->y{s&%5YLp4x!K;@?@RSi)+Rt;4>Q~8S}HbSbd3X~jF!BP)Z zs1%|aCCyZYOB+;?(h*gZRG=C!6{#ltwY?>XZK}5FipD{_khH{9j%u2VA#E|OZ5l^0 zq$8%WOyeYmG-8?qH7&%Du9$K!O-nIUO-#R|rj;1d6TN7uaTY`Ryp8^|{qm5D*cFKN zCx%?bU&YoghFXgusI;YOd&DzmHo_dk^h?@#}E0SyqH;+Kl0 zEpPu~6X&(fgXaIUs{empSCds1swscXr`g{l|4+YEmZD$06EV@W|929hqE9@D`nKlA zHK!yV5ifXiW8?oB(EKyN_5b_&H4{rsrJ5<0$t;zkidUJ7wNp=(@Skbc|96`8H68w% zrbgrTKl_CaNq_C!R2^oTXjD1X`~WXMwPN7qr==Kr`DrUgUVb`?v6r7lG4b*<)>age zItq!@DAh&kDyAZ^p>_l2AO-0sI z>_ygBnu)Yl97NhE%|$j)97Q%%oJ2NKT8M0{v=rGyX(iHDaTaN(xQJ}3xQet_T8nI^ zv=Ql`v=!N0X(!TAX)n@A=^(O&;wG}C(otk9#a*Pc(n+L?(pjXd(nVxzrK`v`iigOy zN;i@16i<=umF^-tC_O~FDLqAYRCTDtI|)ThtgkUH^p0| zr{W{ByD~sz55-qxPi3IUUdkYmy%j%^eU!l>y_6v$`zk|4_EY>t_E(09^j3z8^if8L z9H0b<^i={y4pf3f4pM?e`Y9nI2P>f>hbSXO4pl~p^jE?}4pYKK4p$;Xj!+^+1}LLN z1}ae^gOq5I!AgwC5M_+WP-U#hk;*ucqm=O?!;}dk!GD$f?Ra zk<*mBW-5!BS;`V7URlb_R+cdd%5r9olE}pS)#0CmMZI+Wy%I-xw4T-R5mdyl+DaaC7D^Jq%f|?en>C84|KeJuQ zV0I`6n4QW&CRI7a>{1RhX-X!uTRFn)QI0Zul`LkTa*Rn=jx+m}Y$ij=VGbxKn1f0# zb4WSK99B*-naXMAh?2(~Rn9P3%30=^a*jE!oM*C?3rvo3kvXAUVse$s%t_@6b4t0& zoK~(edCGO>jB81lmh0wa+A5B++r>&h0G=8Hgj3I!(35{n5)WN=9*H>TvzTf zH=B{X~+)ekMs~zmNp>D@khh z8%bL1capT(A0+9pKS|QCzev($f0I;={X>$TP@HjbN+jvCGD!xkxIx5_Rg+}IYLR5j zYLjHb>X4+c8j^_BC8;`FjU-c6k0dizpQIYB0ZBDkLy~H-MkLi{jY%?RO-Qm}6_PAj zBB>5rous;~DM?nW8AK z+14a=VcU?@m2FFs2iuOMZftv!JlPH;b!Xj3>cMs-sVD1BQZKd>Nxj+5B=uptkmSX7 zC8;m#K~g`q8%h0HPm;XZ?j-rJJxChB_9V%d?M2c+wl_(G*ghorv0fw%X8V#fgzZP- zJCMTuB>A)6Bn@MINE*%#AZY~aOHu$kkfcC%5J^F-pSYok9ZXUPJA|ZAb|^_BS$~p7 zvBO9TV~3Lz&W<1{f(;-kk_{wjG#f-x6dO!ZG#f%v3>!+)7wY6G)oLP9$j- z8%t6=JBg&(>|~M>*eN87EdXjdq8%Wy8 zZX_v{-9*wZb~8z7Y%)o^*%XrYuv4rqbdcRe z(jhjDq{HlPk}}ynBpqS*l5~{aM^YA>PSP=UKS{^g43e_h10?0J2T3}?9wI51JxtO` zHj|`N>=BYqvqwqFW3xy)!yY5)EPI@!b8I$A=h+;RF0dy^y2$2|bcsDl(q;A(Nmtm@ zBwc0mNV>+JA?Z4MmZTf(Ig;|(^CT6p7f8CvUL@%jdx@k%_A*Jg*()U7VXu-@#9kxm zE_lFHa3k{+{nNqWK-lk}9m zN76I)K1t>51CpMz4@s(EOGtXbmXh?6eMHhLwv43L>|>JNuun*O%RVKkl6^+fJGPvp z_v~|$KCl%eePmydRK>m|=@a{kq|fYYlD@ESNczgYCFvVmNz!-r9Z5gf_ayyfKali` z{YcVpwu+=b>?e{{>}QfC_6x}}`;}y{-$+)o-$~YDe~_%r{v=t4{YA2d{Y|ni`-kLe zLiw+Ee_0|~k4rMi`YcE`VAUiWvRWh?vDzdXvpOW3uo{vTR+nUAtC3ut)g#%I)hF4E zH6XbLYe;fU)`;X+H({+vwq@&)Y{%9oxhZQ+vOQ}OL9xrj^tKsQ<9xody-w)W+c0^4kWi`o0HszbtJhh>qK%p zwgt)U*_I@CU|W&w#yXSSk#!;2opmL-6Wf~P&TJc!yRdCZ?#i|!*@JCQayPaE$)2nm z$=%tGB==z5N$$yZBDojandIJV7n1w1T}k$0JxK1$b|bkT>q&BdwmZq*Y!8xs*q$U0 zV0)45%l0ODAlrxJL97?aer#Wo2ebW19>Vq~c_`~mvOnuX@-TJ)$-`M+l1H!uNe*BK zksQeSksQPhCOMcLLUIT@l;lv>pX8D3Fp@{H!$}TfN01!O29O-V29g}f29Z3P4JJ8? z4Iw$24JA2-9ZB*Sb`;5D*)WpFvEd|-XCp|Sz($fhksVEPEE`4gBsQAl$!rYCQ`j*i zPi4oFJdGVk@^p4Q$#Luil4r0JNuJ5Zl01u@L~=YkndI5*6p|CzsU*)~r;$9Dolf#R zHjd=^>sE+cszyPV|pY$C}U*cBviWLJ{BiCsnVW_C5n$?O`E zQ`og6Z();2-pZ~cc^kW);V0V#xkWC}`5WAb?!|WcCGugc)A7S^Ae3VTmIg8y- z@-a4p5zJgyd7~QIb!yStRGN$4EZI9w+%Mn@#dL zHizW%>2(WKWWOi9JQ~W%e}5SJ*s~ud-)IzQ&#<`8s=!<@-y}U$>r=rlAp6BBv-JdB)?!Ek^GV^ zBl#8knB>>&6O!MsPf32uJ|nr3EhqUM`<&$WYz4_5*cT*!WM7h8#l9l>6Z@Lv&+HqL zzp!sf{>oO8{EdA_@^|(<$v@Z+B>!YTlKhLUBKbG_iR3@5_!5qa{X&q~uLPO>Mtm{Z zME*`tvp-0Djam4Upw0dw=&-*D8ukxCSBSsf{UH&maY-iVu^{NPYJvePzAlD z9fC2dA(*hb1cj|eAXbl1oz*9pvIYb*){szxH6qkxjS00_6GClPA(*p7uwbhbELl@R z9oCG*y-LCw1S_^Cp&na{P@k<$ux8B(Hmn7q0c%NU$krh=V(StbvsQ#AY(0W4Tc2RZ zS`(VG;#N_6wgI6T+mPVEHX?CPm#{Ixk!?b7Vr>a6SUd5Jc(y5_6>Cp$W}6XQSOeQ(3|Z_=)-ytyx4ApzN{yqAKP8rQp@%r zc(XkTK5Q?-0Jb;5m+eCs$a)b5v3*JWhJ~;nVKCdDFog9c3}t-?{_Ft4FxHnaoE=CQ z!44t>uzrL;ys_JCQJhjU~)vClO|`lL_(c6vAwFDj|WLMwr7+C(LEz2=mw(g!$}D z!UA>{VIdn&Sj5gIEM^l3OV~MtrR-e7GIkzeIXj<_$SxqPU>6crvWp0-*u~=4ci1I_ zHSAKtT6P&BiCs=u$0icivn#|mv)Pq|jqEDICU!MpGrNY6%&sM*ut|h1>^j0$c0FMm zyMeHs-ALHMZX)btHxp9XWWp{sg^A?#+i685m$2z%M>gnjG|LOQ#Xu%AsOWU#vk z2iP>iL3TIc5W9zPnB7arWcLw{u<3-O?0!NPn?X3n9v~cN4-&H3LxddmFyRE7Nyuf7 z5Kgj338&aB!fEyxi65d79w(e(vk7O}9Kt#F1mQfJOSr(EBwS=q5iYT(376SC!WH%m z;VOHUaE(1jxXzv@++Z&d^4W`o0`?N&CVQE1i@ic9WUmr#v)2fB*z1HM_6FfDn@=ca z3kdhvn}qx9Ey4q~knoVbO(GeL{H6J|(He_{3ZNzFwZOrPD+Jvn}YK7G!HL?1nuFe{e+LSdUwHa$f>Kd#uscW(( zq^`v(q^`{psmLzS`QroiDq_$&iNZpieKx%uoA*q|OjY#dlHYRm*wh5^n zSzA&&v38_x!8RpzOV*y$t=MLyc4i$&?ZP%EwJYmL>ej3isoStENZpogN$PfND^j;- zok`t+bs@DI>q_d5Y->`xvu#M-iET^j&TKnUcVXL;x+~j()E=xGsk^ZqN$ttHle#NJV5`Qc;_O`8nP1U zcUhTKQ~pM(C99BX%c`V0vKpzbtWK&YYmn;8-$@N*O;STyi_}QgCN-9INKIs2Qd3!v z)J)bVHJ1%YEo4JdOWBCjN;W37mQ6@)WK&XG*^JaqHYc^0El3??OHxPKiquKACUur= zNL^%GQdilI)J?W0b(bATJ!D5xPuYpoOLivpmR(4FWLHvO*^Sgsb|>|hJxBv&Ptriy zi!?~~CJmN-NJC^_(oortG)(p<4VME*BjiBRNI8f!N)9HCmP1HmHJJLE>vPPvJ+OKv9ZmRm@BZYS-RJ4grQPSQcSi*!it zCLNZ0NJr#e(owmObWH9i9hV14C*(oWNqLC$k338|C6AC!%cG<-@)+r?JWe_%Pms>b zlcay;Kcoxt6zM;CnsiZ~!7h_$u?+GYmQkL^E|>peNnXHA{)f4|h{YxCf9-D>uuMuB zu`A@|*p)J2S4qRJmX2K`1G`pc!mg87VAsnlu^Z%7*p2dP40#Q9le`wYSzd?TBCp48 zl{a9w$s4iTCD0km*aPw|>_K@q_K>^> zdsyC!JtFVJ9+mfFkI4tH$K`|A6Y?SKN%=7LlzaqxT0V+BBOk+_m5*c3$tSSq<&)S8 z@+s^^`84*Dd_hoB_K|!C`&hnF{0943evADgzr%i&-(#8O4_FrYBbHVE zgk_UIW7*{|SPuCsmQ!ZNa>*=MZkZL!BeP+7Wp*r|%z@>XIk5sV7gkW_#tO+iSYeqL zDo?_0jnk}V%23OtcI+N{VspQYRW2DEm;+-EvsR5WOb~rtbx^& zzhm`fO{{^eg*BA5u|~2E)>ziXn#g)sQ&}HtCL3VQWkalmY=pIxjj>j;3D#OR#oEYb zSXgKUL$l&!H&vJKW*w#B;0c34;09_uDMVBKX$tcUD`^^~2lUa||; zTXx0z$Zl9)*&XXAdtm)#Pi%nfg$hRA-{P}v_FCI?`{V?PaoAKj9-Ag7VAJJ9 zY=)eK&6Ja|S#kiVXNhGY>ixjt(AXZf5|_wb#f*4w_Js-m#eW2 zat*douEjRVzp%}69kxaOjct|dv2Ah#wq0(-cF0ZGPPrM|CAVO^h+_L)b}q82d*a!A{Ae*lBqT zJ0p){XXOd(oIHu0m;Ye@%2U_{c^dmqp2054v-oB59G*d*$1}=*@yq1}JjwrXlNWKn zr2nt|Jp&#}8SzZ=a{LOJ@GGU^S4qdOmVsX*GvU|DEAZ>&mH74YD*Og{HGZSK21j0t z-z2ZYZMaJzJk9c zU&Y^+ui@{=*YS7d8~A(jP5gcN7XE>J8~;$ggMTF7#Xpws;h)I&@lWLk_-FD%{B!vc z{)PM)|5AQ}egzd1Q7xugrnxlR5GHG8bMz z=Ee)kJa{3Q7cVUH;YDPAyr?XI7n23?;<6B4LKenL$|86vSrjiVi{WKtalEW7ftQmd z@$#}1UO|?|E6Or>C0Q1)EX(1)$?|v=SplyqE8^8;CA_+z$&16Hoxom{Dkd5({ zvI*WwHpN@ZW_TOf9B(UI;O%5fyuECNcaW{|jA2A1Vjn z!{k7GxEzF!kc07&atJ<34#h{yVfYw193Lx3;N#>-e7qcmPmrVWiE<1+Nsh%Q%W?P= zIUb)XC*afMM0~oOgwK$Z@tJZ8K1)u;XUl2$9624ID`(*Ii%j80Qxm<*=kc;s@PiqL9WF&%D?bUavi=|{*7;u>+!8}1HMge#J9^$_zt-l-zm4?yX01U zx7>#Bk=yaTatFRo?!@=YUHAdH8$T%b;D_X1{IJ}IACddl(!Oh$=is#Ls;h^OV_#53{<;#v75@tk~$cwRnDyda+;UX;%gFUjYKm*w-sEAj>6Rrw`@|>m z1L9NpA@P~~i1=K7Onf0fA-J8@typN_+EZZ{2;#}ew5!5 zKgsWipXK+&FY*WCSNS86S^h+1kv|hzi2U*;wX$UHL`7Mas3gk~m1TM2H(7zGA}bPAWhJ7TtV~pwzY#TL z72qPDC-)RDgvb!APWo~%XGm$iuovJTNu)+HLrdPHMcpJ*Z*5KU!6 zqM2+&G?$Hu7P1M^QZ^-8$!0`r*_>!2TM%t!OQM}@MYNZ#i4L+2(NVS~I>~lKXW5?U zB0CUWWk;f$>_l{zorxZ@3(-?{C3?wjL~q%h=p%a&ePvIgpX^2Sm%WJrvJWv(_9X_% ze#BtepBN$s5JTlaVwfC643~q65poDIQVu0X$zjB3Ih+_HM-XG>NMf8EMU0oDi3xHH zF;R{sCdqNcWI3LgA}0`2(#NToav0knvHpst-jdC5aN&Zc2mg|Wvas#ndZX~wJO~iJ& znb;w>5If~oVwc=T?3UY!J#q)JSMDVC$z8;Lxtlm3_Yeo=UgD74M;w;>i6im=aa0~8 zj>$vBae0_HA&(Fz?EnR_&I@g5nDhkOrf%Y9dIY--hkbE^KF;-7WgJOay!+ z{s4cJ@y@ou0-8XR&fU3tA7}14JHP+@_yu4Vqn5jA^rDzX0fvTB8eJ%+Fs|%G z3X{rC^}ximh8Yd_G|V!jcDYlKydfBJm2JMuO^3^xxfXb~tj#kdgSzb(rWwX7u2(a8 zlUsFR?(nT0Zdz`m;Yza=*p6?$Qm*y7CwNt*E!ffy78#a{7sxvcz}R3&l-(-9%T#PB z-n1HrblJ#6rxY_Zf>mib zu20Oo7D3&uYIvw)0FQL^Bc~&StlB=-@dR@WBd6(HY1vLycnrhuJEzNw!{g;`nb$Fo z1%}uhJw3&uh9w=#ct+fxoBF~G0YC2hc5_=;frb?Zm=WJzIk+r;QUfI)gYL{DYf$H`_&<9lGGi9N(h2Z(en?OI@?##~f%(FAI{q_6%PXIO43n<{S4jl_Ru3%9|2}?Tpu4=fZ zBagB&DjG};%M8YIDMhgE@h}u2!;t4Y4C4(yaIDbVtLM+h>b8njmuCja1 zY}9f|#d4gVA9A}TtOLGxz%AQvw|!}KBG(J87s|EO3IuPGu&m_mnD9SU48uWB4Z~zV zjdZOh7sSlKb{KSS+ae50)oPXDQTY@yD*Z~9IU*W;)9(bf*l^XYiz(@=O0NL}XAO*C zMZ4`wYeTJDqrwprU&wJGFO(rB4e6e41GiP3z#|IMb7+z z{9BVwv@V7}#pp5mo+A`+f&L+;F@{;P=deh*#>s0Ye3-x_S;UMfI%n~28Z(q%`OXu1 zs5ng6xtRuz>qKZAXwO@^pJDU@?6rZ$oQ%*WIjT#TbG;S88E1MzEREe~SkXdf=e`eO+5sM;%@GE; zftw8LP7ozRA4hMrmBy60m6MkrW0_l(X@>BBJuzIrxa;lsM8=k3^WU_4i{H5U<_zt) zG)8Lc1P9eaF)X>q`HL7$jT94XphwmJnD_Q!H#oTu^$)kD8y)$^9 z;T@jN*C3A#)GGs>O0Z2ar0xzm?^l<02_pxZ^zDt=z2r0z=N2 A$N&HU diff --git a/out/production/clients/org/apache/kafka/common/utils/Time.class b/out/production/clients/org/apache/kafka/common/utils/Time.class deleted file mode 100644 index f286a627113d1f947aa96a124f115ce3fcc562a8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 193 zcmX|*IS#@w6h!A|B}>5>sFDUZoPi<@1&H1l6P!49l!e1lZ~zX4a3n-CPnvnfyg$zu zfF1G}X^f1JcS>tDN@2})B=|*H5i-M#{deK0rR)hoQI%&xdbIN(NKUGg+lD_l zVPl7y@t%vjWG%n7oC(`?*06a}dSr`9ST8Qq+|=xFX{4AU0>N{q06%aC5CD$hj}cgra6$Ia~NK0GZt|-1hkq;1zCIu6DN&L86m+QfGx4FBDf6Ay| zz?Z}y;EyuSwuP3U2_J4|cXnsynP+D2_n#lX04!n6goQ~H5An#rl!-VdQ^;zaGck>P z3I)9>4qyhe2IdUR8(3gSu6ojo))?Z2;yy!SLsbRCV8xSSyIDUF!7e}W$&#)phx_|H z@btc8Ohh#=WU$^yDS{23hoPWtzM=xx<_+%Dgni5pkGburdR9Ptm__PKQJ-C7`cwS3v4PBetFhRY(2?8<#QAjqSa4?1oP z=8kFxj(Fwiy37`u1s#IQ-jt57LQlF|BC4sXfs%zqJhpHXgBG4(iD9gpZ6^xo>Uz`j zt0G{S*y~i25$EM(Jei4FHDvM@p5mE-=N4XInIV>6v9N+=17!=Vc*&5v7HteeT?B6r zj)X%T6uYK2W!ZBq7tFrqvg&J?SBfFwpR|GUQN#1A(S!J02NAljNm-<7F zJ;a^kH(U{I^EwTJPS$0yQwl>WQtdUlYj%3EF!YHaP{iq_p}k52WxE(!lrTU~mEi`x z+ZcT`TC^LYl}(zF?wdIS``S7lCYjVm_6ccPYe%|(TNowHa2sPJwX;jFJ5D?sUV=@f}m^v`{V8Ams`sWGt6$g(f31etb_P67&taolMo&EYQY(d{_y eQ<@kaV1jgl{x}(o&?-%~0n$K_hxo?HY8Sl?#$bFeE0b_ zKK{o24-(NL?YsaTSC zyMnx%-zKAr<*_HoChyfb>^X!@LGEU&kbAgS9(_TM@FlWqUl91BGU}H{Odjzd5AfS% zl#oYK9x0vEL7K_u2Wbmms`LIJAK-&JU#9crI$sf_T0Y;;?~wPEvUHV<4h5-(FVy*J zKVRed$ok_qcy>PZ;)jI8cLw<|UmN6i@w?@5U63a7_40U+pWiF|-Qef<`T6|<<;Ea? zfNv5yKUl~&^N~XS5Pw)6AJO@vL0T)4IvS)_zD2%#Or&tD&L0obMp^kpklOi^LB5SY zB{PR~zFk=ObdW#8pOw+)bUqfOPMfpO%k&on*B1j2!W}`rlkXC7d`X1xWqI5!k9%ax z@gU#JU(xxiK{5q-D-R0k5K$|i2=EZ!7v%f-q&yywtzQfBgCdhwzEzH&()sH_>gR9B z@o&oGA(?(yMvnyeQGU$N-x4`}+s}^+4c`&TeOKr21^Eg7zRpiF8Evtcl~@xslSwPd zRJ>vHw$6*UY~G5;s?N@JTRWI4+AlFLH5=2ZNVG9w^;!oSx0tDvm5426(mL9iO6(0$ zGuGRe(#(QNcp6G2%Mtc@r(%2sF?X?nEDp(eYMN%!;Q&W2ZlXp$L+hUr~ z9*J2U>HfV|qSM?P#S9LI&FBSYA|l_NNpGqz0zzjiz~+D%?z0;AnLYc=#&Ep9KOR$b zCL6bd+9Ys!)2W`tOp|MNuC5)8Oa`7g_Ch4yxILCk4-CW;DXV*3EFA9!E9(w~t$|b| zF5LR|9!!apw4JM&#%)WP;eDIT0Yw-bOt74ZqE_}6B$ZgeWWqALfhE!#v%1@2DW-Dg zObqB(1N`cAPYYyB3LMagJDK3T6e8)cGk83CXs}se*V$7nvIQN+7G5LHVgWP*U%AVI^7aE+IGz@Z(NpnNiG+D@O#Lg9%U~X1G~i7P(lZbj z=8}=iAb4*#sMqO6fR4pe9qDKkKH(;O^WICWaB8XBft)E24T{_WAiFR*-+K=ZWIWi@ z*$H0?Y=aM&sdNGaOk393?Bd%2(NcqrwQ9338MUkd(T38x)(_j;&r}$*_P527DH8%k zNN|&^BHHK_&vc<`i$<+pGrB4qwvx%LWrMuDO$tRSVoDfrZ{jel-384?$~0-@+_F&1 zQkCkiW~w=ZOj#XOwt9a$f&Ugj_Y%L9Yc7Jw_leHTlnHBu;V47%CM~NQOa1sH@PT3J z_8Jc8y;f>NTr4zCtxl|Vqt1qJ-R$m$QNu{|elvw23!rBvf=%`kczYVfhK*=J3_?5Ko$`xK*IiQ-HXM>z)JT@n{pUQ)Bu zwd`1(Yi1O(v-0sKQwa!X;wTVc+evG{N|>oQgy!9;EO_M@CN8UQ5#=&RDk>4LiNkWk zDlfgo`LXU(~)SmO50vUbui=@w*t_H8DSF+Q)!bi0`4-f z*Me2I>0}PsYpDHn{sB|h7y^0|uy2v^Ks0W4Z%M=lps&y{clIUX`=zqU zbN;M_IZ0osBDYoagvgi6U-tekNx7m1Z2+szQRKfZsK!#+#Q@?ZK%oR-+Tn^x8OtNr1_#7mTTuRlb1-Wtih(RRgsg7ULYUf&KM)f&hDaxF( z5qcOfrSQ-lw5W#ipyrVRX(yG*W*rnT1*5wn#3m$ybQBHTPtYg97y9>YlRB?vwD9S6 zrVVDY4?97(&x%_85ZRr+*{Mua`zh1vF}NK8 z#6IC>U$AC)51OhrnKgg@e7ICi?Pv{^e_1%{v{b=u@pK|=i4qZewo4Z@Pue$wuBK}Y zdM6!*39Ek272_d{?VYWQvCKbXnx&Q@wKwaAU{^UP@a|Nk-^!Loh)Cx#lECm4Qi@Gt0tIzMCZFZow^(3cS|4SrU>{F;9wkKgM2JA;4E&q3`uG)~^v z`FVrBB&Ysh&~N0_3;d$aFB$wte%atxFhi&48wUT0Up4s8{1=1&%D>Y2ZwCLJ|6%Yy z<;cJI-!l3Siq9MZDSfw~G(afL@?y{v^!>bDLPjEy9*`z6(;Vvjnn6$Vf0?Rt;6^Vl zQG`iPJX-5|p7gOl-e@EY3=L+YH{EZ=Qq|!&4kyxKB>WWyy@RgQ zH8$uaO+)!-XdXyg^J+drE6{X9^J@Y4XS1ONwL%PbB6#g?24$L|YvT-Uyf(q0f70u^ zR%B?!S_x_ZLz^fdCW#a$DbAHs<Vi6mwj zJcTO_ZMHT?*X9~pjSySQ&ly@stAj0S^@i3UqZ;}KqO76K(;9I#lOa}|SvXu~X!EtR zb!~y6Ez-`>wI+ie(#|!s#o7`>Yu1+P+A@P{Ih1i4B|awz3m3}`twmT^p{+D{seoI> zuj<-rLtDe^nP$ujt=P45Ma|Cn4NJ_1%XT%)Yu+8I-L-o~vzppnS8Hf%wRMQN2sYL{ z)VQW9{tT^^UNGn?I>dC=TM&${tv9p{8oC)*AhN4(mxkIlpWz6KANvN%X*|$+<_slM zSGs}ZZZ`Pksp)VYbC*dsv z@XHJ=zupDW{mV&?d{< zhGL{1`y6PLIS2!EMJ>r+rAqoE0OC+^-KH&_J1%Z-+tvxY!mm{Ws!7WWRZh@$B%ol7 z{K>5l*5K5{psH--?`y5dt2VvZXaD3j#?|=9li;B?mp=ZFr_l^LNSEVxVGm7_Rsf?4 zDFN}GLTEMdo=YtKCZ5;gTO;0#w=Z;pw0khn=v{agsL2q$8)L)XK-bar_!gk|(0ef> zhkMD3{X}Ia$+II=;yppWyW3BazN5rncY*?^)WIMX0-Q&I9gjEBGR!Oo*cJtTGC&xZ zxq;rN_LtH7=|=4O06rk&Je_V*NI&Qx4Le9D0^uM9&qP>2`Ur#@F>@aIsXdF(??O0{ zZdL+v5U#fo9-$982p0jNh$t8uq{3?_k#W^s_~w>VmrG7T`Y?S2>j64KA9WzCa7Zo; zRSwelQ9RsELxw`mbdWQ_b3mR&{sV{9yU2N$A2@Vp zmc|lbCyGJ_{WO7MR7PLA^a-qg5+C97Hihg{4&@gDnP_-csPZuLLZDM`h0q=e z%F2aBc^GZ$x!lFlEU+X>1thxNVd>NK83)T+@+reGLeoXO*9}rpSI!oLih}DPjO$$x zjjWIi`Ye48kOFj!KJP%>3=0sh$Av1Z9gxLmfV>%YcSM1#R_MoNL5_3e4gd>N1FVI(^E;PBq;@qB5dt zU=*FZ74ikT%VBIk*b*PARf?$cTyayUJpu{YVUZeE6_w*Xl88Gg2wIslL&RDLLQHVNl#)rek?uK0`jjCm75Z&Iw+&!OfwsfP1R%Nv z>*8cS(t>C_uBUtGxC8Ml3xa^5BvDo@C!4< z_iQ9n)QN9`mWQCohr!b$2yzdBug4VP5McVKlI{g!5&um33hYGw&7`l=AkJyndk^M3 zimXENyiPtZPkIg819XCh9O7QaMgg$Jv&^$RbRSJ-YN?ufJFN;;-A82_HF-*{Qxoh zhal}oG>3jn^XMnEh<=Kw`7>HUKc{u{3~j)^4tiFxxd^mXAdcTh_k&?CZBz_vaF;c7 z60D20Y@r9}Yj|ro^`NpY(ZeJvc!LU=`7}-8aYe6@_8-OhDf+s@`68SaMF&G3#FvU8 znsT>-l1mZKD;jJa=RzOu`5Xv$0;0%nVbufv4#229N|W)b!e?sbo(50V5KTL#rlvP} zLk)ve-4!ZDT%Xa^AgF9jPlMwSy#nU`1h!tKY4jJwk-t)i{zhlhKeA$&1u=dT?1|?u z%sTVJtSv8ub=(3Wcp8;Ypgg;{=^` zDkMaCbGZx~yo^$tJuW$nr*F}>vF~wwY{|&XckoT54v8A{T?{n(9uVRh(-BoCW<*(m zX+t#Y7SDKi(GgE^rSLy!J?CfMD~>Ap{HR`HcO9# zbcyC0A^ka$IBRyCq*^c*DybW!ddxxr^@B7I*}L)7SnDNcyEZ9!;%9RK*9$`DJK|9x zkITr%wQiBP^sUuXJIi7Ux|^|9wD{u!8f1ttw?<GQmt9^e)9B(I`pc{M%H>*&wiO8>$QC3Bn7Y7b;Tmv+&w z;A4K;4ZS{#Q2;SHu0j%gn7#CCRR?(BG{3T`O7eyhAPK>wxRVdQ}q|BHT)k(Wx~ z<c#BfyFw8WJ+fbb5KE z4zdCWORqWzBM!pp`3Q^ad_Xuvi-%~*XlQoT z+LfV_=0RF|lxAiedztOn^(D(O0QW6$?_)H@S%ysN=`1<^|S zJEE>F3Iv*(3v4O;6Kb;q&1Q$3Kxl|o)D6+fk#JKkxB>b%{YSwKK!=9_H|PDb_Ft^o z-hLW9F+!aIp1Yj6;pc_X`mSTczmza1 zRN28?Wf;B^TDXcP@F6PatLZGhhU)p9bPivusFivG*kG^2c+1G9)=vx2L*a)Mly-Hj zAr*KO2etuWKdc_Ll3J5G*D;{iX@brLI_tA-?${5{46vBtwb*|@cFMW=CuuDr^tyK5 zbdp+kpiNu9sSd<%sDG47@oqat<2x|cW!!M8?rz902#-GtXKllt1E~?e54^n}e=ht1 z{4wrL;O}Ot;v?YkX8bAd5t_#zRy;~%22L)dLwNhKW+2T7#5p`3#jB>MpGKZxAn-u# zI)jTMVx5am!yUQ!HSCtswE%l>W297`$dfQ9JXwG)@GDQ!M(}hVGEDmp6GbCFm{H*hKT`oYd*q!~2-V6GgA2BoU}tmB|eLWm;9 z=_hH+jyg#1{Ei0T+B!(v8ctFtsMtP87pPjpjrI-zyAu)eE}F{6UD}_{RZ21d$WwV5 zkjQ8{R|9R3N_hs)1cw1Wi)T3mwBWl0l3=I~m0;HpU6^Ch_OS;PeA`-rS@=N*et>5q zZrk!}0VKg_9H$-d+=~EkXRf|i%{4!a!r&29b&o2jcKKoOTt#(&@k^Ei>kh=TAtuwkpsMZE)cNeDK=C00WLJ7p*MKJvy@IFD)_(|B+Q}_$-r|E3|0X6ZD zToPQIt;-f?>$1fv7l=168m%rXgFtJ>u1PN*DbQzJtY>Rd#xHOXAIy+77^f7JTk9Sp zeTeqJ3|mnMnc`e~Pf!?+=7C{!cfe4rW^aY}IQ0Z7kTH4(sjt~rSAkd_8Kg^Oecv%M znhRu5QE&=oBrU~CS`X;!sZG6YCk(+nh<^`jeh$v}JXP}xaH$s|oR?@Nzf5cS6z6CVSL0MkM^$<%F%&`IK+9y!41uoM8=YSiZ5 zdti|9qqGpOf%dvbX=dG~`UhJ)(v_h}n^a%wUC`{S@D0)14|`FQy^MW{u4CX)-r_zs zO$(6KNrO{p1?1Co!Y{;BtOcl4!~d+H71BJs7ir^Yi8h{=X%lFbR;=ikkXef|cR9B} zKg9@{D|n^ieKoJboL#zBkz z_vhzL*=xcj&d!RG)DWfbr9~x|4$=O5=|IWBA-e2dVv(bL+`i+RhoCSYy(}z>JFuql dX1W4z*d%r04I9&D&=zfGf&32@_iGC&@IUloR`>t_ diff --git a/out/production/stream/org/apache/kafka/streaming/KafkaStreaming.class b/out/production/stream/org/apache/kafka/streaming/KafkaStreaming.class deleted file mode 100644 index 0ee57db387c9bdc0e81d729ceff66fa1bb156027..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2626 zcma)8TUQfT7~LlqW)eoxcvsNaDng*7wbWJ+tYWli0Ba(6tq#c$h72=tW`a`t*k947 zE?;==#amsq@BJTr?X$}V>^_qa0!va~X3q8d_TJw=>_l?-=SKpDH1IhW5|5D?=6+r}K* z^L5*w)~%`$CSq{Owd<3Ul6&nbKJCP`-W@lmY2>2v-uzZF}0mBZPOnWFnT^LQr@XJ zq_}*(YFb6oYPGADi%3Xy&MXc0P76fFoFXZ7WKG+ctS-(PZeE|YI6}z^x;3r4rtH@T zg~=WMyh*BktukGa%p2*X*0VKy0;gK-TUTf^FG^DgPz&_8iqJ?}=g7k5e7#Vf&?^B4 z+E0N=^9jT=j|xV`H(7TD3$=2(J}uKDg=wo5RbZr5x^=)?lvF`4BNr^kqoyI_kw7xL zpfBlZOQ$2or4QU{#W#u#FvC0vbhChZreM|G5YIVPw_sc}Wo344RChpz(~!c5K(Dm< z7Vg`)OT!5y1yU_GY?6|eu&av$USQaPy5%*LVJTSDV8am@ZV~SPq|(rXUdf|RLl+JU zbg^)*PrD`q=*b$MWIc|8iiQVpHJp*mk2HyDSw=~>xZ5`xstO#<)3gcugNKI0UYFmKf!+a?s^6FW?%nv#B)D}#mc*Xe&Oyg zKH>8)6h6lpFM}8^;{dMU7_#`B@6(vX0B2L2J;ar1Vv--aA$*F@_?E4cTq%OqAif~J zQ*7@==p8pi6nv@RD+Q;y;4PIH;+$k672*+PtHvgbC}$>qMSDVfiESYebNDA5m(a6) zuSdwH^9gqKb)6iF9Ed!__JG8{E7D~{B5;*6h7{%E@;E6g>NvZ>pRUA_AwGeNxI}{@a9_bnf zaf-UManX`Qu)Dz_@-rs3BC)4NV&imYgX7*6#D=0Tuz#jju{E*>0 z3GAy^b(C;{5=oo%P#N8@=$MgXJZfe2k?IF5ZrFW?G&Q^LCy<4SQRJt`p?@{SC zX1rIW_o;Myi0)A7{VLt5(g#$!tC>DXA8Mg>^x+oz2z^wgkF|jFakhMSi0)y@y()d8 zneL-cHq)o*(<ly zyoumU#m7ZAizu;dBeU+Y6=_!@QReG9Lo>A#pUgC&KM^!qe(s7lZQ|b8-y`a+9 zLiD0Si+fW8dlHu^)Sh(QVwYagb0wqekX|^XcjX+*a%_Cq1+&=ID?a*+LbpQ8(wSH$ zp6yEvWgGgD%CYu<@ z#E16BlG*hw{Kby`B#bf>1%wBx-Nj7JZ38k}l@7uE{ zJ{0fE4#j&@Lr@uv4`l~qLzzS-ks6qA>dceO6UNs2QT8U|v4MSq*}ix(HY`WBZVtIn zUXw{>kcfsGnN)UgDAgNJrxjW`LnJg-9Y_rHr4D2>iT=2}Ok^5gOxAc7*6$z8#;?i@ z#d!)2@gQXVdYaL@b%O8d;jAjWX_wpFqyqeepT<&+zq6Fk{W` zR4Ri4#|C+o{qdnm*H@*7)0sHx9!GpYXPm3vR6WKO`GGn zMp`cDBBDaNTSNpaW)Ul-_TtT9{ zEQPksY<^|au)QjGKaoc6e^Ne?W5Z@tMD?2IM7dR(9?D$M%T*&pFNf(H^i74?( z?y(Xd`SMK5K>xKWQR6Vy%z2J`qF8}ZQJ1c%1p?@h;G*G1Efk??`?hUZjeW=)lU^Vp zz5_BdM6ZPDJM`T!eUDxZ(f7mj1NtEX;2K0tC+b<1yomsQL_ZGG^|VK!wi$_-;}qiR zjlwg8=NVC03DHl&^fP)rL_ZJHFX>kboz-W$0IcKIwK6s`S3xT{t|nCQ*Kv(#n0`&a zf$x3+qo(Z{F5*C>Vfrl_`(2oRPj7_j4{Y(0Ov%s+u5n|{uyb{7%=AjySjo(lG|$nh z`lz9Km{4)5qU4%W@M+h{8P%%eMimNGoU(8Ik8E%k8`LeQW_w~b>ZO^sW|`%(Sv7#Y ze3ea)!2d(qm=3dcQN&={m6`{85xZtFo`998SHedgj{k{XSLi%84Jq20?}!kxHRG_6 zt9b@8z-k(*&hEi7>}?pS6$*xH}UhP*ZS2lFJRsnX(dR+i2JHH%o^xBAl0vZ~W4Ur1Jlw&2s*7V8Mw31#$4vW0Y z@>qI}=Xj`TBO>hAAm$MLnaleZ;4Xd`=Jf)~FC_%!Ghs=$>smt@=p8fdyfYT%%_Rc3 zIc<2t8!;OR)-~I)J8h!~%mEW(UjQuSvX;|p)c_<7P;ko76Co;>jl7R3VQ3O(#wg-5 z1(*;#JhB}P8@s9(8Nh$#YNyf@{Xol=(Q5=sut^bu?rEk~DI45-04LP1Cx;Vn*Q81h z+8h{w9^xXxg|+;mIhTm5uv4t{*t}6xtcwe8o*Q zxR**-$AA;ucjZZV3%G^_z$mnN78|o_ER$Zgb?gh=7n>aRVwJz7>EObI)|nRPDQ?HD zp0n4p%E5zSW#g_aU#oFB6IQ3{^8J7#^A%^yivd-PoY<4N$&h@a%uwH%GjE}fytm`GZW>TZ*Yic zVKT+#SUw5S1kP~W)d3}O^rZ^?u1d(j4rZ|S-o#~lf#ziD0RG}znBZ>ie~puLZ0zL~ zZI{c+%!KA5@#BNc2|7Qxe?)`JhC1LVl%0MY6eIeurBzh_2S4s3P$cH7m74+2-y#(|%5*-13twgT?eVs&K z0{eLGA=itL^DNzfoa>;{)PxaHbNe_cTaVI$r$98(2>!MR!GomYThbip)&4}vlK~1U zRG=b^l`szD)908N9mIG69m1$g7Dl|#y}?2|+m8VKTO*4ii;q*Ni4JUysFCKzu=O~F z7cg&0W3V(BJP8}VUmGZd@k0pYVOmCy&}sB2ZKB7hlOCrX^n~Q=&cN42blx;yeU3fh ztAkNx!q?jaUpoU|mqnI09E>#lTG1HHbZ{LlhJ)j9@L4!`43#)eXVP=Dm0pmX>zr_| zZH9Bqk)3O%8_9)>oXBn@l#{(=oK~V{M`+a%TFq5kAXV)pRODr`m|vMCR3#5{RI1Ua zZ%7#tZW8?`a_TW!^E9n}p4JT?rPCs(kJI{bT02e~B4->SEwXW(&RkyGc8ty%r%jQw znY9_LWqH9lC(_28E#uU_FkozDMh6(noBk27I++!PR-`N7Y-7%La8}Z}$LRcVIwx`g zvvx?V4Ur3()jCc)dFhLe(8UqNEr@O=A_39E#EO7;8xufgc5Pj=7acdm%(Neylh0;2ustZB_DOClkKq?JTDJTaxZJe#Mjblf)Q^9}Y z!UZG_sAql@V%Bk0LrM;w*`0mw&Cc`w`u*c4fG4m`lyJwuT?5M|idZpl&%k{Hs|=+b z8OUUpp48+^fypiIYh1p$t4neB#5AFJkAQoz`B5-PK1;h9V!0{Lc)w-8$wc z-0?Yq2|}D^uCgB_!V^)m@qunU5JafwPBT2Iop66lw_eB{o>3Xr^F7{mkA<7$dt?>W zV!kcCfG6XKw7i_w8EM9#gUQU!VI+Fe<%!tI@&7;AE_q?L!*S$_S5g^YIX7xcIYX6~ zf$N7cIrBy&{jg`?frT}!TbRQkDEl_?t- zwr0RDXiATA$YsE=WPhBtu+hObJ?Rn{40`MZ`p5;^nY3~!Gs?w{?_ghPq2Urm1qPnM zqG(M3D5F9<8V@y!iu|0ilGF4rD(0BYl({@P&6o;N$3m*6VG&oLRbPQb6e-i(e*lU7^#lL_ diff --git a/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$2.class b/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$2.class deleted file mode 100644 index 812932cbaa85d562c888d47a6968e03d568d803c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1423 zcmb_cZEw<06n-ugIuPVVH{W%->3EakG-|@K#4MU@W9|bB$>OIAyFy24Nm~}@uhK7) zxJ3Qzk20QHYN#-qEU`&@&wYB%^PGFmX@CCu`VGJ{)RIWxzKRDbR+5NfHHiows(6&d z0@hR%R1{T|7~&hcsk@sD(L%Aspln%3f+5q;P4RX#Xbby*w+%w_4XeYA7Pob|_d&(& z=?+6`+cbr}WpKw4v|Vplc30y=-suUg&)@gC=D4=t1KsRu;sYNH4dH0JO%L0#+NfPlxMpy2CKrmS&m+Y!iT*@Q+ZCrRZj1!6NBe@f^YcU`d?I}q5>35p za(85t{qg@Nu`#b5K(cD;3LIAv=1?Sp9e2es6Fa|u){O` z(~^)D&(=SjaC&rk_+-->*&Xplmot}{&PY`b1tG7^j$t`86nnz$Sw||$DOB({g-b{= zlrNAKLvA`~eiD+VDjym{cKUX5<`k~rDnl(qP~fw}M-C_nba;G_lcwIKfQ32dQ8Ka&KgYxK{F(oIdEU79S7Rz_>AGzR-j zMg*7X8TY`a$dD~v$pKm9X=SpyC`yW?OXVMkSLpr%bygslY1$W1e7ak>?a|)B TU0M~~!!k*OiovKX@>=)}hP8nS diff --git a/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$3.class b/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$3.class deleted file mode 100644 index ed441ad54a5161704aa4438939c0339f7dfab8d0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1076 zcmaJ=O>fgc6r7EnxD5_TQz+k1N})*_>I#rJq^gP{kOCRPa~$ z16;UpDG~?NGd~LP?6|6-Z4U9<9lx2`9l!kk^Ya&gCwN$dhUFqu+|{t6;a(91tZKNg z;em!VhQgL~Wwgzp)arc(we1ZBL%Azm@p>{Ih+vNo973vH&*V;@2QuELpc;*2$e?#z zR|IW`hoPYDqplZN2KTu+5{At`+S~}EK=842Ekk_bW8V>>@v4_#FTKI);}%1n`@Zv; zp;qhe^8;=;oSLbHIIRrQaU)@gpjCfQC$W<~g%|qn! z{ugNKSE$N2f|QI*JCRwQwx%*f!Xhpu(q&x65=mVlULLv94y!yPm#eAEQzFJLrN;UZ pl;#oCL*!$R^Nw0dJycvvx~}6!Qg7lGRTZ~!huo25P@oFT{RK=U0(1ZX diff --git a/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$4.class b/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$4.class deleted file mode 100644 index 531928937226d4988b0db216b783b0e80ac55a2f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1009 zcmaJfgc5Pj=7aqBwdqfow~Kxv&))fKA5Ayoycs1!&6DJTaxZJfo~#<3&Yso+0x z;Q|r|)H6S-V%Bk0LrM;w*`0mw&Cc`wKKuC#z%FbPCEPP`-@vMgA|4ocXyB27HHOlj z3}mv;P_P?a2K^xH3x-Nt2I9?R)DzKr-t(zaZ-*}TyF8L=pB43FAY%rr6$BzW@Oc~y z+CFWEk>~I+cL%~5@=rtV#7QLhNCuuGKJ(Go7qRoYlh$5^y|vvYLy?ci{uc(@ZXfd# z?)aR*1R+i{S7`-_@I=&Xe4rZ-1rh4G(+p2)C)}UVt(S9$=TwG`e2=%iW8o(G9$7`T zSnNnI;K?K+Eib2aPMS05a5}T|E)sp|@VAz_YGGhb7 z_8j;nP3cjNxC|Ip?2oe+HoDlMCtU)AL65yaAGts~lU5F8M!C581MC|uG%Qh6VBioI zMQaK`85P>mc&JfS##D$pmQytiS8)wm^#!S|WNOa{t<=h! sPfnq2oucp^Gs5{Rl`~--*VFbla5K%faGSD@JGhH0v@)qdkuuHw2e0k*Gynhq diff --git a/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob.class b/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob.class deleted file mode 100644 index 0627958ac3024a8520b4e1780276369baee076f0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3323 zcmb_fYje{^6g~1w#ux%hfrhpeD33fyOu{>%p%9=vlJIaE>W0$F^2SjjTdpLh34K6c z{FQ!gKafd1^kawVO#72MJ!@G`oMhB^rkRO$wR_LGchBCvTL1aaZ@&XLi=R^H#e|B< z6e6;{m&E-f9_++JWRsXu@lg`fDjp^Av5H(0pCmD(;&BQwe5&F}3YtWHmc-|>tEX=**dL!yVnG^PNt9HSQ#b)zJ}N4{P~j*@-!7Mh zGiK?oD_jMM%Vyd1MioT6d!`k{#_WPn&@pb7#bmXV7tWNPw>YAW+lFpU>y9bk8-p=# z&Lqd_aoZ_o^onlGiOhmNyP#)W&k=gbEEh9kNiS6_;b!h;>tlE9{0RM&bhE5rfA`Gz zyuPSsES;t_AY!B*hwc=~+_}ZHf_TNTE1d1{u~pBsG81}*v8MtZyKiK1}o ztW)i193Ja>+SH9DLsUG|F1sqYm8q;~~t~Y$?^raycUK&*_a$-R&OEXiq z%%?Kp&u?;N*rk$P&ICrfTu$R`X4e7tEb3NOv~XS69#=!L7BU%DK?FW!U>CPX=BLW_ z7O3Wj*2>rEnI++RtVRQ3iE0_1O)pnAd7+XUOV=aavJ5J^FcnbT2pHrheyLzEgl0qL zH>!q$JsT?@>z8L&Ok*&OB`mY9hLCPiR*!L;pQiB@zHT2%m*3!9mah<(!8-0cqF@@j z$CPe2!ZYnd&ZhAU&jV?O()bSF2XCb72?@Bh;>n%*A&np9nHvZ}ZJ}(-Uik%&-=?+g zYo2-qyVg!~{Uk>#Rgbycy}4p;9_iUU#1C%++WgAZKaaDHT~cs$%g<*U1(WqPtkeWM z-Y8tWB$Q-z(!00iLM%_q3PBkvthznh!uZ9FVI^iw%adi`@wS{bbFytec}FiBa|)hN z%aE)Hq*&)@TlNj<~sb!{gzWmjy=E-nYxkFc

D zp8o2-okEC%JcpEvH{oC}%=f|}5)Lhc!{BfUN0f^qSRmm@2}emdnyAM}sFSdeKIY&yr8K%s!cQb@lyJF(pAz8;30F$E ziaxI<@G}C}5ZFZES_0P*xSqfb1a6dYlZ2ns=gkssA$+TZ+X&xI;0^-6AaEyvy9jJ1 zaJPhvgaHZn(CA(XTOE@ zzD3||0`Expjf8h4{8qy6B>Y~&dqn;Nfj>%kpYWdud_dqs0)Hm(7Xlv<_-h&b4gT&G zyzU=F`6q#Y5&2^R|0eJWflmp1M&NS-Ul90`z<&t*m%vv9zDD2?1fU#&$LlHbdWy@% z^KdW%Pl@!D67~{@l%6u_DW`Eoxp>Z_c|AU_rxHYETf_0o8`^`RB@K0p8rneAHpP=Y zwR%Di_ZqdU_3qVrZ8#o{#$&aKWIP=Y$0N1r{)CaLZAXTvegueNZ4K?MEz8;)92r&3 z4Z*rab-}ujEq%ot5Ty&-T9(olqgw0Qf{nq(mSrJ_kr+kK%j+9LYz6J;w<&Ius}D(`X*b*|2rXUBVrHL%Sh5CG7n&yZr{qo)bkD?sJXztR3(k}Ds@xeXg+}I5?L-)7o6T&YT>GkXC$pOm z&z+!M$J?ZW){M@Z&59b4H$$>~HqFw#(jMDdPpHQVzr4AjEnA-L2+{h9g{CdNXnt&= zd26q6h34#iM;+JFxGZ#RTg&oR=g^SA2|09Yr>u?Dg{EnpKWDy0iw7*-eusO)MUntWqr|3BN@~?BS`T##lw1Jg`PBNY!Qpoy=Dr; zj6GkatyVXH4S-w_j@WzZ2@}NhJuk0Im`jcR6t4EQr}gmaW<9|wcs;{Fl()zGl3`=9 zNqYM1oo27n*XkM^05u?HeWNA?#LT_qUu+tYE)ZTcBedG+_j-nFo)I2D9=2ew5iq*~ z@$Ntxzmmq9J|mS5P%AVWb@Eh!sGvn_jbzG<$FN}10&8Tis%veWjV)I=o=h4MJ&k-J zvrF@gqz#tT1^QA(a#pG@k%*W^SHO&=jUFQzSgS|+4BS*t2iBR9NTAaQB=l4YIXcmA zNK5xxDy0(fSPDH>hT$+`X_h^TWFMtu2T}2Pl#YNYipJ*|^5vjdIYfmj?TX**nFxa@YZr!D?~j!sp~vo-M! zsp^g4nL+nqtjOHXWPCL`%Iw1675BzdX)7J*Gp0ukJG_dIm=f_MGR>yN^cjewS9(oU zG@TWvhr>o9ow8FYg`59|HXoVLlWCKa!$>CM$xxUN@H7&}ga8qousPf_17>$XkM-mJ znnPqG#?aDi?mCmmSsF3)E+Yxzuv{L?WIddwWYALq#b@nmrZ5F9B~ge>8kn+SonlL3 zQU+R#x}aalktwtqlb5`OUwLbb%oOh4;9BNdLb1byO7+LW*{s^NCYpGc95|eV8FD*s zD#x>S;wG0fg5mj(7S`Oo4- z0dN$>TADPl-gJd9D4J)i$FGS$k`~2SJ5YsAHkGdhQr6}BRBya5(uLW{p-oc-!emay zyZXX-A`DcOs$jY=8N&>j9hfy2Bh`gTkmh(}e)ggIjqPoyx;Je`jd&lP2#y1d=;-R} zbQ8@-DXMT$sG}+5FIT3LEt+_rv@NrFP~|GexR8rl?hbcFr=BuG@$POmdK?-3CvwyW zKR439dBS#1~z%!EI)Rpg0OT8}2^y0E|Pq*1$R;xDB6N5nr|DUDLp z9J$JI26|>YWE5&4)K;`AHBoxID(xMVGkuEg6P|2qt4j^|15=PRD(@lzbMb1Yq9S0N_6z`_8mu-yM%c;_D2&o)=@o5u-{M zaZ)ZdlX?m)BB=G~teafBpclbQ+jx~qXpFxuU@_R{Ade(}lo2Q=P(hy2$TJ%I+;Ro_ zCijHnzGdXC&VH<4sDte%9+Q`NHOlvq4#ONLDKcK#x$d#KD~a1jf~nS7xjEAoPN|wr zPOgSnRyjTG%Eo$`F26@8ddh1J-w(> z+Q29zS7To1rJj8{U^qE(1OY#RDstyY0;5O{9$)7uf~ep9bw;|m_|+BXWPmC<=-_JF zXEbdwhQc;1g{Qzi*L;WQR9oDFsA8Dd#Gsu#W7U{0>};qd!aD}9Km%|36t#uD1{TRq z0~->(8|}w6*pW~qP`GmgO)cDPrusTl;iQRIK{}3+Ihr(aYLvThVHEXPz=$T&{ee_E zNj(%rRu9CCb&km>VPe9r*dZk+N_{)7z|i#cgP2b{*h8!N$~7KSQ?1P2Wpq@=k{8EO zjH+w}L(X0)4aU>_1oHAk0+VPsnLvPsQ^b2-&v?x_SZaZY#tT@rHO4|HQmRjHMqo@M|pnOu-m`17M8L zamM(nWQ@-@#`x%BjPEYS`1E3o4=cv_WMPbN4#xQGV2m#h#@PHb#?GEGHusFNKWB_B zI%Dk88Dpc(7+YP&*s?Olu9Y!1u8gsFWsL1BW9(oVV-w35`&h=<$}+~TlreUnjIqCD zj4dW%>M|K)qsbV1O~%-6GRBURF*c5ju?=L5eHvrz#Ta8d#uz&?#@LiG#=eX(_Ee0q zTVjko3S(?O7-Rpz7+Vm=*o830Muai;B8;(hV2s@ZV{9N8<7LPg??c9T`7y@(k1<|> zjPVv^j29teybBrQb;uZRLdJL*GRFH5u}1*?CW&wwj%o$g0K_HJ2L;?s(=Z)jDHA8c z4E)v<1I&b3IFc~iUOarWm?(;cs2tobJRRO0qNt;2I|6@kMwIwV2Zgu2qnHVj2xUwt zVM2LERQNR}lrq7W5taU7L>Lsq2gC@HSe%u}{C;P;DsOt^b}_00B@Bvc|7eG63~`la zg^Z<5$ML30t?{(!1STl8CKAUaN5EtfAhQAjWX2SifPIL6s!PDWMA(lB%C7yDW)-B_ z0ZjL@y)`cSG@`TF)AKZ*K^xBGWlG~&r15MfD2;2C9E@JIe~u$(F3Bm$Dn3tfO(w1b zUE&T>^tnhM6bCCE=A+v^9Yq;2rhPyhVh`ry;Ltp&qzo;yf_xYW%VRkl{o!IcA|n=9 z8@abUGSBRz$n2wCdLBb=t#g^ZP$_<-TS`4iaplaS0{Y~s++ee~vLsLC#i$Egp!|Jo zMl2a(fm@44+hg_UvE%GPBMz44SxFoDn_N~lqip{&mz6CP?N%aib{tOvT*qwB0@_F- zF2;kX(lJkB@PAclISFtj8t3N@NjWK31Owk zh_3dIl8orV1FG-)ikTwJ6h)?3-VRw4Sm$^Fo&5=RjH;Mc0>;TPYlzOv zl6lL9guwD=l9r{MWocUGN{Os8Rt@AM+{fFil*k@9>jdU_d~Kc@g+0F3?ag(xWT?m2 zxp~%SM0!Bv-wK#OUNHTb~L`U zfX4pIm_NtMmy)4Bah7eQWv;Xx;^oU}o1eP&y@Ci=G9jmuZ6v{0IfAdw6FlTRSRbOG}2v<61Y*b#@4@#t>X@53a+(4SB1xRe+1c zjd}Vw#wdMm%81GSpF6&|*_M1GO1{M&+>C=;NzU%-{cYs!+u1ED>{ixuKYNGEU3|g( zf(feZ-$^&lU5*Z$^EA%OXsZ@dGwM|EyJ?9lH}Z_!^BtE_VY%Babig)$Gn#*oJs7~j zy|kUH&}3b1RnXmR$U)gc0#Ks;*0A;E$w7OoEx6TIdoYNDZ5eT2@u@q+_5pE!_Hze+ z-d{E#c4mnFz&+x@j5w~H2M_V!dNp{M2N$ZrBRn`n4Iafo5!P5WdMs!3c+TjFoY9jw z%81o!*;70?T@9Y*!7Xa=OCH>#2G8(dD-KX|rR-8QP|Dt*21?nB)IceFs2aGGMF}ot zapY1KM=oV?q?C;*AxhbAtASGXRy9z{-m3;m*=_Ax#j{JN?_5xHS%#jlFZCADHl$ra++k|)H=C0~fwN+*doOAirmmmVqJDXkapmYys=C_P<#RGJYV zmwq6=^d1JK-UyU?F9x4?BUE~?hT-13V1)N!7%7Wjf;=3i$~7=u4#GV7An%iv=?Br_BPzD zeFz!tQyB1-z&*YyxYsuYw)kpc(6<1#`Wj)IZw1`v3&VC_0(SV$fStY#@PO}fc+htv zJmky3!@dXM5#Mw0sP8R!%=ZC2?)wCus4RjfD~H2Vl@sBa%364~vL10GyimCkUaU;O zOO>1Pdkeh6KS@*zPi3tb3AHds6jiPi%heam#VC#!C0|wU?}d2)pNW0G5|Tf^6sLS9 OO7KUI1Chl82mK$HaX{Sw literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/SecurityProtocol.class b/out/production/clients/org/apache/kafka/common/protocol/SecurityProtocol.class new file mode 100644 index 0000000000000000000000000000000000000000..4ed34d00365a64fed25da24e9b4159c2a0e97a46 GIT binary patch literal 2453 zcmbVN+fv(B6kW%%k!|Dv;!-dqX(^$$DRD_iLW@Ij9h0~*Hbv$Zno=Q*5eSPBc4l}= zANmLRgTC~&g`_~4X7bz*=zIU7Pf7QY1&B*JQ`nJ^yk~(UjrD15r7vH zK|I7G5it>&Aj0^XKOXa)EPqV$M=ppK>iE4mS%RJX`0py;KtTB%w^hMBXwiJEAruoOR#6Ds+BdJ(qvm##8R=6vy0yz$wV*4VemT^6iJRa7WSk$z3#uD zd{4K1tEQEqhEJSH!JfeSD%IkqYsVqB3C(ayB}AfB1fPAl7(z-#)i&sKI2BF1oYUp% zW}=Q}@fi*(X4TS)6FQ%V_VWk$CO?=GqWr>%Nm#=73`76>8juhNMY)A#2{+I$;S)q8 zT*Gy~p@~?LP{gW)UVO~0ZcFG!56!E6E1Vv530Jwg<@B<6q;9WKNx0KEL+j~A(z4VY z-e3s>_*6m(Ya()2qpavGOcj%utKW|`7&J01laGrzoE)c9?1$KCLhp>A)Fx{9yV z4!V2xcvSN1aQiuC8Q#;jh~hS{-D1?@Q~s0`gFutZ(C?1oaB_Q}&=j_+WIl3gc{UJ@5w-yW^(%OTMNT$Y~$bIy6 zq9LHY7Cr|+bC}N;$sz*0TVy#%QZK#vFSrZ#8$53y{6Z4LO?o!id!1w-$7hcHFkx|z zpzMtu!1o4TZtpo`KSUa-W>v$ceVR9Va>S<);G=hIY@lZ!4Y8{)UO|j?d0)Z*>q#h? z(gp=>AZUreWem|Dl&rvA=|jy+z*jeL$I8lj{s{xIo?AKMH~11F)o1V!odV`4MN;?= z(dEJ1j~@2(yT5pg1iRZY1Zf>cGqs`@oz#YF#P>QfBrVb;8tB6gt)J5r9#6y~*(Awc cz*poOg6A!!MF@=jLj&n?s+Yn8+Vmj(FQFYqzyJUM literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/ArrayOf.class b/out/production/clients/org/apache/kafka/common/protocol/types/ArrayOf.class new file mode 100644 index 0000000000000000000000000000000000000000..5deeb90da021086c0a2afc305a39c2f477bda2a2 GIT binary patch literal 2464 zcmb7GZBrXn7(JV8vLP-l0s#u4Vp~gI2%!~Xl@<-y0xd5J2qIRygoSP!vYD_{@pFHO zQ$P5{4|PVvj2+SOqrb#ae?ez#J+R8e4yb&4GR%0M$v|) zC^0|Mu&iN4f$Hp)O$C`5yRwxv%0}U?nZ0jp-Z!!Zdwbh1Wy=-Yu?u!FD}g)NC61#C z!dI=5<%}s9O8l?PnCCko^?>Xl9?&hXh8TA&6MrkWMzkbgwIHN*N zvG_a92G`vv1+j+746=o=3tW%E_U!e0B;2;v@a2y66)80@I!59CtWiE}vSJz=3X+YP zCL1Az1!;bh1pTCX7D;xHO(ajMnZKzK$6jSsg>Tq@eTYrKiF8 zu2tMHEA&TxVIjYo(D4~Qm*EckOX_$N!#YO5gAqSUeYsHuxf7nZMIOj)qoxq~&e=|% zQR?%^uH7D#_SoYHata1di0aFuAbr9FafpIoLXJqA+u8BcL_x~C&UDF9(3P0>y>mQ7 zZjCL|iy@GlF{4~IOZ0uFev^H(z3e|{tH7*f7{{-;z9x|PpD zMVbOiXpzR}oo!sFM0$ zna2$ExOcm~rk>*oiUg2AHCnbKzNuUjj&)2?Rbj> za+K_fE3`+rr$#tcBb;*O<(0{Jkyi-!xzvM4@;3YOc9ZHET=ofz%(6ov;U?mFmJ7R! zSKYe07vNV@=&1#Sikj{btkFG6k0INqc+8( za+49asNhUZQTY!=ED&U54;2^nE>OQKc$+iw@Hxut9h#Uof^>}FKZtq`=y=Y>fQC^G lS3y@?Yst&C?DsQ76u}9nXyA?))#$2IBV1GCUB(CS-oL`Z0SN#A literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Field.class b/out/production/clients/org/apache/kafka/common/protocol/types/Field.class new file mode 100644 index 0000000000000000000000000000000000000000..c9935f4538b93d0eece48454f4876c34c2275030 GIT binary patch literal 2201 zcmcgs-Ez}b5dMxWTed8dkpAI@Qd%H5vK{dgeoE2+Nl3{wz(vw9TrnrI5=3Mx<0uTg z2yf7PXmLFQGkt)*1usI`U1>Zaoj%RjP$9wkM@7tCBdH4En0Ml5uFoKI^jAEM2 zB{rAYTw!yS%};EuS(w4BMbXb}=Ga^>;RgGgCgx2nC|GyzJzTuCbn~~BHHER2$NGtO zL*3nS?`=Hxy?92U5OmxAQw6oGV03lI|1fVg?giZ~$}{3;JM>>`CF*UtdPjSYe0N)K zZfnYG(e->g@)U}@bi1P!+^p4BO>UU*1P2Z8K{*Y3`8gAL)~|VA|j0ItjH9wTbD-O3?N1?sYc&-kRPBMb^mE z;kxbx{G1*d@uOf@VJa)oQs9Se(xJK zq?->aB=4I-BVz?-O&^bp*~#0aPBnnw-|D1C_<_EDBlI0LNgYy! zqEXh4{85%qJSixi^b=2Fk{@L$$(v9tA=&Tg=`C_}0$g$iP|kQCIj7l2-f8t=I5ne> zf>SH>QFLm>KFnI-50rj~)v6W#Lizqnx|qjFvJNb=6EJWVWz=y5lhl?&O*ROoi8E-? zZ;C$uE>Qq(C!qKdr|5~`g#-7GSVhV;tOsXIe(<$OK#mAxkrP8K}Gs?t- ziL+wsCUpU!>!fr$gj=Musx#hf)r?v}4PY^ioY1fY_yVoeMQL*`NglRY9ky8=wprzk zl{Txqw$f&mcbD2^r5h=}DP1djEo z2%I5s%@poFRba&`4x!Q12CxspT_d;|fpe17OmNJpm4f_A_qc&Ffs9oLIC2nT4kcU{ zh?7a;5TZO}R8AQw@>6MHmprpAIIm#5pdgt~Q4Z&&pq(V71xvJ8=jeoY`jkwOsK3F; X16u1pQR2Roj5sNAjGoR**$eLhc`nxk literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Schema.class b/out/production/clients/org/apache/kafka/common/protocol/types/Schema.class new file mode 100644 index 0000000000000000000000000000000000000000..32594be6aedcf51eca7ebaf5bc6381349db73249 GIT binary patch literal 5250 zcmb7HYj+dZ72TuP$g+%$g9!$cibuefYy@t?!v>Prm||i~z<^6kL&mafgk?#QMg$0H zF(FBlGzsKEN|HW8AMS&;(1Iy(d3@_y{RLh9Ct;Pi``j5xW8v$~y4^GO%nF_e@dL@2{AL%%&V?2NWPWo|9zTc3` z`2Z^LrkK4Ym$&8e;{Y%3$f|dByeAJo37`(|%Qhd#$FFt#MuTrS5l;@~ zG(-n_(%F%ykuhS&;?YrKc+`l-(qm)kR5X)Ko9S3O88uI3;<@NfS<$W`Xf5qH)oYB! zHB|Q;H%=JQyqQQw_ZS%(RP-fAQihq&a(3U$*{#-N(nyU&`^;=2HPSx6?YGmvZ)+lz zFt=-HZJcNDO@}miI@3cc>_jTwn;#pDXAc;INsiR^q+>?%kdaNuvpwil$!b_Pzc4ju z34Xe{op~%FFqgZ&k7XD$n5Ao&%c}ZJBQ{D{RXD168%RJ6-dD2;lR<|YEksj^bhP7? z8Slss5683Z4w&~29*@V&c1hiP!bs*-rmAGB3idRE5TSvzPQOm}SAq`D)XV>HivlzdgR*4d{#|9pq z*D=5pa*5ONz09S~_UlBTNs1)~BycJ}rr1tr?naq#kHk$48ydTt%B^FeTRgdBnoB#J zJKD^jzps?)xLQYgwkwIPX`aoX!EdIm^%@p6IzcUZ)dn@xKeAm%K9L+E%jkiON(>po z#7<|11=5v^7B>_sXxLo#3e_i0lKOJWiH0W}0e2>iT&~l|nNCX&TW3rO^;a6!%o+C_ z4HD=}=d-c+&V-0XrPZhF#4d=9_??F4`RHg6>#;$@!jg9GGIGb*=s~pNyFm&j-+*^sCm4+|@@&{{U+T8Z!P654Hitz#;Pf8e-=nvxfbgF$>BhuQI#ACVP@AIafP z5I?|65)LD|io0^TMrnun}&?NVd6qxJ1g=H+;f z<)u2?qE@a$`y`ix*PY4p4A|_1tl;bs_f8OPBc7?0=XsEU0P(;nbAfeI&ADxW$KIZJ zE@zCeM1DRfs*`FnP75{2Evrkp*mgRXSZ|44Wrt(KHpdw*NH#QfIwQ4eTw>V_R@&(O ze|xOpD{++X_Ab_GR>GRAQVOv56y_0WHKAYY%5GhHpU&NABK%>n7ac zDR>Uwg7+qT?rja>8*tyff=r~LwU_t4mK)G-!|%dnte8T@;VA@YTv;2OM3v0(YZ41? zq57VzY?y-UaD$BVQ^U_fermbhV}0U`PC2CvTW@2L8~x{9+Qjtx*BO)>TX(X!9m`cXvDi5y?`*PwrdyrnkMca%eX~?cA&H#UgjhvU&ew*SniEXVGFPA#Og)fo#y;mq9Q2m zho3f;*oAKRXz>ho<9XgK!%_6m#>@GW3OS+3coDgA>_IOhkg>g7>!Rfuynua-MbON` z_Yi&^`?YD3(5FD{({VsXh$?tFIL$cp*)tr6+mCks>o_zmXwPNhDVpf${{|HgxW6W+ zqFtsQK6@F*8C9G+2<>riUv3x5XjXU2$-0`zBv$lIU{{;RS(KFoTa3k|R2Yw47=9ND zBSVhDxLKGlr%-n|+F(>5T6Qka;A%OyUEd=Xd=COh+>lCS>8!y zo?`q>e3p86SfLSB1-Il>til<)3Kht)vIDGwVE=+(U#a7WHlR>`8r}kgM+A4lw~7BY zzOyEYtUXC&9~6+~+k}NIuGz?La&L>wZh_TGDA)ti=5{-%CRqLQHpSY1H(X_a7G0N; zW#jN-SY%l%~3nIKSY!&Z`)Vwx{HT@;s-yRXaN~X#7{zy%m%-e(1x=F0< zx6TO9&=mwCHO(@wRQz6~wo$F&c_4u~rRprjWgOKwM+tj_@_nB5f0Oloi*ocfYkZDu zcn3!)>(US*u3d;7iVso*!%oG|rXOYNSTbG(_d_fswY6=QCh#TI%2Vqijf8Te!dr(P z?33h{x3+oeJ>+e3J)2r|yMF>3+PqF#XksE_Q<4Nqe0Wy>Uv0j&M;6sm0;=z^&P&+h zVb%CYNyNwu1TL_dE}{k>(drZ0eMZYq(Tq!!vWslF%Utm}GyOs#u#0`t%)i}8bKMd= zOG0iHg=EJ_m$*4PfDA?0#Wexiy`s1{TI9|s<)zz3Y9qW4MO;OqSK1Qc)aTp-v=G8+ rn!A*K91ztuYo3$eHx{JqI;$akVG3!>Y~n3;5YJa6-U*h#g;)Ovo{0Tj literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/SchemaException.class b/out/production/clients/org/apache/kafka/common/protocol/types/SchemaException.class new file mode 100644 index 0000000000000000000000000000000000000000..2a325b15ccbc39126624a27272311233b6fb0986 GIT binary patch literal 498 zcmbVIO-sW-6r5Mv24ib$Z7T>0Uc?V@FCs#vm#Ux!FV)_+aZO{B-H>cWf0ie~gFnC@ zCB8)Qpm=c)@9hjT`)1$U`^zhUeXO{cL)$^eL6^|XR2s@?q|z)j`ugmcK<9+|!060M zZAUVisR|qOLxk#4s6*Q)bhf-Zc`rpIbs&Z|4Rz4l9udj|bE60iFVyOC7LQeWCC3pn zEzkHoNb)$wUu4-%!i;d}nKTeGk^V%9sl1&^;hQ)%S|q8l#y63$kBQ2}kOi?kdGJ+Y zId6~P#45{VPz?ALh~N&*EcMlCm`7;;uGgsnX-PwHxJr$Z7SGQo-*tw26SqzY*#;~Z_Bv#`9157f^kYUK8)wUX~ z)w)zo+oUzN(U?}FMjIhD8l!Es+SaaiN$slLt=i>}nqR8@?t6D;9`Mki_7C2?ci+9= zIp;g)o^$Tw#dn@K31B|{<$(kH+?as<^745PO7R6J?snq_>Q#kyYljqMEafw-^UN+;}7NCj|4LN+Pi~20fFkqXsoL` z&>P@$^|nCgwm@}zw5KN;sqT$MXx>x!S=X(y)+aF#g_}LDjy@X1zg2O0eeHVBPcMgF%$_l_xH2~W9tKLVUFZC zM%x47je%H5KC3}{ygNkQ(l;v}>(}+J8LPTdx4GpNV>D7AuPYc|9SVj!T4F1Moq_&v zT%bN};rhxY6;{kUIwh(=Rk|t}SON~+qmobye`c?X2imta1$uRYxlaUav_N21xoP^k zcq|m@vewHU3H0dhN};RiPW)KF)g9>5H4!MVg4`_Nra&&CqXf2$nW}UHJP`>ER$8RX zsi7wKflKP#9gR`tx-qM4VoE{~ZlvzXx22MZ44)Jj$6s?vB_5Gclx|b_Pr+dU)Tc6zg+sEi*Q>svvBK^df*q0kDwETbI zx&1F^pGzO`Glyra{B1VN(RMnWsY?fp}1*%eiB8f)wIPU{I-eRK-zId4&q{5i(uz zs1bPr=djqB5{{Aa1;keV(MFHX-A36QaDb+Z5dYc)ibt%f?+=AL7Fo8rKb#VV%Z zr`V(6XK2yzbF>I7Of&ZyUDc{x?ZMu7C>jx%kU}?jS6q?CrchrW^Nm-p*51x&%-bmo zzPC)nY5YRNdTbDwT;|=`9cu6PvZ{I`QE!LQVi=36G<+P_O0!Rq0-rbu>TUFcvYa;r-R_j8Mj^Hlo!qfPphHLOADQ%~Q zKjSYNc44=C@mJ}q*EQTMU;GVkX!tbV*QePC!$UsS<{#xn;}BRjcbZG_L2)4)pd0BWz`7TcrGvNhpy{hGhaHhL*^z z$NoAMY>Dx7bG)uO6Ng=TmPd>9d;axYz=cDsNaUvx3Z<+t_CbF<6t32zj;z@`L*cMM zdAUV1VnRxQ(`3t}$TaQHOOb%jx)51|m6We}FWQH6%n2RRrI+q7PGGJ;ILs4WoE#S= zk8&sjD|yX_o!4SsC-Mu$B;=q3t2p8{mSHtM#3usiVuXGt1!YhG6QUAx37<3g zt|%qOcMKT=us;Qdg}f=0Hj`*T2a%Pp z4IrC?IsC~V$IAe62a)$QQEWPS6OqaLx%`Y$L*nKkm*0yfVgXHCWFoOXX;eKL(L{Yb zSd3=0Q2z_L-i?g2$aUhP9KK=|RSGxowd6F*=O0AAyo^uT?9pGGKcZoTkd>F~w|efwVVer>|1UG^{m3rJ4G+Ae)+(<06xlRY}=ZvbCU0 zH)iTu7*@I<&Z69jjeN5SmnfYsR7U0{bb1a>`(fB^N0KBP?%ixMGcPGBPl=K=I*>7L zg$$5ZY*y&Y6uR3-s|rtHLW)2G^>P!cJBjK}lGln$t)Pl5pxQ=(GPn@dH}|5?Te(si0)~lUq|$aaD2(CBqfF#)PINLrY7B zRuElQXjIG!?b=b$61sIGWMw89n><{x8%$7EG2e}?R!~%I1k_EVKqa)=hOiYBwVE(V(0?F_Vz$dGnxtV#z5`afB1yHYy^5`}Z!9QG$R~adGRpQPHDCI*QEkQPXzS5OYqB@$S~YK+=Y zYxn0~szWSv6gZ0QCl6wVzt~>jsNvSyq>tD;4#MNl<*r9Jd*TfN+uk7sd%UCJO#5Qe>f0XqYFmu}tLf3(z=R zEOK#~$U~RN$96FuyZC&KD8Tii5I2emxKkA2fGEZxVmv4&;W1&HIF^&94XD6%*uzAy zmVt3SR?s=qN#zY}n{CXl`|%0Bm61A_B>fTGh)JX(&DbwH((~&D?kl>k^DQ1z}GBk;DT*BYWMFj%FhhEOvA!h4Qcs{dd6Zg}n z*q6I`3i0C>66fGaWTQ@;o$sSMakAF!QpDM~k8UNIJmNE>-lo$kDO`liciL#wT{D>U`#Py4q%BsTIN^A>;FC@_b~81>D;UQ6v_T{l%Cn&V^6ZGQ^i)kywg4u^f%!JVZqut`iNo zS**k@e0Qf6RZv(wPBsH9Y<5$d4PO4lfAB$H zf~8enUi~P`p3MjwqzZjV_snef=`*K0^Yho|uK;diQAZl4hA|y6+|Y3ox1_l(w|7z) z*Kk+EJss^ZrMaKN1L-HEnbeWOLmfSs((p*bV}|6U?b^W!@v|X`O zuatzp%1aKNq>G-#oi*;;a<2{(!IoWPFcw@__%jZ#)dX$LqUUd#yvnUDVeasa9d24) zrQ*3})%OC=@*FdGQx!FHmF&Qc35K>;z8z4Jp=@!R?{d?%J+ts85QX~2hVUnvhYoi) z&E?Xzu!0FmHCYP3GMqXxiGg{Nb_f{FrAzvPml<+LSIZrhlC0Tp#qtIfQ%O<8+yX-? z@Kyrfb~hPLW}^_zQ@UhompgWu%Xu@Z1=XOUNx>er9SRvIzr zD_-5V#GEY~*e=x`m1MfcKKt4d)xh>#hSLXin(7qlwo?|qfdxDxb{V^talIhJGm^%r zJbrHA8m>#8MFWG#GK~CV_2!T<ULs`Y@OKU+{O zZ~aa^jBg4G(5UzIaDICCL&FJDolf#hBggc%MK6MHiC3fc_w*@!DZXWV|HQ9h zjGlxvJuiSpKN<#5D6LHLjC|YB9@s}(#L!1J86J+~G_8$h0Qzy3b`0lmo~%q>rsz2N zj{NX<#2Sb z{6Om{xv3DduNkwCnNpYxS23WF#uZX5@%|9TOavpgj}Z&mNL^?m6~+j8sZ%_y9VRj# zQ7NuKm>gl1>L3x49L)bf>>b^QwEFM`sr4^tU*CgH_dAg`8t7=Ca}QnLWRx4bWc;|ysA7#b)|&KA*#`Rn0K_hNd*|kFvZ6(f>H8` PaAeNVN+-k~^3;ysWY%fN literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Type$2.class b/out/production/clients/org/apache/kafka/common/protocol/types/Type$2.class new file mode 100644 index 0000000000000000000000000000000000000000..e7b6e580052331bdd354250c1be9a19495e330ed GIT binary patch literal 1720 zcmb7E+foxt6kVN!Oc)2jfY);|-XI~u;0=$$@e)KugFJ)+sq&?nbYNsMotjL%{fYnJ zgT4goSamM1ew5|zP74E4rRRa(JnlKY#h_Zva=ZXd@5T!jz2^F4?$@D{8r_ zKCk64ZQ;6wKWy~FRm+VWZmKw=mRTD){AuF|Zdtf(;SNJ~)(`#o7DKvNT4Bh{$+}<| ztoosN(rK=XXqm4Ebds-1j|VF}^3}IF%*301o55KKLlMmdyxkV`=~iX5;qn&uHif&z z8(ZATzK(}qMBXT|7AFOU=ah8inaz!*3u;-!RrjAgS(XuD#^C}H)5$l%}h}e zv9!RDi)Ag2{BVO|q?m;0ozf+9FL>bBxw_|tgFq5(Ynw8PsU$dCSN? z`h#*K5`$fnoyZgSeKo;;CHJI~gXTLQdVy>c-9-^^%DRJLjL`9}Sek-^yQo;0b8rvy z4(_Ak-~k@ez!sh?Uz%o^{>e$#$O=t9|H>1s*q0&0@kA#rTczU%brCsOz$1pCg5NHL zGA{6fCNincA3L~!QAM@tU<^fuiJxrWGa5trf8>*TZ8e4!9&|`O`gznChBfWI_N2+S zI4edF6;A83=pF@>xJ^2L9V>LT4mW_1@tJz&5p0X)?}g6zK?ufP7fGe3TQ`U2n*=5(ZCY8cfK!v!4|F(#Kw z@_0FgaSc~AT-DJHQ!dw1xGwF4Tqbp-a6?BQZfdxt;Wk5Z(su3OCPO@zUt&m1dlkX3 zuWY;GVY9X@{6)U(&`rARS=?FTzAew{G7+rW4F;p+y277!c%vccXqG*{YVta_R)x9F zSJt^{d9|A7nswg`Jj-*;;8|TX%tczyj!iIhJ@M^;k__g`YkY&7uI-t{XMrd-S5|~S z(Y|!JTQ%pG*Mt>JNUF)@@GHZCU4s~yCuxU((O4?d7rerd-#uG?w~%DRelF%$D47b1 zBIZjBslZzZeA}%u?9W9Z+P73>YJ)p=h08j#yMaW~mfS#8sc@o7y3`nY76NXq&+)nv z{`_A6luQizg4gsdF>A{nc1yuWq#)Gc8P{^W1`(bU!Kzm=(2xBLecN&=5(cJF)G%$} z4rUC@qG;eQ?orc94;RNSGK~LYw-;y}YJBFYCF+6gxeN!lbG1b(Hf^UOd;=xiXXww^ zjg0FB8SZ8jml65tfq`>4FZq@YWRYVS{znGwd1EO2hd=erc7rAEG)X}Ek5H@oL*AV; zO5<2>21JwLP_Cr<-|0f#OjbqPN26J2hVwMAYzre6vXQ#b zWGakdvQnpbTH8tFWkjX80-;9%ITh+45t7Unz9aUQ>LIP(e@5!@XLLW_giiJCNgFNn zw6J#*`@YI7VH5U-dcC5Ky(SIcM7U;*qqnObrl_AHy(4$R@&(IZQj95c|kdJAMH!xpTe% literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Type$4.class b/out/production/clients/org/apache/kafka/common/protocol/types/Type$4.class new file mode 100644 index 0000000000000000000000000000000000000000..9420cc087225383150804df8eb0b8cd493019d3e GIT binary patch literal 1714 zcmb7E%Tg0T6g{1U%rFj!!S^#NN( zsO=dYqZ-a?IA@>}mKx6MxS-}SHH;h3anZmwT+(n^!xe_~xZ^p&C5B`!KhKbwkY&NJ zwd8o>PNTXY{5ihh(n+QyZSKx<-%y$33Kj5xY=IWk(n~}uQmZVpqh!mMpW$2m>xV=2hYmw{6 z{}LT}#9++IhHs1Ojw)cMa(hU*LFJwFY**Gv?X(D%WZ6V7wlj3Ed!-bZxQe2N2@}^a zY2rGHCT`#+m22wG+~^3y=s(tZmKveTCm-3O7C6#l*tMRhWlyo;xMks+n8GcRmv!n{ zPX<|@RXT>$@of{wa9jy3ndnE3VelWbx4Mj>@E>@x%caco+-;C`^w%I&y`kv#)@0CX zjX;TH*qyr-)x8}=P^ugLx$cNK%*~p1qezji)Kc1$G{5MM_MGXlg4y-_!PTnZMFc|jlHyE z*oXZzs^A$U=>v2XPJBb6iDVP0yXz$CK6ovyh7?OA45VC}3IE*6^ z`8e%Wyk2Oc;}g;!XdM^#BIMlJ!mM%X5hue@^hHc(X;w_h9{v3Wl2GV#%!Xi ziS89_{i?ErRp<^4dqX9AOD4XHY3*Or+K)V0M^9`R#mt_ol(`UP9;UqN=IH^-(X&x^ eyu9g%Fo_d{Phb#3G^fIz*+VOX7`M?(_4ox;o^Z+l literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Type$5.class b/out/production/clients/org/apache/kafka/common/protocol/types/Type$5.class new file mode 100644 index 0000000000000000000000000000000000000000..710bf74ac9e7b3b6eb029cf8aba49eb0e39c780b GIT binary patch literal 2113 zcmb7FTXWk)6#mwZEk#wD#7)ySEnw&+c3el{($eBKP=~Yx90%ewA%sh0Wn-(6B_qqF z<%J*7C*FDNFfA9S)0A*|VTOU>4>7=5NpTfOhT<8YBki93&UbFBU;cc!4d5jh3Xb5S zgjW<4a7n^t36nBjRgl1I3a+3i#FP-P3o$Lst_pEY#v2l56!hbw5O2zOOX!jivkHb$ zmM|ybZHD9}(>A?HhIl5s$j~$8tZ|0^l4Rj8<*Ee*{a2gHA)|#&4Ifi3t-t8uDX$vGzohvXT zK5SCW;rVQby#Px@E5EA}MF~0*P?wp6Q_+HvZc_Od*3(aoVoI zubQI1y`mW7B3V@ZY1^=z7V(?qUfo$!aTLcG1|Fv}@G0RP74xXjiO5#naXp6fPuK$$ zYLbRNeaqlY&va~ti2yrow$hek*SMSZ>bgyWryKe$v(av(TYm1+LCVJky9Ei?RV-pj z#k*Km@g7!Gyf2>S@qvmD@ex&}vM@hWzQ%AQLe@trwoPk|yDC;e(GLpOw&SJsbU=kX ztfH!70?(^>0WUJ-pD?(f!5MP@!=B;fe%5Wzv|8GAA*o5*-ueZG;b?aRd<$98*SmUe zXM@J(kRSc-38Q{%w!KPFxucm%*pj=As4L@S#c(XMQryo%uJIilx=^Sy;UCYmzQID=8b%Mg+b_6J5JvZD(ZxsYI;>;%D~ zTnEbK6OZJw;BjCn{{ZSeS`UIv`8E&f1GI%+Jfg43cM%RlT@FJbhXh%iMV?Og;y}oO zVcgH9NXsPc#7HZNoS&Dz)w^9^L1KUM@rMJeF23SNCh9>$8AgyNE$S0b9ehkU0Qosm zop`r8G|J&oi^38#Msq(R_Bmuh=ANj`7W$VrF+e#POr<(F)WMJ_mGI)%<{)X{SL&4C zsQAA}xQ=$Yj?$O>G&vHvkAtZLSLhNs7kZuu6ObGuABX?RJK9yr5HODO1Qf%|xIntc QZ`G&iNg>2x(p2ET0jLSz>Hq)$ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/protocol/types/Type$6.class b/out/production/clients/org/apache/kafka/common/protocol/types/Type$6.class new file mode 100644 index 0000000000000000000000000000000000000000..c8c97b2387ebc24c6f0a45959d2e4de1f0edc5db GIT binary patch literal 2015 zcmb7FT~ixX7=BK^NtYIZK&{Y5tR){LTNSZNTWydk8iJY#;>S#vWPvT4-OOfFt-rv( z;D7L<-h`QUM(eG&&iHxZ*NxtIqvNIHb9R>iC2D1c_nbZFJ6K1%Lw9Q0hRK!TvR&8ARn0Nvcd!_7 zw~e-dvSeAhvtVlNw$4wrY&%=3*3#U8Z8^4U*KJdEZ?^QdT4Ovr zH!l!<*D+j@w4Xtmr`RJ-mXBZ?;!zU zOW7Xa!Luw?bG8^;HpPl17VYR-Lm=CqJ=+f=(l*|w;P4I2q!RhDvPmTY!a=$$2@uBa6p1;Z)ND#qj}M*6fz!7?feR%CbuuPS&ARk>DE@VZ=E z#hQXQP-B}*>$SzIz}$WMqe=rcwZ-@9ddoFzOJIDznf}d{I)>TM9R=%nQ{Z&kXs0dP zO>1eHSfq`L`!*`dthW>tp~`CBR&XAV2^8*AjBi$f{Qt;jUuiPK7taSqD-1X+iBC>? zA=c`+OO`8eGP4x8;vVNqmv)#i^RL!Ca%l80p8tx-x25}h(Y9&S+41o`Gaf;Rnnu$g zaZGEqbc>Ip@E?{s)DIf^P<~+Zqs|Zd$NWQ2XD@%1ElTlEp!|vRmmY+`gPf@x1;=P^ z7vf`1LO8`3^A^9rL!2vq0w!>pUjk>4W|W6NV0M(_)XZIkDuph>bCEoUNEgvHIf<4k#|HA;kknvZK zCGF?-SQM{NK6Ca;79S!iA6tMeIb_Ge!^ifE@cZ?f2j}-I-0K%#XK>VHIg)xh{~1Cb zA|ch?x&vkX4hGkEah&&aVtA;F2f7&E#mJ|U#k0e4PvP&>^9Swlr$;m9L-kZo<8ih} z;0f=PP=>D1bh82Tc_vHdSdJW{_v$|AvseHa#*@qsVGd7mjCkgq=IAhJ>(0$dX+i%3AY>z#-3@L-T}j0zHm(G4xOH0SZJEIXpIM5;dXe}qMfCtV{q%3yQcKR&A4}F zQjLj^uIW)sG2cArmt40^N3UObqCW2Tg&;Rw{WGj89pF;A8@ z!Hp?0WGzgvI7ODMr3r4zqL^RQ3Y7&^p`t_GyVS*J#i^e%=i+$NSOGE0=@ QWJ1n|OodE@3`$J>2KmMvw*UYD literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/ByteBufferInputStream.class b/out/production/clients/org/apache/kafka/common/record/ByteBufferInputStream.class new file mode 100644 index 0000000000000000000000000000000000000000..65d92dfc1e4af7120b7c8c6b1c589ff40775dce6 GIT binary patch literal 951 zcmbVK%Wl&^6g^`*H8E~Vo3@0uX=#fgacImVVgu?TNR=X`T@X|xx|%qX#3Z&O*Fj># zl8@jUx^NtbLT$J+;h*J`T6Vn4*(4`4CJw$$2II2xQ$(D_N3W2 za0iz3?&`Rwqr$Mzo%Z`8Vpwax;cvMUc%jpL7mH?+eoBC6Uf{)t45iAIi0UbW_B`wf zhNZR_h?A+`712xH9aCnd9lCsc$|Fz4v!oUey>o^q?JycRe8SzKa7Mg8;*J~oei%5B zaKosV8KV_Urg0~Z1ovqojm{o}TB)`e7CSL_N5_0JYpz$$nyr?s#BqiBb61LUdXn^+`Upy-J53Y~`i70j-Ef%1tW1xsWL zgayjbuuVO7aD$@t6pdx9P(=5VuJkM@Vs+NDLl{Q2lKmOVCDf0|DHo})E#vq53#zFZ zhCS-BPcvBwgPGo)F{~w6CX+=XPlx_MBWYByvIONc$bG}Yp>%6oMxxrnCV46{Gi?z-LHQ3$k-1d> literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/ByteBufferOutputStream.class b/out/production/clients/org/apache/kafka/common/record/ByteBufferOutputStream.class new file mode 100644 index 0000000000000000000000000000000000000000..2a3ff69562a9e1c5db03f757769da8aca119999e GIT binary patch literal 1474 zcmbVMT~iZR7=BLJ5E51p5JW?2HM9*uT>K#Ttt6T@8pw<=^uh~02^&~Rb~BreUw7Vk z(~dL!3;hM{OpA^e{s71RC8y6hyJ|YZO)v89dEa^WeLmi^dH?b4I{+83mcl3&k{H67 z6f7*Na5lyCoI0IX;X(pS#*JS;{F6cf%c^owg-a@2R^dtlR|O{S-Cn6yYo(QqN^R{y zd8M>byC)#Z0*6YT+YO`}Y|2*878tDenoZjmn5;gMkEP{0o>hDn*hO8w!iKAk>jc*Y zrgQst%xwx7C2xzfMyrl%ul3q>+uxA&7AxabuOVBT(sxuI6^&rW=?W}YJ%8Jh9og8i ztzFsNl~%)Rw>{VL?S|)X_0g#Hf=(}34}4p;Nh$uscLGi~autOx>iktGZ9R~U-BsC% za1y!u#ft8T*Qtn*vHL}V0k7F)Y}u~B(cIj=q6AF)X-B$Sp$&xUI?sui3ajfnF#@~Y zVQt;(`3<}5D8cD3-u06{qr5dSiEAb%a3q22CT`%Si4_%oK*mH7B^7Q7Ed4KE61Z)m zj2}(>go=qfxND-Srmy0rz!AM%?zW#5UC1c|C-&LAK+?C{lDl`ebsr5LYN%~F!7~=P z4|OGq2l|Tm%cB_#mLR6 zy$}7;W!Rs47`;(OR~aU@Em7t5>t#vnUz_O@tMlM zhRDzVjo6>Mf0}Uw1B@qN;9Dd)`53dQu$$A<%=n(3$}5Q(WTKfZzNcnp^M4_>hXLbv z9L|Sl!+7`_adl4rwuix=71#h2*f?Q;Sq$Sl=HKHKFUl9d>}S9%z9A(+O6pB9t&vYg z$S*iVkBN{oeaM+l$n!IqJtV$@&J%i`(5I2bf`-mVls;*lC8;ct6wO(ZP#R}R_XLUb z(U~Co&|Hg8)vg8q!9z?-9n{4|jbVljk-V7tI#^Mgc@=3XM9#e&Pd@r3{RXLPGjA}w zwh)%h@Np=7j3{CGh5u}EiW^t|w_{l1_6k_UGEW!#6c#?4TR={mJ4RJ=THzFx<#}&t KHQiAPPX7yCu^gNL literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/CompressionType.class b/out/production/clients/org/apache/kafka/common/record/CompressionType.class new file mode 100644 index 0000000000000000000000000000000000000000..e2aa0a11d84fe3a8249bfbd8f3dd39669226cbf7 GIT binary patch literal 2310 zcmbVNZC4X#5WSnXUBV)eDkv3+h>}DlQn6a4%DYXCKpUhDwxvs0;>u<>T@qXU+W*jV z`WN~Mu|Q8bZT;LI)zdqhwFIOeFr1wynP*<^%-mW2{^$2U0gRy9f(WMLn8AG&Srwne z(T|*nPvz!;h`fl0B4$O*iI^AhnTpR<6yj*df`&(OE{ga<#FufLKrxObJl61)h6@_L zj??KI4Hq>mNA5AdhD#d0)i9{RP_d#QGCMart>9YT^~zbJYLuRs*>z)W-N=^QO2u`u zo>_9e)$F8Ose0zdhUGd7&#UGgdfs2k%`1o%X2<8}iweT|r4a=oYgK`gPnPJz>4F%Uz>349COuDPY z(Vn*)bGBYtF}($2#imB}GMV6M45}*o%f|NrE6tY}9hB+Fb>iBa-BD zZzrBfFS2>HCl)JiDOhF4sMS5f?N0rladge~a-7KZRPG&6w-steY5k#5ebW|3oAt-4 zjh5^Hd^jbjCYsPrJN3%waeH)^yoeXvx>qu1ESbT{LzEj5BIy{CrOE0jfn^OHkF`4Y z&(v`fYdTU$>&W1;jw`sT;{pbBToiFh#2^L~^zTzQXWM4ku*bb}y<$4G=^sjFwZ>gw z@Ar)w{GO;=_NwVgnsEiakDPVK{oYA(o!%@^vc?TcTC2B{I?8w=!qV{+>*8apsOWIu zD(E@NSJvuwQufPr{911Dd-;V`Y0a%X zHA^*i|3)(lCv9V6<4%)Vu)NKl5|w^k&3=JdPB;-G8q?vJt{!309Oyvf81byIbszoV zYpv!sREJ9mk}EiMR6-)^I2@E9&F|`l&4oJP6Er2gG&amPXurHU$Ib{z0G{NDf08Qx zNw(0UQ#j?Hq?Lb?SwKgRaF)MgXu&lML%|39)z4Et@hvDj2)#yllfSsBlrjIVi}oY9 z5!jC)|+e`i~L@9e|eGaYYV`WKn<2)pOhW%g+eB{?G-=ylWualgOTd6#X+th|| zhjonN^GvvFK~cxWi%=NWw_o~*!CeGe4{xJf)sA z0mfxw_?7M1LD$bv13=9RA6>J;huW+-h`XD(87*-b_a&wWD`?}uI|<_)(f0FYkmc~M zv(;m4d2r UiGyspPZos&225a*x)7%R1y6nlasU7T literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/Compressor$1.class b/out/production/clients/org/apache/kafka/common/record/Compressor$1.class new file mode 100644 index 0000000000000000000000000000000000000000..a684492291092e00604589e93ff07a0c320e95e2 GIT binary patch literal 884 zcmb7CZEq4m5PlX84)7{itF85|+S7tn&X;OpG$sv#A;LjIQ)BySIaauG+>$-0@$dM> z&uU_fG5rDkQO4OrC4Os@o0n&HW}ls#IsWzI2*4Jc99D2Ci!4^Nf(MX225}s5&yCO$pC8Efe;m(J# zU`XuO88Xkkzz_Y&3x;H|w98;rWRDK>4L=m_bkr4Uhj#;tEHjPKI1ZPL(HWzAnoDWtFOYzz;b^s-YLi$PfEXF&W673ERXi3%9Xh;S4MbMU*Vu z!6w71E^z`L_8nJtrrx0Di=bCkN-7I?b=p0x$|x~xQv1%VeJ5_;nYHiC+IQv;u^%v4 z^)M8w67VPz5kp~4)ao7xZ^BSI@qG;C6XyQ=_31+%Of~VNVrjo|y6R^%Be&&N8SbCH z(xgrBLmtG_rturE_UdhhbjRIpx8G8)X0OvCq_Ba9cue*Q zw(*i|5(VN*$E?RVM}C52k>nyWZ%mv=^M6GZOL5i($^bDelfFrkqdT%>QONuTFf!KH literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/Compressor.class b/out/production/clients/org/apache/kafka/common/record/Compressor.class new file mode 100644 index 0000000000000000000000000000000000000000..b194619ead852a217ea678501b73085f4a10dea7 GIT binary patch literal 8487 zcmb_i3s{ubnSRgAF#j<8Fo1%h7$BlSK=2lCs1Xo#l#2n>MZ48;7;t2mVTVCuYP;Je zF-_~HY13`9ZPPTSiEY!gY7}TpV%yzqcC*>8y(isXv%PG0x7+Prdb1n*e&_$^0-~5a zo9E&D=lXr``R-@Ti?2QNJb)EyqZjLNCL8%U>&4SJC%yC1j7l?>jd6U=z%vFe$nvuW zp7SCH&&%RPFPqOx?+XULD1*O}wZHaa8UDt=mkhk%MG^j14*s1qFM5%Uzn9)i()$N5 zzKnmA^OuCpS6sLC_4T>dzx2{2q3~4?z9z@NE`x8#p>Im_EooksxmTq5wh;O!X}%)> zzAMf5WcYp8ZO8liveAVfcuD++240o@|7_q#UaY{6J@|^(n(cwfOn%EVHjD!?!zUov}c87-JuaEIQEjKD{*v)td_LK;{El3p+N9p zsD2>OHxQ@~#s&vt(fW8O7>oDTH^&Bt;-TT;a4g!*6Ah#m7#a#id%F_xP+(ADPV1q- z;Xr*jR=+8b2y~1jhDH+Bn(RF=($^P?C-<(Jj$6~wM5sw0nMBA4N5hF7F^Y7?k$55xpmw|~(t3avTTOQHE% z!_iRt$l!rcygP6p!kNO>STGRT6^MsrY)`rp2g5{CbEB+caZ1c>Nlc+=O4th72O_cH zKv(!qvd!&E1cC!?fg#P2erl*JL%9{U{{LKB?mVk%(KV3xk_w=0bOsW{%PIccBLS9W8eRLqiE3;bomeqS5TbV)oXJoWXdI(M7?a8MVfu{S0reaO~x@JJ#1X93nku zBorkw91BpaDd|*T~7dU%0xSXnO>?OhF zT3)!@GI`!$SnItgG@J+zQsljIvEkG*J1G{28>p=;QUc-Q*VDAa??lbu=je!X8V^NK{F{SIeS?01?tfS`$w z;9(Q{5l~n(4XyOu<$Tyg3`b1diMKEzCG%fKY`e~tw@Z>uSuLq4)5JSeu8Dh8o~e8) zUwQ?mDpWHJHPci@s@TMv@W-Z_Md_;9xJzNRzrLe9l#Hb1M-GOf<&qb}(f;y8tUPfr zRBmPIa?6=&%OxgrRrr;dY7UN@YOX3Z)jX zO*LOtnreZ{X1q;adUDbi1`8E#((5CMaHRgu@KF7hJ^me&l%Q@=sHD{E?g%k!M(T#6 zM0~WaOMfQpRaI&c%}-NIM+$|RNer304OMNb8a!gETD6$|lRnjP;7}--FjbwZH`EeS zEmg~s%9|KTC|<#36-9NrD5_(sV^LIRh2uJ{!`3}3nj{rxa%b(9X6TPZLj8eAW4wQ4 zFceLsbk3Zb-bB}?TAD_}k={_8)zr@DKs0tFT26+^x~p7DwYBA@TCP?|bE~OVs#PNB z)k#uU#|~LuqcBf%nxKaa=U142Ee%ZAl{WEo#8FuFdZd&=&s1wwgQ3=$YQ5TEs@uSv zTtiwnQijIjl}nRan^@5))ZHKrJ2xvV-8)^#OuBtCohx`)ve3_BtPYrM{Y=U29qlbN zMph$MV$ra44a=aMtV^_obsvn!j!1#nAZ5sG)-@R?QkAbhHGx>>QkJ+mqjCGx1YtiDNw5GBqnc8gVu!=^JEf&->YMD04u|`rp-U_IulS|l}JjvgkLI4WHuUA+vB?Q`vPSoCqG|+dzVBr4`I7 zbug3~kVFG$BlYAVz(5z!@&SB1R(P(nYG<0HXDjPuK@_GT#CDjqXw#zTD(#(O?s z`L?zWk%M8^;0nblht&{le%xY(Og%XT4cbdIbi}XU>p@=amI*Md9Mh1A!=7SYYh`4G z#nVV*LP{W@3HztuFnPZx5woTwDrG{kW=XMVDg-(tSt6x6Gvc+@v|9^~Zo8ab^ZLVP zURtwlfy6-uPga_I)Jf2hAR6am1{M5$XB~gI@?s8lVi$k8*v*jxWgP!e#<-s29_&f2 z@8y^U689ELIjX;`>hza&p8m2jNgY9)qf`Jr759 z%_yAb;i}F)k1T22wP#^Gj>4Mib8zLG`}6bmkHWjV`T}x#YQ|umM{YHja%<;}Bkw$X zXGqS80k+k=7O;j4*76su1{7dDim?H6u@N_;8B4K+Gh4BZzcoelehmeKh;l?wEIEe= zRC;nrhwl9eujNFX{TWbKNW^7e*g(R-h-{VH9ky@RO4j^A2;S? zYbCHY&Ai;&nPTouG56Zcl~lks$W6osc#blrM>ho)&-du*Btci<>zMDBw` zCf6ym&$sgH&VK)gkmdWVDBxZhzbp1xZn0!psMXXB56z^2hB1TYAu=nr$k;4Am}SEj z`FNmyj~NsVnX+!TSgVKNhQZaZl7(*+GfpA5^#Y1Sm&F%3=g{gNWlt{FDrnYg-HXi+RH29BxYYmK2s9n&mDmKXl;sGg{mRt!}0* zwRXx0bQ6P59#LZst!O46LO0Px7V_yt?hcfM>EDuwP575RQq3j-~ z($7+LU!ep=;#)~Ahh}pOZzaElc+!@54W7W;XqygF7{H%uD}4Zs_%n`Oq)j1jAu#h9k|QV5_wsUIhz;um5rj}6nyQXrox*Sk752r>#iOmN#3;> zS)_LtwSJtlcY|*#TEo2>vlzF@S-~t0G22EUy}OA|X@4@{*iI}t9Nt4BmZb_1eMErF z9YrO9v{^uEuLtDa1ad!Fo*?5tCy@7C1w`zBA`pphB9sgu_u`#4kmELx`2vJixqyEQ z`FG1Gs;0y=gU&HgI1kbsKET=c6VG2T1br}tX_hS)1=&VC#|8v9(F(qLAQo{#XvWF#z?rle9zecn&LGWQ;B@X~uA6_zQf zHcBd&TlkK)0Y1zdXY)u!P3__$S5ekDs#m!ekE3R#r@Gi(m(@>O5v%%gEpQD8uR@+AXey@Ep}T$ZST=HsL)QCMS4ZrDcz*%qaBE) z3e*qSP}}LGa?)hhU@UYxMzJ`J%E}vOsJyc&rEzZxjhRAY+Grf|n}SXID0vBbUv=5M z5?4u9k-NwkN8M!d=aGj|OK=*b0ZH}jx#!-)t~-q;PYdF6G>~T~vG!-6tJYj58=c1 zHT!u^FxJu>C7(RO3tRbw;ryPb*mvKT5yB4@)*m|Y6!%-X&f@w!xn88Reo=F^QfaNt zRe}UL6w{>mgO&c8+cTg$(k@P8i!ZH$dnI!bxa~nnI_Rl z)kbIvhqdr6llYQ0iH(r2tjjmTb@@KZR=$}g@pGEQFK80KRJm|cW#LKX;lE;e@q)_HCh;c9eG3-QY#yP)f*gGepQ4%u zwW?2Y?pSK|q15Wh)anbm_f{RJF3}{Oq&DQiA};~OWMvtG#o_T3HR+`c!FNb(KsVYD zJj^6^8H{feN*0OibTo`(M{CXV>zy0sm7c`xnt99Cy5^L)SY?jm_J+Hi>eQ8=&2jC2 z*8Md2#L3uNs<$$Z>CMt~-mbU*SR$wu%NFt$piCVMJ4JjEmGQfPb?0K#Vk6sje!Eex zB_>r`5uHPB=;CUg-E*Lm=}#VWaq6kV8Knk)b5=eh>j^v1a7oF{h-=adI-~;$5 z#M&SzpyG<%o6*kA+`hfPyaG7EmWw59l+i3>lTeSPRU#Zp8!MyldKUz`CRDpd$BEF% zP=u*0vM@VDD4i>~rz@Is*{-XO6`Pud4VLZNHMl2G%NmN#iMlJ-`NLS`Di z3D`)m9pf*ukW5ufIP;C2cw#Pssq|*zekQ!YM3K>+m4UHiucsFxRO5b*E*}FqPdILy z!?L>ZI|;!ZnA8UHN@d==f9sF4A5^dmmvH*81ccVl7W%19R3yLSVGj=Pn)ikl3Mg_f zF>+Am8)dd!SD5KA=Ws6_9zBDe@|g-(H)pq5xq~X>&Ce;UqQ;2lQO8>TR)dj)^$#0D BcRT<9 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.class b/out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.class new file mode 100644 index 0000000000000000000000000000000000000000..3123be34daa95c2d889b6cc357a85f01f8c7043e GIT binary patch literal 5162 zcmbtY`+rmQ8GcTa^dyH<+6DxnR04>$0nLP>B9}o+kwQvYQ&31G`A%u+OiFG zUN?2h=2TGUoH*3MHq@3*Q0Cm`W$G?&(_P%fUEK9Q;4}8TCnrs)WuFi3hkWnf_dM_O z-oEW?|9S2O04?~piqC4e95-pW8MjDts}JYkHVwDqbJBcXrgvy;?v&A88aCn!GP+xu zd!*Sb%@?Klk~H^9^JNY9;eHhlsMzPlehpLcpzGYuxo!;);bAYnqM-wi$jPsY#78xJ z4Ufs41718XXTI*mK^0GEcoI)(_y!KSZXjUS^`}mqlB-;8jfOA!*}qUjE;%q<0_u_;suRbKj_5?72j1*v#Gnib9K+=?)Dw+T@eNT@Gj$8 zBa}4KeW8ezOQidj@labPowtm%wZ%vlOwKoU_3Vgj-n6N!yQh7Pf+?-xuC@(3)~$}L z+tJw(p}e+r3eH{A9%<|D*woY2J$_4%nbAxtn=|wIOisc4aN4v&yRw-=+S(mT?rsT% zx3xr!xVfg{7;$)`{bqC^Uq~rvJnhQhV84;?4_$rrI(CZ}@o05#As#m=yJ~x@g1XYz z%vh_luBVK_)?_9+5J~K&N=kA#qYa6Lp&Z)JM25Bl9H<1Z-q{q-LYu=*3-Ij@w zh1zf;ZEh^2dd*yq(VOIoKb(mg$t^}MA#-QZZS^O}$kMaYq4JX?A(2*a&V+cVP5WTf z%vuS09VetPwvIosrmIQNR`)wox0sz;(!@0p%ZLti8d)2o3Ucg|o3=H^`0B}YbB#Bs z+IgqBkmGe@Una-C-m-saWe;M&kuZ^M*SfX3O}XN3lx9 zE3)#cj@R%b9jno*V+VHX=)@)+uOqEvEw0qjhBZ3crTLglyQPVsN5{wU2^DYXNP#R( zu~U$EY1f!J6kK*Tm@8DK-xtF$V5VbcEWoUjGOR+*Ob&@QZ>spQj-TMC^mXT4A)PO= ztK(;QOUF8dW%P3uztGW(UkXs#aE@~!o-e zR6@S(7KK`cS?>hd0f6C-v zI)<=Q!OSwUZl>o%${dqM#>5o^>5LT^V~pmr{dD~5jFM65p_Z9D%|Oe~#%3hnr}UYY?N^<~U?NpW6{|=&3d^;EDch+bl@A0- zRP)YcgMyVv=0lsMRZuTkb8|YM=u4ZijU5S*75j49fK<^0ukloR?HYD$2Y{Jh_JlV!XHE2(jK==dNtl*LbZn%1)%;}X7^FXz*GGG<^Yt^gAX zXH}TS`SObSOwLzS%$qr1Suqd5FSU?&{Zaw#o(0eDS@Z0kRgis^6coE>eYAU)L%U~1 zw0jms_LGQ(^_J(a;_M0bRqVsT1xHa89EEEH?qPU1svdzl3~%r#e8J$e&_*z67`hDg zW_vNe2G(W|D|IbV%pk@_{+-c;^;m=rWFkzhI=Q=% zvo37q>1jng*oN(#5t-C+V!noBH~XpF=QDsU@hkHw+r^y&byJRe!@;_0LGmp83tVh# zM=(XYM^JMNQ@8r39mVt${&N?|T3vN)6f;UYGi4`eZ+K#hN6|2hS;eVarj5hUc*$Fo zN{Y^_^X{(4sXcpFEC_C;v~BL4&WA@ZdvDJyTN1C9Gc{560Vmws? z*GV+1FlU^^j?>9wjdsYRbqN7@hr0PkF=u(he%hMmuRqejCH(;A+q}*7Jdb%>T@9l+ zKeE*=J2D!@e9@}Gf5D#pP~8XNcCDDT50kh*dE-I&8~hjUS2s(D79-n4leFihJht7B-UGpZo^8I)P5AYfH5Z=bawt-fnng(>Ck7jXkSEC^l zbY1f77|qKE+%ATL!tdQp)AradJ>=XF^EE&_3BS^;tx`@x=WjAiIlYOGtK_7LNKoX- zSTrUDc{t*KAIM4lK6t7Q1xIiZEyDo5xV%QzDm-?vIPvm^L%dh?%(Nk%!UBXq7so+FnXR8*{XoMILVj$-$U-n_8HQOS4sgZ%pr z&z8koMzLs>t;H>~joYcm?8CfRG{6O<9rP`gfBN|7Uv+qy26&YQc#Q^l9T(yaEXJD^ zMp!b2e+fPNI(pR^jKCMVgSCad!p}%jlfbNv36WKHZmPC--m*WS7f(4KC}&_rp7D;X zOL<0Qc`3&~g2mL1lp*zKC25db+A*|j^)ES!OHXiC ziR|49)wMCzHHN1I)EowL30Vte*;?ANE}KBsdlk4!vMROQgX0Bp4L0fRYKIVIz;p(TFWnc>PvN>sB0;YjU*%W2-IZiUHeX1~>0z+BS)m-j(YWHG4ay)UX zZ;Rc?34Df`d+)EAwe1MEZFzAovRcBrBzeANs3cZdJ#y6|Ph)YE(Gt-UA$)7a;(pt@ zGNQHop%xv{F~r^0V_`=O)0O6NkZz6Rch;7+@EJXe0==Fr89-*xs3`qNda{Ii4NQ7P z?KI6+q0k7>t6433h|KRa&fpZadFsJ_gvNftX__0uJd`j?GpR9`YFsA`%9R?k|E2MW zGTW4vl3T4vFnMCph&ZT6lDVd>+NrW9!f7E|X zxkDdRNzI;*=ozUV!+BIvjYg`mD0fE&XZyq547C1p8mY>_q|whG%DxzpT})*&SW1dl OQKmjm1$6pmkb4JW5;HXb literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream$FLG.class b/out/production/clients/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream$FLG.class new file mode 100644 index 0000000000000000000000000000000000000000..f67b8e1986cf3966138f04fe1ea3e261cb411460 GIT binary patch literal 2914 zcmbtVU2_v<6n@@hv%5*tHcgvCQ=qX>NduAsrF^th`UOS;g$AnyMYqYeEKPRf?xq6_ zlkqQj$(7!D9lpGm?vQ&U^NKpZ9z`=gn{b{`fP1 zaTFs6U`D|l>-h*0SO}AKmBlp{@3F|Sc%Q{}7J1%$L%~fIixG6ZUvQKre1MwTeK8?{9kR`|@|kXF1%( znp>^81=rTK$}xhQq)^g#IU+$t>nylhapQ(o_3w?ObIl)bP}gj$GW*!2AZOCEqi=d` zaIL&fx6V>&T&oF7rJ<#|sg;eA#`h!)=hiDdD(J3PLyES~aHy^AE%H9bw%yz(&lNWu zFZ))rJZHb}Z10%&A+oN!%dNgqOvIl^!K&Fs{i?w&I)EB6=^7P%;ZaeqGQ9;-%~GaH z+k8(MPO4@)wQAL}UA@GYeJn7r)S7|RnxU6VsY=apQ>%JvOkl__c#6p84aF-E0)wp` z4i*2kuOsvRa1bEVG39tnd~M%=U;ord>9udMog^1mAL>O{!6#AN$EONDi$cSyz=i)? zc%oc7zk^S6Vwmfb%>JGoZ;m*(uG>7_{F*GT3G}CPqiyH>9W-2R!8XB&?rWA^(TK$t zce(0b8%i^~$#!5rAnJR7UQIwn5(hB=YCWw27(|@OLUx?_;%TCcr-?P5CgOOSxZ`P} z59ugj5Z6eK&}y8p1Zg=F+=BR?tN`97$!{?5CqnoO$I0STva5;RcjULTW(GBti{@GYPqYNG72)&_P?H zJE>+US%?rMAtNjlbO{x`B8&kM!I0>{h=}5Z=tNd@d7&>g@7oT2%xeUfXUuB_7h{a) zmr`eo+aY-tHzed)+!EQdxG4(4wC}W^(eJe4jAoO;r&4_?DUrOK43V5p$|NVGq{1qc zR9VT%nEX_(Pbe=CSQ$KYI3XoM2|1xq`M)ESW;=%&QS?9-aYRKg;-Zi8OCTlsaafmo5dNNrL!`o{M`82VVVVqZmrw~FzV-)qL-U3&Bd8H!X7d4?+0WzPspb;UCxQeE|o4!ItqZ(n}9otn&m;;0vX zlch6$YXbaM`WOE}Mdpj2@Ws#2`2w9Q@vbd&|H2Wp95LUav>ZLYLv1h`1MUOavgW@EH#VL%6EY6D4xFpVC zUW_9z&S6=c$9-`DkHw|744?QJ^0cUsdt$Lz&oJe89vA$4=crYjZx`n)eeDk=SFJgZ z&8l5)!|ZCq3?UG`LjT#gxGSKtC*aI}fR}a!RQCj2*bi`WSHSR|fVuqur*;L5>RD#_;wpkXq&blBts@MapsN$ ztdu^qwYDm4wFRryYD=}PN+k&e3sx;Yy88ZASAX=e+OGcW|1Ryf&rD`+5c-3am2=KM zXP>?Iw_oSZ`yZaW0H6!M^1_cpUQFPy3`c5k9q#wyGk8FTqe6aG$1yLe@i`$ruj30| z%*7Xlcuha?1_=X7jrd;Rn z1fCTBZ+Y=;oDdl&-N$I5@I57b-;ujd%kYdB-^H_DJcsWI^Yc1hsK)obSdAaZ@hKUe zmy;jrcu|8ZuvLQ*%4DouZ`#b~t-J>Jw(xchRgs?Up60NY9?39l@Th!TzCeaM$*M9d26Gq@hg?xApf#fSN9Urmcqxr&^pmH)#OpKWM5&xb&J2xo8u-g`bW*=sOhK1 zv24blv*OuY!oO4Ys3wrk#z*@K_ISZ2X3HE~PN!3hE0)l(>Dr|Pwo=HPSRt9Th|sk^ zprL*Svy}*x_Kh)fLRk<^9bq_)B7a{pnYV1BPNp)c{D?(Mw0)Gek)*v@LrZ5dQmL%p zY59iGzSV;oJiXZjWz7qxGS;rbSj@@|n6Wf>8pGMRnI1HADWOZ79(yFkRJ-k(k&4T+ z1zNk{%0_9ZiHVPk2y4~4w;WAxa3XGv+bO2Cj+Znnx*9Y=w(u&4ayIu+E@fLgEHfeg zc6TaINNzCK)RA)YtRRG~%&^V#+M?G7vUiCHruL;U zk{Qe-B4%!sHL6JZzB43>wU`FIbAKSDD0eoQq)BNj!-6g9>bJ&B1~W4}V?$LkT@YIv ztW3U;vvy_eZROZk#nah5?ea#mgWZ=j6iGi2#ih*8i z)$ywAoHp=Nyk;PXZ5lRQbEO2fikh^65bn^h@tWnZ;zZCH1KY8~Kp(mcOk&EwE?MnF zSjV)1v-p{T2*L*L#HS3Hh{>g>hFh;yvD_chan8VbWDLBHpX<0_AdZU$_F|uoHw=u* z$uIDx@C+Gv3vV09i|PboI(}*35;8j8G4L+lGmrx#zqryyL+P|NY^Iq%wpGq_4Xpzs zR!3g7n~qez!%VaBBo23|I48ua_YE8rCAP?0q~x*K8F|~vIZIwc)68M0&!J$EvqzR1 z>NYcOXLE2)dZV{En8K5#GFWt6~>C)hi!q6Xq(fnkG(c@DVy>P z5GyfLp;FqsyzFYpR;W(~Mcb z1!=WvA4=u?nGny8eDzG5VrNjy>^EXzV=AwZ&otSnJS9}g7i57`r5HS8uUEaV{e1ap zUVSBhqlcL(d~|EfT+Tck&yF*Fe5Ed|z7zUl2d%ihA>{aUnAuXjui435`lwhvh_(Fq z@$*xn0SmF7-**~rCgmcv5Zw;?7Sgvm=q02-iQCHjKGGW;=Nn0Ha?qPe-|nEdknVBN z0W?Zmr0zy(hH7RHQZt*Hn%RHU%+{l3HYLt3X;W%uH{v{(HnAIVe}L2n@bC{v`qoTC z^PPoj3hqgGxT>1MoJr`u^Qhk2I*po1c)6atw{056Bx-$RtMmEJz&nL`lc<+v{p;0} zXpoghs3|ncMw774_f4buw6bG2$8|6`Zbua{%wt<#NDM33LD$muZZ^*zw$z=p{BHJ8 zlXhFQd5|}JMQd>v_K>38(*KA^R{M$5rB-+Ih8y>A?cv#{72=^1;ybt}ABWYxHLdPx zEU4JzRk!bf&KshuqsUj*RbAFqUD7pM!-0~9+j&py(0yz4X?Th*Gei$W~UM zmlRaQy5#jJiTwkeC1Q`#wGHmZg=ewIr%Mb-`u3hd-9FEOB3V0ymVu9NGf=Ivtt$Z> z>fBRktFJ!NfRB!Dyo|r)xUjaDS*NgAMZAJ$C2DBRIDb;y%Mj+UlF!X193rv_;yc3e zemuY*J>tqD(xWA$gXrYjCB@`K%6F=WRGUu^Rq>WBHlBp%G-pYu2DfbRzlnx-h{a72 z_vRMae2xi7hKQ^;HZIxBxxJg0>4`Xw>Nu)Q!-lc zxgjYUKt0C?nGO%}C)LAf=Z~mmc!X*CC~m}KOyS3I3y!nMpI~7=$%H+@q&&$~eF_;o zO>aD-uy+xQMPCVGDK)#~g?64=$_vYwu1O}oi?aJL%sn?f2$qPu`a`VMbfmQPvwX!j zx0DrII7|AATiV@%!oy;#M5#_C@hKvB5w&V|S;NM`6dXxS9Eo{Nt>|XB> z(z`6x_sRjMwPlzw!VD{y$B-ppDdHFlN5b69UUx579u;sMuK%KWR;)GoR4X|`8kxj; z5rND;e+r!y0L3``n%Vmsw*24m+4w!1>>nNRUsN(syzhZ)Zmq-pF8Oq% zE_os%5*yBqqV)u7&tvu8Mml!Q#fr9Kmi>v^KHyXNXNQ(XhZf3dpeTMTmH?WT!iS~-ZrA%nxFI4zf!uuOj?eCb6e=tV>#8UhVtMG4! k5uFYr2z7qrET{!aa^XI{J@UG`DET^kf(1L5^E%Z04|_TTAOHXW literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/LogEntry.class b/out/production/clients/org/apache/kafka/common/record/LogEntry.class new file mode 100644 index 0000000000000000000000000000000000000000..40c5acd8d8f888c4b03ee35e443cf506509e1c4c GIT binary patch literal 1218 zcma)5ZBNrs6n^fyt$QhMR1_7E@iNvHwu*vp7(ZwA+U%$Wq1h9q28RW5=Mh-`y zEroTiH&Q5bUCBVl9TA%%?uxj_kaU~Pp5-&JDjht_biD>cspfhu-RK(Tp`{-i&0|A1 z-A>1K^e{r-4FY!v^2By*|0%<|QoSTd-DikDcN-Q%u4X&d>wc$hd2fw+n>=#OHI4SZ z;o1BgIphAJ-D6NEC8)Wr7mn|Jpt=M_rG`{xB4*S{%6IpC&vsf2GXy&_-Wz(`a9Vok zCC+%y{zxSf^(%(Vp4<0K>!r=Bo*HJZaF~Q^*pe^}S;AG!FwBmVJ?q=;hUGD&hd@Qb zHtuurfFZU~;MPOhiYnnY7DYUgP{0zy+NBml!zxV(942PS|E=y#{m3%uF;I)Nq@>a9 zS`IyjvT}YL=O+|3MT44Gsux}VLl>spxhTFkp_Ooj*rZR8r=Jmn%x6!Q%*RhU0Y)pi zPI8t`2N38d06FaxOj|rbOe>#2_)4xAZjh9N>p7D1NMiw4XazUPUmzgCa0_#!xy944 zz~YGnvZo0XrqPLw-zBtfh*{wEGAwd>I25{|lyP5`lMv zN&w{t;s>Wl$jJdjUc~?@IUU-<02w(;_5jj2zjlUS85Kv*FXG>hI?EC#@l55Vcb~>e Otem09jwqIcsFgpVQ~gH( literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/MemoryRecords$RecordsIterator.class b/out/production/clients/org/apache/kafka/common/record/MemoryRecords$RecordsIterator.class new file mode 100644 index 0000000000000000000000000000000000000000..b4dda1dc74b6c1992a06143ceeac380b53668328 GIT binary patch literal 3595 zcmb7HYj7J^75=XESWDS7jq`Ao#z_i!*m4^65dzr3O`X_{QpYZ}NpQ>~Yk4hiEbofd zDvm>WgaR$J1xk6Pgm;thDBzY_?qG%j!|(&j<0msP4Dh2V@0Jt>j>`IR8+GzN5 z?%i|md4K0z>BldA`?~;kV>XUmI3B}R7(#}HWHgM#5rG-UD6+yCi<4M!oWQt{oQ4Sv zb{s2lydI9Mc_D8T;>Mw)5QB$e949d;E zGeX`YpLYv+uZH)F=sn`ey|Vj&h7W4EPs9CjB=LZ-J|yJB8a|>B9WIWJny$jSo)gAN zBV}7os&mRSJAHOLFBCl2G$s@h!3CZlFgzn|=Zjvyf5b~+Zz^vp+|c8=*_4qtGGk_H z+!!4p7&hGjV>m}nLywa&az_l;lJ7Dz?2TCrrGCN~H+#)foQsoFG;&4y5OqdJ3ns0oL-e#@ohC10 ziY|r!$LtE0wNgkbqn7pa=;-R}?mCq*^B#*#56ieBY|?$z{FVKlkr_X139@hSJ+lVI`C0E`G$^fVp_*zcwEOpoY(Lz zg}pVz7nEr62oSA%b-WsFI-ZbF*7~_3RI4$pu%^PULa@U2ZzIXvO@(!TdkelySRgIQcEa@pR47qfRDQ%W{q_(&&%kN!Q4CJFX8{g)_~y z+Gt6Px7U(d6hwQu{g-@$0-(SAh|`r1*hr6KXZ<8_@@c|I+nj@(D;%6rBcC_z5ryl^ z+41K~&@L65RjhIcaUN`JPFGK)1r5gIdd~@1i$_;kHK*fJ2IUp$?d$F0Y%b)i45wOS zW$-OTftbiU1xrRPFXyZYi!o_@P=l$tcHTW-uJWYEB>e7TF2|{_H4k;B)BeLHCFL_u zMmeUJ8Eb7mRI6e^zqX6QXn-36tM-#t7UR2F^|HuM1cjSwOD~3r!p<64UN9G!Zrgu* zE6w58dSJ|TCgrL2gVDx}f;`Z;jz67t@z-Dk4Km`nYmnjWublh-%Bjz_P9_W2Fq8}$ z(%13pZhnWk?nur;C3nuCF4;PRP_k(T;p7}5gUKflnniTylh9^R|Cq0+gLDI;q(7nX z(}>}xxC%eR8vGpF@eAC-lVqS4`+TD`;Jc@IvO^vux+6PrEySA~MS(+gE;Q5wDnf^1 zXYm5<*Ow6A7pyBfn$BU<9Q46CtQ?%hRSm03xVou?)g?5{V$D;$R`)p8UcfElcnRy; z!og;J)A2o#b&=qs@e!;@mT=AYkeC+Ex=0BdjwZrXbnVfL*wFNy`Zl>*pNNFW+n9(v ziKe#j91??@=dfuo5uU~7S!`*GBq(`Z39o32CZe=^9&w6=&mx+LOqb7I*`_g*TL%+b zB1*Y!GuX~M+e_FX=6foB^voeS*gA_=iJ@}JBx}gx5}GlF@B8Zy@Ql9-|2eeyd|8Dr zYY0*v7aK``Md{yQ6@H6G{0`UR_e}RQEXgIL@hk@L2OP&Aso_t&|7T3$FSrYT#Ton! z_fywH_&XoTe_$Ff;3@o*7A~{cFXGpD373?@v#JiysSuu5VfM4!X9K!%8+lLDkAviC^y`(0{qb+e?d}Fqg3HXSQ!*!=flv#7ZDCU%A+8|QQx-iN391URenSf zS{>S_u0FshhIlHymf|X^BP{v#)7;egrScWwhgFIy0fq1;|JIxF7Jq#!UduI3C0p@6 IyauuV0!YcqhyVZp literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/MemoryRecords.class b/out/production/clients/org/apache/kafka/common/record/MemoryRecords.class new file mode 100644 index 0000000000000000000000000000000000000000..e828471106c9d22dfb84428dbd379d3cb7a35647 GIT binary patch literal 5566 zcmb_giF*`h9e!uCo7rq8gapE&l!Q`Xvq@MAlww0Ul5!-5U?BnltDS5n$-ri3-JNi> zYQ3nf+FBK@w1Q_#)wc8?8z3#vdQ|IC>wVw)Pxxqkf8We*HZkqO^AsNDo9}vm@9&=F z<^Nu|2w*k-6~Rrorvee&8^Kb1T;4t*k59_uJ{|W*pyN|Ac|gafBOE^?kIz=%I6kN2 z^I<#~!3=yMf)jX1&OR)UFUs_!yq(hVB^kdg@L!STugdgm^7wj)9ORE7vq$7{vJ#Kt zw48lRrr(gaZ-((L86Vg2gpMa8ScWsg;VB(YM-an3@_42a&*H2+Ms?)E7?Z_w5yWv` z-Y$eO9>#Mre_rMn!?+~lWgRbQsNQKMoJ@bV!?iM|>tr+-9jTO+X-}Hjtd-ReNjSsl zjFrvGUQ?Ho8H}50Gcjbv51Io9%{aG*om4!dVB+nC!>t;^2{~!IM>S|28Y;5(9afh; zY`YrD`$h%^EF#u*9WW1>@s#bv+m5iD*XO3|t#CD=?H5+ZPQzMk;ZPSf>>-lAb{v~hypq*4$Z zNZM)IhI;rVYEJ&_h-)X~`O+n+mT=OvTE#AVFlD+U8KQ5R_UyV+p6PN1H>cdp(bn}+ zj9h1zD^}7lI}evMQ-g7Dl_x@$9M(`W-U_%RD+&?OQ+ui^df$kh?3V-cy#qx8+q^Bx zoin*mvC#O%YQr?BSEIC!@36f`6avRu!Ap^Hyvw;mr%VVQ~ zP4Z|%I}LKh4MUK=Yv5+E$~Tv|HHn8@+-hJKdUU*K;CtxSP_Md*GFeA5X$_i5mQvR$ z#HWUZ9!7(gtJwxS+mLeHhJ4!*(3f<4-@v;?s~_NH13$!%M60ZUALAz)mX;vt*Cqq6 z$i1K9RRcf6&kg(nzclbG{Mx`EhIIVKz;E$81HTtJ??uMIAHZzc)-z-^469aB;Fz+k z{sz}+=(8GBlJ_?ZI2lFR;u_yA9zATQ`WdQfOUk4H7q4oVHBFz@u&#s-lMH>GuTehX z#72KKFoHu&-wA_zN#Vz+fxY+>gUP_3F<{^?_$$j|38DEMVHNVq>dk`kHv{{`XnXN@ zmf2=*damQ~FTs+9!0cXV&&hmnt1l=f zjH-{HGR`&3o1VB_sPgQ)hYiHFQmU;C2O-o-#pVy0TB-bMWbx-1kovDi_6CkMTw$H&t!B9z%E(6{k@ZYm!CoR6I?DGIa16 zBO_o5g8Z>m!Cy%=h+-}pv5-59(S%iK!CEXQ*A?jG$<>~8yd7Pfk%wZdskiOuCRbS# z=4I+lbP8ez-|G-~jc_3y@6d6pj-48BRq!wSkFQ)O%GEVv9Fg8wQ*gl;Dxc!jfTCK? zQy_xSXzGH9>dH_EjiSz z339ojK8IQv>w-DVk{52x&Y{j*n`zvh!(4A^-ku4ciYGydI^(E@o|wz{sb>U589}RP z$_-?;l^zsB#^@b#DEM<5d&Pdmc^kXKZR9u?*Rz%9y;ugVhTD7b9$$@fJW@`GP2@C1 zjrqZIn7>9sazc?%UXl4!=`Bi+kf+B4|7#H@;H~7pll*%ypN?Ea7sTj<)fByvHk^pX zO$F|o3fwp0c9uSEj=z83fTk+)W~IDwTMg2#ki4sV@}1!hw- zRXJuJe;9MQ8p>~Rt_M#eSoSzYc?mT`i5;Rqo9S~v5%r>VZGq@oMNgt=t)drDq(TU+ z*~m@|3j;o_K{A(PEgr4K#k7u}QXw(H<>dcyiKuu^p#hHK;RR!h&qI=u`L?V~3FRl1 zFSL)hL}Y!g)tVHK#Xg&Nj(e6gukSk0f*?X!~AcZBbbGwtRHt4iL5UYN!u?Z zzD?T%R7u;wQL?kLiQwhu(GY-`vpDJRQdnMkRu;@yiGxTIYM7_RPAUF#kmge=Dn7Edi^r;6@}3-{f^{XOI^!WZy9 zgWan<*Gc8^0GC|GQo$_N@DXbGD6{dgBJ2f|HI%x*Pg>ec%)gyMAzqbQB_zAXv8=aA z>g5=s6O6qOeS#VPq(b(5T<0T)QN@^QC-4zM1sGSFe5LY95iCSys(Kw{1#TpMtal7e z+nZ-RLXaIVZ@73Ztfy!M9t@|qlO zq#Dl6Etw%NApQV;(Npcjkk;63@V+%dUc6pxRcY%#?oaOg9 zw#w&O!Y;C)USi$6%=CUiDJ#Wy8Be|s@24tYID2h1#1kLDT~xD*MeBq35LN#$_a%S5 ny^rus@^=OT|G{h>AJuW}-^2{yZmtIKF|6Yl;i+YOwxZ&H@@P9_ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/record/Record.class b/out/production/clients/org/apache/kafka/common/record/Record.class new file mode 100644 index 0000000000000000000000000000000000000000..178f2ead3471fd470647edd6dfe764716e41a392 GIT binary patch literal 7678 zcmbtZ2|ygx8U7!;v$H!4VL1~*Amkvz0>K;-b7|PkCLtl9KoT*wuq+F#z%K4C(b&dV zTdVbGwbj%{O;4qVN4yp^6_485T6@~lo?2@!d)iB@rr-Z&*qy+pNo)dd-v8eJ{`+_{ zJoDBglSH&qEY)ZY-Q%Ksx>v(+Sfl&sex5zR!-EVy#Iqz1<1U(@hgEunmyc+aO<&~U zQ3a&vDE=`tsnTQoe4K|T_{5VceMzHYx`&TF#lx2w{0a|G^YB$3zQ!BRFq^My^elaY zXW!)ETRePQrSGWpU5zT~9w$A=&+qZ@eJB0ENk8Q2k2G3LKV~Y=^NF8m^i%p7&wkEp zztAAbFByD+!54Y>6%Q}*@M|Z%tkQ3I`?nhXj(*RxKk)F1Mt`I~@$AnU{e}L@v%hKd zD*c@;_YWTa$rR-9FF~5G&DYhsX;ZtuLy%|^l;ev=;)y^cu`@6<8nUseOsr;q%jS+P zykZxmZ*Sb(ls;*KX%rzSud$<}t;xHi!{2Ul;IoP#cWR@{I3Or@o4>QGz3F_v1=C?& zoiDwX|X}g&@e$Guqb|iU}%e zzA&&SP#+0L>%DsuA+KCs3xsvyNH}qppgC0;F{*b8viqXFA;{Vsj)Yo9hkHV?jzG^4 zR`Q#p!NAbYKrGC#xoA%egyVuvYmUbH>jNW!;6SK;Fwi#`s1HVmhoh1DSST2c_13q^ z2Siw1wcESfdm?mSba*5dipRszNXOoh(AuUZ7R@#o+AAn)H!P#<;i!PL32X~G{Zt7G z%9a#b`}*RcgdmrM+QXNESWbH)5FFed7?C=vG_2CU1+98pg0t9X;N+>w$LCbZk!K2K z@_BJAoPdNancesHEvvy>(%?}M)rhuS zlag;B6da6?4hx!_*#i&`ia1B2@o)kJ1XUi_IHQU1P`ut33$9rH|D?rMGznxzmE?=; zGv!mman#LluiQEx(H@G#M`NM%8mChgThcKQh;QS(%1x!fR8lCzf&MU@1H>sKSdvoE za+v-Ec|SdoQMKYj;b3TEXed04ScZUes+x}HLD2(`qY1>2q0U6K9pMq_7gRXq@}WSa zzus6~i}tC~Ybw1Cl|mPd28IwsMW&Y|UTe>Vpei=Ymbh`f}u@eE`-@eFhxZ^Ql&zzT*B(??M8j>p6>ZGBS& zu@>$%MD$-042>AsGF!GSi&wp);h|pkRE2?C5sp`&pTtH-5*1bPL^Kxatw8;#SYM%w zLQ%v+F%rKPNHX=Oc2Queb^0WIip@A%7js03PVc7!s+g;bQo2&7TUgn7qD&X%V!lqd z!-t}RuF~l`u|O9KF%xdFKv2LCq)Mg+_*VATR;25exxn1bOyQL{0+^{)aeR3sMD=>~Nr~>9| z5--R$Yr}%kevM9*#dTB(#9{$RWv{B{3rMyErz)jmbfi}*>Ff^~7gk&;j5Wzgk=5>m z*|&px0xEGV6zJU%iHG|mpetva1{;jNwvM=2vSUhn?4|E^mXjMf&C_H-9!CclZsQK zNb{QOKz2ae;Q+K>oxvj+lE*5RX=o$@?B^Q_#1RjvgpqvY;#yjU7pgUQ5pYupZJ{Q- z9Bl=ZMcV*3Tj1@0Tc*LS7Wf?OpKF2J0JmFU-tVx$I{@#rz)XLa1wIdOrv>Ky^DQu+ z-)(^}0Q?Sm=P~kkS@yfJAF#mNpf}nrbgjS{v{{UOfHpz*EIc*OI0;Y9LzH!&v`0V2 zS;VhQTgVPNF4$=RP_cncVY(0y&Pl3h6x^!~n&_Ag{3r5yYMv(5HbzR#WUVpFFH&wL zXOS{S&f3W$d+!9MoYWMl4G{q#!zvYG9SsutXzW zAmusB;FOZbi=sXZIlvtieOOBCr6`R^UA%M=bYV(fieaWu9P>_6njA-Q$l*{aA%8~k z0-WU#M@h}J2nA+mccqtA}NtX@hlytMC?3b8yn@zeb z`~9S4pxh7H2P~Ae6y=$+v&4H%%3e@50#MXsraTDBLz42$;A0n~((#qiq<0;xtU`zAj3syr1` z@q_#k%w?|L9!mt{6fmze|3`bJhmhic*-!8*q^3s4}v~#j0!y^ z_DS9?vD@UVutZ7kRu-w8Lh2Z~N|ebWM{k4EQ{)a~3iq57FpqyQUl5{P~#`V$)ZHR<(Q7HE#QJgHd)Cm*JE~ocE zE1q3}BE%8bBW=W~)gx`hU+H?JWw;ddNXw`)-H<&C+`7vTgSa5V`%K3TAPhJV=6KAa z;h9XA57ddfuxHc)hh%+&B_!rZ=0Ko1W*Bo`7n9xDC^jRDk<3PBQx=kvxn5~@rjJ9*WqboZjoekpa8*4 z4HsPvvm1wdpc`vnY+HxpMTclssaxgQWtZQ$t~9N?Q3r1W?(Gob5FGk(i-5&4C|Q9* z=@=mm4n^LeLe?8^A11#cq#ZkJ-VrM99H-eWb;y%B_$?{zo|mM#bxA6{JEen>BzGbV z#%MNuR&q2-)I1m<2s}`t>!e9|#vzjh>_P;}_0Wttf6(A8&IVr&r+p^pHA;zsG?-^= zaMVi8-9-+!%2TM`;3z>*V;4c8mng`OvN0r!;z53t;bG&+T*OIgb5W6<)5AVSYRzO> zZ-WW~szk0S#}`YEPo_J|)e;+0t)j3)i20sgT@fh9s|XBVsr2lbOMJvrk5n9sCp0 zSPCQIzR0^BaWZ7 z1Yo`yfP&vZO-d^u7gKUE5j5qB=>(p$5OAdk;KkJ>usJ14k`^G47T$D_RJg%DBXgg( zP?{2j^HrfN_Xf_Xo9Jef=pHPxRI5O6(Lq{%gcf%mp-M#TlKj(>RE6aQNvh7rljjS0 zUW0j((EaKjOBX^TZ(cxQe-Yp9U!n$j*&@X%Qy-SXMxp^c_tODWhUIv&3{KO$(<{U) z7DCRHVep!4o?mHNR^#(-1oP7A6kfGZnChE?KAJME*OVX!D%Rh69IL)=A(CU!2UgAA zfC9+x>ZU{utRytj-I|R_PSEMo>mJG^U-yiHrKhxtS!fvXqO(@GJT9B5mvCmJftPYq zj$>jDxw(9rVMMNlKsx5|BAI^PTe#Tg0UBY;U@wdtE&)`AJR)ypPm#kA@z$}D7Z zQ^vp-gBfzLCB|vxbOMVl1k#G&`-4efnJI_raXT^`udy6gP5#^g4w=XAfJ3>OFJm=I z;feM0Zo{8y4l|gfGg|6O)$SxU>_Syv)l!H0ekM=VU3Dy{Q3scx0+CowF0q1g#7fFV z7R(l_Xudd;7K_#75o@Sktd)Xr>7Pw=(RzY74YsC8Jo|)f0xsMxj3&S-B^yvKXA!<7 z86qS6jJnjFzuN71l-6`|B(9yHb?u#Y%+|}9g4tQ^oen$}OwifwohoJ^7j< zO;BTdrxSoTz3w&FC&-u9a3QxHu>pT{YD72mQn~O^rPv6q{IpqYq6@H>j`?m=Czij1 z>__p|AyoP_*;V?CN_XG~_X?=MYf5?zi|&&{gdOmK$#me2@IPC!)>CsQDfYY2(G=TV zsAL=S^ONL1D51wGe>-~=y%ApBl)e-gnr(&0x8W~b&6F*+!-p+YB3h|boI@4jT&bCH z=dVq9s}`;NbC93{dH#94&}GSvqmWHiaW!HYJd9|Y z=`mM<@NbryH&>{6id75NtHzjUPXDVsBI%;}UpO&Rs%Y$gR&+>`2&I?}gfg6Gn6%H$mX!pG0dmflAW!hAsltKj=pofm#CMY;5X3mg{AY2n pDZrsXZo|)`U)b=Akis>Upo|gQ@y^ihAX0(dB)*!wHP}n+{x>%Ye|i7_ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/AbstractRequest$1.class b/out/production/clients/org/apache/kafka/common/requests/AbstractRequest$1.class new file mode 100644 index 0000000000000000000000000000000000000000..f67a99440f432b4f35a6361532da703046fa909b GIT binary patch literal 1164 zcmb7@ZBNrs6vzLUjdhy?2BIK7p;Q0p# zrLW+bhO=q-Hs_pGO|zb}EvKgVzUQZ~ zz{Bovxy$7qm-}26xh!#cfH{U)YKwH(7OB-1>98%5f4HweINnvCfcKRphe%P?7cYF#QER^jEO-AU^?ohB*3gmb{-}77>~O z@4tc_d{2Hx$0vSAz91S^KOnl+!Wdy^8wJ8x8{>pR8xw@_HYN$v$D+@0J{F$hd@R0; zbE1|15ed5bF@R1CVg#e)V|0e+$rm7FgnOq4jKbAB=kKIsDAM;QsQeTgi2j#r4QjEc@&pOZ_!9n@osXYw1>as zC-j5%_`x6GZ}B&Hyt8>JC$gqZlXLH#x#!M(p1E^pcmMk5k3Rw2!LE(-xG{<`w=ijA3a*XYn6@#4S(}3I+V~vzY~05K8(*MkV-623%v<= zh+$+d2m?LOFq1D!wde9C_xFW+!0QLx_2uEA4Be)ZTKcl#>Z7KJ+=^B$Ul$8I44IOw z35N6KAQT&|!>Ul*yxJhog|hT{V~4B2%sbAE-VX?JCJ{)9pIRc)k-JchwBo+rYCDR= zlN^M~;3lpd_9oS7Trv&7TPs1u``Ak!qZn;c|OReRvG# zDw_KT>%7@H2D>6#$`?z4IjBp0rJgcn$>9Pn=CFW83#A;s#-kjTuxw!^2M?rExMHu^9ZkDABv1btqI{CC%Gm=qr(sllV*z ztjekk!eu2}&0c$~4Ns&$*()(v5nSo2;4how4H)z%D`u|aE&7l#zclC*0T)SMqS-uB z^an<6PW%q`3yo5^OxljU2lyYkS=HR2(G}Vbu#5 zmbw=5#mlA$!oXQAks|Jf+pWa52pX~;2l)@ zUFAWwpD+AAHqC*BlhE?G+^z4M(J=o-p$k@z| zZ2GR)@RarAUCf(Pgcf?RVBrER6L&2PW5mKJ#w?8Es)cJRT*N(w2i@M$zeZ$X0@oS( z+q>6}lbN&Kh&xVpi{TZ>36vfa_YcJxojvzr6q%nqz6b-j-V*rB`5l*u*%|8@zI`q%z%*cO2>;r9D zbV;2Y*ECk^LPW!q(UegQEzrLLrxf@E@#8l9^f`R$6i+jX@T*i!2}&!xeS*Z9o{T=o zlOy&yE$LG&Xw99ciwnGo4Gd|faXR#xm-bjRhOFgS8*m?UsjKs11 z14jHfe*%-VneS9!e@ALlsh!b|rB#Wz9^e#71s=k{5{V<2uCYH&9xh~d_6FzOt0 literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataRequest.class b/out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataRequest.class new file mode 100644 index 0000000000000000000000000000000000000000..750f7873844c90ec4d479d18285b931381659842 GIT binary patch literal 3355 zcmbtW`F9dm6#ga)L&ix$NK9*Dm8M1ssbcNol2l6wDwRcnV5x2C3rxtEfx%&jru)9{ z{WE$_kF_?*si(`&{iAw%-^_#{LbUA-z-(B8cfB*VBfT!@ih`Bzv5SAcd zUV;x1zC0`9oEzu)W|Xg==d%kUVtt4sA<>YOa1oaz>_SRJ+Kr5e1qoSPmf*z;5^{L4 z4_ELKA6}O53i1+O1(UFdf&>qWd{2=GJ0f8TCB9r1p$ZrXFQihDL?)jOpO3^t0)jE! zCy=>D&`rIdYe93Zs?>sM5-7770Ym3f$%Tvg=zRV{B$rQw z;t>Jf*j2X5f*Pxof@#xGD~<_STgi_HChzlnpG>Rd0N13|GfvN zvI2U-dXY{H#?*?EsFxQNBf}Oo+VRHp0@Je0Q2E-~>@kfb){z3f}Byk ztQc{{WJPAOl>LFo9`koR-RcO_s29u`PSaJh=Kscv_F(~@l43>-LpM@NtxBxOwv<2G zqbF2tMZuJ+kx#79`gbYKWGA3S1;pHOvpHdt9&>jBZ#8{PISU( z+;9gD0^C+s<<+8$6=*UJ;(&}YDgsXa2T(f}Md&iBxF*9uO@t|67I)cGchKDFkn=%#O#X=?F^k7|Kh^f^IU^DXIyZ?fiN zhT=QwGj)=u)Y#ULT)e6jOc6I^tl>2ouj367Z_0QJCq%q0;~l&!<2}4D;sY5U;v*3s zbIDIcd@AEJe9k%hWqg4zMSLaWYkVW)TYM+r=zV?Dl2ZSwLfEB}2U2%UF$FBNDRWJk zWm~yJp0}?+l-yKA?6eecu#Jwk>3gV?=?yJbU7~_I5l$x33-L%QA5JDy^U*{olT76k z$xJ?!4Mk(2xmZL%=frElqhi!ku*y2+SenwR6OV)?+g37@M+}9LYo|K3pKbDvJ>;uG-7u&y zI`r)q6fYz@w8&Pvb}6Bz;iBgw(2X(PKxj6G$8dlij*~R+f=CZTs{Aw;X^)B?jZ?JB zqldl%`vZ3%30_%#Qy+BP>G&YX%J8!7j2)UPu2X1~xEw^A0>Y?~v;^cyrz3n;1^#dIIYh zS;y{luJ;y3Z{wj&JiLiL=}nAnVEhk4>IS!dl$e1m93c*qmi}W%(m&_vvl%paX7MDw z@i~hup2jm|`Ei<)yGJ~rqJkkm^)8x%z%wU#|(LskL2YuE7oh6o>k=F+8BIoS;6G9uz pGfnio4yGHYamGQKv~Ym_EwAqX8C^V>sA|z@+c-w6=V;Z1&_6qDi3k7y literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataResponse.class b/out/production/clients/org/apache/kafka/common/requests/ConsumerMetadataResponse.class new file mode 100644 index 0000000000000000000000000000000000000000..cdef59c526d7e4219746e8ab745b1529921752ee GIT binary patch literal 3068 zcmb7GYf}?f7=D&Z65=AVf>uGRR51x)thFkFXbiDNNn(?L($=P%u)s>l#?6M7-t7I} z{*r!Ztp=ygbUOXmFa1ZIK4*7@6-m*_%sKCS@?M_za^ACl|NHwN08Zg2KdyGbhigGd zm_+)i(=7(xQ!)0$`W?Qr?c5aI$y}euOw123DGIT zT!^YwwX~>3m(=+sHCi&3myJraY8sYNGRjeFwW`&kISMSRV-j{Jve`_w5YJ2|3fB^| zg>)>Hkf2Q6R#(($S*oH!3wttynW^xvoj*iDxp|$z(c~r-2RKU1?z} zIq5m-ypqY~z5Mk|*2{NjrfHaQV~$p(oP@TDO&)9-O`87`k#AhD=+=aUfl%)MWy3SH z>Nx0{(koiJzFgGIyjm>Nr7~rd)bfmK>VkI<+pI;sCSk0pT~oVL*J@TRD$-J4*36V< zsdK8OX0=+CM*`rEl3|#0dPTJi^LeUcne~!I{ga`l?l%|IQ4w+aYBo)xXc|iz9s5K+ zlXFaNl4)2pY8JPuVRCB?g_-a}p>R`2Zm!#_s##;QvLUB8D!LK9v}$RW>htrOIo4eN zS@Ps7+Eyg=ZfRUXV7ydzYfZ!qGMF>!W=XrOivk{KR)XWAjU|jc>#eX8C*vg?k#Pdc zVi}cj4re8dHnXv9d<1LvccH{8_)(Ezpeo}IOc^z>27XbVf<7msjuk)N6$0iK9KPtF3R`_AIrFlPh@;5L_Wjk68f8A6g3pVeBsBJGQPssetaY2 zTYM*6j`;Dtj34l$gtN^ItOpq@vb)rhm36xRaj!pDt2(nsRI$r5W)|&!^sk8Fl9}gt z3FX4xF(m%ym1wPLOxd*Ch29C}5zZ8EYrH_sn{w;h!#>mP5m#bU#kX+r{03WeUOTod zFJAXXu*r(0Er^@BCzSLC>7Fyp9^S|e`SlqbSkSDTlc}Ry;`8XZ2@BQ);m+(<>Pr~h zM%7Ip2M5K8Ny4#hG&X=UgO8nc2(b3+vAxUn2=|7PPY0e9&GO8**zPY*I8#%Z{ICoC zse!eTs`Yq7&H8`6Ut(4LnzqV^p*x;Q=cZGMY$27%$0lQWKEIKsLv*t~D?GHVuj}H? zp0@Nd-Sw90rm0nU@0`cQdBDZBV*_n#>4`@lhxt7*z;BxX_KQ(RYCn!*kl!80NVmXG zKIGwp>YQ;WNgwol+?t|!U;gEszvQsfb&*?VYNL+cva zBKOg*`1>9r&_Krxn+rCuBR!(X!wqyk#7=QOqI8kl<&fPZ755KEVZ_$$dw|`M;RopX zm2O(_3fBmNTnDj}LH66>h52z8;eZ`Nig0fd@GZhE67VTo-iC-*ahe>#+963cC4y?9 zt@Aj8vuL%08+C*Gi^2LRbu}_1Vj5EX4fHm!hlqw0#aoY(zFYxV9+>J$~BD*1!Gr0Id z>%Y3y7&4c^ICMlq>NyHOQ@4ivo?FOZ*+LSgg%r{zRv2pkU}VVawzkXtzJ1mm3Ws)Q zRP=T>|I`-SbcO`oCWe9@kF0{8kTh+drk)~c5xRy!nLw`k3|3v8AXY6;U_6m4hAheA uXLy(79+mH78a4XB8WmC*y&`FCX&UxvX(EH_X9&eGOVl7rx-b@zN9hBphJk$m literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/FetchRequest.class b/out/production/clients/org/apache/kafka/common/requests/FetchRequest.class new file mode 100644 index 0000000000000000000000000000000000000000..f7c9b5011d238b513672d1b19dc0b3ea3c3719ce GIT binary patch literal 8557 zcmcgy3w&GWRsVmsq$|nSPOMl-6(_CJ=Ak$br&*gOw(Hh$oLZ@0wVb4Gw>j5}Y^#+e z^-4|?R#u={$GWkO6*gdljuukZ73jLgc4pQV7z{QT1;%Rw#`_K9%_!Zx|GD={zEY%R z`K`>q_}=e)_dAd8oc}rBx!3vZ*FX6ZfSXiY!_Nny@VthP2Y7!WhK~ z+@c>#L0rNo<=L_yzu?EGG`ti<6JBn>M!eF1U&Jpp;Fs}fx%`TTU-jcNviR5hc%lI< z_;tB_R>S8s{DvRD>BoEg_1cbzATqN6Ip*Aq@cgh@RvdC$6twcf34wfg7{nfom~E2!#{Xe zoSy&=;va*!8UGaI=$|$GiyvRn@UKDKj(-c{-|-&-{3rfPF8>|G|KO`Z{4ZV$;{Wh^ z5Iy*s%za&)M^~*VkW@-jbqbB6hFM4zQrXNAqhKiLu}mgu_N9${KABfo*Ecd8KR(ny zHaXTmIvDHgos1n(P%$$0Wi$DLktv)o((}oBu~VU`@A%kQ|M2)^ylp$YTrQdKh;zZ5(M64`iYD*upO_r(9qLyI z4c=#*F*?#lW~L)vFjJWsE^Qv_y?gRxZ*1JXa40rBd3a*HKkk}bGd?mJ>vPXESu?IB z>qdLW#$)3#Zr3#%D$Tkk!$i6YF%VD9WQ@YRnN&Da{RH78u zuA1x2nOWZ{-^lL6 z)LzXiAuF(D-c-2t+X@D=E?1tewY^xGUj=hMQJ@!_N)q}r1@oMO*SUwwBfHg_76=vu zPBuVSnR`vsAoE%m(_ktu3WC;pN2pEYZEl-zIA>g(#Ja7n+n%)2JeDaWXOiZ(58qyw zCb6+LII89XHN8 zb-bmtrpPxkbzk|x#fhYqgN;^-6xT5YEt=U>HgEC!@>b$%%FEMUQ$8-5Nf!D|Gi$E$ zk42pVWF^k{teJgK3U1fVn)@%+*xo5BFcJl)`WIyqWj$ABcFqmkO%(?Rca^jYOyEq? z%(KCeBT&(sPNAu=m92;AINYul50#_5SRiWk6K^bh6do*B|Hk9Cs@EjF>!Ul_Uo}T} zmRrClXDjF4c&n_ApdyrFl$Y?9RJbE*Ta`&=I|TWM=clJxq+{O#)ZASLCHkf;))F2n zBq|)OuIlPF`MVQor-Pwynd8BDcHT@RkEWzaX?EMSSK3U4J+)9)${RP6jv3^>2~*W# zqhlP$$*DA9e@s(;T?JH7#}9+WCFrz`ld031YS5Lg8gomC-Ak^snq;eev`$$$?L_)>x>~E6H5Jm;I@Q7gt*fwFud5B}8aCXkOr?{J(XmQV zURM#dQCF?%8eMIY)s1DV6}qmNd-m9t2vh{0am)y0r zOC68nE*>+wx=yv}YKyvF$0^*U!;s6AToO2~s~gl-U2VhD4DhPS14e$9o7B~IwL@1? zwNqE^YL~8dt348my$aiF?WerH$m$MV?Nc`@bk}031WUDy*VKMpy-D4qtDDsUg}qlN zOINqxqhiZJxg1is-rB3(Kc&;j86(|m&dkrT68E1?By;kfr_g!AF34@$PPe7pc>NDu&zebsIKl(V+#AOxciFZ zitX%ba0Tr=a91Z{g`{Z|vL-QpuU)k)$CQp?*H-Ml>8e@=`-N4?NzG~@nV9eB%Q6xP zas6?j}5@&+aOYvYf zGebN@kGZ`qleE0vIL!nJyUlR0Ci~C|S?AR!9Nk$JkD{7^Q`q4a;Y7==Z#BBye$hy= z*HKWFzpBarnTpuoeg&0X>Y(H$h0*m9AK$;G6e954)arQ=!#UypsR}o-6sL##Q z5eMCm9bdXoUi9weQ}ki2T2g|t)kIt8PS$i)$J^aHzm;CPLLh*i0VgH6)<-$Po*N3*Bvr0i&3$KGql7j! zmrEyy4QUk06Q~r-=oS6cn@cev=qRuC;Y*>t=9@vu6%G##t4m4SPe1IsqM}jxB~?@yKCyt1R5ub;)QGc}Krl4sqYOX(IfY0$qrolRb*e-#y_Q?Dg4 zZ<;LIj`{Y7S2_8Eyo*i0w+uJ0n&H7 zXr1(gi*6)+kBbhIzSl)>ApK?+y@~W&T=ZtrZ*|evlAd(YTM&|7k#>coU9|4(7_B>- zM%tO=xDL~3;CKeJ9H%%l11~(h2iogeH|<$O-N#AP<33)4Tml@1AAPu=qjq~C(wHMf z7E6(`Mxshq2c$zKe}-e7itL$b&e-a+mhzmfyzX^MTmz_)_GQ#hya3N4yo>NfJWJ3z zz3nf;9}2WCA-I5slNP0Ka#M|)9cmd(6N^~W>5F(TVXfjHCz{y`Y7y-cLJB{M&&g<= z!YgR$^jm_$3s~P7hy+3#BK}KY^DiojVuCUkk>wj*;u<0ij<{1Ya>VHo-C5h(*+_Yt zL~v82DYRK0ID$!D8)jtQLEJ!0--uz{ zh7k-9*J+0BK_dH6;`(vodYQ=n3~~J`k@aQb`76Zm3hq$>+^clFS+(LVeDQdziege7 z!YS1QL-k=w4IrV08F+!oc{EbSVLSknqgQbVc@${DGWE_=o6g^#!&%M+sPh9jhwmcP z(hS<$_{1-8C_;=Q!8BpSk)WCo<4C|(Z;b?O9)wjV9^%N00j1%)DbHsCu!nbB>)WZ- z*7a~n*Y`NO63mIBjPEVUc$mE3$K6c8vqFdXH9VsDcO{&+{=Gw^%jlV5=$Ajx@JCUZ(*eKKoMpE+#_zu25hBI~UL{QQj5Oi^O&k zjYVPyvu79aznh`o6(X|t4DEiYr~ZH^?AiPjp4c7s?C6J4ZmP%0^Fj5@%16$7c%543ASo%DleSvGgNHhuZa;$BS64&qM zbDgWZ@g5?jkv#k4m75rR7F*>V8*9nGfs#x#Q~C(rhaV*GNn+>yW&0Z)+DXvcZ{ z5TAM_C`DXy#7`LvnII%42sr|N#C*gG`v(;9y26b#xMnnb(0ceI8l2}-6&zos3aM2; z<{;s2D%U9ff)m_fajmNyc_JRC937OAQ;xjLxOrmt5)LfjmKGd(0bPse?)2@E!pLGI zl|$n3%*v-*KStsF+00+5)AG}@jDr(A1a5tP89fuB+kBtG?GyD33n`F;AL{#C8wPU2BxDQZ^vFb_XwRdjz{RschG(BWZ2%tnaA)X z?-Ddg(jyGWyYV58_F_9e%rYSEzEE=a1^hUJCRScxFrQ=@@bmtW(n+(PGdAZ>lGA?n z6ibAJXDvKm!!=gcKKdGZTKtrA3QyZ!MhEXAR}PzP&v8&^)lHu@`M71EeT;b>V-)L9 zHRkP=%uiHho~p{cqmuc9Rhbj`DcUEew%hFJ^A%=)s0y=PYkry~_X3`A_I?L_#e#}; z%TPRgC8n+IONcGvSf$7(Es^48F2~Q{qrA(=4vPqV))CrG7>GkdwopOMoh&6)WIpXO zGvvth<70Tvk+jW{12h)lHuwp6q9-QMJ#xPha#vNMAy!6lW~%Q#DAxN=G)6cEq&{p0Sh|d)p-^(CiH5gN(H@@!>Z|hm(Gv>?eWBzS7Da2&`>) zFJ7G4xe*Bd5G{0+rprmzBYQ1-W2Ro}m_&|mq*XruTMS&Pa+dU2TBn>c`dB(C5T0vI z_Gm2goVb0^F?JBkJDH>uk7R!&{LqV*|C(dHE8s1-~ReCI|94Yte0 z&@lp>FkG9DpW+S97#xVW$^cm6-+#@5SQL?`E)|g{4>(@ttb`Rl4eklX-N0+Ko&bU} BoNWLA literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/FetchResponse.class b/out/production/clients/org/apache/kafka/common/requests/FetchResponse.class new file mode 100644 index 0000000000000000000000000000000000000000..850d63ed0844014f13477a4a8f1c5ef58f6834e1 GIT binary patch literal 6236 zcmcgw349z?8UMdzv$H#yB+aHtyKRxHl%xk+D;7v%HBGnemR#(nN4S>Bc9Lwn*$uN3 zN&)czJh)X*5k;*B-a;uks1{K1Kt=Jw6BIAJZ$(t%|IN&9WuZD(qN#fmdbU?v-6udWp_XRK>!25$}!tE+PfDZ=nAxV5# z#Ygbb06r#(k5}Uk+$kRirRKX798$3yH%WDO%g2O#Osbf|v>e?dH650tPe?VNl%pdm z72T`gJ{4@)1*3YKKqwwf^!4^6qKV|0(Sc-7M|YIWu08ridL*M~ha(Bg zOlOC=vZgba&0Bib+M#F04I1z7?TdAKq#F7<;{CDy7`1u0p(5AAY>39=z42sc@204S zyJ%}{%hu%1j{a!8yCZ(4XQm+@?d*+jN+zQH9&Tf-XGcd@Y*SLo6~zR^wyGKTM_@s; zyRUyBIjcaR(b--$or`STZy6iMhlUK3JJd`gKgLZOPO$5oW#d*l6Y18+xFDEF4`+32 z+~oH6mCm-8jeLJ@ES)Omv90Vn)3|Wl$XofyX2VL2#9ddG7bT@rZ5!x?zdfBzTN{|B z=29QpT6PHdI&-_}UR_r@YxInd4jN{^KA0gV)Rjx=nH{>BmUMwRB#o8W5ZLhF`ADdY zmBRM9!_qRxQx^AFThtKETIPO%D)+yeyFyZy+iaN{ONQA=+^leiY3dYS;E7XLIxh{V z_Ig*T&l*XyoYo%YzKd$|j$KEmYqO~|+^jk`abEeZ`M1qw* zk{%w}sau9Qs+)T`ZyKqbxjSK40t<_>1%17PdrA)%G7P)kYwU6?tYf*`G%L4+Z_E4h zHaHc0Rl&mwzDCEJTS{R;pzNQ^d*0$t=ehAbJMI=20&@|1=c**D}fzW4KKk44RL&3KJp5_q2ZhOmOz7j zC9wOh-Qi?EkxQ=D@Cd%G;5!<=i|?@jG<+XF(C{c8D-kO84h^rw*#f7Ob8G$}*wAuL z=e&j=;zt@D$733PjGt)uskBuy>mY3jv>%;;()LN$i|9?7?pe$uLbdUqO!17>un!k2 z_?d>E;|ZQvvJ#T}3Z@X1OBMtPx!Nfzi9X?{-)tc{9VIS_=kpn;$Iq`#*D`L6wHUH5Li)e z=cS!SuCCOCPgE(wuL(s26cN;fDylVVx>yl4ny3|ZiddkDdJ)plg>He<%PBABtz(cC zs;8{DbD^0n>e|^2O7yT5%ratZA-a__8TfOYVz$r1VuM&@7~Q91Me1G=*hq;R?fZEH8!`*3s32!yX0iA zl0tCj6$( zzPPbf^K#(khFW+ZP^iU;^YUj=;s!pLVFG4VY;zouuVt*Xmhw90oT&m!niJcc-Orhl zCEhfTqL`eh92;jSPxnr{dG|!JfP`nV{zLPaTQzpSVU3!GPNWrfHrLnrOf;A4xJ45k zV`&y516H*;+TXcVU}f1W##|XNB8bsvU*U=m8t>S+W7rV$iq^@V-YPz7$m zBXU$Na38eRs+EUO6RMlQf>wXnAF2;mO(VolQfS!eDrr?DZ=qb?=usC42V8Zda^$KD zhJ)_frdE~u7D>g`;p)(0IpHcM84jz5uw)8Lr6JPGBz?>jUf|NlO8PjPP{P&IIKBcq zZ=bnAY78sWIKlC7P1s+UI?36`n&RBbRd?gWI~h42w)1&5zn_6|tz~2z7}ceW>IR&} zxW}=JG1VE<>$vh3?7#u+#GT~cj{!VP$wzSxp1`@hYn{gn)A^zjN#Vx@q6xZKia|;M zLr^K(OUYdv9bm=>C|OON=kWJD1Zn9WoKK#gwhkbP3!u=02S}aA%rcMEl%gg%a`N6F zg4j)+6<8_+3`$he4k+b!^$t_7qiv+9E$wRS;qG`;**f+Vb&OEQUh3$BZwAe%Qjk$F zYX4;wm`4VCMCjlN?J;I`F`zIClt;i~%}94JoUWMa?8(27>V?a2CI<$ASI zZ44g3o^YU1T_=$xmuq1yw0Hu|6KGjq!*O^5D<`mOeJ#h!Ca`(}Yfh`<_}B@Y(prBQ zO#_X!O?Bb=Nkm#h1kQUHXm8!UX=H@GXd=5$&RoXDxhy@%Q z4Dk?0!)U<>KggzW2KF%Vdzsb@pQF6iXK@9?c`e3pD=x%A&QEdu5m+`{E@9~=2|J5` z&?vD5MBRAqbc&ot{LO`3gbxva6hU2L}~))ciJcpS^RSr*w6?1V387_ onNwak`Bdp8a02tt#?dt#S1{)7{2$_|iMeQFOV;?!{akIZB$Yzs(O>IkuFocmL6DJdt?)$#) z`}Uz<>IZ8TIQ4Y-+&`+P_stAp2tlxL&dYmm-o5vCe|LHBpMQV<6Tp4=Nx-lhPKxP1he3fY*x($^O~xwDK!(+7xHo;7^eeSsh45L zKx}OM++_6hSzp)jmF1EBC)mGLRwRc`RFtmHf95IPSzw{QFDBcMHxIPvQx~;8sBjC zDLF-dh{5k)k14u-t7C$pI;>6;zWQM$Cyy4hQ?izjrZTkS9ad9PWd9m|=Tb)+3sx zYB9NxCz{BqnBPuAVpdZxbMJaPHbho(C^SV#q|}N(D>}(9_q?nXR$@X%nruw3nxLGB z>r!fNM9N!aHu^hkCrhbQ}7iVTYVs^)7e_#{pC_4ss^&Qry# zS{R$UD5rE0Z{jThZ;N;bAp!4-cn|N3_y8XY_(;UZ_(Z^`oRQB2d@kY(e95tXCE{y* zBj8&R-{E@^Kj24(o=r(8tFSw27-oa!IaXDWb%yZ@9ne;6rgyg~`^pW8`tT}4(VQ-) zWsN}~f{XIl3`4V@>T=x)-Xf?KnFY>ppaL|zRH+d-Nr;Rvv~MC|At5_1rF7%8Z^U=W zHKR9DHy0@tY#DUQZf&)HZQKIN6cUW;lyRw0(Z^}wkMp>=$eVCRDv-g3HI2{MB%%6M z;;eJEoHZqz&&Z=vmO6%iQmD& zSlMyGrlXY1s|1#EsHuW1kA`SmFV@}6FzAm)O^szY!WN=#J&{&!-7U9oqhyIGO_qrA z4h!fy`>Re_eiDbNqNY*Xv*4SrGu|$4b1RlYz2-%SJ@ig&qgSke-TWe<*>3pJLGRH3 z%^k*mr)}+~?+0z`ZS+K?=1*t6y#E`&bOq3_n{*t)VH%In>St8bKhPhz1(v)H#}cZR zP#w5|8n4i~2-kUI>TX?zr?lGEWlTG7LJYLtL~Wq!ChC4AEe?8h1AF13aUbfiAG^ql zcJkmbjSr9qeS{%IJ{+YJ0n;+>r!V}k-$2Za8OQ7}W*l{uf%%w)d5+fnTYI3BL+$hm zi>T*VI=${iY+u9|(L7+OW<*gx?eHA23)fyP z6Ao7t?x|?GuH5oK1z|VBIBi)TpiuEh6S+=GPEJ}&U>SRsu(w>*kfF+qVgn*LLt-Ro z@xzr4SW5edAI^o>RO+Cx?E4$6g3d7;Jg=qGg+W9uO>KrApmrsm`+tLj*8wvEX|lVE LR%dC?fua8ZVur(B literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/HeartbeatResponse.class b/out/production/clients/org/apache/kafka/common/requests/HeartbeatResponse.class new file mode 100644 index 0000000000000000000000000000000000000000..3e41ab94a3ebb839b0ae52ca28f87942601a7e92 GIT binary patch literal 1934 zcmb7ET~iZD6g?dvL&gDiCxQy==gOJ{Vb-r*cR|+>$HgTOO@J+*Qk}7Z(J&KxCRXvE zd{7sG(kjbmf7F(@XGV-l1Yc&lZ{I%m-qYva_y50s3*ai2HB4rZ#*_{QONWVH)%KEx z%l){blgBl6@|!aLu2$ET?GGJ)Vp=B~dC%a6`rOp<7j9`NG7OfMt5th(sa`A1*$ZT_ z%D!B)c#Avh!g|bCA9Ks`H#U6FYDquz9lvRXn=KJoHFDVC(+pX=TCG&;rOIu)K5sv$ zFBTVU2DAK#Kjl`FduvuLl&-f%t^+0C3qtONE4hJ%FH8PITC*BMddBtK@FqiV zu9jb+UwkxFc0IA!-e?H9#2Za=Hp{-ln=4$pYM&T;!*w@cxZ1S>DW0@N5C+zq;4*9o z9#%!rqLc*Xb~)Ge-j|h!vh9S^D#3IRt-pu1_%cIJE>9KrwYUtZp!r}Dut z+pDWWPUpMk^vzn*jp#bV@V=-F{WDH8p_=lbLiC#7mX5gNDt1S{;%7=dCd1@mwGkC* zAd900Mlq{CXAPXg&%|>#jP4atdXZ4-4k<-L$-r&c2JYZ*1IKZ~z#Q%x_y^}T%o`|U zLBpbf3ho)Wj}scI25MNMhBz>Eu|W^Y9nwXll#lZGE_K9xp-3I-Xn}QKh78(MCH%@N z!%XkI^~> zjZR6{wC*F@D8}gf6YU!4rBmQs;SJb>mq_iPX9vB7E%ce%*e?3Fk@>eU_6quo2eM7&g)&s}vV{p1X2R6Aabz3DX8{Ju z3Ajk9PExR`4)E+Acvf{jfienmJ^{C)J(Wo5>(2-x1(k1_%)q4x#)#H^2}}m#R7cFj z3{|E4%r{6YA`06Wcn7wF!RKEfUF$$HiGv)S_Y((&1Zh0N0R~=TXbV5=;>Z7xQuMKo MbdJ+*A_Bhn0kO#Nx&QzG literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/JoinGroupRequest.class b/out/production/clients/org/apache/kafka/common/requests/JoinGroupRequest.class new file mode 100644 index 0000000000000000000000000000000000000000..9994fe442bdd2734c29c0ee2bf30e1bf6149c389 GIT binary patch literal 4986 zcmbVPd0!jH8GcqstPqQ_WpItJC_aTS$aWgHG1LymiY*(2ivSnWT3N&bYfD;XwQ@r5 zBfanYzS1LYleUgy4%?06-fencAE4i&eAo|GfnEvme#4xOt3@?Cqs1?)QV;_@Z0a)#B}*+>qiU|K zN!XfBq|?d4RAx9ikQf{p_Ru#E51voeS@j=$UtIfrJzme;i2B)#OX25(aj^N zbE&}#sf>q2f;?>+vP{*|re`UpM=$8saS3Oms|k};yt`K) zS-sOfD#71pOj5Vze!ZZj%J~V+99AcC+zIy^Sv5DRnz~rKn|^CXr=bt5>1k7YysVY1 zQv9r;7fuVc~vz)cM1yR6u>&k!} zoVcX%HJw4f@Zh}!y~9~TqZ2N@rm51an;-S4Tw&jow$$w90kvo F~BB4d~#Hmz9+ z(=^N>tyE;3&{0EC*ZFqJ49}Rx6T)NN?Q7axaZ>Mu2(PT=`e=!#h}$cgDFR)@phO|9 z9xazeNN9<+d)`#?wJzGu#0W3F^k&5VRhu#(mz5Y|uF5t=0N!c z4k-mk@PL8@z9WW|f;VDF!r`@eRGeQeoW;hja~XY7!FTaJ76rH3Wh|%M$@sp4XYd0V zKUDA|Jj+;Ks#PSCu#+CMbcD0m*%m>6~&up`G#6;V)yfip~k>JFjc1zcyE-;2#o`pB45FpmWVi?|^H zGHxokg~x=XpGnxgEGe1GY13-1*PLc;E?9{tvszJ@D&g3ulPi&ZlM%fXDHvAdikj0W zyCUpX`eejSrBY4!7EBso< zZxs9%zmxHM;f9wL`~iOyivCH#pYazNe^u}|{9VC6@J|Vc?g~a#_TN*PbUL_*=(>`| zGB{GBgnJ5uTiDi4qS|GcHw2FZ3haZL$#zV1%i7=+J4%#QVATnhuMPHQrRLkOa z)hlla-Er!%me(r)+uegbTwyeNg7{%C!{rLU4v#quX_-H3UnN-R(o}HS1APWlCo6J= zVjHx4(V7*>EI>|QL8>dUdHur4h154+vjo0Q$W90l&VCgD##U(9JK zHP2!jy(==kMg5#MOA(>7gUM9JzMD#jt=UE^kh+9a!zaq1MDsiDD($?u6OZt!NP$f` z-tkJ7TStf_{4t!eVJGJ{vodyT|8?<(Fz_*ckC4*<Kjo)_<^7HE`=RUTMZFrq$-N5zhJ?loU zAM&i5xIX4tZ$KNmJ?r}r79Eqk!=h>0Pxej9>$T;b$jHTV`#w*Y_aIs#$2V;;c^_OxNgB0`l_W9JPh5<8;%wyDiunQv$d*z=A# zG{<(#VPmXw4x3_a{yBtWZGky#exBm$aE9X%!US|;1G;Ed90zcS;2fsHM=(h6h6&vm zmA**uCi&(Pp-PhCxT7-8;v830-iCmUnKJ=`&z=bw>g<^Sz;Dk41OfDOt`G)s{sd0) zlmpg)4NZz8p}{NMm-#HjIs_seVR;@c!m5sN!#rB&v4z%kgd6AangOD>b!-7`H1%Mr z^Y&4H-O)g6pw<60c6GJ}jvT8eiksT%?_k>vZ1>^9QvvCjSN}~;O*YRkXB+I92c0gu z3UqdW)-}_v9kgr@_3ffvr>Jq7&oOGBC0=>%2xMZk^At}H;=GMfp@LCC-1unO9%7az zvObLP{5JSrC9w4}MrB-(@h1Kx0`ngRvPACVE*3Y5gurrq`?DN@DE|e1;&1aU>=-+^ zh@JD;`Fq z&INQ;v$|xn+Tt8%-fAQxq3a>RbjoGg!)O)W2s=!D43PL8NHusCJnDsAoMb@Dg;1h+0t%#%J-<@3PT={V6zu>0`wg3PC literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/JoinGroupResponse.class b/out/production/clients/org/apache/kafka/common/requests/JoinGroupResponse.class new file mode 100644 index 0000000000000000000000000000000000000000..ed7722fc7d867c79a92fbf509634e1e7416044b0 GIT binary patch literal 5149 zcmb7I`Fj)B6+L6i8ZCnXfuO=9friBkAWA7sjN@Qr1q8P2NH)YIsXCU&WMDTU__vHnFJ!#c2MKTUVJGhM;q7>b*gGxbogTc)hj-(CaeT&u_xP|G@Acq) zGT!e)BOdVM1NfjH58^|9d>9`Q!$)O&%!7{$j!$?n>c?t)(t}Tl-%rcV%p;u7yvkqbJpHIyXL^%Z4Y6oS94K zGGTLaLd%DfBp6rsN>~+5B>EDmNMCm}bvQbdig)!!B?Nj-sHfC$M$L|elcu3($4I&? zlFQ~zHESMGGX+h;ngh{zG|@E>>x-vi-45>6k-m6xus510-(KI9OvVnxqur_guEao0 z@Fg8wjRSrCv4~@1RaLU!VEk~r?`S+#rd@&*tDoaP-e{h{VTA{eN$_fhkuxH>5!x;# zB`9NBRx?yn&t+pH5`1Z1uQ09|Vi8dDd3`Lajr6OAsSA?4gcY_|3Z|Y3_vm?(KJg|= zrJ4nUGTSSTJ8FV4kekrcCCPg`DahTSXLYkv!q(ccG#QFKzEP{kew8y2%vU{Xq$_6?sXFE!lItZ-|v5G8FXXf0ZI-V(!NF5^iVmn76Tw*zVsz?1tYE9ri-*+vnYdhh$=2&`;vl5!RxR~!9iRWLt4g^f@#bs zIF2|WuO+MY&1j}oc~fbYUKz6rp28Ic-@(%?)?*^$QwxK8H zp_S+&pM-s<7~cxMi)%8zr{EcUpTVx+2l%0aAK}?DzEWA@?tFPqm4Y#^;K%rhg69O6 z&$<0mvD3Jq-&XxIJg?vd{9M64bSc=64h0eHR?v;8f?wd53VwxOOW0D&VOBZopRaGM zbqg)GSZ5Txh;ebr-zfMkey8B~$SC*&{wU*53jU10$oQ**zv1r+{(&wTFDdvZ{w3k2 zS}IEzbqzC<)U=tfi|>5JmB2U0iUm9DGRD%faCthBDYdYUnWBZ}4Epp=NA{I6*&)r| zTT<1qUnwHeYsRn$iF`Pc%VbzO*rx}@nzyh5OSHREFUBIe+;REi zIm2Y(zlG~g$N8-Yh`vMJ z0nvJ_Uv?ksmko$~w^?=_pk;iiB1RhzvAVwm7koTYa-%)GjUq1t-v|6@ZOs}AEXd27g-JynAtdKB^mHhgVL-H&F63(Jm%vMRb zhSlvJOGWllT{>-T zx{T{v*I&jBt=lhSLu<%&85^JAja)d)@eut1db*L`4AL(*;uhl7$=R*+%0B4ynMse` z&tClyTlIPFUZAgL=%c6UrDrJNSxR~yJ^T;c1uyA$q8D+_bYjwn1bs>mB8dV1ikU!y zy^}M6MV&PhXw+LXfrpFM%wiBn(7@FLILgtDJ9ys8Gh6OZQEoR5l<1F@=fff%|F7QOJG1wT*BSd<#Lkt}Q@pUfa9v zYG?f(SI~9cIi$7)UAx*FLJij-5n^{>?G$b#y6!-Tl!X(%Lz{MWE~3R|YJ)rRqmPr5Mo;u#mmhg;hMo+&g>p+&&h7VuA@m9zrR zw!ktfBBlb(dJD}x%s0^E4d}-T+>RFdekX5uGd+I?y_x6lB==9#pAT~9EQat1j#o5{csZjtY+dP=!Rb85<0Q`76Rj(h2t1#PvIKjlZ2S(9?4u-b4d*BLW zQGbYj8v|_`+bSiFJ0u1OwMi}y3OH4iwAqrwl6(={FJZ?F!cU@3#122pINIK9qU~?t Ust)%uPQcp7cO3ti*YEcJAMArpDgXcg literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/ListOffsetRequest$PartitionData.class b/out/production/clients/org/apache/kafka/common/requests/ListOffsetRequest$PartitionData.class new file mode 100644 index 0000000000000000000000000000000000000000..cad10ede262af6139bb6c3cac7123d46271d4409 GIT binary patch literal 595 zcmbtRTT22#6#mBblD6ihvaF;YdN3bail8eI8G*87R0ute+o+@NuI-M|-|9hv9{K_O zsOXIL5bLO1yT)vMvAD?gU0QOKyA%?{yq9|yylt3|o6^5jet_T%(hYYOFkaqd= z)yQqM+o4b)VKs*MiS(p8Wr&sQwXG&YwCXX>}CpW{up`8kh=9pfg>)Z7E}0Dzun_c+(ZhiCgMn#m_aguHHNeQax-N0Y-_;1 zj@4-OgrgXYvi_{yzt6@Fttdu&iy)_kl9$t3>7|X3{#mjnMfWhM63A3u!7A%xL@K2* zj2D6;NR!P^()(l&sQnOgIHC(|P$RX`$9J`_x8(?sLXsXS1qw||Aq^=kBx&iUN4Uatvq={=yJ2=x zTA&}EFIwZPAkUv~DrH}5;% z_x-GU~!dQX3!nhmv$l510d@6{0!?+KhmfQDh z_)Hi&wpU>Wc2wa3JXnRDct|cI8b*WICEIofaZME>cvvoD8a}Jxks$U2ab*x!g%HJ~ zVLXP%gZP|Go~Xj-@ddejQDlBeE?*WlUlF-aYIrJ)qp@99zABfm$>r<9;TvI+Kds@L zVa&(3M5k|S_)Zw##rNd${VbHRjN|c)h93p-V+}tEV=;aj#?SC<2tUU!HWbF@PA==}QcwVM|((q@6>Q#nmrL0sobE;t(3VJM)Nt$hG zBbQ6&6ehK;S-rZwtGhSec6xhft3q=})*NUyhK$6B?>jG+gEkO+FE;Kr}m!N-qYLF+S#rU>A29?WHhIZ%s_M8GEjlsI@{yj zt(~hpi!(Y~*Y|d<>Fiy(Vnw{Y+shZE4cJv>%Qt9SS zV~EO!;;Dg*VGWy9{IcS^OUoLqJ3Ew06kM>R>^?Jj@o+L{<(fNEIcue@$7*M(zCbOC zEm=lQ0!vewl(mdTZYb7cN#j}tUt6|MY}=8_B)f(OHzdt&V?&xNk&bM_NUt@_l+4{l zpS3YX931xsG@~Zdi&+vFF54?6X!fRBNUTuP5G!Gu8pp*&CRJGe1`a>LEyVUvaiG2S zK(fr?ghgXcDsX7eSmqW5zx!QVe3E*5SrBOnx*WWY6StbCLEZUcftY4XcDgr8_EV@X}1f} z)I!9pPUePKTbSiSg*6F~)F88}F*w8we+FZ8(AdnB=;T*sy=jHLdHR(bE-bEHTu9o{ zS#1YTejkI=o-@vh)@}Mg&PJw&zp{dB_!~D3B&~MS%$no$tMeweL#lhDnY~2P@{-1~ zb|}<{)(sS3BrLZ=BHtA>B*_$U6Q<5~b*)RDrvUg6j`z*W3^j=~Z4ft}_molXuI=~i=Kc#wIxeRCo?B#&lV~mS?*^A8kgXdG+oVBM``M4 zT{WmiO*QFij+(2hd1}6TlI!E|+;I$n;sNLMGQ6E(G1S1oFZu9m8k z6c+6F;1enqd);vl3&>e>#}{H&(lo5BNob$riqlHHuI|BkY?Yi*HW5ew z=9=5G^h82TzeeU##LevRkkq*4TWoA8)J=HIVf7wA2Vu+1=wR7#t1P!{B820zR_!v@=3I0N~2OiyTZpsL~)Vg5^sBwguN%2D!62N zQT@uhiZt3rBiEJOOpeURXY(;MV^5~#_j?tl0xA1NK!Vx2t;88$|A_+Zu+B{KxhKeh z`+C6vnJu^@n_;@HX*k1cZCRw|l|~-}q}Hp3i#0iiW@X*yqS}VWl6Vx+bezJ`UKXym zyz<7q%j4&j6n$L*CFx6w43Q|0{iF6H)m4NiA(3N;`u&i&h6$x*pS_Ds;^b*{ZgCX>gb+1*zy&PojqrITI8AOT+-5(W1( z?DyoYLn%UBj@}WAcjw<|DKwSs`U_`tNvU5&N?v#IfOGaMZ>e31vb-Sr`ABPl9 zZ-~Vl-%V&kUw>LDTO$gV^}Y?IS3V+(-U6I11%#T4LBVnU7udo%gCz(*8#; zUQ+rJ9L^WOEBLlNjG5B5ax@cXq5}$@99JO3d6#Ei!}&_jyq5D-p7~79&+^P?alYC! zKbZ5lXMPx8p|hW(tcbL9l)K)R+e67pIvwh8E?3S%B}ahPR5521Y6pjvIFDC(TMjHk z04L*oj;1@?(2EN=BZ14-;8-V1XQihgXoD>#;fSfY2R>8-YIf5YDtaD35e{_~z}#B!%6W%y!Ek_3y?MgPLf7eK!uTz`DEl-YQI@!QD8*Q7Y<-YWaJ|IJ!rDg}Y6& z9>S4Lb00#zIKgK-;d~s#@p4Q;8*0!_RINZ0P9ug+r$1vDAhL#tt6S;$d$_uTn0kyj zdYagImcD(C7@mfHf+FwQ45Tsl%{d)uYFD>=3F*yBvK;a%P?0yTV$$Z}Lkyc;d2# zOEp}k!P)W>`uAcM{6)LS)CrTzZS-#Cl|iDOfuCUnJl@0@@i)zhghnuX1V@R(#&9(8 z+CaC>i4cm7opbjrt6b!(^&N62Zkk)`TPWat3>7VbXdqHMf+k`$AlPbb2}Xl^7KJJo zg=@pLp*ztV4c3Mi5_(m#-3Fc`W>K}EZUl2iF!zK>f;e}5${3sD zy*z7D-fcv4Klx4NXC>asJyn#q6>lSLfKn}7LmX?=V;kPik6%ol#}-F?&Zk4f^D*&! zlkGg)rPnE9ZZGDditfy>Xn2P`e5VHY{)Pz@FH(fU^>~--=k*k>k^Nn+QzbRkd-8@) z!qrYgi)G~I4gVO9>0u^YID%u{>@|wxS_1PVRWWTy3KB=$z4wV(_mVqsFh7zp<*#N8 z$M?(|#iH$FIH4zUV&D-h?x~b)81L~lji6-&OX5BL2e5P}PTGZKyKr)R7fu;P>!YN0 z(nJ$O2n@3zF6J+qCKE-D2DgaiO?1v?n(R`#;4<9KZ*iIgUppOg1>VikeEv*&14CKN zeOJNUchPZnD0A&ScrU{+h>zeVJCr4Wr5-qx?<1vi@BIu>anBU^Ud0SM9=E)NlWT%J zZ8uYg9ZhKFx18Qj>mGR@2Nia%3)qWS*$O%Z#qTPkm`gsuT~e>S#P!=J5N|6%?8Ng2 zS@v$hhg=O#qsCI*X!{_Gkyv(C(=IF@McYJ{*V-(d;#Y$YGiS+PjcXiXdYj91DV-t5 zia1Ow>AO=I$R&7Q?@=@2@)QH!?sA%K^8u>!PH%q@l@hwjW2hr}f;+fYfsZo$q`j!* Wui8NthbRMj22z0#f5tvu=Klp)2;9{G literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse$PartitionData.class b/out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse$PartitionData.class new file mode 100644 index 0000000000000000000000000000000000000000..4fd667be019fb5d1b3e35dde835510544a1645d4 GIT binary patch literal 772 zcmbtSO>Yx16dWgMw@J68Ev2QDkAe^?5n_elmJ&cgNJuM+NY%TuIH{YB9lTy7einxc zi32}?AB7li;E=)zF81^Iy?HZ!{{G|ZHvms?yM-m}HqpSfusI*lYKL_naoe5)8YIJZ1_`6VQX)t^V(}`UP>td>&A(k$?Q}m6FHj5B(u}0HHlLnN>%uxvU61Ve5Z<;HHEsnz)9uX zivm8)ev7#Ghm*fJ58he>Te?iW(!tM8c)`V%55x1uh;b{%GNKqOXhzr)L(6@`X1TwxruS-4xlDv`N_ VZALppYq*Nb+*{0P;5s|){sgcEzK8$- literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse.class b/out/production/clients/org/apache/kafka/common/requests/ListOffsetResponse.class new file mode 100644 index 0000000000000000000000000000000000000000..3ae4fd4c234f8a8b9720be24f74e44c3c8bf59ba GIT binary patch literal 6013 zcmcgw3v?7`75*mK?CfLm+GGR6Ln#HC2XqnINE5_{5G1g9m<`a-mUOaAl7(b9ot;3j zwV+}Zv|8(0Yz%HD%NLxYqeHweLv6faP*wU@1L37%%)jNd)l7l%>Ix2 zfA{|P|L(p2%(9Wve}mfR`h-9EfWEPSsIMi2_#CpveD*864LE#n>^KHx*bhY$Kug%1VrVSL1g zVbS?$5Ff)~@wiujeO$&T0=NW+1i~l9|p zS654WcWGm=H3xK9f?t3 zSZkCJ>S~E~cC^P@V!fMMdV1TN+FCdm-Kp)?B12koAQCf;L~?+V#m%W?+SHQf7Hw!$ zr}FNO&emqvs*290uI|?ER-(B^L)lT+U`0z;S4UTGbH}B= zW=DU2S~t5KdlqGv3(FeU&=l|LL^5Hnq5GqZxum*y(wN3*+~iJ6vIs3n)7UGa$oUQD zZhVn7EKbQ39=FeMYH^ccXk1+8S|l7z`G6(wfO^cq1j&Bo^VxhBD{U*cfgQzFxt?cW zINl$2Swx!I+gNQ6T0zm8H1z@9c%hinj#yEhm*X=ZyjbM5 zQ$=pp?&#a8TcKK>4b@EMVl-yQBuK67>xPjs?EECfB*-=*W3J*_`v@)4XEB4$yr8Ot z$}Cz&0|l97tPsu&F-2H?MtjbQASe67iD}&LV0zK zOWrOAw@x2>X%CUxq&L#c>QRSG$wVr$cCV?g9qsRD!l=o6z!#`B3?*-$wO)dgVW+`L z@U4yyIpwFYTF@d8ON|%Wydc0D>I-C&?UB;Ic zoW+AAigi#{UAB^|&?iLpEBFc?lJQjqU&F%;HU(eDHxzsm=W-ThGnN7qQ3)&anRNj- zk;sNqozn`wg+~;88|M^!2j5ljJ)y98%1SDf(DXdCWYrcYA z7?SaQ1wX)}3Wm`oVTtaQN>fEXlX4+_WX6pH%P^{vhLz3jTyY%lM0ezv6ES z{w^&3hk}3NUlLa3!^@sUQy(K(i}P}1-Iu0<-Zs#pg`F%yQ`>Acbwe{#22;Y+HtZae zGh~;F=ad}n%dLolyz~|}r6bL$p&{l;-mf={KJ^Y5snHQ}S6I8(a+ib!)5-#4cxn@N z*751$8Whu+4RezchZwvr+F3HbM0V>{Ir%E%60VU4wRF3F6A>BC+jxCUTAj*_d~Qbe zC9JD?oT1+Am}?0*Wj24-nYpYm-wf#HCY_hFLf&~LY@Px1nU&$WtMO5d^Ey#FMS|F1 z%Aittw|LR$=x2$kjyX=ZcO~Y9kIa;FlM7eZ%vh-kwH-T{Gu_rZRZRF)Xmf<; zc>8uzOgKJj7)%(MENQ=ri~QDl@d0FKI9$RH2P^q4#D_VeGP5-Yt8f)RIW(|cfKv7w zUHx+QSG)SN*ngF)Kb!qEuD;6tt6lwh2#In<{IwQ-Gm6+=hZgEuPlfy7;rA0r<+Z0F z^&CgRI10z%sXK|Hw$t$Tj6-f;Isu=A3HWKGNA*k~AmMI2B3eO-w+W@8sP+VkL#1OV zYw)VxP`O$(fslkTRBUxX8e}n6Da6ArIG^fs;QB<%f%B_=N4BaVK)hK3IH(3gv&Do1 z%&w{iPGHVS%oU2*pC|hBC-5>`wJNK@3B0@jTaQfMBVbfHJGV^rW~P?g#xKjx)z&?L z1;^+v4_?FP7Jl^sT~GA%kyTOC({40RSM!7sSG_t0s0Z(x^)&-fU6sc7e$bGokZ+_pELKd9p%?C4R;Z> z3o^BzB5ntdk_Sih%|sF{`=BfEG&MExfal^r*vb`!v=dzAb#Qx#Yol$;qFw8tU5#cE z#KyQTi?NLu*At@?o=H@rNXB*Dha3K$+PH^nfwxws*6xMsfp81FI@Y1knMMIHVbQY4$ zy)i5z*(}mr+CYAtsrMJw2f~4{|1j=Uec`|gK_wxss7h$|7#5FVNqsTf>KHB?Lv?)# z+w-WWp)^C@GPSg!TrEFSA1bV`2v>wdhcRC*4_B;cs8q|;%Co4k$#kGXIx_jhX;k%u zOVqM))HalpTnkR4u1A%_#p772mQP?=A+{a}P;&TyhtGy7sJ1ag@JfNZ=vtb{<+kHVUR(vm4xpGa#graE zB@u3>e3UW23n{MH$(cPC>JSOOAKSPp$t-k`R7;UMWyG9CQf@>!EqEhFh!UhtM{%P? z%Bye$2JMq+>p0S!B`JADbx6q;-wt@Jf`AHtZQN1f3OenBcM|g*I>lw|mLx%?r3hLq zu}5U=v1oKtc77u&^UmASl0;#Pc^2e(ESRK$Hf=L{&eK$hF|*fBg@~|6^$^Ky!BI8} ztnl|*gCmwN#SSRo?9Qg{>ljQo;}*`4y3ExcOD1$0B_f^Hj^T20Z5)y5h^7lMgxBL% zuJ+L*`yHgk77kE+9LrDO6(_OceiR6*NG`J$(#SV(tbj*zC4WO~Rq-INfJ%3V(eCyC E9~PUE_W%F@ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/MetadataRequest.class b/out/production/clients/org/apache/kafka/common/requests/MetadataRequest.class new file mode 100644 index 0000000000000000000000000000000000000000..15ab286616f3fd77806888de2583857264a8a332 GIT binary patch literal 4481 zcmbVQdtVgS8GZ(Kca~uQ!$pfUx+bE_BCZ-sBTyP#F|Gny7O^C4&9DqGy6kLbXNk1w zJ#Etay-j+h>BY9`#TdgSG{!c)<~#I{evAJ4Yipk~GrR1ryA>-kGv~}X@B6&ZdppBh z|9R~ufTMUzMzRh8%qWncE9l0o7|zL<58`k=3|ta#E{n;Ocv_H=R-nR^krA(-lwm2@ z22%{O^{|l>mqi6n;YvN0@Ie_LQm_M0%lNQ@c6>y}M}zp7jE^h$1U{+YemtY#Q}}cp zp2c(G@)-r6#pe`!9$!$ z;+f&8i(^Bh!xEZDE@_KeG^M5Iq6ynF(sNwa5Hr(RTT9!MS}LbY2-s%ENM!7(wbZ0$8Di{B`s{gw$c|Ly!qT72=~+7)9o21ZMzghXcY;8+mkK0T zp3wc}z$#iIz#a)#Rb1`_+sY+v$HJh)=h^8?rTlY3=L!;(b=KaChh54npNvL=BIOyo@rlPo^A(Sa6 zQ@O0ITN3tEg%)#{aP>&({tDi$Dx~057fb(Ozl?9l_$HH|-VApYN%k^i-yM?`^ohLi zXEaNYe%gz^v|&b%FWLI>-0ZAw^~bkB`kgTzn09s-m@N}3AzoSA|D`NAkW6{GT^knZ zp(Ma+|M?%k5tC$nxDh{AW#XImVaT!!GgyRwpRg+N&$1>eAo>%b# zzOCXr_^yh>IHKZv_&#&SsfSL5bFxS{QD}OC`B7dfRr~oRVNm2arH zh2IIlzgO`l{vhLzD*l8&tN084x)raM6&bd!O<&Lt%*f4goaI5YO_qpzDB5o46rtR> zDM`cxOO9=Nb+#?sRk706W_7MR?I_53*c)0P_8+SVa{` z+6+C{RtzZ*;4uk7qrlPjlB$*GPE^`?Ejy-PprsN zt}?`mBJ+`Ah+AnqWdez`Hj^XDhl>SyxGK@VnJxwJQE#hL4PqvyXrb8PP6BE=A^oYlxFiZ=anr+AIYJRB1;qVrbV~6h7-lCIkQw7V3aLr ze1EJsEIh+2Byow1{Q0;feZ0NGkE+way#tSgQ{i~rbzE7~z4u(-a*uTC6=|__jr?|b zM&6-+GcXFl-kQu=7Q2H7-~DC~weo+QI|?~h@|}S_{Cer;S63}|if+xPoj8anKdKJ# zxdt+tMaSj%A!mG)GZLFXFW`rE_G1^qco1DU#+mJ|EcBtDBTDsj5Aa!~UU%kb0nNJ{EQ7oX zEN@b}%zG@-O$go1@(SuliA$YYMFaI880)#&>#ObcxB1(AFJf0uoBv2(fKZk8z%4Yc zqp1cLp7BdpZ~uK0>Mlp*0r*LdN@{kHoF0-h!259$BVb2pP7j*!ZahLd37}v=Oq_WL zPbI(@PSCtzzTb(O+Xx{b(=8+}P*uY4;ykL)$vG*W$v}+ke}@jwz1-@3xA!_4ztpqEwK2 zP?P(wWdrR~?Y^E>?7X&tj;ZFi2VTehQ?(sy*p-;_MOLtT1);>0|2p>EzyoV|a1DDC zYv^3XzBees=QuV3h0nbR(%vr4^pc-rbmlqoHG@HzWaw#f@&fOIxk1|3g$A4pd^o$fjKi~-+pjiT0vn#ZQ%y->_R6*ug5zJ;! kW)K(ggeR%fkpnd3T^4x-H6pUvX?b;R4RNG#RD?(`j%``8ti*O?TXx*al2Qxn;b`R;k%CyQR@U0` zuGn2Aabv(KrjU?e63ja!!GyXgrE#2CwlEmeQfLE|mJ$jr4TZKqTlz>#+mJR^|2s3g znpL!R@_qKqJ@?*o&VSE6=f8JH&t3lfMF5-R77eEZ2;=D>1ZRTSjb}7G8-y2UgXq9? z2@Z})>0Dq`@zT(FNs^^dVcu>8+s%(5s zJ^omk{F4x#!+8x~_v0IWOoi}G{HY(`Qtv<0@aI8Xk8cO@7x+$qt>=}-cQyQ_AK&xi z`+odY5Nq()emtz+KTwaqQDDF3$07Bmhc$S?kBmb3Lk)i$#LYOXgdZtrKh|(Uz5h;u z{d?8=2Zit_%K1-~_%r4CAC-}RQb7J$!_U>&zbF|08pOZh--EREOaL$9Kh)ztgZKsh zOT&K)A{~1N2D z$fR@mcshT3d}1moSTxwbr@Ldgw`;I{u(N%z-KAgM*WcMS+}-Kw3h(NV4Z7t${R3{< zw5@dw4D=5ScPRBaI1zhnuH)K0?E{0|gWdgomiu|$ubUfP&|GwGe9mN5Pgi?q*MQ4w z%|O?lp6-rzcVP7Hn0tzrn3&4tlUc!vvSIJAr!auNWHy`0<^)xCc&GBIiI(2@B!>dA z)c$llKb2)%drC*Qzj`mq`YLX5m*r{O4&`-wDxJ#j5NvEHt5?8S>UrDR+Xa;!nNfnO z>Pe-OeN%@OQekxIlFSHp$BJ)4am?n&j8>55d2kfLMBd}4f0srH(( z?urw;(_rdc;GZ;qotJ~Nq0*ton_o9oJtr-*H1Sx4FA+oqDae6FQ8nU9Zh zO#{i?Buh;#He|=s@ffsBIzoME{8`h^9yGde6{FzJqV9A)xj&h;vZLKKSuJ9FS(wG# za_Ga}H>d?JH#c!SlgslpnaT1cDgHm`PDPa?tm0ydP<PgW z1+{nB@wS+?f8;>YN`2i)VKs-Tvh0qs>vL+r&xCd+%K>+4Djc+6=b;VCP8us7L>(it zoV9fV%}y@P58Ew9m$gVaSX_t_tgLe^?{wu0cig=BUnj;f;FI{2faEH zcv(H}W9nL00_)5wvWgn|oR0s+|FPJ0yn@TDcpb9>T|_*BCFOjo?A7r}u##>nH>Yfj zbg7U^=5LxthEt=uc*UoQri)(!x&$Sp;{nliJSaY0!V=M?N~(3Kk*FrMB<}f_8&;5& z1}1yyQYR74wFj(|dpMy$iy`%bHRV7TgK0C6F4xE+O-zMtu`buj5?!v7rGi!E z43wliPpx7p7SRdn<~rw)G3ayMj0zs&^6{`N({V_a>#~9mycus{0nKgYu6S;o6{KSr zZ&gdXUYC_}gD$I7=vK=b?yxlIvQ`>(X_9riG|M8@w_cYPxshB|ZW&IE=yH>6PzoD0 z*`&+O(yGa3UAD+p?iwD^q)nG?vR$yHoS|YCw2zRu;)(o#eO?w$NwU}6a<~+3=A5k9 zg)yA#=;m!Lp3jh%U&o` ze@sx{(Cv_{dj=O!kxLT%-UZOu4Cr3K>&I6b2Zy)mG(Jk)hTExgc&Zp``HXWXG8)z{ zc)Gxx*S$y@XKN*WcGAcn$HgS%oTdypX#!lO0s|PQeXsm`?QEXQh*<%dc>@ zDPpGi6)KYDO46AVNu^_l(}{uN{ktSerS9En!7B)Z%A_NWmIpj9?^p_jAD#hf+t%l8 z%mOZ*F2OYo-LCy|y1P|PS4>V3$rd*fC%Ie-g>-v$7ms6w$KKynZYvAHaW*`dQw`^b zbanxy?kflXDtmL4VYO;)ER(f=+O1PLS#GO~L{dyVpT!ZZr9}nKvhD;OnTZMBX8E&> z+0){zX#R*Dz4^Tg&TGD$SX|7C$?|&5ig6sxyCS)3%-K0X!&TSVKAB?8b7_9{QA2%I z5HwyrG>#hA-E>tKd(=mmz5L-%FGgZ2n@y&9pxM9o>@PCvWFq%o)l$h<9IW66qTT$q zR*l8#;>Olu^r4?0m-evjL4@*cF8Mmj11?$V#awb7EsXZIx!_3<83WA*NbE!6k&+yp7L7k4C0Z z)#fvO(Q4B>jhgd_PNQ}o_JYn;Jm(>H@f9_%|c`=o87c?PCH&2^l&Rnq(7 z8C*;6UejxPmm{iWd0*1zvox<$-ZfK;E;W5heVsDB%tNHh*ty&iS9l2b`f02@j~mpS zQ?f>lox!TPzJgnSld`kgUUt9?ILrR2YFUogv<0c$pxlPcP;_mH6?Qb5!D%!Vq7aI% zQ*!fMUk!deAEuM+I(l98k^2y6Z7U~rb_EsY<`*?q(JBqMmiNnVGqh zl{LVc8s;;@+PRzcat~|deXN5M%>A>>_e;#_=a|ROGk0Iauz2uR2_Y`)F(R9JDc*%q zc_Wf?5Mz?Zet8Jv@)+Nr;qRg|I3O2rP`-x=`3c`Y=Pmx1NP8-f@%S+5F_7~tL*8>e zraT*QmuCy^_H^Jd_7mwz^1x2CVjL;f$7ZzS01m=S`w^>EO{;dRhB&@~3C;@DbvMwm#S$0XjyL(Vk?=p-$H8h-+1DIkge5&GbC+?ze)l!uzR!h~Y z2lbxyj86^~I3cHrF~_+62B+?T&+&Rs(d#|*YWq4uU&`7rMv69$6m1-#jknXrH=)mx z{=6FAq2Zkxj%v79!*2pFW9SvEw53;Yn}&C3IQ9xw+dZ=kM0Kf-QnP2-Q#8Dr8XA5} z!+V6wc?m0KaRX{zvsQ_siNF;U72L>6&>dQPuSIJ2@u^7d83v*rkRLWSSzKl%Xs?r~ zeNBq5wp_wi{%fnOX!ROiv-09e^qQ51cf%!!O=Z()o5r?QP0`piwzv7&sCPO8&4$MM zzJSn>p_zUoa1OW3U`GY^9rLolo-S0lw<$_?D){nMv}&C0yLeJsOrN1yzWk;?x_lbH zGL2tt4Y0jb`3f2Vn^Ydhi|ixS6l{blT6IG=L&W-vlxm$}G@G@C;Zij$Rqq9?88U-L#H==>X5Bfo+w^iw=Wru& zk}HkCdF*teN$i)u)wqu2LZ0au;u+B?Fumu{Me3Q)H%IuVrML53e+LY_6Dx5P>v1o( zFtTM~a2i|uR>g7*%eSrjhX;l{heye76<8@>z+<#riF@TOc%1C!WxV#_Q|$9uJhz;) zY)!>vw&qFp+4Dbb?K8;3ky(bvXVC7AtL)S^iZpJa3LOM7rWS z$a6~RGBzq%dNn*Ti)&GLRUKa%{TJd1&jpnJIowg zc{_JUIUvV{)wPiHax1Cv9DZ%SrKen$qCELOOD#Ws51(@^t+MO@;WOCvEV|F*4bQ-% RN>^NKT2ATr+2_Gi{|{)gD0~0_ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest$PartitionData.class b/out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest$PartitionData.class new file mode 100644 index 0000000000000000000000000000000000000000..68e9bcde6b2dfd99d50ff4134ccb61f40597300b GIT binary patch literal 916 zcmbV~-%b-j6vn@4yDc4y0z#qwD=2E&L^mefX-u?@iA|*l*!$^rS_Znab$9A}dC`z~ z;RENzW+67hyj_MDkBXTI;ZGyCJ`x9Wd1-%8r8h zo|gi}csNY86Ih6}Qv>Ccz{+3Ehb}R8#Eq|vHSV>*dZXR`w_&FzP-w;jEpV%2tlpbO zeVrUB0%5^8RO6mX4A0Mt1$S(SY;j*D&M;;xGq1e2RwvD|N>feCtxlYb0yRXK=LI3~& literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest.class b/out/production/clients/org/apache/kafka/common/requests/OffsetCommitRequest.class new file mode 100644 index 0000000000000000000000000000000000000000..e09a391500291df330e407bdff978a1124d0de39 GIT binary patch literal 10053 zcmc&)4SbY$egFQF%RQIN6A}&*IG`FR(2xK@N}&xeZAbzHNJ4V~OlytD2?$Wyp zEpyIIu}-VbcGFthGK;p%+HSQ%0@BtxZELHp+BuzW^QG%_b(^|%o31L^_xC){-94Ay z734G5@Zq`t>+k=2`Tky>|J^fZKl>PfH8O1A3zhgHp3ujWK|F;o1tEAkh^z5T5S93H zB_Cfg@Kyc%s{o$W&#zTXxqsUGx;~!M$6p8VjUfI8-&7jk3gB;b@Y?}=Cy2kp-v{vz z_-+8-Gw}T&=HmxdxCGBv;UDor75)i7)W?qu{Br;=>h>Q8a9GaI6xI_*no)b>Y8j=RX4YxPHD=h5y8V>Epk3?&tdWg|hTN+Im^}_@#kg1#ty_ zpd-Hya?@FTOc{b8Hi?ieDGN%u`1DZ`6u%gT1cK<0N<)GHsWQY2Vw+S4WroyHXHpxK znGy-sKD$>B&<~XNHm?c(t;W7yZib& zd-?|>?b|xL+XO8;ld0jB=vXv%z-l=d-G4CJ5=)MbCKD}VsbnS@OOCW;4v$&smIxP& zM%N2wZtd&cy=$ycKbcno{axLio|zf; zTF=0&U2T2+UAojuAe0;REUjo8IEDDiK^ivS7P43^HwlWT)Y8)BHGx3p@?&uhquZ+Zp6Vc3gic8;CIJ==Fy!w-4 z@mLP5>r1XnSvQPZ=}fw%*XE&};EZSboWaGp{mR<lDqmsqI4{m z@NMVh_WxOvs+)C1nCdz&t=@YfJ6{6{FF!v%w-*7=-LdK{p(&*SkcWPTJY+bljZ@&rSw9=eP~dpq=Tu5*cgQO8uY3 zt~Sqs%4%t_OwY}Z>NNh{>Y-7T>7t?Vwe!i};M@Z6IdW%_b5 ztspRJWuilN30Iq&*Im8)t}m>&IAGa+Q(?oJ$vo!{_F8}1KF5&xtR?9W!&asy*onOkMG~l@Yi)7 zI4N=vG2-86Q^^zGu+H@V;B2_`PzOkK4i!eQ{lvlA*~V8>d6hvT<&M)lQn4wN$dlO=*!;hP=U))p9ALt0`;bGE>&d zx`Iu)3`JAhQ7diA<#L57t+LLP^}4z`e*{+xHeZOI3ab=aI~P*ZCXP{SvOzYQII8;l zIQB9ln6gQ(H03I3Gi9^1o6;ejrfk7UQ?|-BQ@Sugcg^#*Et)<+{!G4j?N9^n)JL~q zX(@IKBZjW-F{M{_8FIBLeG)OHUv?X^N0m=_Tvm!s33xcMUCF*tu*l}qsaZxwtl{WL zTWXl^eu+%yO)+asU-Sg6dz?Vsuw}txOyhVA68nVxnm<*Xr3Ht@B z&MDorL_?V7b+4#nPRtKnJVDRDK!YcYdujompmIwagg)twz2pQ>oBY5YWJeR zxi%0ql}wY<_cG*_soO|I(0tAVcj*PpZoX#gnXZ|gTNuHmja{!2c5~-WySn)eN*d?3 zMKn;N-uT&)mU4r9DY+h$;MCHP_mqO)HSihiwNHG`JGjL5?2IswfXM?K2?S zrA|cUEll7X;`q|`kppiaZeO&; zn7Z1FDEv29ub)LC!LG61%3j|E$ddg6$#h-p5OP@_9U{65X0sONtdU8&Z&r1U%ZfT- zB~Qz`>C$3nc%b8~qpFlw!cK6>Idp=Ci`o@wIuN1sXca)EfQ}BXY3|HZv4OtVsa)>%RXB4vE6&Qg`hJ8=A zxXP>Sd>wxtP;&^JmlSV)!Y(^kB*I4JyE^mtRCU@Z93y7L2}2yM!x(Mdo)URuJ8)%g0F~uy@%%@ zTHq1xsG6^2c$yye+$z|YvZ8dbWiBhz%4P4-xzI2>Hl9i`KJ_^_rrG?lTEq4^ZC{W+ zu#n$0T+MHC>M&0$MYiT)5BBn#jW@Dg#@;*(c={psukrMC-?g6pBKF_p>Ca*RI!}Kw z`)~I2m$5(S={K?e7EiyK{ivtEiv4};m%IF}K#Y6>L;UY2*A_p+*pK0y1P9oUdni<~ ze?1Q7=BwEsVLwZ;x(LOPmh+?>(vsZ%WQlHnvO2dvS)h|*n(Z=VFoW%J9D)xwa^w@J zAa&sKrZbR%hf#I{<>jZ~n?%JV{7ucvCt)--pG2Uk>?DFsp{kQGn`Wc>Bx;(LpG57a zxwjlQ@wo^AJ_l*7D8B&PhX(Sxgt9kN?$wlf9l1Kp6%EdL_#M2JJ)Ugy&vy#8l+!Z& zF54!#bVf29arEd&@U{Ydsw}_f%JLn`Qo&t)9$9=IS;CacXgYzJq0oX;m^Fdfd+lD` z3cG1bxj>~{;Sn|0!jr`h{Q!Dh@8t{2Epd%T|=heX5U!j5DCmprsPr1_I=YXq$`m7nl{#O-U zbuy=(x|&-|6ZKRI;z92woZdraycZYa7Ocekuo3UaW_$o!@kiK+4^pIC z=?=Hq%Gg9VAaglo4CItCkWr z0a>{G6v92Wh37S^TxT$UU=j;jeJf7mBEf0AjxQk<;fm9Ez2FW!udN2b<5<}0x3?{t zz{Rab*a$5S`%hzuz)K~dV@q}UB_48>;Yyd>joNa_1;asiZDVT{=`B;jX4nih>4Zy| z-R5xB36O2u4D1t_EAPN8ZRt!)erA>S-=O@217Y(tR+nM#@u_zy?Qq~UE_HYf`@%+c zZk{7j*q57C-}TwPdkyN@{t&TxJ8JO}MxBph1MZ{|?_$Kgn_hjC=KWL7evJR_!C&wl z_H&FKPvc&E8~5RdIF6t34&WtxQp#|@nD~^`-~nmCgSO$v2$tJv#&_ZfTOY$NybCwe z$8TY5csBtQD4gV5b4@Zi-9yF8dbM zGn;~MY-+ZHb7_XAE>>{25$9pJ}D*H;3pB>${gfwyAurug-VTF`Qan=Uc4~^f;u|ANGe9 zlREvyulQcy8VCm-TU%MaHdq&|t2~Ay;Xqw*HQ`^S%Wd#aKv`W|t;lx{UpIs8RZ6Bd z2mggmOL$WjrVR%?Lf=8xqDhWfh*5wRloC z&|fq!9ASuhnWypw2qCU4r*|E}Owt;qf2eOf%RG7yGekA%?$$dahT0bxQh8m3TGE{3 z?s$YeeS&^iMV^k?KIA87chD>M5-#t@C+VB_Y8H}>IUm}n_jLr7BK#dNrzm!r4Z%AZ z5$Y=D`p;sHL0^_BEJu|yWA_c*Zv&m^=d6PFI@Fe&cUFs!-U9pzrh&Vrm^B@=(_L&5 zzaVddPvHRvSyJPy} zQ_s!6pX8XJoIlOD^RoZXpkrXgBsw2DgDnH0t^P-`ZJ>NUi-X93uW15Z6WAUZsBp@n zQ`kYOkyF??iSEylf)hKJvPuPi{S&}|s0}8P*cJ#$j7AVQ zO}Egbdy_7qX&TznrgTf2ki^EJrAhauTe_x8+WdijX+QPphtl`XjK(8j1@h#9XEgWR zd){;IIqy05j$i!Wa~A;Ih*xEpLA0TwK*BKv8*p5~-I$W`P6dA4(~1b*)rx!Z?pC}9 z?-h^t$#}m|xlhK406riT?+@T$D^xrn1|Jk}ACmFm06r4HodJBb6(7UL#p4q~??D+4 zDYy#vh{c~2k57rmrxkn#4~w^x3O`Fbt_T=zzdU!OE-j?)8RYNbAXhh$p znVN(JwigvMmyZuve1c_laXqg?OpVG(v&Kg`h;%GpSixe?KI{=eFfOj7TG`G!64Bt^}wlt+GSVh zqA;+xvF00v4rNR&dzfBZ(PVpK6O!>&8E4qC*ytn-)KR)|gf3qxq8c{F8nR>l5DTS6 zE~TqvAx&MG1S!R_ZWu+wt~DtmVX-@3)k%*Xs%=gb=vK*uEZ$Yuv0|(-*MPPGCodfcQg#?~}Ccvi(UW@?n&ic~QUo^dxd za_coiMS2-dn<}d~hglWRVMfJyd{@QygvrpHpEOAqT>6c=nv9-B($OqW1eTeK3;4c@ z=kbDyP1sB-&4KLD%98|A@dNx&#gFh~6+gjGW&BLV&+(#)U*MNAex>5q_>F|MjW$(V zlOk>}srW5^r{eecgN#3__!C~1@n;o(!CzJU4R^_SMaAFo4+)zafx40E8)GqOS$1%% zt}NeNfiwL{k4+}4;c)_y>ohf^7y+pcX56mi-^ z6l+Z&=LTd+P1mRfiaPS7Rvy&vCL)XQb{@0^Yp}Sw-V-h1K+d{pWm%vJ$6fDTtZ}!P zzX9`pGohP1b)NpsJhUb3ng{gKOul|I0h)1cht1l-gKSD(Lk>a2_GPo^Tni$xj(JHW zpqwSwdO@7?a(<;Cm8$$&JD-+klMav_8(uLa%xlM^QN4#$tK3S7BeZhn?mgDQ<4o)z zBW{=y)?CgP>mte3q6)`Z78j;Z1@DQ!nWr;QA=-9f;8XFRyO-?h3WGjZj)==$|_BWL1WT`7J6}y+= z8h+Ak;Mb^(<>K7r+j3lw8~7!-k?&1d!0}r><3$|b=ov5J_$JSIDaT!&@iK(PfkeEU zExc~D@Vy7U#C{tM48n(2&iuRcJfwYR(KLhR8Ti(nL(9N<`1j2~9$Y_*fP`5DC3O3u zzF8;|9>EjhtySWgr*^k=K7&wr!88_j`=kEwqG-!3!V*vL_B{?rw=86f#qx*;E)Wek za0&6|zy*o1M1Uz#CA>5mbip`WCWgcj*ikvE%p%%^J!dY?3Z1Ati{kEqDn#c;Ke+3U0+eu@nD7Ked265ajw8wfmq@`v&SKDNEr&YWDHZ3mX5_a0|6% zT6&c7Bs0T25N9WU#habCcDzc=&CJ&dyo>>AG}B{H%kSW(iEE<`xoEdJXq&hvp3fG> zu!}K7j0`b`;JZjCTV#yL*k%3SEn^RGktTw5QW<+??2}=y_%Biy$u*yM5fN>c-EMJg zKcD27bdtU-;M}Rsb)*)Fb`@#0k||gxD6?u{{e>;fUA~BKI*6>RQOB1$B98mVX?qHOGLjkb#AYE%s`oyOJExTY(__hr*~^EBSl z)yDL%IFHr)qH-iOgEf)1b67izYn!p>gd&}}IDNv$=Y4_?&5SBoKOsKT^m+&jaT_`? z%$m$_ZzJTxE{tL~4l_!}IeLh)lU(x{_Pco>W$oU<9IITl7I)$R@s2ZAqqvJ4+e;h` zW30OCF@dbbg`2tNAUPt_RslL^1s6`aTsS3eyo{#ESxV$=rIRzJV?vTdzSg7FN`)I;k^!O_VgeXLkeD$9C<9T{N?(Qtj2ULgn@3FA z+D&Ws+Qp@|HLZ3@Tf4YK2vVc1wYIieyVve^v9_lBW^I-J_s*N)4a_vg@_lo8cRBw( z_niNnd)|BGg?o1cuy%byw?wb>-}iK4Sw8+o8;L0G~DdN`{m3BG?nAo|DcSq9Y(695 zKP#KhNwGU*bEjLgAXW$+U<9;l|EgB{?94VQ0{(PA9<6cb3=JS3m$9+<( zqkh!kehm-!@C6NDlt2&qaW=l>$3ysXDIUgGWb;)&9>LfA_&Of-<1u{0kF)T&9Q&r9 z>hSR2!nbAjI~u+#P|@1k-QCvF(;sQw-nO$vAk?0;hC;@u5gj%|dyK(7Mktyb8A&EW zqgK*RMw9W7ePGl~g(6%qVl)XXxS+eUx2r#Ve*ccPzW$Duooxbv_A88iMksD1hC&hB ziY11)u)H;yNZCfh-ettcOo4elon7J9h)b+OjkpFXyIQ(?!ad>64)?M^cGR_08?usP zqu~Jo|EOWvF+0ZH1m>qT8M9;YPURlvT9X6v;O()5*)cY<+q8O&-Eoct+LKWuzRR#;vhNI* zXU}oIVH+pn8rvF+%9EBUgqWR7NV`z33|_@ z9ZTBy#1TE5NSIb@+(@P5aJB20JB`t$Y86pE_Hc~T>kFFGGWU*|DLWPF92`uU_BPXw z4tJ-=DDGEErP5l>%iSY8?dZ5L*LEbQW4{)coOFF*-_E9X+J4f-9WIRQ`I^&%Zg#tt zrMH^lf5z0C6=$})Rub_4fhv#KMsyEB)Onf9G<=UKNyGOw{D3)&nNOhe_|zKRE;R$9}&H=?#f-3%s~ zQ`kzgtV%=dGp!WUl?+lr!#F{u_%)*qW6yGImyV0_u1q#c_ph2wtwNc6mIB8=WloHy z-M2Jnb4Bf{x_XxxvuQt&8FuI3<~f5Wz(Ts4RFM?g>99KyONO={u+6PwgM-X!;TPe# zoHMChM5lRoD}byEX5k97=il(9?&oWc#+^H)xK752Kaw1?qUN@kycJHGtK({!R|HlS za&6uS)Ug$>9vCpG*+$1m|K<|ZA###1_egQs)kviX&MP+^?X@mu^($1`|Z$1%CO zVrCb)0rSe7JgejP_=Ao=;!iqupiM_RcIx;u{-Wcr_?wQu;~zTyiDw1QEOhYP_>iag zS7s-5!0kqAn92%lErgKYbonAjVBJdr5nwU*oQ{9vKRTYr3sT8xo#{jHNfZg3?lMd~ zZVnmo7Hep1g!g&d{-`-Bt0PuFyV8lLcG*B}ELEFG+O_+Pcx+&GElaf6K&_LPQniCg zt9Dtc*09XlHMMq<>zPPrc1WjXT@(wCE=q(~6Phl3xK4$C8 zGVG*97GCPgT-i(J1a7)?F)SmvU}#F}q(ZGpdLt^$-z)p_K9Y5-%-~xODB=m6<|G+K zHpz3|L^E3R6wFk`ESdy*3*PSP@^DU4I6LJ{7ex2Ty6-?lZm}RIQ*(&ZE}N<~zF5|h z3v-rdEKC_~*hqDl`-#X*cL^)RgzDs88d=l(V(L{P$}HHL=Zq(QA}_D~9VQD{A0M0- zkYZziJg#=*XDmhqH9P54nALUldHErsoF&WMAkNj7U#OHq8E&4LPs+u_>>Vn#XB{b! zcORETrr6{;W7!ui8l2_T!aSRWC1aNtinA;`RJ}y5>dXVK^s&W_I8&&+2^Jb)+b>-5*v60cZ*|6Lb%ZEj>l;hJPoR2nsPuj-kA~k-2tG|Hr+g<%i_QS4zko^l?{Y41KT8UTz zSuPQ`L*aH3S+KHr8TaVsM!oPr=ghH&DTuziP&9$!33yf>K}ko$VR!?+72_zqXbS$m z36wQ@R!xF6$t22IYLo;^CQ%`9Gj5Pi^8_Bi{6?=5TQH75qZZTxl|k<$ss!99J~>t` zmoJ>eNkl3QmO4lw`J|9eZuC<;C~?YyWr0OOzk~?-f@PDaDZ)jErXQ8_LEj`;S*STr zP|M5>rL_opva<~#+nI_-*bST5=1m`2oB;1c@9$Gxt`rMMhk${j;L$K~HKeOR9-$sb~e~ZfaOThLLZIQuN1)0NHZt&Xq?u7jN)X zdluh@_{wU}TFI{mAR4_vZ=iAFArrB|gUmTi#_!l;=Ce8eU-BrA#)D>k5< z&ta;2A-@UlKs`OMhW6gXV_i&(_A`F=FoIsk{BXG^4cvK+WczRl5i(pO>WtHq9~>3~-W za$_1RQKpQa9?|e>X7<_OkB`aBKeSU-*pk-n0B4RglTh(m9tNc_XPI-7p`X=TWKRWaK%ltC7ic2T7n9ju8LLla|nm zi|^n=k>ZnA4UQ;fBqSF&D)8$A-o#Z5j~wA8vxWEO5l+Y0n^B6l;H`v}AQuo?+P!iL z8nd~yWKF{noI8QdvmwV6y%C$4jlmMI*dJmcgoccVxKo^ToJbgJ|&juqkE3=4UI718boahbQ2sjm*#AoyR_ C?J3a! literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse$PartitionData.class b/out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse$PartitionData.class new file mode 100644 index 0000000000000000000000000000000000000000..692588d5234d90d1e547fff4e465ca245735654b GIT binary patch literal 932 zcmbtS%W@Jy6g>?C12{1Oj7faOM>H^+Oj(tkmK6hunj&C{m4&-z(!d0m#-7ICap}g5 zmS~j=KfsT&yxme|`M`~v`{=&+oO@6A*KeP{0NBBH0XbaG!^IV2){VKE$2D9pkh$UF zri)t)S=sF-LNl;KhTKSKe!?|pC^b&`D;^Aa+zZ-TMRD&QLtZE)RZX4M?`!qk6Fs#iBVGb%c7(J45O(HDt(#$6Q4wpB*~7A0-S6Pr1^Oj$~Z7 z!+1~PSg6{NCyAizdPAySz{fo7i(tUJ10IBOG?H%Qz7m zzkt#&S|!q`wZ2& z68_dfIys?zaAKk--iuC9>yj(JmZ3B-lLys^J{!DAS!!rvyOx3V9P5VrC;lowku7 zcj$B#YcuM{b`+rOe}MBInUBc6hU>h6lYUDrX&BK5bgH&;s5za+Q QQn*BO!g9OVpgGpQ0~-9)a{vGU literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse.class b/out/production/clients/org/apache/kafka/common/requests/OffsetFetchResponse.class new file mode 100644 index 0000000000000000000000000000000000000000..9d205529d0d129ae766fe3ae822a3137f2a387de GIT binary patch literal 6111 zcmc&&33yyp75;BBnK#Qz+Dw|XL(66tk}eEZ43L&;noeNaBoi}fYYG}(rZ35~lbPYo z8=wMGwJOR|K~WJ!txMgdr6dj20xF`YxC`!zyST52O8D=4ZzgX-rUgFGe*IqVJ@=mf zo_o%J&pmVRfBep)0L~POf@}P!!?ghjTo=GuxITcrc$0z~0`TI-AR6)JAl`zTf_N+5 zCO>aiaI;i-hk{#tc&Ai+mk*Z*q2gBAdADr6N5Oj)?DOG$J{Ufjel+3z0ek=-^r0X- z`+fM3{QKbmK7x-1@iBZ{em)_QKdIoh0G8lJ33ot#4$9B%0UW{|vNbM|CS>cdRG$nG z=uQPk0*K&K(&SGEa1?h1h;_XmcjF%U`Ah(x#pe{@y^afwBI#Q-yQ95 z-_YLQ?vgb``}*R2gB|e=QJ1tX*0ZI(E4IPG5D*({rksBQ!JhbF5z2?p({qNN&v65~ z?Z`Ry4hv=~9qHC`+@3#?8p&v8!Qj@eEFWE4(X0L0Tq;?L(VB|$4E^GQo;UN6`0#LE zH#>DRIojtKw4|gjO)lHm8~JL;n}d1l*pCS^QqI%Gzv+RJFP1rL^GzbOQ6R2+gQHw zC6-u*$&@a)gE+0+ZWtPs=em@-QhB+6q7yeI4~xefP4Q3bGuk?l0@liTduhAlEZbD> zcOEdu_ZgQxWSPB*wYJbYT(OL)kLbp;Mx1ung3L>J zmrC`e<5qG6%^iSF89MzOLXK zD!z$t@t~;qHol|cK|EBZRLVChuEbV>Gb@>OEJu;DhI6;(ReTrUQ}KN~q~Ztop^6{L z6>F!>_#J+);14SPh(D?LGybCDulSpa zzvCY&{)tDWXI=@OpEM` z;Atg3y9k!&q$2+%m-CU1Y&y+s#X5Ym>~q}_BU{MHT44Py%Z~!5%*gpX=F?kai#J%86b9$8^NIvY8Qj z;1cZqrj}|xHO9OWigXjesP2r8;V{f zd3d!eG{$!7lH-`-_|~7MkDb5}Vk0r-erz`{Jq||E?}5CWKaIlrVSe*Q*DjoWC*(nG zngui_2dZ-gI^=BkuoFEOQZb%S79Ar6%;dh6HF(ktT_aywoE2x>#cqO?5?FJk)LB}h zJ(uEfVuDefpT?ch{*LnnmRBqpW~!3NRVmuF;<#(|$|nIQpEV{6hQaJq%+>b$fJ~;b zr+!-Pbc$2?nsF8%ZGt#K)(LE#fVDW6$$uT&RhYy6i(LJB_FwGk&u72g)nCB=dRITp zeut~Si2V(&{z(WUiq6t{=dmAijc-Iq-mV<)w(RLa4cl?_(zch-!0X}R!w|%xhp|vD#Qq}LKhf!&Bs(V`#wk``2?r-}Y8AHbpV})C!^$L{XIr=`>@5y0 zwfC{AG`4)j9awyTp7WrO-)(%-0=-#>9yHL03+cmk9El-;m!cmUHsd;Mp?9`oA0-Dc zfP1L*04`uLb|K4|m*Ek-9FJj;b;d=)3r#d(NGwE>2*5A`)QuBqGh6!@*DXW}67vG0 zZG)dH-O2V=z5;3-zmRweEx4DM+Zb8K5rephNU~+ey+Qanw~O&!Cw6uX^cP(E*YLci+NXw-R%nC`lW!)=}uB&eKGefJGpwYh%JHo!kz-q}XIb98_p#|e; z9!E=SE!*L7EFZ^;v+CGhBw^;Vand+ej$>78{Sh<`G}VRYHr7wz^tKR*v-k)i17W4H zb^>RFLz8%ZHMZ^vi2YMH?eXwCBxzJ_(P%y9vMpUa!qzBSkir?*fpf5vVNNrIV{Bz` zDYDp&9InO1xCI6dQhJ#4kFvOyba@4j?jTzilMpJ$&f}asYN@vdti?&w-B>LPajtSL zmdOGf^+cLaT5V=xkd(R%yGXDza5d2+tZx8Jjx{m+4?FwC-J$MKfVbsv&!Pz*r>9+ z=Cy>ESvyzl3651ZXgM!FDmZWVJy9nNMxR{8cl;PpQN D7@Dyx literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/ProduceRequest.class b/out/production/clients/org/apache/kafka/common/requests/ProduceRequest.class new file mode 100644 index 0000000000000000000000000000000000000000..c63fe22c77164285a97583c7e1755299e702c4d4 GIT binary patch literal 7363 zcmb_g349dQ9sXXjo7vq=0xSt(Ib=lwNgyouksumEfCxF74S^_a95zd`knG0USrl5W zwY1jWt+rNetF;!|YFha&dp#-0hy-yb7Q@Fnv58$|LPAGWLgOhU4um?wqQHBw@c2xFGDR@Z1X&>g` zVGkbh;Cc_v6ys4mCYw)7!H>)4GqU-tg3tM|5Tg?7^RoGZY`!Sf_>$~Bq2Ng$YVl>M z##3_UD^lODD)^cYU&l9O^GyZca*x@6KCHmEeOQg}DEO`i-&63k4;%1(AAW!zdhsJX zBby)l@Du#hho9l+KGfkCa`;Oh6{2^3gK;9w#i8qyO+B9a&g8Ha|V$xxUJ2DN&Dl7`0Z;hybH-94=h zElmRc9s9L|T4+Fv_l3fS9*g(MMU9Dg($L~Ymo|`!3QX;6ZfR=U+3DCfy|b;oxv^(+ zLuZ3yw$z#(hp%XF=;&NTY4M$CvM;vLaQLht3GDV9AcBQ_9f(S~S@HyrDWYeq_^A+6a@H{^9+XJRN8 zX-}iqTifEXL}=3?Bf2TIZ(mfe-$?MH4Y7F4*eFm|6>iQ3udnVBa5W}+DR;__SUlRA z8r&PzJGH$7obm5SM6`h}O^?a3Jz2EZL>4H|DUfi|;PVN)p{F7S^(@VJswr;hhXmaA zui6}sNA<=5Et!nceYW7V`xcn1q!SJ4kjd8jFc?8(3T{{deTHkf$S z-t;WXvsp`@Q}8@1E-Se}coOOJ`|8{>O6JUps=RE;?QpqiT87{q(c}=z7jw*;nPcIw z*~CoE%%)ei?cJYU2ssfo)5dL5PC(62&sd@*{>(AmZ@G}o!7cdE;Q_=04-wJ$&kim$h`-ndXn|7xtA4?CffNrL4-iDZhL{@ z6X?t)N;)UED*C)VT_DpM3S65l>fHH|PnnEUG6lj4?}ACf&4OH5RUH;!%j02M zRdu#DbYT0$xpUXhJScOV0d5jX(yh75k~U#U^Em`OW1CqFsphuaSpN%@XG3JbKBVXt zriw3|Na>O2mY6(;re}*@DvcDV%@>WF38PTS6suICL`5fds_4Ql6}!<*Vva*^(~|v!S4F9qqKc_vnkxKax+=Oby1;;nIfQySz@*-=HO<5P5CxYc7DFJQp8+UTp;GDA}A4;oiCzP>_XYh z7npAvZXLq|1JOQhph53T4f3+kbWJ2WByVy8bzN4C3s&|9W65AVVFV9q1F_zvL0(5= zy+OMKCWHGDdayDX)bwa@SANAN)^?jMiq;(QxQv4 zQ7e|RfU6=TmPs9#t73&%sfbmoSS{A5Vy#$rE+$VX*u1=qM_3Y-$IJLcvk}!bBcYR- zS6CIpbV%mVahPBoV(U#VS_eaYfTtKxS?-c9>Pr#pm>x$Go+MrFPk82d9Yas0ij7Pb?G3 z8Cl0lI>Owe^lQo1=ru%S@x7dq;^v?%y*@kio|yUG6`@LvIoxyp&M4x&WP6l%bPqfB z+nl#IH8c>5Xa=QJr1i+IYcfb$FVm^qsfEo@X@+7X?8|?7Rdr4lOM&*1MNSa=W}17G zmz&a@Uzlr`BZxK--@&+v5f#WO$FXBP?DTWm8&XgCQpW_S&M$G2duHTVYjUV?NWPt} zb3V5H=FlYV$;nezr0h(pbfTmk=h%}@C|^=$o*mvr`3iIv+aIdlywJh%qM?zpR*c`SHcBswD$V zw4`C5X;!HTleF~xphl*wcY576B}YOz%U8zWtn5tlgDzIh8>#*FATN{6%@%7W(D?l8 zN!~Lh<1FX*CR(dUHM%UFS(D_C*W}FcR~AX>It#I_zV+ow9)bVmBAruuJuc)ov~B$I zSd3Znipt(BY{w3Mw`*a)03ME89b>t!%`qNVa%xW1U&wVfN}~m`3coV8$qDXE9Yj(<+B}dzJSk$+bVlDTydb-UOQ(PbL&(> zUO;UBJUMUS`hx+fY+Xsf8z?!23k$I8*w{(AFyK7}-VUYgfIHwxKfBCQG2qTTt(koi z7uC!;i3K&aCsFwTz34&@+r5~{b_Kn+l3#sSVLrdPF2fpJinZv)IwY_j*YVrxji|>7 zet~@)7vm{x#M8J0&v4bVXu$K>gtKTAE^HPGn#4?O5p%Is%*Qsdi2S*d5)MP<-i6R8 zQRPk_BE&Bt=4wRIPo^dC9>h53p{#4M9|suYUfhKNK2>CFF^D*kWKVKKm_5mk0<$Oi zQHV0W03_IRnYs-T#f_`DX(NhkSvqA|5|SAShAm+?uE=1dG8jo>93(~;Tw|oYqTp&l zL$5La4=J<_a=v)dynXmNrvr|{YFn#CkC zX}z#}1dABD#ePz+s%6QUjfHDm<*vDR7}zH;x~XbgDDeJkonJGop9sQ!7ws2RrMH6`o^hq0utl=P`QjoR*jQeHBGrGe5@ z2o+-25uZ3V_V5uG+nZ_@`#r;0HjL$xSd0C|76=I@xm9R#Yolqw4qDQJsdP{^Ys50v zh4rijyP5wz%=rV1rNL;vg>&!Xnp^qg9>(thdgV$Y?O}|1umrCq7gVke;(A8BgnKIR zI&!I$>sR6iyq-~AMs`UC6;Z-&9AQ0CD6=1L;8VBsWtu_k@eP}vmQih%QJo7n!(s;O zMnPtcQKXD#dPc#U%CB66hg;ll$FNm;*HIQ+ zvgVjoX1VOyWx-u5vzPgQ8Y{Y&jAG?5R+VAR2-aD#I5zg=?0X4MLXj=9!$1FNtnRKI z!FtN7>-N_dJ%SD0g{*vPxZA}TUp$PB;coYFT=F0;9Yq66Vt5prM$q^ucUuKvDIE#A zT%r5DbnrAfYAsW=3;P(Bex~Doq}bj-XWq%*E4g|VZC{IdW~xd@AI&)WDBj9&N&SxE zZFoCVbsE$B9cHS^w4A{iYv((OVLf{{=_3O(4Q}s?s4%na@L8<&dkEBtn=J<;#9s;Q zmJYs;odT0|MdqY#>ZMu>Y;6Q>xP>!JudK}rCT>2MbMtO0Zk{$-n=9|1Ao*~P%~pcH zm;SsJ@3S@7N_$zjF#R+Xo>(#>vuj4Nc?3-pA+IwbrK`Bytf1_$XSr5_-e!YtU=XEG zNCXL5z?g4gn&-g0!J($##`NHJ++ibCnm9m7nk-vSpg^X!aCkXLmEO;}0(<~{QY?FA QWaPdgFOQVfOz6OW0O4cJdH?_b literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/ProduceResponse$PartitionResponse.class b/out/production/clients/org/apache/kafka/common/requests/ProduceResponse$PartitionResponse.class new file mode 100644 index 0000000000000000000000000000000000000000..786cb1307fa43b9fad216fb6872aa1f516027883 GIT binary patch literal 1077 zcmbtSU2hUW6g>m9e7Fdu*ov)+SnC39x7u10F+LR2q=eW;Oq)IqFu=C3LuVIXjK9D? zqlu-7CjJ0_l=03&)1)LM#+P&Fo_p@c+?n5he*6Nkjh8XpMlOmt9&1<$<4Fu5Ft8d!tj=9=&}zBDBcj4E``UJF?+wF@S*zra z7(yd)qG~(h-Jn+&>I1KLDLqw{4cfWtcnU%CkLNI`4(smySz%W8ZOv zDtEc-3X;67O4YV_pEo+fI^(S~ZZ%}DCml=(W5=a`D zLyBSXW}9&}1JCfBA?z>tRaU!O8n$WiYtrBHh<#yT8TZ53Vc7luH83Qv2Rf*qiUvJq z(A2Lt%lm!dG#Qr7^5k}|m#}V5Y^XF%zijSL4_BszH~;m{w8J?4dmsR&-+@l_$&eZ| z^GK6_hrAO6>AM4o!X;Q?=>ma5_5#6gzHoss<%b1f3sJPGXa|eLXDN}=a2HEt^+~s3 z{z*5|jp@n8ZN88?K(W$KgibCoQ!OSx4KaIMOofMt92X1U5f~y$e(cH*4sq+anEdR^ zhA_TTdH{N)93q5W!X7nsh?97TK2~uK3ko}M{Sg;NEyp89u!RStW5j0tT90s#Y>b#V I){#NxFDyU(yZ`_I literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/ProduceResponse.class b/out/production/clients/org/apache/kafka/common/requests/ProduceResponse.class new file mode 100644 index 0000000000000000000000000000000000000000..f1e1efeb39c1d22903e436fdc4cab6ecd3116e01 GIT binary patch literal 6010 zcmc&&3wRt=75;CsyR$nxN!o4GvMp39r6hf@wPH$>T1}I!-8Rj}Y}!%~*2#8~Zkz0e z*;%A$1*PHxR1jbI07S5gBG8tkp;|!&1r^^4;`^=mMiEen|DBoLOtx7H-v`>Sy}hq< z?z!jx&pCJJ{-^Fd3g8S8RPfdi8t^t10+*{;g)3CN9q&-^&M@ZUN)-WIC427*<7&J+ zjQ8NZvU#6^_e;5J6nr3v4O04pLF@^m4j+<(4@<#ov0n~8uHqB8IgC%@Q?j{5s{OQrTU9K=wNlq@vbkM02UHxyA=$e_Y8aKhF)4mn z!MKXkai>a!N2I-yWOKKQ&*KXUz9=xWM>nmkmCfgRjlxJiS1<&$ zL@sBT9YcDdU=#%EJGS)p#=HB{sg4ct&FuoQuDm%I(?|5oP9wHkAK0zOGWp@*d@eR( z=B<1tKNPd}j2MMjiW`RYHi1ZQJk^uzPQ_E{P4VsN?)J@bPIm3mFVJH{dTub5vdnC5 zkUQ%-XoIEatgZS`k+#(LC3_Mb9;p+1+I#yFeTigudRnEE%abR~iud*=d($1s&bUWY zeWH77dsm_}om{^@74H)e8~qdRAA#BH+Ea0N$s-gpT{qD&4bBCMR(2@1Ss$VGp;UG- zr&~pnmhYXqx~8hr`tl>$Oc{o@s{2gi{Gw5?3b7tDzoVEjN`NgY%a&%^*3y>1nrtp> zt)-uvrdrb0yj8&0k>5ch8@jSNqq{iVZ8SgK65hJ0=!tg6t^iG&spXk0l)iy8L1K5J)|Ft2>px2BGU5>M9cvGL`t%T`e5oT*-Bw78zmdCVcFoI9*>ChbT` z=%HVeJkW{<}~v6n)2G|I$N3g}QdJeg4wj6TCO z^QMz-M2dMSKQK@*82L5Uh(PrtE3V11Bo&uZ+|=@s?B8W%Wau{7LFP5bZ_lu>lvkW0 z7bw{LRq!PRUsiA*OJP&9?Mu)92j64Xs!xb zwQ|8*1^R7a0XJh-=oAR9$qc#Gqqa$+tfulsGh?jJ$`UcVq5z#P>$t%3XZk9z9j~Dk zZ5q08zifsTd_}`o@il>2_N%}ys&YH%|IqLl{;A<{JRz{S z+6AYEsPxg3viX;Wf8#$2p3*QOpa`Lf8d0kWA1@&Mng|F*;LK`D%ZRl1Geq?aL*9P( zOh$hFKnd1v!a>}U;&ngo-Ph1%zRrxQu zT!?k#hlZG>_|o1Y$2@n?%oj)Gv!wh+IJu$-FuW(PM%Wm;zC!jQ#vsFz!1|gJ^5mfX zU`ffOIyH%LiXtAu+Ixc8vXkaa$!{g+QAXLR7rKoLsgWn0&69HWV0LN0$AjD*%-WCk z3=QjWuV8O*{P{FVYWlju3^iz2n+!hBYVA_7Wg4Z&qFX%|Ry!uhW85Da4IUQk%*)rV za)JB_k{n>dYf8C5IJc7PytOK0FzHq^hm`huS3WmL{xxm%3du>z)apBSFz>sagUpAJION zWehRXC_TN&&Mh5rfhAR+3zbsjYE70VEj{irJ@QAto6{OIMbl(9DkW>@3xG_hiDw?2 zPD;f>ezRD`k24|6lWzs~=3xz9%1<3@*{{J2j$h^(M>u}DXB_3Y-7}uY@jB0VKBDMA zXIWpI>#y*vFF-_=Pp)sY&)Wn)`(4<~v%6`)Mesobgtp`e#P$QI8AI(Dd`l0*zxfCP z+n0^A_F)XcZuUX~<4^@!eNo>y!UEUeZrRfW?m}Iwf5{=#M`ny-W@{iCh(x0Pam*4J z#q4cT)fgIE6*)IYZlCK>7mNm7b=zdmRTqke+`T8Xs?-;y;>bKX=_rlNkE(}o;$fV` z6Cw-b;01?qG6${Us1gm2V_^-p?Vq?w%0`uOywI^_c{EU3YI7`JUS3PBvTZ>os9%>zcO1af|C3@L=fd5kHgZwZT#};fOW0JTLXK^mTbFRR4oDGFK zZ^b!W^-Ij~Dx6DAvgcqt3s2Cl1hHL+N06pOEndwHYpKK0t5dI|t-q`-<7(T0P2`57 zV@Fv>KXn+?(F5NEKg#(P3@8}1|L#3@gU*qFQX*r;7a(*YESO z9hSVQwRy9aQ4hwLpLd$UnPz%#1(9A&tj{9OX`;HDD8GgbyOd~N$=Ur}yM=oX5?{%W z5pp3-Tyxy3arYYJIaf!S74j8LPF~7yss%Elf%0=YOuqV&6@2fc*T&o;zTtc?K0dM3Axj_&3>^+eqXppIj zh^-`;V|a0;#tS_fBlOcgN(b>KT;^&z)z$;l9l#lf@RGx5IfxnwpYWKqfJVOAHuWt8 V4y0ubzehJP4MYjUTx<=5{tKahpK|~J literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/RequestHeader.class b/out/production/clients/org/apache/kafka/common/requests/RequestHeader.class new file mode 100644 index 0000000000000000000000000000000000000000..81e838af4724bbc9a6ac2ba5b58f5335496a6e5a GIT binary patch literal 3097 zcmbtW>r)d~6#uOW*(5AdQb7u$_N5^Ru~u6xXyp+kH3BAx+E(Zi7FY?{xY^Ls_xo%A zk$z}x2~M5q^m9M=@9Fe+?uIv+;y9VP_ndq0d7tyk{{8PCe*(COAEP*p(FodcLzb~9 zVn|2v9&QSok>!>w9#Q3I?=HW~_H9j?)9@#uO zHawBdrZR(5@@o4yWR~ok)|v(_ry)va+H%XbQ`8X77i_EOrRO!oG7rsFGg&Z;i^-hl z+QmhhZOS{YYZXk7ujP@J)({%7i?(-7!-e=^qoBS@-s|t4)({+a<}D4)8M|nWS61dM zcgmbA@FbRT@@8S$bZy~lhe2=2E^9bn*N|&Hs#s;OoXq;fm}SmeEw8wMxB=ptIR0rbS%>J=>_kP_Pc*)E*luWnGRNRRB zZ;G~)99r|Np~}JngX*puN&PI$DNzlr2Wn&k3@{|Wob-VqMsrTZ&0C|ks>!ab_DO>n z=tZA_hqz?mWxQfw83i3H28!r6a0X`$yeh|Wp=~O1&l)1VeHM)j3eMnDj@_nDiZJG%@$k z5|pkBGg~S3|C{^~WIS!m;toCAwLA1Mt1$WJ;1aZ1j(YgF-3m2diJa^<)wtVtem5UJ z4u5orX6^iP0K(`HWFyrfP^NYQnc9h9++%`sYA4F6o$#i1;u-e{h-2vEnk4rV0yx6` zMxti}S|af~jyy)7ieMED>j>2j^v#2C?I2P`bRCT~&ZwfPilggj{*|hZ;5^q3gt*4h zKy+=R17``Rm*~JKjlaR0Hs35R;7xLLfL}+K=#)73R?WG4ln(OFNOX2S!3*cQHqf$d zA*wW-MAw z1r4`W(RSBo^%J|;NbmNQ(HeY$$Zcz~-au8Y2RTXD^(0!v&oe0^5MHKQU7s56N$$k@(Mk6+{@cX{8y zAVNQr@Jkd@hu4)6!+%gSqOouX35WLzj~po6P4D+xZv17B<*@^Wdl>e9;pV-TGY1Gq za1HO)ga=d(I1W$IE=5gL@zP(2ZJ=YX8Rv89*sU_A~#fje# q5VA{=x{xOo84>c7A`NL%ej0L(skZE5o1SFTpW^hM=FN!8Me08nPwFKA literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/RequestSend.class b/out/production/clients/org/apache/kafka/common/requests/RequestSend.class new file mode 100644 index 0000000000000000000000000000000000000000..61f8033796c59dfca9a26c73528ca799b6f6e204 GIT binary patch literal 1900 zcma)6=~C1{5dMZe0!t8BPUR51*gJrDa;S?U=mM5dDCHl+?f{XHtl2CSK9yFkkc*)UN(|i?ZKB@wiLCj7Pq;w zqi*h~O3~ciH4Vk$AFI4#SCrT8X5R6bWC(AVr2_^f(6elrwplcF#XczWin3x`)uK&~ z(J9T)>=_0-QTU+lsfw-|TjE+Xwk8A70^$g?_h(P8QQ)&?i8HhoG=ncycQ?4Ts&43X z5-*rVRbNvrP3YcX*xuGE47os}`oLDWQKBrHOY~}*q1u{h2P8c(EI6nWTDB}ih%IHI0Mh|*rT*eg%vofCInS?nR z&oM7!0WV~{M1i5Fo;P3Bbea(Xu}BnWSQGB_Ov_lpvWyhc3{qxD%*8YVOEP3GI#nVG zhIEj*!R>w1+EJF=)wA{)pLf1R6kVSS!uGErHAaORb9s{?lt?ZxWP|*D%SqK#b)Ck^ zrb_iD7QAx)TN>&-`B+Zbeyz>gLBfs6d&amRm)}P5wkwea=x3q zs!n2t<*8WsF`~z$~l*t~WVdV&+ROS%j zRQ3>&FLd94L6ULDb^z_9b>k}G43Px&h=W1eFlmH_Yq(AfVkg)eoSongAtpN4W7p{> zM0bO!8lveMr1!^YTufzZXiAfa(bxPY^O-DM{%(pMgxo?HaYE_AEn*mPVi!J z6UH$ijvtBRNwJ!e@Hm7gm=4jMXd|AA%`*uz44n(9bUL1#TTi4`w^rhtTT6*}a*<&y zX*#?y?B6b@ z^SO=*Su(oNEn^mUWrWc#<2hbPcq!u*dS!H=lVP;R+Iu!gn3FM&1p&M$BaS5*udz&a zR)eXMuY`n*H&|h4sc5DxJtrfHw+z$&#hc68j-_f&+N<@S?5V_bPJKm-Ja-ZXEf#8S zNw34BUI7ZKkyi+S=iSz6a|>O0NF~t?XG1NN$p2_H#b^Dr=1&v#c{-k1ORZ+&Tg&me z#d!KE?MOqquTm#pVa~h|(J|7Y5S!X&=%Q}EG|hm`$+CkhT2=v-%ITK~%n6qu+ELZi&&N zRaguZy&FL8qMM#KXqQ2N?f~7f;S;b}>@(_)5h$a+jD{}=exy%zxJl0zx(Ga=@F5D1 z5G?5JhN1_zX-6QqgS!MQ2=4g=8tUC!1F=&`o5yH8L6czeh--_sJm-Y0%9YkO?K+TE_KH2naWs>!zos Kk9I>YZsa#)Oy>0f literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/requests/ResponseSend.class b/out/production/clients/org/apache/kafka/common/requests/ResponseSend.class new file mode 100644 index 0000000000000000000000000000000000000000..ff672c2d9d4eaffc8d2998171a429f758984fbdb GIT binary patch literal 1762 zcmcIkYflqF6g@)=3)@Ac2%;ztdG!S=K9M3SCPtEq2_+C?V%)Y_sHAaDuM>iJ z!387^JOB@cn6;f|oAywNgV(z=^UeHbcK`hS^&7x*JkBGFjXW~gw6SI5p^ZljxrPj+ z+GEI6w+|Syufm>SD7IuEK8*TZ5q;)epDg88=yLynM^c|tV^$r@A;XJS7x6&d&PYk$ zdCv#+Lycu85mc)<<7SwX!*^oLzz!Lf&Qp#D6Vk2ovPN?BdEhy170JM>*Y-3?TY3Rk zF-diX08#*mv4kn5Q)t8_USkzK{f9fp(z% zb=;sRE&fDbM%%P7V+faH;dF`g2H b#;|KUyFXyfvY5lNQx+@j=Kfgc5Pj<;H8FvN7TQ25v``>TE6kx6d1}Xe1 zxPZg~app%MW^AW%(;g}g-t~OEH*aS4&)=WF0BoaXp@7FGRxISPZsLiFrzV~W%r#V? z^o~HTwy`gee;por0_Bzpybq&(*NZ;Mu1}u%R_IE9Uq*`U)S1`EYAE2e!l-Az@p3~gDRD0!U6>~eCLe?#DT!#d5z=2MCbZ>SR*lg8T9P7j#SWd>N~8Yt$Kmf zv69-1lFKo%yyB!wYM`3JvfQ2KMyE+ibvRHiPcm*Xn?Z9wlv)ZHaiZ&oL*kl9$Yy4+ zd)*Vy)lSU)_M-oaJDIMc(P{D!tadnxTjd(f{AM-RB>0}<$rHqpfa6B z(vQH4tInvdgkh%pSfNwk!CA+f{Qcs3w1y&ZTfMgOlGGThuhA zUQ1}RCc7Xilf~ooa@S$9TdFF`@as~JQ!UA>o0!51d8;%T4sQO_+OzTs49<mn4F59#e$1z>friRwR6@?-t1S(`yiRNj%nYi0HyViOg zHGM!l@IR1v;kCR_1X6fJ;$I;C1W$xB>vgj!j?{Lk_$rrMEmcPUcbjJTyA zNG#IH?F9omjY~*}JIUl;=aF_J64^>6PK+c70;Q1J6j}8-{fzIKzI2h~0IR-gT+$g*`o9EdAEg8Jd z6}!W?N%J@36!_h!|AD-CT0fNEW9m0#zMjoKL+QkG|N^v$=e?=11-(O;EckblZ(Bfd3i0p`mwpT&$NRLJZixh!G@m(ajv z+~)c&uEaI1Qoj|hD!4#R$?17@8+Z$LT-^roi>zXvSMVj`vv`}H738exBPNg%S0vt% zs0#B?{eKAHUGlss%XBwQ`CmhpQw`z$Xc{u}5t-K#rB;~BG(^^x>p#OXWuDegnGnw; Lk}snkd%yc1!t`?G literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/serialization/IntegerSerializer.class b/out/production/clients/org/apache/kafka/common/serialization/IntegerSerializer.class new file mode 100644 index 0000000000000000000000000000000000000000..a36441232ffd3fed07dd4b22736c04ee444cca19 GIT binary patch literal 1355 zcmbVLYfscb6g{)ALM^g@toYz5=)R!1Au(nNiI|uW7mZm$0)Cn;Wf|zUo9?X18vg|U z1ixTnjPaX^KgxKg?Xm?lA%5ue&YgSCJ@-ue>-X310M<~^q2aNP6rQLhuc4qLgQA9# zhO&lbhRnKfh1_6B<_o(Fsh8e?%`jRMuKlLlYS{h`Z#a}0uXz@CcDXOqK1!zKk?1fy zuX%pc45p=fq}>M@b8 z9rF*|>`LL7+q_-fSCX`_g8GbJIc7+U&TG5J!1ft(mlgKfeU*!sVe8@6;cnBcOJBIn zYH>p;sf(t|WuPQ~MakbWfxO}Vj5I(Yg2lOG-Jok z5T9M`u{Q8QEY)h9Tj~ zUG8*g7WzZB=rd{bq&>)NVxIme4OB_LT$bQQDDcj8T;WzoRFqatfG?0Vf0CTzD7(6?7aE0U5&`OcF_sa3Q@qyGv(Nlu6(o zMPx0ls?8~BR&6TfpGdPa?c2~^vV4w2cIwmV&R!}Avnosn;gAY56ef4h2o>s@q4`1V zDdNr$vw?@yZyF1j!7^rJdMg)tD^xQb=&exIEa{pH_?U8M{=&i%C6$HO>|2W2$3oWZ m3z};onrn@;sk!uGsr&<}bEK8te_enc=}JT%J;Gv8z4Qn9k1aC* literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/serialization/Serializer.class b/out/production/clients/org/apache/kafka/common/serialization/Serializer.class new file mode 100644 index 0000000000000000000000000000000000000000..109d005a91bbf833a497559d7962eff258f50ce3 GIT binary patch literal 460 zcmZWmyHdk25In_6Ou{1xsrUnj0#Q)mObVExf}ya70;-EfF-jCEqu3OD8U-J~M=_k` z7(Ap~xQU-9PsB)6 zW-#}R=pSQR2{f9FC%IE5i_d2}*;YD*TD_BTEo<@`%E$RqiaegY+%P!XO=wcnwCZ() zwp+3bqB2)JO|SP&X8Wb8qAb5HP;#Kk$ zE>_5eU9YX-^;VcGLVr32Lw`G;yIP3B$d{Uvy9^k#=mqo>L7PDvI)u7}S`ZRGfZi|e Q6F!6^->VqMaN^P62c*(`EC2ui literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/serialization/StringDeserializer.class b/out/production/clients/org/apache/kafka/common/serialization/StringDeserializer.class new file mode 100644 index 0000000000000000000000000000000000000000..7d354c2acb18731e0d0c261619f935d251122fea GIT binary patch literal 2147 zcma)7+fo}x5IrL#?IJ|L0&|INtT@CG5}-hEf)ODB8*;IXlL$zF%EKa!#TubityWIR zpZE*>N~#j5n5tY}>?%LxkCMun-IcWFQg%_y>~!~Z_vt=8dime8KLE^ND~5i|>-eM- zeOS=(sgBROu!toccQo9MLC3O&6%C(j_#%e5vi(x+U#aC@3|+{_c=f)9RSgd`d@Z2M zYSAj0)v`b$|Hyc3WGhCsoGmyuWjTT9yjeA!1%Zy#*oHv(j#ZKZJ$bV#R~!2~(q1!m zDwHJhR?(yk(cOM$IVh%Iu!;aL>pVt^IwgnypLPSInOc$K=Ti zdRI23(q;mkMXUPIEH`Wh=}UPjHXO5(y>HZVTPi2bvtXc}Q)+>TS-&R_1jLp=yk=W9 zX*&n2#y*{OH<51`m4=I;4gbN;BUyA9ec)AC2el@&(N{ywpRHYbf%m%$j#1oWs%}Nt zB-Wjn3TC-#I4-8tDNN^ka)EZ0+NBX_UJUkYdui-!B1BJkH$8=EW!J0qMy+Pqjw~(v z&0qehC~K+*OgZ9MHM7W8mkh@c2yHKsQxq2~R-JxcAGTGztw1im@C}&|YSemd+@!>p8~EwHkzZYF=+ zmDOZBXU=tU=fIKMKO`M1S$79LS!zgiYP6}5Y?dV%$2vCR_zvGIK$~%7@qxhXMd>!D z_oA5=2o%brg~0Tic?a(80z&br(Tq( zQLD*nNnj%At7&lNj$d$+Qgz{dCv2=fp_>U_k(KwhZIry;79mP`f`1n zr`wbZ?p^6)h;$EAE-%8b){$xQ-+(l$(FJN4V#%K7pPH zKSd-FS-Ze~!{NW7KS4Y_9R3TsA2#fU9f3wy*BEYunO|ps zBu0s43O6u^cimKGyl{Rh3{3qEH@zG!MwyNx)O$qSK_55y_da2y=+^_G?8j)IrsNXB zor@YW|6+)%hVjdopg4p{8jzJ-6~?D@5avET{sIwkcmjQMJpC&=j?sCzdV;Ra6T~-< z(Vckf2t7){?FhZcxb!D|slxZMqERH=y~Bk%;KE4J?-(^1YR1{H2^V6T(teg;3eyC^ zxGk{L_>jO<+ot*ZeT&`|3=N@ZUc*OJeBA19h6-gU+Fgyhf|)=^h3C0 z=kKrTdvw;@#J!={8bSWY;O}lfG!388hh7G??>2#rzZ`1rbmemMyx&{)bz_&Q2K8Wz zSao75TUA^415DWym^yn3fOa>F#cZ3`+Iw|6hl%@(#7^_eq0VxO>vp~2x{l|Ym8Gch zOFtN7DT+)mPsUkQv$SF-=Pk&psBLc@CUzGPvGW~yqWoa$Fwue zj5)%YdN9cu-^pz6`{w=cGL?oYPmN}CfU*hQ zbxpe>FxAPT&1-lmDmn?7^qo*7Fp!-%Wk`#`P98V#4&UJcP|a^l(v&ZJoWDt4u5a=5 zG3A1LPwogJmyZ30u3vbJ;cc!d?x3+k^Yi?rg{HWGF`fi2B0~%5<{$2H?xpjOkxVN; zBc6`0=BJ(@5yPx9s=PpNKCQNn&oMBnJVWv^wA`rj7m`s}B?vnPm98!@+!!;z$N(9P z6U$Xx!YtkmQkf3JMX4|_^_PP5O0*bfI+9lJ5pfrNT;}I}!pPEZ2twLV&^|}WSt#8L zD)RqggsY0lK}=B`LxBe5CRd5^Cpu8L7bagIE)FJhzoP39-3O~r(X;s!+U5~@(`OFR z7b^M=(SL-qf6|XEdLOHqz?oq03SbTe5VG_+K~0{TN%S+ataOgje!{)HPS)m;Tv literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/utils/AbstractIterator$1.class b/out/production/clients/org/apache/kafka/common/utils/AbstractIterator$1.class new file mode 100644 index 0000000000000000000000000000000000000000..70b9cd8ad0eac3ce2998c09e15776049008f8cfb GIT binary patch literal 897 zcmbVK+iuf95IvhXb`o3~LR&((73#JLml{w3ArKHXi4>8GL{Jgx)5cj6H;%VjZ^|F= zD?9@dLJ0K(_$b8KEmZ1TD|s$wJac9=JHP+@JOHqXdwElSWUxXJL+>3{UKH}3E~N2-y-_qaC}&V+xMaL1F=sSKT&_Jhc&4I-_$r(0SmuBCGN znrp$3+G#OlpL>BH`uYV!x>DU{Ft+57e$Kc3P;_V0flzPxAfQOGEj=D=bLGeQC^$!4 z-h^gIFb&F(@5`C;MAMIp7Edz9ihi^XyKNH6&tHqvysO+8|$dp zsNyz5IWBPn9**3u?9aS$(-*<8p_EiM?!;+#QDL}GTW}Az;3ive54YeZTX0Vf;yhrm zTVW{FR=}f3L=1)FM!mtV@HE55=|~wiPgn5oEiy-MV(#-`7SHmuQr&5vh3grE*{rqN zjXJ~Qvocsx4gHV@$x+ai81-Ja!H{{=sMX&WmgzimbgryWir*^sC}Lxd?0G_;kR}g! z@&oMPSN{SR%blQlz3RSroF=WC_lz@?Q}3;bZ^+ literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/utils/AbstractIterator$State.class b/out/production/clients/org/apache/kafka/common/utils/AbstractIterator$State.class new file mode 100644 index 0000000000000000000000000000000000000000..4d5e2942555f2eaede40b1804071d8c3d772d8f5 GIT binary patch literal 1323 zcmbVLZBG+H5Pr73_S)lktGoyzqN22*9AE*l4WWS&YZ{t(NYePhY|o<{T<^&B%J0&W zh|y^H?2j_eE{zuPi@i%`GBeN4JTp7{_juR}C{r{o0*Sokb`HPnN?r6?jeKwXH|bh6({BZqz3Yf6jsAW6i(*td zZi{W4(D57_Dqb*Td)Xl!)5z&~h*=%O7|}5*#F!A{7-1Orb1kAL+F;A|Jz_PI&UE!# zsU6!^NX!@ij|9VN_sHdg$aUrYy$aW9FxCdHr# zsUV3-+yTQBSr15yPiupHLgW(B4`eaiB^j5oDZ)?VUI(8=M0&N9GXEK|ODF;#xrU!1 zkA8*1eX1bhG+2_N;xeN21M2+3)CJ-nWw^WC40*1Ot9T@+{V1m=t f2s9!MYA#FEqa7X@%#$yI1>7R75mJGH)YjxL_|`U; literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/utils/AbstractIterator.class b/out/production/clients/org/apache/kafka/common/utils/AbstractIterator.class new file mode 100644 index 0000000000000000000000000000000000000000..594847603c6bf95f8b6e9acc9cb3967b805d61ce GIT binary patch literal 2342 zcmb7FZBr9h6n>U0B9%ycfoN1Pd{AU@^v(r5Ms!mS6Y7co2goPYogy!53kC89_FN9xO_r zhw_n=JM;3)BWXSk<4G8WFp6P3RiI@oOLrDcQLE{-Fjf?_7hK`$eAoQes=1CRyBSw^gljtq@BEAc)zWud1-;o9;-yHL!rDyb zOE2`Y%e831*tCROcNi#CEYj04YgmRmM{{(HEYwANoon4$!_ss0N=bK$qGZz3o3+cr zToH~T?ZBzJTSiU6^;1%1*n#74uc{5mo2SH@WfpS;3~z~A&P$w$tSRVZIm+7)MKz$> zH#+uz1!*50^UL7q75gRgRI9qaO*mniOSVNvq$0L;AGxO@UX}DkyHc&Yfvf)*?1F8Z zy0A!RezrK>8gH)E<$GjHQ;kkGQi>Jqx>MGd4O##Gll>l-?bUD@pJ@0BaSa#c<7=#H zSi>^~7mwPXF-?6_n4Un=J7vA<8n&h2LL*?@sKpH{z9ATCJzkUglfd6-_zbth_*TPr zNNE^HN27!`>GU5uxPj_(? z^c->VZwlpplxCw!)cNUv6B7!iPI<)Z5$|a(Uwr-~J-@W7;BKPugW;C9sO1FF68=a? zcq1X1!NsI;8F|5rQIRVZd6(f+d0*_Jj zHChx0QUUVM#E{6-Z=oiCM`#c2lHx`)^< zS5%M30NUj)BzSt9Os+HjB(61anr`7FJ(AOP{yLy0qKd+}moUMcB!`~@epi!wIP(k6 z96(#$N9X&X2&CNgH#$9H-IADeyyiKd(RpG5HxTB(xNh9U0B^}K@ADPhZen${g_ZO) zSgC01J{R<5lH6__c4ax>7CS)CYVX;7^uA{_zb74D_3ku9iM2$L9={(`hxv-2(|srN zJ(gFO*Oz(bwZOav?v zaimAa@fAF&s&XQ+3W3c4oTId5SAPt`B^j=d;gZa@{~fxd0!iRkL_&XH;5ApzaaEPy zJk{LO9Qov#`y;CJF-B0J?ut$Pk}dqCr>PDjOwXtGIi?BTh8cXwcQ3y#@Qt1br80<- Hd*}ZHV+ZRU literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/utils/CollectionUtils.class b/out/production/clients/org/apache/kafka/common/utils/CollectionUtils.class new file mode 100644 index 0000000000000000000000000000000000000000..b60cbc9624c76d39b1131a7da524c8f483a6c989 GIT binary patch literal 2901 zcmbtWU2hvj6g}hEUdP_1N%K+6N1J}eNt+r zXe1x`C&GJvk2QQI5S_7X%gqUNrAAi-!gG~%Q{Z^svdzW%_L}JwjI}aHPUI^^ zqr74`mb9CLVRyr-38eECr<69TMsdSTZy8UwjC8TGyb=NA_(sPw^*(|zN#a@T9UqNRI{V|Y=MaDK~H!yWy3C|mt4oPOQh_s z8jfqpO$5XOZ3UKF$Z&$xq%v*S*=2jvYxM=&Hl4Y$QLCAZXfU|oyh3r4@Veql$foW2 zP*9rPYD=DxAj^!;?FHL4OQw@86tX$;&IXa*elv&4swBJLinm45RD0gt-;cF+gE zw@r}`2L!^A0UcX%jp7!)P?^G<)bNBktAVG{y=0Ya!>v0^tM6wD8J{fo);5_a+1$Qd zI9e$C66gN~a>nmU-m19=rei9umneH8vB|SLbem{_dZ-p+Rcsk9r^fNSl z_74$WtrI?d0NlF>e)NF^UP38Y^L{X{9iv<~?>!Y5N-g+TwQdmAq?;=eMHh_qb-s`e zaA9;6XK<2V3i8(lAyb&o*J!2L3bv84=MX=!6T<6!M%CaD-e6z%062xyw7M}&vXERX zwn?UO?CakVdiWe&`LPDV0u4k2?p|&nD)1dlb~J%M5nCK@pj-ChzaWtb`z(Dbaw!>Z z;8=*WJ(*}S`WH?%Ni$JSC8G@-@4|y*bTINOdbzYu?jwhO*!%X;WAi`j>+u9@aPMDOGgWSU2i;vP|sulbJPgirlDE%oDz<4X08&i$E4GNN=I z;NcBsr}ddp^()0XP?Gn{DhPd#2Cq0x|yPSlbE7U*Qnk!7TJ4*ES_LS z={rF!CC1ZtlKqREjnK{#@vc&74LMp;)m6OLR&|(j9{zpe86{EI+o;*5mgl9G$2~1y oAg5uD0&2Lihd!y^4JBn$y_?kS78yc3?=iMV!*JR0r(7ti15kN^Mx literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/utils/CopyOnWriteMap.class b/out/production/clients/org/apache/kafka/common/utils/CopyOnWriteMap.class new file mode 100644 index 0000000000000000000000000000000000000000..cf57ab5dd079055aebd1872834c6e726ecd133ae GIT binary patch literal 4034 zcmb7Fdvg<26hE7!Nt$daEu<~jM|cQrk`hqll~{pVp!9{drh>e->9*Zwvl})^vCgRD zFryCR2M|94f8dM))fqp4AIkA}_HH(t*G%~%H+Szn=XZYRdH?Jncqki0(z` zE4trJgY7O_Aik?#}jlZ44KxIoSOLS)5v^LX9 zNy%@i>4LIRP||s`STqfrPB}eomUfqnRZFX=g4#eM*lwu?c;TIpJZ|)lH+q<1XO1=< zy7GJ(&6`F=(Twt(x+_u45!DrAGrhF7t>!D4iF*=B1$NyIt@lczo|<}2(W{7$v?Ebi zH7eHbvRaX-Z+yZ-2uGO^J&~wuh9xADXAMKOrgf!URuRCn&C*$qGD3FJR&(id-u#NPyz5bSWtxv11+RbHEcRi*1mzq5TAwk4*L%uh8o1D zO7$M?$?x%DD^J$OQM6#^FX5Gb#t8}0AK4uZc}6Q)KBYOgNh8R+?Eh*o+da# zMfX;XqPebZ*z?$yiZ_*M#yf(9QcvBMtrAq9R298U$MB{M5xM}s>l|?W593wy(FpZp zk9=T_c;XHzXzAlJ8IQ|Bi1zFo+J|pP&+7ngZ#-o>hjS`91FjK5F)T-e*|dMH5jENrEp*OjPHbJDAW${!B+TG2lz7`;n4zM z9S5l!FvAoh%zlqT61~Egm;5;!T_hvokgrqb6Q(R8a~@deEC;(eGGSi}_=OJCJ4HsA zxd=1pU954KZgQbpYKeblUoJ>I3fhMy(i)bl`@$4_Ghm7d)1Gg%mG8{Huq7T??fJ^B zd~fdyQ}Dftp1kIE?k(8h5(&3-Z3H(&OB6zP*E=@cQY3uBMRJUXsKa;?g1Y( zV)QA7lOK9fn!#DXZo9gs=yTTOauS+1UCj|!^CmP!kZu7T&w=C4FOc97Htca{A%$^p z&_{_>^E?us+y}1*bch~;*(2MNxWI5d8K&FzNR{cJ&EIFUu1g8lEHww#LTUtz1jBhS z#9QSs{1w<7ztLufeLh;OQJid!b{?qfbQlA6m^N&V34!8r9Hm8j&d3<}9Ah7(BX*4a z9=4-S2?#0XD5P$Jm0)`jY?s{fc#d<9XDpF?P5r-NJ|$DH>Ch|wJj^5m5U;%(1qzc+ zy<~WT7;_26+;WIH<-p@h1d*jg5Lrs3o~1bTcOc~_sn?V51x$MhszlEq!2CvSelKXy zCI*VJ7Zm0Fz-r*7pC1B2RAjsiuPbM-F%{*-;N!F@0+>bchWj0-w_| zh{U1z5_CAdV-BGq42G5QvazC+f&$K(5hC%$%KJIqI%kmhU;Y7^l{2yjUI91m(KmzvZ46Vm-N(*4{M!9JI7 spUbz;6G4$~mJ%#kO0Z<9w&|`F<|=&&RzA8*AL2iP0yu|u{UF%=KdsoAoB#j- literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/utils/Crc32.class b/out/production/clients/org/apache/kafka/common/utils/Crc32.class new file mode 100644 index 0000000000000000000000000000000000000000..a6658fb0a5ed8eb0f0f03140cc4986e6f2493d91 GIT binary patch literal 28501 zcmd4ZXIRu)*f#7*t^o$=xdjWLDA+(mPys7qL9D1Cii%2Aihva>Dq^o#v4Rz`AQlt_ zQL%SW6tRF<5Gxk!?OXTW``OPv-sAi6{(SYA>&#kAR#wVnl1yeU|303pQmI;MdRJ4) znoc^J&TJQst{fiKG+KWbyXlIerW}!xrSANU9*EUxz6Ng%1Dax;ZolKu8GW`E zV#2GI;YHAnR@fZ8dK(Py&v(M)2U*+k==O^`SbMX0K6H=tvP0Z~X(#YJ%U%b|Pg*9z z@~Ckz!ZWNs;N!6|o=C`bFG1~lh0`(m;nP>BD$MMM*|+9g07WMGA~<5#Df|ljHU$eq zF134aMf3_Y9){Z^+@E6W$T4Hl4o?fvQ(f2%Hx1_e z!j5X05ondU>pMCfOp3()>`QmBH|JYNINgnWg07`QC*Z*ygB#dW(B2OXi<|F7zY%W?;I1*~#9upt9FNbu0j5d;PTIHbfwy`b79GJ5J+D9_$BKhRg9jM4EY=KES zm$t%4^{pk==iZ2%G46rcY&;3lQ?M$yZ2}BNhj+k~@WH$A z)bNHLR_gxQglekvMwqU%`2gM-t?Yt%nr$~xNBS@dVcL1W@g;DeKW2uEyn4*L($R7C~J97q(gj=TeLwFu^?uokwcK5Kqnt3lcqsM2stEY@W>4?fR*fp|r z034&1R>C7JcQUR799oQ=(Al-oAo^ws`j2|-jH~*wtC6kO4X7`h9m4>vh7LG)ylpNH zp3rLyyTgN5VCbQ6HS&u7q#`r_h9jC5Z(f8UCFz#f5S*8R*Fhf|VnX<~jgUvLtcH!c z+G%)Y=0zVp+)MKlmD$9)^I3Keu2Od7~>*sy&{BB7>W6(MLVo2d!}^52+&?1|WZg*%x#V z>((0`qGInOy&${?ith}5fR3elp>Vm|_8ro4(!EiVz4;uv9lS9E%`^YJz=;zdEO7aF z-a>dETG<>nhqvv-@qA}3Tr1LE37?XIO<-L-@+1z0PHBd-0X?$dKgwK&ZM5Ag9M;Qq z!5RIfTQFFw)ExG5XD^Y0g*HqdeB%AAcevEl}w0R*Gwb4L4(T zg3VSePBwjl)Po*Y$W4n4!261_ov8Sj{~TvZX4~N2xq~6__fOxAE*{CR5FK%~F($_R z3PWqhh;&%l4z7frj&4&Z7OtaF>f!tm=lyjK;!TXt7*s`sG)K1W$1m9Gn0EvVED|SR zxlT$;)JV?x0mFnP$I&3IWHQV^E20JG`6hrG*!Pab1w- zWj_!1eJsvnmFYqWOANC+VyAV*Y@Bp0&Vy^pIRn(ouIhzqYf_himaMssp;uM9=vw6H ziKy>^ix6ApeHq8I{e7@4CDIi0m$=K`=74t_2&{n0sh%?Y;pazammGtz2xU1Fz^dK%acqU`UDS z+fZ|1@?*3}y;={pIlo3=pH10*9Cpipi)3ZCJ?82jjKn2Rn?1<)Hhqa7aUM?5QCRAlkB3^ZZH1oRIUms9bIBo04k?Mjh`35eJTL$F8^3?$oy6_@#A&#C zC8a%PFLeHn^@%#!IFREr3A<85TBGgkq@JjslvWM8ncpwLIQ`0e46pR}f=5-PAx0Ow zUBmc1!^QYGSm}(H?$$C+N5-7S-B|aTSnFKW4NL9IwXs8$c@8<|2?-eEzOXh%4bC2d z&ao9q7#vw#3{!irI?#8H8;(Zi_8ZYbW$_Tll2i&(W;=RgQF`EVtjP2(z{9F@W;k0} zH4v3~sjKk0c+G93>*Tz^QHv$Ju+gz30&{FDZE-nb@*CWW@!X5A{^rpr^RR7(&=Th| z%s8i$0-p+>AoTtiQXlfckI!J9mbV?v6B0+FNpea#tgD9MkYVa3>^5EV0MC3>=J@I5I2c8N zf$MO6viDtdFY~H_zTe|~F}28kH3F|%6vBSVLPE=~#fgQsjqu@0&Lh0a zU$ODz#ujMCmJrC?FP5RgK>z-w>gHeE2dx3sobM2hWv`1f(IcH znK&0WxC(E*bzShq({&=Ubkh%Di<0~f3*D}^z)G86VjAW{ zqgk=+$JLv62?Kc4%P;*ad9%_YUeuC2K!$r`Sq(3k@&~PV=$Ms5t zX}iJIQT=ywYnWv%oq`%up7w@oZ;wZ4y{u^w+Fn|*6m9gr9)nw#x9!kjZh{`#oodq` z?d859a2{4;5L`A+k4CF|XPnWpW|{=YfhQBteD&=^aJm)o5-m)mhp>rwwp{GZmZhRm zrQ>fjv>$Q_)`7iyp?*rEsjwiNsR5Sh~~1Eb@*%tN&6_Y$#B zT>b(PnR6o%9{YL-MmhGE5%$rw6$1Cq&>(2EsT%_9_Jm@@i_`rOva_fV!NDhbOm}RJ=_70v+I-4P5a$` zbnV#ZGrGjvoQC@~r!MGdG-w<;buYI?=S7Px;q!h}Ukqp=kHo+UIUV48^o}mPzn)r* z{%v+=qwlopkI^qT?luNLZe4*PR^FxX3w<*egSO8*0RLBsEitso4`5hy*TERR&tWvm zTA9zpqc3BQ;_-nSZ}DW@j$)LzJ2Vu}e-}sKS=LA=Je^{oh02CKT=Dj)l@VTV-Rz4u zBdP*W;qdq=Uc8xq3$OM#FT~5Rp(pT*vfklm!OaW!y*6?){tOTYK;LT{H^q;;Q_S&g z^Vv@LIy8L(KB``L;zQo-8F-h_z9HUw3^YNNuKOx{x?;BtUzV)?j?Z3yuAz96{|MYY z?lKVfel9S?{SHrBqh!pd9F!hPUWA9A>fOWxm!8j16rq0?chW*DQCNBR2yWXSoQ3?r z?P?UHTyKF}; zu^Lx;@85;X%g!a^;-#tQaY@hQJ+5}q8INmoM|Z}JQ@3j4x_nHWFYI5_5z9yajKvDO zPR+3LMdMmnv$MlStPS?rgVpt4e!!|nGtVJ8=fYQPj@x$w8(nD)HhrJG1xcCFHdq&{ zMcCk&-2>}C7EH&I1+AK6anIp8SZYu<49jjT3d6#gn_gp4r!V)hK;1na^UqsnV)pvr z`G_B6coH+~WRzppgL?%?Sb5b4bNcNJ!n_(a+hFePu?9$coS%YSR>#+3cc}V0_H2*( zhV)mHreS{*r6=}9?^UpO-}weeeLt%vcD8usfbA0oOvH|(Zth6=+T<*@wCVg2+ot{6 zjjg$BHzD)vxO^PeuJs&8I;Q60X#BMWIC$;8J`NcjXo~~gjRKLeC}aS#i|UWUaZ}IX z$nyTIjbn)$9g$PA@GwqTJQ7DEgNNNiZc@uP$jTWRg5A0o7eUF~{}#&v*A9eB$vY=B zh#39=n~GeI;oRZIm6)#aT8QWqkAhJo&6CjM&dDr%2*3Lfk4j@(U{H{;FZ$LRn2d@Y z&A;QjadLHZ+WU13N~fN>h>;gk_hG_VO=n!Y9^DYD_sxiby-7_|xNSfB1M_O;Z$|AK z6^*cU+_Fx{IB&HVnkn5bA;oEb2h?18aR{ol9hr%P^Hlk8=<6DWwXcWkV~*du$7ug- z?L36cohRP5+dMjq8_T=~Beq3jH@tjmJQw{3#XdpDH+K#2wcp86cxU%@3q02*pT*$% z&28~`^FS}m=pOwU7d~lL;&#VWacchaR86!>&hLXI)<^A-yr$+kG_swMg5$rs)q?D9 zH4Vm9%T{2QXT=w6s6MDYtarH$f}`bfaoD)!%Pmx&cq1HpFYhu!?hKtWXrCF8fMF>y zH&Jd;<0z`qvi#7?w6vW#h<-i?VR17bVd@nJL!62AjKCEGqpcWsVAKT!s4p}@aaKw% zbcs&-grB$mEXUi>zH#uq=VA`SV~u*_h*rnO*q%{&9v1rZw_(=(jM`{DvSuF=&j%pFEoCEa&A#vrr=3UDz~qHSWAW7ASnR;b=3+ zu?CQ&o7E$Fen0s1-%r-km<3z`@?a=x;_cUsBZ{mRcpZfHGQ~ik_v2e4=8YJ1} zPJ-3iVoPyOa;Yayd8RkR&0o?fgu93Cz?U_{EZ}9^dNTZz*Q~`e>-QhfJEE2rJ{84{ z!1KU@r5IjvgrSd37eonm{%rA9(iRMAR%TWK$w*agU->intiJd2K zQaXJX`wmAr!&IZ|4a>28)?>%@CVy~r>Rb~TT__um6~@KaV7oVW7hG$Zcwpv^iPpG# zK6C;`Op%N+_D1?|Tpo960|st?Umuk<*L1_LeXY~b)nwRJOzizu8`s}22*QId4VNJJ zt9u2$IZW;c_my>8!n@&tdw8|}jyM7QcK!wQx^7vFTJH=4v87j-4%%(#my3i(&iAoq z#ky8#*6ep*H1s*P4x1}Veqyoz7zHiM%wokUS+nafS=fCK;8xxh*|TR^;mM=Ki5N2U zGokwnzh8JiphFTOoL_W(H{x86Biu9*~@S5z|ZoI zI9A?rpc+0_uk#c=cTJuJx)N4|J+X!daV&4kP)Kp-TVrF|@0qYMUH1&mDb9LWYSAwW zb3@Hi(c#{hJT#3iX^K_1j&+Bwde%D}$SPlrZ3f+^!u&w9I_P|}gC~AO`Pt*e{m*C6 zZ)D<5T+K+Ui}Cu`ry%rLz$%nzX}?GN5lu2M<&^gbbW53%k54mqp2EdP%VMyseL+=AGAM-!ZT_eR3k0PRt`llo zoqdc(9pCiF>Jg5euuE0*11eJj>*MCBo-5FI`Gd|F{qEC8*!Ep!jT+5&uEY-GDYKCj zKp|SXFE|U`dMV@4XV$+3bOI^G`$ zK8LFXy(eJN7?&M5Vp7KxYR5b+wC?vR6LIeoL(n-f>o&I3%bJUg?upL0U-qgT<+Jj9 z;j^WV2f|Lfd`7c?-VH%g@ERoc``8V89q)a?tHhK>xbuF&S_Bsim##u(?_r)u3EKD-&(D-M#=UJflHf`=u0U&C`BZeewQ&z-9LhAuwUuFdc=xem zHja9?iNL}ZbRU|aglcGoeaA6&TTwW?&iuZIUr*{^#;MtQQ!vkZ%x)a-Jg7F-ImPzI z_Wr|u;mHRZdz7p&*o4S~1xwKH=GXQx8yo)ycFG=qq;+by7^~`=w!^LX=+}7r*kdS$ z=B@9AfUSqb$%TBWDa->OZN#^mk8a@X!AmnRTRF5JxntMYg?@hzBRD!mpTxu!rqSs6 zpZ04m-O#0p}y*F>B2aZ{&Qw`2;_FceTKI=UKBc-pqd`x=$Q$ z4Z}NzAJO7Sqs}PMZe)ztp@u2gbAEh`H~{iL1)I6E@=>dN*9Z*rxS4^VhCfs&n;2Pu z2WDLZu;GZ?0qneECZYQ0%O+^D=E)XBIjg}Ycc=mkVd#squv*M zh}Cq1@dZtF*r(Rpj&PkJ<1ujL*t2-nExizT8_f$w^1NS%vGsYO8dgU;--cyTcnAi^ z8fRi;jn-PY?efeN6$7vC!2T~sCSb+di;4M?}{d)v(~x z3_l!R?fU{BTQzNs>jShFVA_P@b#OO**96*`3%{Z5?QPw0Y);E0Bs{Zhj5A$Fe#P$w zeLT@iYsDap3CXU2^SRTm(A!ZlAB_tyi0jXX-$tW<^;IVkJ^q9d9{L2-#iN#<`;ofJ zX$IC;ncqOo?ThQ6!Py<@2n;Tb!*KQI>v&^hTY~&9h{D>ZlXI~tsUrnaf=@2t{R(;4T+u1(Dz#6A>8qMVhKyF=2>{{9X}e4 zjQU-~mJQaS2*2>|3}W+6O-0UTDnXlS0Wo;*d+;`JN8W&o3ky1+Pn~VcF{e{ZD1omxtsexv!iSj| zeW0^fOI!zjcV{Jf1aEIIE=9y9VPpGlRu~wtY7PoMmD{0qTC1ga^=IA?SZ(w9jh!wH z`(kv&r%#B9I@cT7PG%KozcbhapT8e{0`(M~eYnxEP=oFVckV>|+X+DOtFGDbIl86} zZdyOhK&=@rVhd|B=M=0b_=aFdnaveMo%j-kaakAcAnR%Map({e`VgO*W=?|9plXpg zt1iBTPRsX<#Dc4n^04ZX=M)TIy`dDv(u*-r{M!`biQaq(jRy|dj`TB4sv#u#`+iJX zf4K$@ov(Eet@=k;;J0DcG3YPVe8GjA_xr=$Bkd;^)}G>rwRL-xVQA-dU2w1H)k~Nx za`nQahYQ!kZkC@L_L;$mbNn>T@TLfAa~x_i7nG*w1}ac0+pKj_*n zy&sm2m|lgIA>ssoQ0vVvP_plJH<-Q`JDI1cOI+aaW7rC$MK^7Sz!rbjVTSXKdN>?z zkpQRPQMUND^~5sh)i*eeQ*kB1@OZiRGL{^f5sl@CyoxZSa^qM$oci_w%o??yjOSyP z?L|}jp}N?8tXV1op8b+B?a{U4IGR(ZE*u+29>DkUC#$1QM)kY6{rbXmcu&f_g(REM ziI`K*_d5DapA&(LFTR`uGHpiS!;>|cXnnG)F;3dK8!&2+S3b9}-s zRF>~FfMuvzXWVIXt{i>Sk9uOmhu}Au`^9GfdheO{4p-bh4TDPT`|!Tis%B^t)@>1T zezj_V*e&HV5x)J7E4F;q+Jr`t;wZg$M;37^7@XGoqO)DiJZyuYG* z!AyVL==$L!)V1pQ;*;0P+2}wW8zO6-(|nA(UuKV}l7iLP5id7~wOPz&RP@-^7PT+a z1Qe_}m5+h4xJf2vE;VjDtIhd=!jF6Uz4HdZr29i zYpkn;W6vHvaJ1C*DW=VS<&J>4_rGEHJa6@7!RjSo&`9p+hzu440v%5fqjbuk)~hm0vwVi zh2iPhJy)?aR_`#@CYv;YV@2&vsPeKQ^mlbgMNodLHaM5sVJTWv_8E!QNnLMYd(7}Y zkac{%<3dXKcmxy$oI+o>ss8xv8S?@oOEzA|>9n;|@Xme7bND*X^~TD?tcPe-ekcrT z!?Y@FoxHUpd>n74;e&@b^*ED$W+p<4FSbNV+~?!a*8OD&m$K(8#U=jN;{EbO>x~Gz z?9>>8>Nn2B?*Y~l+J3NHhD~$K+G6i${hg>*TSvjq{;@wWxWP0pL|hBKiEGD4heBiN zU4iuQK|_%o-{~}3f9gH~yMlJ~hhg>fa=86GJQeE}oxFjeO=r2|`;dj7aOKeQQ5b!8 z!vkzs{Hhh&|9CVD`t;obyMy2FLiCv%4RPu4*%jh)QHc?T+1<`Y7d>(D?D4d?Yq+&4 zxGyF=iGGTVF8#t`)?#o88r|>M8MF5G{DRo0ZR?<5W%J#5G}YDx9{TlWWA>i9259i0 z`Y}{*X=sCkowb(3`u2;Xn6vdUaJa|U%~0BZv_;YSYuhpA&3R*V)OxT4rY$HyHDF9QlTNH&67&(+RWBpp!g57UN#7c!}Ga)(yj=BfXx%{&cr~uxR3c6~~A9 zic5~)Mtw%{(x6VL2%TJtJ|<%$kmV=GqfRqjXE>avJy>v{W?l3$YnX_a0rm}We}PLD zrvGlIkJqE}?a-s{{bWr2a$3g2nU{~CW}_-sfLG^BFQWH>!fD91*CCkFw?!p-W;Gd$ zH#h2^N8?czo>*G9#!sC5Vi15@GqrD{+n!rdD0^`C9`ak}b;HEYmp3s|j8y zm@9Xwj-xM!Er7|U0qs%PCgKpr_8y^w&O4^8$FrMbnt|3!7m)d;+BhU=8NWsSvF3yE z{C0yvxNmJX0x>;W{=%L1VxM!>Qe1Kj4`1bmA@NJM;@77+W^nDForKK|GMZxlwcQ7x zdwh!){(LBS4Zk_}24U3bQ|EBK_LVWvJMrlc_Fewn6C3M4y@NIbDg&|S47$SLu;xA5 z4OJqs-mcbN3}4*H7vFv~eT<9bJOfd|?XMuIY47^zFvMdGj1CReL)zJaClMJG=7`JH z1Lxq!&q+1mzi8ZEAZ2F_>?qo|4XfOac7T&-&O&^tj8~&y(t-?(h*@cm^EyfE(A=v+ z9HYBFIg8Z%Z+?(cKfJ+N)zu&b#$G6bck;s?_*8M{H-h8Z6pJekj-&Ch%+DTDGGdGE5vZ(P6W*FFv$%zKE~y-3rihh?^IV7rO4n z#5h+c>>p;k89`OHD!iF${{;04?5D%7x=k>i=h}Qnn4|cZa>oYidH6Ol&=;*91YSh8 zxA-6^_FG6MrnigSivw#So#0k0q5vNcMR>t><>V(QZ#{WBMjeUUjO`Y2Dx??3)vl-Gn|9QA(?_Ud@w+{Y}Wtzdk@NMjQ z5$oeUeNo-OD-(B9#fKI*1I1@;buN1B#qw_MUKskuy#P%%<-Nf(m%MR^$U3(ismeJ+ z>?^uj0|C>oErBVjOyeRfr9w zywVVwUX{xsFDPGt>(=Ep&~a~B6&7okg(Lq}N;WLqQyO5!_0;ni7MwaroZzRsBS|m4 z6lMu&?Zi$dZ7X1yZHOD&vzNo&E@vE;%*uI#d0jHA(DQ9(IL-weTmaoG2Wwz&-^Aq@ z@jcNHZzrsIgZiaw#-VZT)iymjMaxICgLLR!oSQ-45Pu=iEbb;+!rpsS}@yhlk_EWr8t_tYChBQ7%^dEcS=r z@5N^@vt)BLdX3-w9(h03DKPA}ZZXuWmaoRe4$Jl6abRr)7Mib}2#1-6-{Xnx;b@HB zA#NktVsuEsp3BGeFmlB4)p${PVj>!JJyC(5A^W?a`HlVJYInz7tuglHu2f8N*piFG zbGBHCvzVP{@Hu6tKX&(iz8Im;pDU;s_2@lp@*YK_na+!ec)agL1tROe)kDhsx2rLE z*U#3-lz*n8?UEl|@TJiYaRK7-hcj^W{NRt17e3`;+~`kMxbyi=DryhC(^_1nD!7M1 zlMA|_f6iThY&N@l2FB}3t?-~lX)d(rnu1Hmt}RA4=gZMpwDIzL%rr`cr)i7!0o;51sXP8*rt$off*DY+8;5rcEcIZ>tW4*qGd*2NYv* z9kb+kYZnyGXxybY58xB`k$Is|JG>zQ{((^jrKn^F8|i(h#En6WzOWBihlSq2ofvi`K@4O%dZZ=oLM7q(eYVK+vcVhlqgChyaJ6=+UF2qZ}rT z4~rb<(3$(tR*cs!EGle#2eBX)I1yf&<2j z_Wzl(7)1_h&f&-*=1LA~!O@bV6^Ap2i^05wHMSZ%eN9s?>S#3jn%Ww3{zE%SWz3B_LzF#D^Dm!!j#uo#EVqe6lr#!VQlswZwa72AVGTg5k@igv_yS6ibilr+`Eu&mJ& zW&Qt@4Mf@SKV>6PHvUi9M3j~Plu4AU|EFv!%4Yv5*AV5J|0&lJWicya+PW%Tv0yAk zPwI%0lZv-(l}cj^+uSUbwC}Ip)fLHGuS(TWV1&3zd<$kJn0+Z2unbpEgAc8XN8}x20(y zhGg+OY>*gF1cqowF3x1nt-XR0*+ z+Y_7^K<^EPTS5x>;J6Nv9r+rKkGCHReB~mDpRcy&3H&#WvV@*Sv(JFsZ4eL zu2ic`HKH=!S#4*bGor;`)6})I(EKx4&CbGDwEI`Jo*4e0r|6rixgDrXb^mThR+(1& zr*TPTs`pnVO!c{G(RPBYNx;|CKlLnkizTB{Rfq>KIUVu1CLRaHW36~h6ORD#@DLBX zzh+uY1*W2J39{7UuO%W{bXNbr|0i2h^*`GL{@OZ2Rkg%C^%U#7w|GJIQPoy?shX+! zsyeCqseD!aRkKw-s->y{s&%5YLp4x!K;@?@RSi)+Rt;4>Q~8S}HbSbd3X~jF!BP)Z zs1%|aCCyZYOB+;?(h*gZRG=C!6{#ltwY?>XZK}5FipD{_khH{9j%u2VA#E|OZ5l^0 zq$8%WOyeYmG-8?qH7&%Du9$K!O-nIUO-#R|rj;1d6TN7uaTY`Ryp8^|{qm5D*cFKN zCx%?bU&YoghFXgusI;YOd&DzmHo_dk^h?@#}E0SyqH;+Kl0 zEpPu~6X&(fgXaIUs{empSCds1swscXr`g{l|4+YEmZD$06EV@W|929hqE9@D`nKlA zHK!yV5ifXiW8?oB(EKyN_5b_&H4{rsrJ5<0$t;zkidUJ7wNp=(@Skbc|96`8H68w% zrbgrTKl_CaNq_C!R2^oTXjD1X`~WXMwPN7qr==Kr`DrUgUVb`?v6r7lG4b*<)>age zItq!@DAh&kDyAZ^p>_l2AO-0sI z>_ygBnu)Yl97NhE%|$j)97Q%%oJ2NKT8M0{v=rGyX(iHDaTaN(xQJ}3xQet_T8nI^ zv=Ql`v=!N0X(!TAX)n@A=^(O&;wG}C(otk9#a*Pc(n+L?(pjXd(nVxzrK`v`iigOy zN;i@16i<=umF^-tC_O~FDLqAYRCTDtI|)ThtgkUH^p0| zr{W{ByD~sz55-qxPi3IUUdkYmy%j%^eU!l>y_6v$`zk|4_EY>t_E(09^j3z8^if8L z9H0b<^i={y4pf3f4pM?e`Y9nI2P>f>hbSXO4pl~p^jE?}4pYKK4p$;Xj!+^+1}LLN z1}ae^gOq5I!AgwC5M_+WP-U#hk;*ucqm=O?!;}dk!GD$f?Ra zk<*mBW-5!BS;`V7URlb_R+cdd%5r9olE}pS)#0CmMZI+Wy%I-xw4T-R5mdyl+DaaC7D^Jq%f|?en>C84|KeJuQ zV0I`6n4QW&CRI7a>{1RhX-X!uTRFn)QI0Zul`LkTa*Rn=jx+m}Y$ij=VGbxKn1f0# zb4WSK99B*-naXMAh?2(~Rn9P3%30=^a*jE!oM*C?3rvo3kvXAUVse$s%t_@6b4t0& zoK~(edCGO>jB81lmh0wa+A5B++r>&h0G=8Hgj3I!(35{n5)WN=9*H>TvzTf zH=B{X~+)ekMs~zmNp>D@khh z8%bL1capT(A0+9pKS|QCzev($f0I;={X>$TP@HjbN+jvCGD!xkxIx5_Rg+}IYLR5j zYLjHb>X4+c8j^_BC8;`FjU-c6k0dizpQIYB0ZBDkLy~H-MkLi{jY%?RO-Qm}6_PAj zBB>5rous;~DM?nW8AK z+14a=VcU?@m2FFs2iuOMZftv!JlPH;b!Xj3>cMs-sVD1BQZKd>Nxj+5B=uptkmSX7 zC8;m#K~g`q8%h0HPm;XZ?j-rJJxChB_9V%d?M2c+wl_(G*ghorv0fw%X8V#fgzZP- zJCMTuB>A)6Bn@MINE*%#AZY~aOHu$kkfcC%5J^F-pSYok9ZXUPJA|ZAb|^_BS$~p7 zvBO9TV~3Lz&W<1{f(;-kk_{wjG#f-x6dO!ZG#f%v3>!+)7wY6G)oLP9$j- z8%t6=JBg&(>|~M>*eN87EdXjdq8%Wy8 zZX_v{-9*wZb~8z7Y%)o^*%XrYuv4rqbdcRe z(jhjDq{HlPk}}ynBpqS*l5~{aM^YA>PSP=UKS{^g43e_h10?0J2T3}?9wI51JxtO` zHj|`N>=BYqvqwqFW3xy)!yY5)EPI@!b8I$A=h+;RF0dy^y2$2|bcsDl(q;A(Nmtm@ zBwc0mNV>+JA?Z4MmZTf(Ig;|(^CT6p7f8CvUL@%jdx@k%_A*Jg*()U7VXu-@#9kxm zE_lFHa3k{+{nNqWK-lk}9m zN76I)K1t>51CpMz4@s(EOGtXbmXh?6eMHhLwv43L>|>JNuun*O%RVKkl6^+fJGPvp z_v~|$KCl%eePmydRK>m|=@a{kq|fYYlD@ESNczgYCFvVmNz!-r9Z5gf_ayyfKali` z{YcVpwu+=b>?e{{>}QfC_6x}}`;}y{-$+)o-$~YDe~_%r{v=t4{YA2d{Y|ni`-kLe zLiw+Ee_0|~k4rMi`YcE`VAUiWvRWh?vDzdXvpOW3uo{vTR+nUAtC3ut)g#%I)hF4E zH6XbLYe;fU)`;X+H({+vwq@&)Y{%9oxhZQ+vOQ}OL9xrj^tKsQ<9xody-w)W+c0^4kWi`o0HszbtJhh>qK%p zwgt)U*_I@CU|W&w#yXSSk#!;2opmL-6Wf~P&TJc!yRdCZ?#i|!*@JCQayPaE$)2nm z$=%tGB==z5N$$yZBDojandIJV7n1w1T}k$0JxK1$b|bkT>q&BdwmZq*Y!8xs*q$U0 zV0)45%l0ODAlrxJL97?aer#Wo2ebW19>Vq~c_`~mvOnuX@-TJ)$-`M+l1H!uNe*BK zksQeSksQPhCOMcLLUIT@l;lv>pX8D3Fp@{H!$}TfN01!O29O-V29g}f29Z3P4JJ8? z4Iw$24JA2-9ZB*Sb`;5D*)WpFvEd|-XCp|Sz($fhksVEPEE`4gBsQAl$!rYCQ`j*i zPi4oFJdGVk@^p4Q$#Luil4r0JNuJ5Zl01u@L~=YkndI5*6p|CzsU*)~r;$9Dolf#R zHjd=^>sE+cszyPV|pY$C}U*cBviWLJ{BiCsnVW_C5n$?O`E zQ`og6Z();2-pZ~cc^kW);V0V#xkWC}`5WAb?!|WcCGugc)A7S^Ae3VTmIg8y- z@-a4p5zJgyd7~QIb!yStRGN$4EZI9w+%Mn@#dL zHizW%>2(WKWWOi9JQ~W%e}5SJ*s~ud-)IzQ&#<`8s=!<@-y}U$>r=rlAp6BBv-JdB)?!Ek^GV^ zBl#8knB>>&6O!MsPf32uJ|nr3EhqUM`<&$WYz4_5*cT*!WM7h8#l9l>6Z@Lv&+HqL zzp!sf{>oO8{EdA_@^|(<$v@Z+B>!YTlKhLUBKbG_iR3@5_!5qa{X&q~uLPO>Mtm{Z zME*`tvp-0Djam4Upw0dw=&-*D8ukxCSBSsf{UH&maY-iVu^{NPYJvePzAlD z9fC2dA(*hb1cj|eAXbl1oz*9pvIYb*){szxH6qkxjS00_6GClPA(*p7uwbhbELl@R z9oCG*y-LCw1S_^Cp&na{P@k<$ux8B(Hmn7q0c%NU$krh=V(StbvsQ#AY(0W4Tc2RZ zS`(VG;#N_6wgI6T+mPVEHX?CPm#{Ixk!?b7Vr>a6SUd5Jc(y5_6>Cp$W}6XQSOeQ(3|Z_=)-ytyx4ApzN{yqAKP8rQp@%r zc(XkTK5Q?-0Jb;5m+eCs$a)b5v3*JWhJ~;nVKCdDFog9c3}t-?{_Ft4FxHnaoE=CQ z!44t>uzrL;ys_JCQJhjU~)vClO|`lL_(c6vAwFDj|WLMwr7+C(LEz2=mw(g!$}D z!UA>{VIdn&Sj5gIEM^l3OV~MtrR-e7GIkzeIXj<_$SxqPU>6crvWp0-*u~=4ci1I_ zHSAKtT6P&BiCs=u$0icivn#|mv)Pq|jqEDICU!MpGrNY6%&sM*ut|h1>^j0$c0FMm zyMeHs-ALHMZX)btHxp9XWWp{sg^A?#+i685m$2z%M>gnjG|LOQ#Xu%AsOWU#vk z2iP>iL3TIc5W9zPnB7arWcLw{u<3-O?0!NPn?X3n9v~cN4-&H3LxddmFyRE7Nyuf7 z5Kgj338&aB!fEyxi65d79w(e(vk7O}9Kt#F1mQfJOSr(EBwS=q5iYT(376SC!WH%m z;VOHUaE(1jxXzv@++Z&d^4W`o0`?N&CVQE1i@ic9WUmr#v)2fB*z1HM_6FfDn@=ca z3kdhvn}qx9Ey4q~knoVbO(GeL{H6J|(He_{3ZNzFwZOrPD+Jvn}YK7G!HL?1nuFe{e+LSdUwHa$f>Kd#uscW(( zq^`v(q^`{psmLzS`QroiDq_$&iNZpieKx%uoA*q|OjY#dlHYRm*wh5^n zSzA&&v38_x!8RpzOV*y$t=MLyc4i$&?ZP%EwJYmL>ej3isoStENZpogN$PfND^j;- zok`t+bs@DI>q_d5Y->`xvu#M-iET^j&TKnUcVXL;x+~j()E=xGsk^ZqN$ttHle#NJV5`Qc;_O`8nP1U zcUhTKQ~pM(C99BX%c`V0vKpzbtWK&YYmn;8-$@N*O;STyi_}QgCN-9INKIs2Qd3!v z)J)bVHJ1%YEo4JdOWBCjN;W37mQ6@)WK&XG*^JaqHYc^0El3??OHxPKiquKACUur= zNL^%GQdilI)J?W0b(bATJ!D5xPuYpoOLivpmR(4FWLHvO*^Sgsb|>|hJxBv&Ptriy zi!?~~CJmN-NJC^_(oortG)(p<4VME*BjiBRNI8f!N)9HCmP1HmHJJLE>vPPvJ+OKv9ZmRm@BZYS-RJ4grQPSQcSi*!it zCLNZ0NJr#e(owmObWH9i9hV14C*(oWNqLC$k338|C6AC!%cG<-@)+r?JWe_%Pms>b zlcay;Kcoxt6zM;CnsiZ~!7h_$u?+GYmQkL^E|>peNnXHA{)f4|h{YxCf9-D>uuMuB zu`A@|*p)J2S4qRJmX2K`1G`pc!mg87VAsnlu^Z%7*p2dP40#Q9le`wYSzd?TBCp48 zl{a9w$s4iTCD0km*aPw|>_K@q_K>^> zdsyC!JtFVJ9+mfFkI4tH$K`|A6Y?SKN%=7LlzaqxT0V+BBOk+_m5*c3$tSSq<&)S8 z@+s^^`84*Dd_hoB_K|!C`&hnF{0943evADgzr%i&-(#8O4_FrYBbHVE zgk_UIW7*{|SPuCsmQ!ZNa>*=MZkZL!BeP+7Wp*r|%z@>XIk5sV7gkW_#tO+iSYeqL zDo?_0jnk}V%23OtcI+N{VspQYRW2DEm;+-EvsR5WOb~rtbx^& zzhm`fO{{^eg*BA5u|~2E)>ziXn#g)sQ&}HtCL3VQWkalmY=pIxjj>j;3D#OR#oEYb zSXgKUL$l&!H&vJKW*w#B;0c34;09_uDMVBKX$tcUD`^^~2lUa||; zTXx0z$Zl9)*&XXAdtm)#Pi%nfg$hRA-{P}v_FCI?`{V?PaoAKj9-Ag7VAJJ9 zY=)eK&6Ja|S#kiVXNhGY>ixjt(AXZf5|_wb#f*4w_Js-m#eW2 zat*douEjRVzp%}69kxaOjct|dv2Ah#wq0(-cF0ZGPPrM|CAVO^h+_L)b}q82d*a!A{Ae*lBqT zJ0p){XXOd(oIHu0m;Ye@%2U_{c^dmqp2054v-oB59G*d*$1}=*@yq1}JjwrXlNWKn zr2nt|Jp&#}8SzZ=a{LOJ@GGU^S4qdOmVsX*GvU|DEAZ>&mH74YD*Og{HGZSK21j0t z-z2ZYZMaJzJk9c zU&Y^+ui@{=*YS7d8~A(jP5gcN7XE>J8~;$ggMTF7#Xpws;h)I&@lWLk_-FD%{B!vc z{)PM)|5AQ}egzd1Q7xugrnxlR5GHG8bMz z=Ee)kJa{3Q7cVUH;YDPAyr?XI7n23?;<6B4LKenL$|86vSrjiVi{WKtalEW7ftQmd z@$#}1UO|?|E6Or>C0Q1)EX(1)$?|v=SplyqE8^8;CA_+z$&16Hoxom{Dkd5({ zvI*WwHpN@ZW_TOf9B(UI;O%5fyuECNcaW{|jA2A1Vjn z!{k7GxEzF!kc07&atJ<34#h{yVfYw193Lx3;N#>-e7qcmPmrVWiE<1+Nsh%Q%W?P= zIUb)XC*afMM0~oOgwK$Z@tJZ8K1)u;XUl2$9624ID`(*Ii%j80Qxm<*=kc;s@PiqL9WF&%D?bUavi=|{*7;u>+!8}1HMge#J9^$_zt-l-zm4?yX01U zx7>#Bk=yaTatFRo?!@=YUHAdH8$T%b;D_X1{IJ}IACddl(!Oh$=is#Ls;h^OV_#53{<;#v75@tk~$cwRnDyda+;UX;%gFUjYKm*w-sEAj>6Rrw`@|>m z1L9NpA@P~~i1=K7Onf0fA-J8@typN_+EZZ{2;#}ew5!5 zKgsWipXK+&FY*WCSNS86S^h+1kv|hzi2U*;wX$UHL`7Mas3gk~m1TM2H(7zGA}bPAWhJ7TtV~pwzY#TL z72qPDC-)RDgvb!APWo~%XGm$iuovJTNu)+HLrdPHMcpJ*Z*5KU!6 zqM2+&G?$Hu7P1M^QZ^-8$!0`r*_>!2TM%t!OQM}@MYNZ#i4L+2(NVS~I>~lKXW5?U zB0CUWWk;f$>_l{zorxZ@3(-?{C3?wjL~q%h=p%a&ePvIgpX^2Sm%WJrvJWv(_9X_% ze#BtepBN$s5JTlaVwfC643~q65poDIQVu0X$zjB3Ih+_HM-XG>NMf8EMU0oDi3xHH zF;R{sCdqNcWI3LgA}0`2(#NToav0knvHpst-jdC5aN&Zc2mg|Wvas#ndZX~wJO~iJ& znb;w>5If~oVwc=T?3UY!J#q)JSMDVC$z8;Lxtlm3_Yeo=UgD74M;w;>i6im=aa0~8 zj>$vBae0_HA&(Fz?EnR_&I@g5nDhkOrf%Y9dIY--hkbE^KF;-7WgJOay!+ z{s4cJ@y@ou0-8XR&fU3tA7}14JHP+@_yu4Vqn5jA^rDzX0fvTB8eJ%+Fs|%G z3X{rC^}ximh8Yd_G|V!jcDYlKydfBJm2JMuO^3^xxfXb~tj#kdgSzb(rWwX7u2(a8 zlUsFR?(nT0Zdz`m;Yza=*p6?$Qm*y7CwNt*E!ffy78#a{7sxvcz}R3&l-(-9%T#PB z-n1HrblJ#6rxY_Zf>mib zu20Oo7D3&uYIvw)0FQL^Bc~&StlB=-@dR@WBd6(HY1vLycnrhuJEzNw!{g;`nb$Fo z1%}uhJw3&uh9w=#ct+fxoBF~G0YC2hc5_=;frb?Zm=WJzIk+r;QUfI)gYL{DYf$H`_&<9lGGi9N(h2Z(en?OI@?##~f%(FAI{q_6%PXIO43n<{S4jl_Ru3%9|2}?Tpu4=fZ zBagB&DjG};%M8YIDMhgE@h}u2!;t4Y4C4(yaIDbVtLM+h>b8njmuCja1 zY}9f|#d4gVA9A}TtOLGxz%AQvw|!}KBG(J87s|EO3IuPGu&m_mnD9SU48uWB4Z~zV zjdZOh7sSlKb{KSS+ae50)oPXDQTY@yD*Z~9IU*W;)9(bf*l^XYiz(@=O0NL}XAO*C zMZ4`wYeTJDqrwprU&wJGFO(rB4e6e41GiP3z#|IMb7+z z{9BVwv@V7}#pp5mo+A`+f&L+;F@{;P=deh*#>s0Ye3-x_S;UMfI%n~28Z(q%`OXu1 zs5ng6xtRuz>qKZAXwO@^pJDU@?6rZ$oQ%*WIjT#TbG;S88E1MzEREe~SkXdf=e`eO+5sM;%@GE; zftw8LP7ozRA4hMrmBy60m6MkrW0_l(X@>BBJuzIrxa;lsM8=k3^WU_4i{H5U<_zt) zG)8Lc1P9eaF)X>q`HL7$jT94XphwmJnD_Q!H#oTu^$)kD8y)$^9 z;T@jN*C3A#)GGs>O0Z2ar0xzm?^l<02_pxZ^zDt=z2r0z=N2 A$N&HU literal 0 HcmV?d00001 diff --git a/out/production/clients/org/apache/kafka/common/utils/Time.class b/out/production/clients/org/apache/kafka/common/utils/Time.class new file mode 100644 index 0000000000000000000000000000000000000000..f286a627113d1f947aa96a124f115ce3fcc562a8 GIT binary patch literal 193 zcmX|*IS#@w6h!A|B}>5>sFDUZoPi<@1&H1l6P!49l!e1lZ~zX4a3n-CPnvnfyg$zu zfF1G}X^f1JcS>tDN@2})B=|*H5i-M#{deK0rR)hoQI%&xdbIN(NKUGg+lD_l zVPl7y@t%vjWG%n7oC(`?*06a}dSr`9ST8Qq+|=xFX{4AU0>N{q06%aC5CD$hj}cgra6$Ia~NK0GZt|-1hkq;1zCIu6DN&L86m+QfGx4FBDf6Ay| zz?Z}y;EyuSwuP3U2_J4|cXnsynP+D2_n#lX04!n6goQ~H5An#rl!-VdQ^;zaGck>P z3I)9>4qyhe2IdUR8(3gSu6ojo))?Z2;yy!SLsbRCV8xSSyIDUF!7e}W$&#)phx_|H z@btc8Ohh#=WU$^yDS{23hoPWtzM=xx<_+%Dgni5pkGburdR9Ptm__PKQJ-C7`cwS3v4PBetFhRY(2?8<#QAjqSa4?1oP z=8kFxj(Fwiy37`u1s#IQ-jt57LQlF|BC4sXfs%zqJhpHXgBG4(iD9gpZ6^xo>Uz`j zt0G{S*y~i25$EM(Jei4FHDvM@p5mE-=N4XInIV>6v9N+=17!=Vc*&5v7HteeT?B6r zj)X%T6uYK2W!ZBq7tFrqvg&J?SBfFwpR|GUQN#1A(S!J02NAljNm-<7F zJ;a^kH(U{I^EwTJPS$0yQwl>WQtdUlYj%3EF!YHaP{iq_p}k52WxE(!lrTU~mEi`x z+ZcT`TC^LYl}(zF?wdIS``S7lCYjVm_6ccPYe%|(TNowHa2sPJwX;jFJ5D?sUV=@f}m^v`{V8Ams`sWGt6$g(f31etb_P67&taolMo&EYQY(d{_y eQ<@kaV1jgl{x}(o&?-%~0n$K_hxo?HY8Sl?#$bFeE0b_ zKK{o24-(NL?YsaTSC zyMnx%-zKAr<*_HoChyfb>^X!@LGEU&kbAgS9(_TM@FlWqUl91BGU}H{Odjzd5AfS% zl#oYK9x0vEL7K_u2Wbmms`LIJAK-&JU#9crI$sf_T0Y;;?~wPEvUHV<4h5-(FVy*J zKVRed$ok_qcy>PZ;)jI8cLw<|UmN6i@w?@5U63a7_40U+pWiF|-Qef<`T6|<<;Ea? zfNv5yKUl~&^N~XS5Pw)6AJO@vL0T)4IvS)_zD2%#Or&tD&L0obMp^kpklOi^LB5SY zB{PR~zFk=ObdW#8pOw+)bUqfOPMfpO%k&on*B1j2!W}`rlkXC7d`X1xWqI5!k9%ax z@gU#JU(xxiK{5q-D-R0k5K$|i2=EZ!7v%f-q&yywtzQfBgCdhwzEzH&()sH_>gR9B z@o&oGA(?(yMvnyeQGU$N-x4`}+s}^+4c`&TeOKr21^Eg7zRpiF8Evtcl~@xslSwPd zRJ>vHw$6*UY~G5;s?N@JTRWI4+AlFLH5=2ZNVG9w^;!oSx0tDvm5426(mL9iO6(0$ zGuGRe(#(QNcp6G2%Mtc@r(%2sF?X?nEDp(eYMN%!;Q&W2ZlXp$L+hUr~ z9*J2U>HfV|qSM?P#S9LI&FBSYA|l_NNpGqz0zzjiz~+D%?z0;AnLYc=#&Ep9KOR$b zCL6bd+9Ys!)2W`tOp|MNuC5)8Oa`7g_Ch4yxILCk4-CW;DXV*3EFA9!E9(w~t$|b| zF5LR|9!!apw4JM&#%)WP;eDIT0Yw-bOt74ZqE_}6B$ZgeWWqALfhE!#v%1@2DW-Dg zObqB(1N`cAPYYyB3LMagJDK3T6e8)cGk83CXs}se*V$7nvIQN+7G5LHVgWP*U%AVI^7aE+IGz@Z(NpnNiG+D@O#Lg9%U~X1G~i7P(lZbj z=8}=iAb4*#sMqO6fR4pe9qDKkKH(;O^WICWaB8XBft)E24T{_WAiFR*-+K=ZWIWi@ z*$H0?Y=aM&sdNGaOk393?Bd%2(NcqrwQ9338MUkd(T38x)(_j;&r}$*_P527DH8%k zNN|&^BHHK_&vc<`i$<+pGrB4qwvx%LWrMuDO$tRSVoDfrZ{jel-384?$~0-@+_F&1 zQkCkiW~w=ZOj#XOwt9a$f&Ugj_Y%L9Yc7Jw_leHTlnHBu;V47%CM~NQOa1sH@PT3J z_8Jc8y;f>NTr4zCtxl|Vqt1qJ-R$m$QNu{|elvw23!rBvf=%`kczYVfhK*=J3_?5Ko$`xK*IiQ-HXM>z)JT@n{pUQ)Bu zwd`1(Yi1O(v-0sKQwa!X;wTVc+evG{N|>oQgy!9;EO_M@CN8UQ5#=&RDk>4LiNkWk zDlfgo`LXU(~)SmO50vUbui=@w*t_H8DSF+Q)!bi0`4-f z*Me2I>0}PsYpDHn{sB|h7y^0|uy2v^Ks0W4Z%M=lps&y{clIUX`=zqU zbN;M_IZ0osBDYoagvgi6U-tekNx7m1Z2+szQRKfZsK!#+#Q@?ZK%oR-+Tn^x8OtNr1_#7mTTuRlb1-Wtih(RRgsg7ULYUf&KM)f&hDaxF( z5qcOfrSQ-lw5W#ipyrVRX(yG*W*rnT1*5wn#3m$ybQBHTPtYg97y9>YlRB?vwD9S6 zrVVDY4?97(&x%_85ZRr+*{Mua`zh1vF}NK8 z#6IC>U$AC)51OhrnKgg@e7ICi?Pv{^e_1%{v{b=u@pK|=i4qZewo4Z@Pue$wuBK}Y zdM6!*39Ek272_d{?VYWQvCKbXnx&Q@wKwaAU{^UP@a|Nk-^!Loh)Cx#lECm4Qi@Gt0tIzMCZFZow^(3cS|4SrU>{F;9wkKgM2JA;4E&q3`uG)~^v z`FVrBB&Ysh&~N0_3;d$aFB$wte%atxFhi&48wUT0Up4s8{1=1&%D>Y2ZwCLJ|6%Yy z<;cJI-!l3Siq9MZDSfw~G(afL@?y{v^!>bDLPjEy9*`z6(;Vvjnn6$Vf0?Rt;6^Vl zQG`iPJX-5|p7gOl-e@EY3=L+YH{EZ=Qq|!&4kyxKB>WWyy@RgQ zH8$uaO+)!-XdXyg^J+drE6{X9^J@Y4XS1ONwL%PbB6#g?24$L|YvT-Uyf(q0f70u^ zR%B?!S_x_ZLz^fdCW#a$DbAHs<Vi6mwj zJcTO_ZMHT?*X9~pjSySQ&ly@stAj0S^@i3UqZ;}KqO76K(;9I#lOa}|SvXu~X!EtR zb!~y6Ez-`>wI+ie(#|!s#o7`>Yu1+P+A@P{Ih1i4B|awz3m3}`twmT^p{+D{seoI> zuj<-rLtDe^nP$ujt=P45Ma|Cn4NJ_1%XT%)Yu+8I-L-o~vzppnS8Hf%wRMQN2sYL{ z)VQW9{tT^^UNGn?I>dC=TM&${tv9p{8oC)*AhN4(mxkIlpWz6KANvN%X*|$+<_slM zSGs}ZZZ`Pksp)VYbC*dsv z@XHJ=zupDW{mV&?d{< zhGL{1`y6PLIS2!EMJ>r+rAqoE0OC+^-KH&_J1%Z-+tvxY!mm{Ws!7WWRZh@$B%ol7 z{K>5l*5K5{psH--?`y5dt2VvZXaD3j#?|=9li;B?mp=ZFr_l^LNSEVxVGm7_Rsf?4 zDFN}GLTEMdo=YtKCZ5;gTO;0#w=Z;pw0khn=v{agsL2q$8)L)XK-bar_!gk|(0ef> zhkMD3{X}Ia$+II=;yppWyW3BazN5rncY*?^)WIMX0-Q&I9gjEBGR!Oo*cJtTGC&xZ zxq;rN_LtH7=|=4O06rk&Je_V*NI&Qx4Le9D0^uM9&qP>2`Ur#@F>@aIsXdF(??O0{ zZdL+v5U#fo9-$982p0jNh$t8uq{3?_k#W^s_~w>VmrG7T`Y?S2>j64KA9WzCa7Zo; zRSwelQ9RsELxw`mbdWQ_b3mR&{sV{9yU2N$A2@Vp zmc|lbCyGJ_{WO7MR7PLA^a-qg5+C97Hihg{4&@gDnP_-csPZuLLZDM`h0q=e z%F2aBc^GZ$x!lFlEU+X>1thxNVd>NK83)T+@+reGLeoXO*9}rpSI!oLih}DPjO$$x zjjWIi`Ye48kOFj!KJP%>3=0sh$Av1Z9gxLmfV>%YcSM1#R_MoNL5_3e4gd>N1FVI(^E;PBq;@qB5dt zU=*FZ74ikT%VBIk*b*PARf?$cTyayUJpu{YVUZeE6_w*Xl88Gg2wIslL&RDLLQHVNl#)rek?uK0`jjCm75Z&Iw+&!OfwsfP1R%Nv z>*8cS(t>C_uBUtGxC8Ml3xa^5BvDo@C!4< z_iQ9n)QN9`mWQCohr!b$2yzdBug4VP5McVKlI{g!5&um33hYGw&7`l=AkJyndk^M3 zimXENyiPtZPkIg819XCh9O7QaMgg$Jv&^$RbRSJ-YN?ufJFN;;-A82_HF-*{Qxoh zhal}oG>3jn^XMnEh<=Kw`7>HUKc{u{3~j)^4tiFxxd^mXAdcTh_k&?CZBz_vaF;c7 z60D20Y@r9}Yj|ro^`NpY(ZeJvc!LU=`7}-8aYe6@_8-OhDf+s@`68SaMF&G3#FvU8 znsT>-l1mZKD;jJa=RzOu`5Xv$0;0%nVbufv4#229N|W)b!e?sbo(50V5KTL#rlvP} zLk)ve-4!ZDT%Xa^AgF9jPlMwSy#nU`1h!tKY4jJwk-t)i{zhlhKeA$&1u=dT?1|?u z%sTVJtSv8ub=(3Wcp8;Ypgg;{=^` zDkMaCbGZx~yo^$tJuW$nr*F}>vF~wwY{|&XckoT54v8A{T?{n(9uVRh(-BoCW<*(m zX+t#Y7SDKi(GgE^rSLy!J?CfMD~>Ap{HR`HcO9# zbcyC0A^ka$IBRyCq*^c*DybW!ddxxr^@B7I*}L)7SnDNcyEZ9!;%9RK*9$`DJK|9x zkITr%wQiBP^sUuXJIi7Ux|^|9wD{u!8f1ttw?<GQmt9^e)9B(I`pc{M%H>*&wiO8>$QC3Bn7Y7b;Tmv+&w z;A4K;4ZS{#Q2;SHu0j%gn7#CCRR?(BG{3T`O7eyhAPK>wxRVdQ}q|BHT)k(Wx~ z<c#BfyFw8WJ+fbb5KE z4zdCWORqWzBM!pp`3Q^ad_Xuvi-%~*XlQoT z+LfV_=0RF|lxAiedztOn^(D(O0QW6$?_)H@S%ysN=`1<^|S zJEE>F3Iv*(3v4O;6Kb;q&1Q$3Kxl|o)D6+fk#JKkxB>b%{YSwKK!=9_H|PDb_Ft^o z-hLW9F+!aIp1Yj6;pc_X`mSTczmza1 zRN28?Wf;B^TDXcP@F6PatLZGhhU)p9bPivusFivG*kG^2c+1G9)=vx2L*a)Mly-Hj zAr*KO2etuWKdc_Ll3J5G*D;{iX@brLI_tA-?${5{46vBtwb*|@cFMW=CuuDr^tyK5 zbdp+kpiNu9sSd<%sDG47@oqat<2x|cW!!M8?rz902#-GtXKllt1E~?e54^n}e=ht1 z{4wrL;O}Ot;v?YkX8bAd5t_#zRy;~%22L)dLwNhKW+2T7#5p`3#jB>MpGKZxAn-u# zI)jTMVx5am!yUQ!HSCtswE%l>W297`$dfQ9JXwG)@GDQ!M(}hVGEDmp6GbCFm{H*hKT`oYd*q!~2-V6GgA2BoU}tmB|eLWm;9 z=_hH+jyg#1{Ei0T+B!(v8ctFtsMtP87pPjpjrI-zyAu)eE}F{6UD}_{RZ21d$WwV5 zkjQ8{R|9R3N_hs)1cw1Wi)T3mwBWl0l3=I~m0;HpU6^Ch_OS;PeA`-rS@=N*et>5q zZrk!}0VKg_9H$-d+=~EkXRf|i%{4!a!r&29b&o2jcKKoOTt#(&@k^Ei>kh=TAtuwkpsMZE)cNeDK=C00WLJ7p*MKJvy@IFD)_(|B+Q}_$-r|E3|0X6ZD zToPQIt;-f?>$1fv7l=168m%rXgFtJ>u1PN*DbQzJtY>Rd#xHOXAIy+77^f7JTk9Sp zeTeqJ3|mnMnc`e~Pf!?+=7C{!cfe4rW^aY}IQ0Z7kTH4(sjt~rSAkd_8Kg^Oecv%M znhRu5QE&=oBrU~CS`X;!sZG6YCk(+nh<^`jeh$v}JXP}xaH$s|oR?@Nzf5cS6z6CVSL0MkM^$<%F%&`IK+9y!41uoM8=YSiZ5 zdti|9qqGpOf%dvbX=dG~`UhJ)(v_h}n^a%wUC`{S@D0)14|`FQy^MW{u4CX)-r_zs zO$(6KNrO{p1?1Co!Y{;BtOcl4!~d+H71BJs7ir^Yi8h{=X%lFbR;=ikkXef|cR9B} zKg9@{D|n^ieKoJboL#zBkz z_vhzL*=xcj&d!RG)DWfbr9~x|4$=O5=|IWBA-e2dVv(bL+`i+RhoCSYy(}z>JFuql dX1W4z*d%r04I9&D&=zfGf&32@_iGC&@IUloR`>t_ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/KafkaStreaming.class b/out/production/stream/org/apache/kafka/streaming/KafkaStreaming.class new file mode 100644 index 0000000000000000000000000000000000000000..0ee57db387c9bdc0e81d729ceff66fa1bb156027 GIT binary patch literal 2626 zcma)8TUQfT7~LlqW)eoxcvsNaDng*7wbWJ+tYWli0Ba(6tq#c$h72=tW`a`t*k947 zE?;==#amsq@BJTr?X$}V>^_qa0!va~X3q8d_TJw=>_l?-=SKpDH1IhW5|5D?=6+r}K* z^L5*w)~%`$CSq{Owd<3Ul6&nbKJCP`-W@lmY2>2v-uzZF}0mBZPOnWFnT^LQr@XJ zq_}*(YFb6oYPGADi%3Xy&MXc0P76fFoFXZ7WKG+ctS-(PZeE|YI6}z^x;3r4rtH@T zg~=WMyh*BktukGa%p2*X*0VKy0;gK-TUTf^FG^DgPz&_8iqJ?}=g7k5e7#Vf&?^B4 z+E0N=^9jT=j|xV`H(7TD3$=2(J}uKDg=wo5RbZr5x^=)?lvF`4BNr^kqoyI_kw7xL zpfBlZOQ$2or4QU{#W#u#FvC0vbhChZreM|G5YIVPw_sc}Wo344RChpz(~!c5K(Dm< z7Vg`)OT!5y1yU_GY?6|eu&av$USQaPy5%*LVJTSDV8am@ZV~SPq|(rXUdf|RLl+JU zbg^)*PrD`q=*b$MWIc|8iiQVpHJp*mk2HyDSw=~>xZ5`xstO#<)3gcugNKI0UYFmKf!+a?s^6FW?%nv#B)D}#mc*Xe&Oyg zKH>8)6h6lpFM}8^;{dMU7_#`B@6(vX0B2L2J;ar1Vv--aA$*F@_?E4cTq%OqAif~J zQ*7@==p8pi6nv@RD+Q;y;4PIH;+$k672*+PtHvgbC}$>qMSDVfiESYebNDA5m(a6) zuSdwH^9gqKb)6iF9Ed!__JG8{E7D~{B5;*6h7{%E@;E6g>NvZ>pRUA_AwGeNxI}{@a9_bnf zaf-UManX`Qu)Dz_@-rs3BC)4NV&imYgX7*6#D=0Tuz#jju{E*>0 z3GAy^b(C;{5=oo%P#N8@=$MgXJZfe2k?IF5ZrFW?G&Q^LCy<4SQRJt`p?@{SC zX1rIW_o;Myi0)A7{VLt5(g#$!tC>DXA8Mg>^x+oz2z^wgkF|jFakhMSi0)y@y()d8 zneL-cHq)o*(<ly zyoumU#m7ZAizu;dBeU+Y6=_!@QReG9Lo>A#pUgC&KM^!qe(s7lZQ|b8-y`a+9 zLiD0Si+fW8dlHu^)Sh(QVwYagb0wqekX|^XcjX+*a%_Cq1+&=ID?a*+LbpQ8(wSH$ zp6yEvWgGgD%CYu<@ z#E16BlG*hw{Kby`B#bf>1%wBx-Nj7JZ38k}l@7uE{ zJ{0fE4#j&@Lr@uv4`l~qLzzS-ks6qA>dceO6UNs2QT8U|v4MSq*}ix(HY`WBZVtIn zUXw{>kcfsGnN)UgDAgNJrxjW`LnJg-9Y_rHr4D2>iT=2}Ok^5gOxAc7*6$z8#;?i@ z#d!)2@gQXVdYaL@b%O8d;jAjWX_wpFqyqeepT<&+zq6Fk{W` zR4Ri4#|C+o{qdnm*H@*7)0sHx9!GpYXPm3vR6WKO`GGn zMp`cDBBDaNTSNpaW)Ul-_TtT9{ zEQPksY<^|au)QjGKaoc6e^Ne?W5Z@tMD?2IM7dR(9?D$M%T*&pFNf(H^i74?( z?y(Xd`SMK5K>xKWQR6Vy%z2J`qF8}ZQJ1c%1p?@h;G*G1Efk??`?hUZjeW=)lU^Vp zz5_BdM6ZPDJM`T!eUDxZ(f7mj1NtEX;2K0tC+b<1yomsQL_ZGG^|VK!wi$_-;}qiR zjlwg8=NVC03DHl&^fP)rL_ZJHFX>kboz-W$0IcKIwK6s`S3xT{t|nCQ*Kv(#n0`&a zf$x3+qo(Z{F5*C>Vfrl_`(2oRPj7_j4{Y(0Ov%s+u5n|{uyb{7%=AjySjo(lG|$nh z`lz9Km{4)5qU4%W@M+h{8P%%eMimNGoU(8Ik8E%k8`LeQW_w~b>ZO^sW|`%(Sv7#Y ze3ea)!2d(qm=3dcQN&={m6`{85xZtFo`998SHedgj{k{XSLi%84Jq20?}!kxHRG_6 zt9b@8z-k(*&hEi7>}?pS6$*xH}UhP*ZS2lFJRsnX(dR+i2JHH%o^xBAl0vZ~W4Ur1Jlw&2s*7V8Mw31#$4vW0Y z@>qI}=Xj`TBO>hAAm$MLnaleZ;4Xd`=Jf)~FC_%!Ghs=$>smt@=p8fdyfYT%%_Rc3 zIc<2t8!;OR)-~I)J8h!~%mEW(UjQuSvX;|p)c_<7P;ko76Co;>jl7R3VQ3O(#wg-5 z1(*;#JhB}P8@s9(8Nh$#YNyf@{Xol=(Q5=sut^bu?rEk~DI45-04LP1Cx;Vn*Q81h z+8h{w9^xXxg|+;mIhTm5uv4t{*t}6xtcwe8o*Q zxR**-$AA;ucjZZV3%G^_z$mnN78|o_ER$Zgb?gh=7n>aRVwJz7>EObI)|nRPDQ?HD zp0n4p%E5zSW#g_aU#oFB6IQ3{^8J7#^A%^yivd-PoY<4N$&h@a%uwH%GjE}fytm`GZW>TZ*Yic zVKT+#SUw5S1kP~W)d3}O^rZ^?u1d(j4rZ|S-o#~lf#ziD0RG}znBZ>ie~puLZ0zL~ zZI{c+%!KA5@#BNc2|7Qxe?)`JhC1LVl%0MY6eIeurBzh_2S4s3P$cH7m74+2-y#(|%5*-13twgT?eVs&K z0{eLGA=itL^DNzfoa>;{)PxaHbNe_cTaVI$r$98(2>!MR!GomYThbip)&4}vlK~1U zRG=b^l`szD)908N9mIG69m1$g7Dl|#y}?2|+m8VKTO*4ii;q*Ni4JUysFCKzu=O~F z7cg&0W3V(BJP8}VUmGZd@k0pYVOmCy&}sB2ZKB7hlOCrX^n~Q=&cN42blx;yeU3fh ztAkNx!q?jaUpoU|mqnI09E>#lTG1HHbZ{LlhJ)j9@L4!`43#)eXVP=Dm0pmX>zr_| zZH9Bqk)3O%8_9)>oXBn@l#{(=oK~V{M`+a%TFq5kAXV)pRODr`m|vMCR3#5{RI1Ua zZ%7#tZW8?`a_TW!^E9n}p4JT?rPCs(kJI{bT02e~B4->SEwXW(&RkyGc8ty%r%jQw znY9_LWqH9lC(_28E#uU_FkozDMh6(noBk27I++!PR-`N7Y-7%La8}Z}$LRcVIwx`g zvvx?V4Ur3()jCc)dFhLe(8UqNEr@O=A_39E#EO7;8xufgc5Pj=7acdm%(Neylh0;2ustZB_DOClkKq?JTDJTaxZJe#Mjblf)Q^9}Y z!UZG_sAql@V%Bk0LrM;w*`0mw&Cc`w`u*c4fG4m`lyJwuT?5M|idZpl&%k{Hs|=+b z8OUUpp48+^fypiIYh1p$t4neB#5AFJkAQoz`B5-PK1;h9V!0{Lc)w-8$wc z-0?Yq2|}D^uCgB_!V^)m@qunU5JafwPBT2Iop66lw_eB{o>3Xr^F7{mkA<7$dt?>W zV!kcCfG6XKw7i_w8EM9#gUQU!VI+Fe<%!tI@&7;AE_q?L!*S$_S5g^YIX7xcIYX6~ zf$N7cIrBy&{jg`?frT}!TbRQkDEl_?t- zwr0RDXiATA$YsE=WPhBtu+hObJ?Rn{40`MZ`p5;^nY3~!Gs?w{?_ghPq2Urm1qPnM zqG(M3D5F9<8V@y!iu|0ilGF4rD(0BYl({@P&6o;N$3m*6VG&oLRbPQb6e-i(e*lU7^#lL_ literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$2.class b/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$2.class new file mode 100644 index 0000000000000000000000000000000000000000..812932cbaa85d562c888d47a6968e03d568d803c GIT binary patch literal 1423 zcmb_cZEw<06n-ugIuPVVH{W%->3EakG-|@K#4MU@W9|bB$>OIAyFy24Nm~}@uhK7) zxJ3Qzk20QHYN#-qEU`&@&wYB%^PGFmX@CCu`VGJ{)RIWxzKRDbR+5NfHHiows(6&d z0@hR%R1{T|7~&hcsk@sD(L%Aspln%3f+5q;P4RX#Xbby*w+%w_4XeYA7Pob|_d&(& z=?+6`+cbr}WpKw4v|Vplc30y=-suUg&)@gC=D4=t1KsRu;sYNH4dH0JO%L0#+NfPlxMpy2CKrmS&m+Y!iT*@Q+ZCrRZj1!6NBe@f^YcU`d?I}q5>35p za(85t{qg@Nu`#b5K(cD;3LIAv=1?Sp9e2es6Fa|u){O` z(~^)D&(=SjaC&rk_+-->*&Xplmot}{&PY`b1tG7^j$t`86nnz$Sw||$DOB({g-b{= zlrNAKLvA`~eiD+VDjym{cKUX5<`k~rDnl(qP~fw}M-C_nba;G_lcwIKfQ32dQ8Ka&KgYxK{F(oIdEU79S7Rz_>AGzR-j zMg*7X8TY`a$dD~v$pKm9X=SpyC`yW?OXVMkSLpr%bygslY1$W1e7ak>?a|)B TU0M~~!!k*OiovKX@>=)}hP8nS literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$3.class b/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$3.class new file mode 100644 index 0000000000000000000000000000000000000000..ed441ad54a5161704aa4438939c0339f7dfab8d0 GIT binary patch literal 1076 zcmaJ=O>fgc6r7EnxD5_TQz+k1N})*_>I#rJq^gP{kOCRPa~$ z16;UpDG~?NGd~LP?6|6-Z4U9<9lx2`9l!kk^Ya&gCwN$dhUFqu+|{t6;a(91tZKNg z;em!VhQgL~Wwgzp)arc(we1ZBL%Azm@p>{Ih+vNo973vH&*V;@2QuELpc;*2$e?#z zR|IW`hoPYDqplZN2KTu+5{At`+S~}EK=842Ekk_bW8V>>@v4_#FTKI);}%1n`@Zv; zp;qhe^8;=;oSLbHIIRrQaU)@gpjCfQC$W<~g%|qn! z{ugNKSE$N2f|QI*JCRwQwx%*f!Xhpu(q&x65=mVlULLv94y!yPm#eAEQzFJLrN;UZ pl;#oCL*!$R^Nw0dJycvvx~}6!Qg7lGRTZ~!huo25P@oFT{RK=U0(1ZX literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$4.class b/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob$4.class new file mode 100644 index 0000000000000000000000000000000000000000..531928937226d4988b0db216b783b0e80ac55a2f GIT binary patch literal 1009 zcmaJfgc5Pj=7aqBwdqfow~Kxv&))fKA5Ayoycs1!&6DJTaxZJfo~#<3&Yso+0x z;Q|r|)H6S-V%Bk0LrM;w*`0mw&Cc`wKKuC#z%FbPCEPP`-@vMgA|4ocXyB27HHOlj z3}mv;P_P?a2K^xH3x-Nt2I9?R)DzKr-t(zaZ-*}TyF8L=pB43FAY%rr6$BzW@Oc~y z+CFWEk>~I+cL%~5@=rtV#7QLhNCuuGKJ(Go7qRoYlh$5^y|vvYLy?ci{uc(@ZXfd# z?)aR*1R+i{S7`-_@I=&Xe4rZ-1rh4G(+p2)C)}UVt(S9$=TwG`e2=%iW8o(G9$7`T zSnNnI;K?K+Eib2aPMS05a5}T|E)sp|@VAz_YGGhb7 z_8j;nP3cjNxC|Ip?2oe+HoDlMCtU)AL65yaAGts~lU5F8M!C581MC|uG%Qh6VBioI zMQaK`85P>mc&JfS##D$pmQytiS8)wm^#!S|WNOa{t<=h! sPfnq2oucp^Gs5{Rl`~--*VFbla5K%faGSD@JGhH0v@)qdkuuHw2e0k*Gynhq literal 0 HcmV?d00001 diff --git a/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob.class b/out/production/stream/org/apache/kafka/streaming/examples/KStreamJob.class new file mode 100644 index 0000000000000000000000000000000000000000..0627958ac3024a8520b4e1780276369baee076f0 GIT binary patch literal 3323 zcmb_fYje{^6g~1w#ux%hfrhpeD33fyOu{>%p%9=vlJIaE>W0$F^2SjjTdpLh34K6c z{FQ!gKafd1^kawVO#72MJ!@G`oMhB^rkRO$wR_LGchBCvTL1aaZ@&XLi=R^H#e|B< z6e6;{m&E-f9_++JWRsXu@lg`fDjp^Av5H(0pCmD(;&BQwe5&F}3YtWHmc-|>tEX=**dL!yVnG^PNt9HSQ#b)zJ}N4{P~j*@-!7Mh zGiK?oD_jMM%Vyd1MioT6d!`k{#_WPn&@pb7#bmXV7tWNPw>YAW+lFpU>y9bk8-p=# z&Lqd_aoZ_o^onlGiOhmNyP#)W&k=gbEEh9kNiS6_;b!h;>tlE9{0RM&bhE5rfA`Gz zyuPSsES;t_AY!B*hwc=~+_}ZHf_TNTE1d1{u~pBsG81}*v8MtZyKiK1}o ztW)i193Ja>+SH9DLsUG|F1sqYm8q;~~t~Y$?^raycUK&*_a$-R&OEXiq z%%?Kp&u?;N*rk$P&ICrfTu$R`X4e7tEb3NOv~XS69#=!L7BU%DK?FW!U>CPX=BLW_ z7O3Wj*2>rEnI++RtVRQ3iE0_1O)pnAd7+XUOV=aavJ5J^FcnbT2pHrheyLzEgl0qL zH>!q$JsT?@>z8L&Ok*&OB`mY9hLCPiR*!L;pQiB@zHT2%m*3!9mah<(!8-0cqF@@j z$CPe2!ZYnd&ZhAU&jV?O()bSF2XCb72?@Bh;>n%*A&np9nHvZ}ZJ}(-Uik%&-=?+g zYo2-qyVg!~{Uk>#Rgbycy}4p;9_iUU#1C%++WgAZKaaDHT~cs$%g<*U1(WqPtkeWM z-Y8tWB$Q-z(!00iLM%_q3PBkvthznh!uZ9FVI^iw%adi`@wS{bbFytec}FiBa|)hN z%aE)Hq*&)@TlNj<~sb!{gzWmjy=E-nYxkFc