From e46d649c2e40078ed161c83fdc1690456f09f43a Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 9 Dec 2015 20:31:25 -0800 Subject: [PATCH 01/14] v1 --- .../apache/kafka/streams/kstream/KStream.java | 35 ++- ...StreamWindowed.java => SlidingWindow.java} | 45 ++- .../kstream/SlidingWindowSupplier.java | 266 ------------------ .../streams/kstream/TransformerSupplier.java | 1 - .../apache/kafka/streams/kstream/Window.java | 36 ++- .../kafka/streams/kstream/WindowDef.java | 58 ++++ .../kafka/streams/kstream/WindowSupplier.java | 25 -- .../kstream/internals/KStreamImpl.java | 26 +- .../kstream/internals/KStreamWindow.java | 68 ----- .../internals/KStreamWindowedImpl.java | 67 ----- .../kstream/internals/KStreamJoinTest.java | 195 ------------- .../internals/KStreamWindowedTest.java | 91 ------ .../apache/kafka/test/UnlimitedWindowDef.java | 4 +- 13 files changed, 151 insertions(+), 766 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/kstream/{KStreamWindowed.java => SlidingWindow.java} (51%) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowSupplier.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/WindowSupplier.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java 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 d3931efc9f4ad..51d383ce60219 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 @@ -83,14 +83,6 @@ public interface KStream { */ KStream flatMapValues(ValueMapper> processor); - /** - * Creates a new windowed stream using a specified window instance. - * - * @param windowDef the instance of Window - * @return the windowed stream - */ - KStreamWindowed with(WindowSupplier windowDef); - /** * Creates an array of streams from this stream. Each stream in the array corresponds to a predicate in * supplied predicates in the same order. Predicates are evaluated in order. An element is streamed to @@ -149,7 +141,7 @@ public interface KStream { /** * Applies a stateful transformation to all elements in this stream. * - * @param transformerSupplier the class of TransformerDef + * @param transformerSupplier the class of valueTransformerSupplier * @param stateStoreNames the names of the state store used by the processor * @return the instance of KStream that contains transformed keys and values */ @@ -158,7 +150,7 @@ public interface KStream { /** * Applies a stateful transformation to all values in this stream. * - * @param valueTransformerSupplier the class of TransformerDef + * @param valueTransformerSupplier the class of valueTransformerSupplier * @param stateStoreNames the names of the state store used by the processor * @return the instance of KStream that contains the keys and transformed values */ @@ -177,9 +169,30 @@ public interface KStream { * * @param ktable the instance of KTable joined with this stream * @param joiner ValueJoiner - * @param the value type of the other stream + * @param the value type of the table * @param the value type of the new stream */ KStream leftJoin(KTable ktable, ValueJoiner joiner); + /** + * Combines values of this stream with another KStream on a window basis using Join. + * + * @param kstream the instance of KStream joined with this stream + * @param joiner ValueJoiner + * @param windowDef the window definition + * @param the value type of the other stream + * @param the value type of the new stream + * @param the window definition type + */ + > KStream join(KStream kstream, ValueJoiner joiner, W windowDef); + + /** + * Aggregate values of this stream by key on a window basis. + * + * @param aggregateSupplier the class of aggregateSupplier + * @param windowDef the window definition + * @param the value type of the aggregated table + * @param the window definition type + */ + > WindowedKTable aggregateByKey(AggregateSupplier aggregateSupplier, W windowDef); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java similarity index 51% rename from streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java index 4d73128839f21..33c2e35e7892a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java @@ -17,22 +17,39 @@ package org.apache.kafka.streams.kstream; -/** - * KStreamWindowed is an abstraction of a stream of key-value pairs with a window. - */ -public interface KStreamWindowed extends KStream { + +import java.util.Collection; +import java.util.Collections; + +public class SlidingWindow extends WindowDef { + + private long size; + + private SlidingWindow(long size) { + this.size = size; + } /** - * 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 + * Returns a half-interval sliding window definition with the window size in milliseconds */ - KStream join(KStreamWindowed other, ValueJoiner joiner); + public static SlidingWindow of(long size) { + return new SlidingWindow(size); + } + + @Override + public Collection windowsFor(long timestamp) { + // TODO + return Collections.EMPTY_LIST; + } + + + @Override + public boolean equals(WindowDef other) { + if (!other.getClass().equals(SlidingWindow.class)) + return false; + + SlidingWindow otherWindow = (SlidingWindow) other; + return this.size == otherWindow.size; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowSupplier.java deleted file mode 100644 index 80e548f94c704..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowSupplier.java +++ /dev/null @@ -1,266 +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; - -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streams.kstream.internals.FilteredIterator; -import org.apache.kafka.streams.kstream.internals.WindowSupport; -import org.apache.kafka.streams.processor.StateRestoreCallback; -import org.apache.kafka.streams.processor.internals.RecordCollector; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.internals.Stamped; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.Map; - -public class SlidingWindowSupplier implements WindowSupplier { - private final String name; - private final long duration; - private final int maxCount; - private final Serializer keySerializer; - private final Serializer valueSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; - - public SlidingWindowSupplier( - 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 get() { - return new SlidingWindow(); - } - - public class SlidingWindow extends WindowSupport implements Window { - private final Object lock = new Object(); - private ProcessorContext context; - private int partition; - 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; - this.partition = context.id().partition; - SlidingWindowRegistryCallback restoreFunc = new SlidingWindowRegistryCallback(); - context.register(this, true, restoreFunc); - - for (ValueList valueList : map.values()) { - valueList.clearDirtyValues(); - } - this.slotNum = restoreFunc.slotNum; - } - - @Override - public Iterator findAfter(K key, final long timestamp) { - return find(key, timestamp, timestamp + duration); - } - - @Override - public Iterator findBefore(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp); - } - - @Override - public Iterator find(K key, final long timestamp) { - return find(key, timestamp - duration, timestamp + duration); - } - - /* - * finds items in the window between startTime and endTime (both inclusive) - */ - private Iterator find(K key, final long startTime, final long endTime) { - final ValueList values = map.get(key); - - if (values == null) { - return Collections.emptyIterator(); - } else { - return new FilteredIterator>(values.iterator()) { - @Override - protected V filter(Value item) { - if (startTime <= item.timestamp && item.timestamp <= endTime) - return item.value; - else - return null; - } - }; - } - } - - @Override - public void put(K key, V value, long timestamp) { - synchronized (lock) { - slotNum++; - - list.offerLast(key); - - ValueList values = map.get(key); - if (values == null) { - values = new ValueList<>(); - map.put(key, values); - } - - values.add(slotNum, value, timestamp); - } - evictExcess(); - evictExpired(timestamp - duration); - } - - private void evictExcess() { - while (list.size() > maxCount) { - K oldestKey = list.pollFirst(); - - ValueList values = map.get(oldestKey); - values.removeFirst(); - - if (values.isEmpty()) map.remove(oldestKey); - } - } - - private void evictExpired(long cutoffTime) { - while (true) { - K oldestKey = list.peekFirst(); - - ValueList values = map.get(oldestKey); - Stamped oldestValue = values.first(); - - if (oldestValue.timestamp < cutoffTime) { - list.pollFirst(); - values.removeFirst(); - - if (values.isEmpty()) map.remove(oldestKey); - } else { - break; - } - } - } - - @Override - public String name() { - return name; - } - - @Override - public void flush() { - IntegerSerializer intSerializer = new IntegerSerializer(); - ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); - - RecordCollector collector = ((RecordCollector.Supplier) 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, partition, 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 SlidingWindowRegistryCallback implements StateRestoreCallback { - - final IntegerDeserializer intDeserializer; - int slotNum = 0; - - SlidingWindowRegistryCallback() { - intDeserializer = new IntegerDeserializer(); - } - - @Override - public void restore(byte[] slot, byte[] bytes) { - - slotNum = intDeserializer.deserialize("", slot); - - int offset = 0; - // timestamp - long timestamp = getLong(bytes, offset); - offset += 8; - // key - int length = getInt(bytes, offset); - offset += 4; - K key = deserialize(bytes, offset, length, name, keyDeserializer); - offset += length; - // value - length = getInt(bytes, offset); - offset += 4; - V value = deserialize(bytes, offset, length, name, valueDeserializer); - - put(key, value, timestamp); - } - } - } - -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java index 2c2d8dd28c924..93d930df93b2f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java @@ -20,5 +20,4 @@ public interface TransformerSupplier { Transformer get(); - } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index a1456f6c421b6..b1827f9a4e59f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -17,20 +17,26 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.StateStore; -import java.util.Iterator; - -public interface Window extends StateStore { - - void init(ProcessorContext context); - - Iterator find(K key, long timestamp); - - Iterator findAfter(K key, long timestamp); - - Iterator findBefore(K key, long timestamp); - - void put(K key, V value, long timestamp); +public abstract class Window implements Comparable { + + /** + * Returns the start timestamp of this window, inclusive + */ + public abstract long start(); + + /** + * Returns the end timestamp of this window, exclusive + */ + public abstract long end(); + + @Override + public int compareTo(Window other) { + if (this.start() > other.start()) + return 1; + else if (this.start() < other.start()) + return -1; + else + return 0; + } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java new file mode 100644 index 0000000000000..b3c0cfab7c7e2 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream; + +import java.util.Collection; + +public abstract class WindowDef { + + private static long DEFAULT_EMIT_DURATION = 1000L; + + private static long DEFAULT_MAINTAIN_DURATION = 24 * 60 * 60 * 1000L; // one day + + private long emitDuration; + + private long maintainDuration; + + abstract public boolean equals(WindowDef other); + + abstract public Collection windowsFor(long timestamp); + + protected WindowDef() { + this.emitDuration = DEFAULT_EMIT_DURATION; + this.maintainDuration = DEFAULT_MAINTAIN_DURATION; + } + + /** + * Set the window emit duration in milliseconds of system time + */ + public WindowDef emit(long duration) { + this.emitDuration = duration; + + return this; + } + + /** + * Set the window maintain duration in milliseconds of system time + */ + WindowDef until(long duration) { + this.maintainDuration = duration; + + return this; + } +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowSupplier.java deleted file mode 100644 index 46a2b9ee91622..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowSupplier.java +++ /dev/null @@ -1,25 +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; - -public interface WindowSupplier { - - String name(); - - Window get(); -} \ No newline at end of file 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 67a2d27c4c0f5..8898169c93621 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 @@ -19,6 +19,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.AggregateSupplier; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValue; @@ -26,11 +27,12 @@ import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueTransformerSupplier; import org.apache.kafka.streams.kstream.KStream; -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.ValueMapper; -import org.apache.kafka.streams.kstream.WindowSupplier; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.WindowDef; +import org.apache.kafka.streams.kstream.WindowedKTable; import org.apache.kafka.streams.processor.ProcessorSupplier; import java.lang.reflect.Array; @@ -131,15 +133,6 @@ public KStream flatMapValues(ValueMapper> mapper) { return new KStreamImpl<>(topology, name, sourceNodes); } - @Override - public KStreamWindowed with(WindowSupplier windowSupplier) { - String name = topology.newName(WINDOWED_NAME); - - topology.addProcessor(name, new KStreamWindow<>(windowSupplier), this.name); - - return new KStreamWindowedImpl<>(topology, name, sourceNodes, windowSupplier); - } - @Override @SuppressWarnings("unchecked") public KStream[] branch(Predicate... predicates) { @@ -251,4 +244,15 @@ public KStream leftJoin(KTable other, ValueJoiner return new KStreamImpl<>(topology, name, allSourceNodes); } + @Override + public > KStream join(KStream kstream, ValueJoiner joiner, W windowDef) { + // TODO + return null; + } + + @Override + public > WindowedKTable aggregateByKey(AggregateSupplier aggregateSupplier, W windowDef) { + // TODO + return null; + } } 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 deleted file mode 100644 index 29239364a8093..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java +++ /dev/null @@ -1,68 +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.kstream.Window; -import org.apache.kafka.streams.kstream.WindowSupplier; -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.ProcessorSupplier; - -public class KStreamWindow implements ProcessorSupplier { - - private final WindowSupplier windowSupplier; - - KStreamWindow(WindowSupplier windowSupplier) { - this.windowSupplier = windowSupplier; - } - - public WindowSupplier window() { - return windowSupplier; - } - - @Override - public Processor get() { - return new KStreamWindowProcessor(); - } - - private class KStreamWindowProcessor extends AbstractProcessor { - - private Window window; - - @Override - public void init(ProcessorContext context) { - super.init(context); - this.window = windowSupplier.get(); - this.window.init(context); - } - - @Override - public void process(K key, V value) { - synchronized (this) { - window.put(key, value, context().timestamp()); - context().forward(key, value); - } - } - - @Override - public void close() { - window.close(); - } - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java deleted file mode 100644 index c71c11b774583..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.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.streams.kstream.internals; - -import org.apache.kafka.common.KafkaException; -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.ValueJoiner; -import org.apache.kafka.streams.kstream.WindowSupplier; - -import java.util.HashSet; -import java.util.Set; - -public final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { - - private final WindowSupplier windowSupplier; - - public KStreamWindowedImpl(KStreamBuilder topology, String name, Set sourceNodes, WindowSupplier windowSupplier) { - super(topology, name, sourceNodes); - this.windowSupplier = windowSupplier; - } - - @Override - public KStream join(KStreamWindowed other, ValueJoiner valueJoiner) { - String thisWindowName = this.windowSupplier.name(); - String otherWindowName = ((KStreamWindowedImpl) other).windowSupplier.name(); - Set thisSourceNodes = this.sourceNodes; - Set otherSourceNodes = ((KStreamWindowedImpl) other).sourceNodes; - - if (thisSourceNodes == null || otherSourceNodes == null) - throw new KafkaException("not joinable"); - - Set allSourceNodes = new HashSet<>(sourceNodes); - allSourceNodes.addAll(((KStreamWindowedImpl) other).sourceNodes); - - KStreamJoin joinThis = new KStreamJoin<>(otherWindowName, valueJoiner); - KStreamJoin joinOther = new KStreamJoin<>(thisWindowName, reverseJoiner(valueJoiner)); - KStreamPassThrough joinMerge = new KStreamPassThrough<>(); - - String joinThisName = topology.newName(JOINTHIS_NAME); - String joinOtherName = topology.newName(JOINOTHER_NAME); - String joinMergeName = topology.newName(MERGE_NAME); - - topology.addProcessor(joinThisName, joinThis, this.name); - topology.addProcessor(joinOtherName, joinOther, ((KStreamImpl) other).name); - topology.addProcessor(joinMergeName, joinMerge, joinThisName, joinOtherName); - topology.copartitionSources(allSourceNodes); - - return new KStreamImpl<>(topology, joinMergeName, allSourceNodes); - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java deleted file mode 100644 index 12bed1788fa35..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java +++ /dev/null @@ -1,195 +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.common.KafkaException; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KStreamBuilder; -import org.apache.kafka.streams.kstream.KStreamWindowed; -import org.apache.kafka.streams.kstream.KeyValue; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.kstream.ValueMapper; -import org.apache.kafka.test.KStreamTestDriver; -import org.apache.kafka.test.MockProcessorSupplier; -import org.apache.kafka.test.UnlimitedWindowDef; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collection; -import java.util.HashSet; -import java.util.Set; - -import static org.junit.Assert.assertEquals; - -public class KStreamJoinTest { - - private String topic1 = "topic1"; - private String topic2 = "topic2"; - private String dummyTopic = "dummyTopic"; - - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - private ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; - } - }; - - private ValueMapper valueMapper = new ValueMapper() { - @Override - public String apply(String value) { - return "#" + value; - } - }; - - private ValueMapper> valueMapper2 = new ValueMapper>() { - @Override - public Iterable apply(String value) { - return (Iterable) Utils.mkSet(value); - } - }; - - private KeyValueMapper> keyValueMapper = - new KeyValueMapper>() { - @Override - public KeyValue apply(Integer key, String value) { - return KeyValue.pair(key, value); - } - }; - - KeyValueMapper>> keyValueMapper2 = - new KeyValueMapper>>() { - @Override - public KeyValue> apply(Integer key, String value) { - return KeyValue.pair(key, (Iterable) Utils.mkSet(value)); - } - }; - - - @Test - public void testJoin() { - KStreamBuilder builder = new KStreamBuilder(); - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - - KStream stream1; - KStream stream2; - KStream dummyStream; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessorSupplier processor; - String[] expected; - - processor = new MockProcessorSupplier<>(); - stream1 = builder.from(keyDeserializer, valDeserializer, topic1); - stream2 = builder.from(keyDeserializer, valDeserializer, topic2); - dummyStream = builder.from(keyDeserializer, valDeserializer, dummyTopic); - windowed1 = stream1.with(new UnlimitedWindowDef("window1")); - windowed2 = stream2.with(new UnlimitedWindowDef("window2")); - - windowed1.join(windowed2, joiner).process(processor); - - Collection> copartitionGroups = builder.copartitionGroups(); - - assertEquals(1, copartitionGroups.size()); - assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - - KStreamTestDriver driver = new KStreamTestDriver(builder); - driver.setTime(0L); - - // push two items to the main stream. the other stream's window is empty - - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } - - assertEquals(0, processor.processed.size()); - - // push two items to the other stream. the main stream's window has two items - - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } - - assertEquals(2, processor.processed.size()); - - expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; - - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); - } - - processor.processed.clear(); - - // push all items to the main stream. this should produce two items. - - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } - - assertEquals(2, processor.processed.size()); - - expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; - - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); - } - - processor.processed.clear(); - - // there will be previous two items + all items in the main stream's window, thus two are duplicates. - - // push all items to the other stream. this should produce 6 items - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } - - assertEquals(6, processor.processed.size()); - - expected = new String[]{"0:X0+Y0", "0:X0+Y0", "1:X1+Y1", "1:X1+Y1", "2:X2+Y2", "3:X3+Y3"}; - - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.processed.get(i)); - } - } - - @Test(expected = KafkaException.class) - public void testNotJoinable() { - KStreamBuilder builder = new KStreamBuilder(); - - KStream stream1; - KStream stream2; - KStreamWindowed windowed1; - KStreamWindowed windowed2; - MockProcessorSupplier processor; - - processor = new MockProcessorSupplier<>(); - stream1 = builder.from(keyDeserializer, valDeserializer, topic1).map(keyValueMapper); - stream2 = builder.from(keyDeserializer, valDeserializer, topic2); - windowed1 = stream1.with(new UnlimitedWindowDef("window1")); - windowed2 = stream2.with(new UnlimitedWindowDef("window2")); - - windowed1.join(windowed2, joiner).process(processor); - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java deleted file mode 100644 index c3dc7e08129ae..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java +++ /dev/null @@ -1,91 +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.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KStreamBuilder; -import org.apache.kafka.streams.kstream.Window; -import org.apache.kafka.streams.kstream.WindowSupplier; -import org.apache.kafka.test.KStreamTestDriver; -import org.apache.kafka.test.UnlimitedWindowDef; -import org.junit.Test; - -import java.util.Iterator; - -import static org.junit.Assert.assertEquals; - -public class KStreamWindowedTest { - - private String topicName = "topic"; - private String windowName = "MyWindow"; - - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - - @Test - public void testWindowedStream() { - KStreamBuilder builder = new KStreamBuilder(); - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - - KStream stream; - WindowSupplier windowSupplier; - - windowSupplier = new UnlimitedWindowDef<>(windowName); - stream = builder.from(keyDeserializer, valDeserializer, topicName); - stream.with(windowSupplier); - - KStreamTestDriver driver = new KStreamTestDriver(builder); - Window window = (Window) driver.getStateStore(windowName); - driver.setTime(0L); - - // two items in the window - - for (int i = 0; i < 2; i++) { - driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); - } - - assertEquals(1, countItem(window.find(0, 0L))); - assertEquals(1, countItem(window.find(1, 0L))); - assertEquals(0, countItem(window.find(2, 0L))); - assertEquals(0, countItem(window.find(3, 0L))); - - // previous two items + all items, thus two are duplicates, in the window - - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topicName, expectedKeys[i], "Y" + expectedKeys[i]); - } - - assertEquals(2, countItem(window.find(0, 0L))); - assertEquals(2, countItem(window.find(1, 0L))); - assertEquals(1, countItem(window.find(2, 0L))); - assertEquals(1, countItem(window.find(3, 0L))); - } - - - private int countItem(Iterator iter) { - int i = 0; - while (iter.hasNext()) { - i++; - iter.next(); - } - return i; - } -} diff --git a/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java index 3da1ca7ed86d4..b788819319d9d 100644 --- a/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java +++ b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java @@ -20,14 +20,14 @@ 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.WindowSupplier; +import org.apache.kafka.streams.kstream.WindowDef; import org.apache.kafka.streams.kstream.internals.FilteredIterator; import org.apache.kafka.streams.processor.internals.Stamped; import java.util.Iterator; import java.util.LinkedList; -public class UnlimitedWindowDef implements WindowSupplier { +public class UnlimitedWindowDef implements WindowDef { private final String name; From 2167f29ff630577fe63abc93fd8a58aa6c7d3c1c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 10 Dec 2015 11:32:34 -0800 Subject: [PATCH 02/14] option 1 of windowing opeartions --- .../streams/kstream/AggregateSupplier.java | 23 +++++ .../kstream/internals/KStreamJoin.java | 84 ------------------- 2 files changed, 23 insertions(+), 84 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java new file mode 100644 index 0000000000000..3ee80756adbb4 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream; + +public interface AggregateSupplier { + + Aggregator get(); +} 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 deleted file mode 100644 index eefb8c9b0641d..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.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.streams.kstream.internals; - -import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.kstream.Window; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; - -import java.util.Iterator; - -class KStreamJoin implements ProcessorSupplier { - - private static abstract class Finder { - abstract Iterator find(K key, long timestamp); - } - - private final String windowName; - private final ValueJoiner joiner; - - KStreamJoin(String windowName, ValueJoiner joiner) { - this.windowName = windowName; - this.joiner = joiner; - } - - @Override - public Processor get() { - return new KStreamJoinProcessor(windowName); - } - - private class KStreamJoinProcessor extends AbstractProcessor { - - private final String windowName; - protected Finder finder; - - public KStreamJoinProcessor(String windowName) { - this.windowName = windowName; - } - - @SuppressWarnings("unchecked") - @Override - public void init(ProcessorContext context) { - super.init(context); - - final Window window = (Window) context.getStateStore(windowName); - - this.finder = new Finder() { - @Override - Iterator find(K key, long timestamp) { - return window.find(key, timestamp); - } - }; - } - - @Override - public void process(K key, V1 value) { - long timestamp = context().timestamp(); - Iterator iter = finder.find(key, timestamp); - if (iter != null) { - while (iter.hasNext()) { - context().forward(key, joiner.apply(value, iter.next())); - } - } - } - } - -} From fb92b2b20f7be6f17c006de6e48cb04065808477 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 10 Dec 2015 21:47:51 -0800 Subject: [PATCH 03/14] v1 --- .../main/java/org/apache/kafka/streams/kstream/KStream.java | 2 +- .../apache/kafka/streams/kstream/internals/KStreamImpl.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) 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 51d383ce60219..357f143a3700d 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 @@ -194,5 +194,5 @@ public interface KStream { * @param the value type of the aggregated table * @param the window definition type */ - > WindowedKTable aggregateByKey(AggregateSupplier aggregateSupplier, W windowDef); + > KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, W windowDef); } 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 8898169c93621..604bf295ad0cd 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 @@ -22,6 +22,7 @@ import org.apache.kafka.streams.kstream.AggregateSupplier; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.KWindowedTable; import org.apache.kafka.streams.kstream.KeyValue; import org.apache.kafka.streams.kstream.TransformerSupplier; import org.apache.kafka.streams.kstream.ValueJoiner; @@ -32,7 +33,6 @@ import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.WindowDef; -import org.apache.kafka.streams.kstream.WindowedKTable; import org.apache.kafka.streams.processor.ProcessorSupplier; import java.lang.reflect.Array; @@ -251,7 +251,7 @@ public > KStream join(KStre } @Override - public > WindowedKTable aggregateByKey(AggregateSupplier aggregateSupplier, W windowDef) { + public > KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, W windowDef) { // TODO return null; } From 0862ec2b4ecb151ea1b3395c74787e4de99891fe Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 11 Dec 2015 14:15:02 -0800 Subject: [PATCH 04/14] v1 --- .../kafka/streams/examples/KStreamJob.java | 2 +- .../kafka/streams/examples/KTableJob.java | 167 ++++++++++++++++++ .../kafka/streams/kstream/Aggregator.java | 42 +++++ .../kafka/streams/kstream/HoppingWindows.java | 66 +++++++ .../apache/kafka/streams/kstream/KStream.java | 6 +- .../kafka/streams/kstream/KStreamBuilder.java | 6 +- .../apache/kafka/streams/kstream/KTable.java | 9 + .../kafka/streams/kstream/KWindowedTable.java | 111 ++++++++++++ ...SlidingWindow.java => SlidingWindows.java} | 22 +-- .../streams/kstream/UnlimitedWindow.java | 52 ++++++ .../apache/kafka/streams/kstream/Window.java | 17 +- .../kafka/streams/kstream/WindowDef.java | 39 +--- .../kafka/streams/kstream/WindowMapper.java | 23 +++ .../kstream/internals/AbstractWindows.java | 52 ++++++ .../kstream/internals/KStreamImpl.java | 21 ++- .../streams/kstream/internals/KTableImpl.java | 8 + .../streams/kstream/KStreamBuilderTest.java | 6 +- .../kstream/internals/KStreamBranchTest.java | 2 +- .../kstream/internals/KStreamFilterTest.java | 4 +- .../kstream/internals/KStreamFlatMapTest.java | 2 +- .../internals/KStreamFlatMapValuesTest.java | 2 +- .../kstream/internals/KStreamImplTest.java | 32 ++-- .../kstream/internals/KStreamMapTest.java | 2 +- .../internals/KStreamMapValuesTest.java | 2 +- .../internals/KStreamTransformTest.java | 2 +- .../internals/KStreamTransformValuesTest.java | 2 +- .../apache/kafka/test/UnlimitedWindowDef.java | 104 ----------- 27 files changed, 605 insertions(+), 198 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/KWindowedTable.java rename streams/src/main/java/org/apache/kafka/streams/kstream/{SlidingWindow.java => SlidingWindows.java} (72%) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/WindowMapper.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractWindows.java delete mode 100644 streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.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 819bd687b18ea..88a89550ab2bd 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 @@ -45,7 +45,7 @@ public static void main(String[] args) throws Exception { KStreamBuilder builder = new KStreamBuilder(); - KStream stream1 = builder.from("topic1"); + KStream stream1 = builder.stream("topic1"); KStream stream2 = stream1.map(new KeyValueMapper>() { diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java new file mode 100644 index 0000000000000..9b1f4a32b9a35 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java @@ -0,0 +1,167 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.examples; + +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.AggregateSupplier; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.HoppingWindows; +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.KTable; +import org.apache.kafka.streams.kstream.KWindowedTable; +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.Window; +import org.apache.kafka.streams.kstream.WindowMapper; + +import java.util.Properties; + +public class KTableJob { + + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamingConfig.JOB_ID_CONFIG, "example-ktable"); + 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(); + + // stream aggregate + KStream stream1 = builder.stream("topic1"); + + KWindowedTable wtable1 = stream1.aggregateByKey(new AggregateSupplier() { + @Override + public Aggregator get() { + return new Aggregator() { + @Override + public Integer initialValue() { + return 0; + } + + @Override + public Integer add(String aggKey, Integer value, Integer aggregate) { + return aggregate + value; + } + + @Override + public Integer remove(String aggKey, Integer value, Integer aggregate) { + return aggregate - value; + } + + @Override + public Integer merge(Integer aggr1, Integer aggr2) { + return aggr1 + aggr2; + } + }; + } + }, HoppingWindows.of(1000).every(500)); + + // table aggregation + KTable table1 = builder.table("topic2"); + + KTable table2 = table1.aggregate(new AggregateSupplier, Integer>() { + @Override + public Aggregator, Integer> get() { + return new Aggregator, Integer>() { + @Override + public Integer initialValue() { + return 0; + } + + @Override + public Integer add(String aggKey, KeyValue value, Integer aggregate) { + return aggregate + new Integer(value.value); + } + + @Override + public Integer remove(String aggKey, KeyValue value, Integer aggregate) { + return aggregate - new Integer(value.value); + } + + @Override + public Integer merge(Integer aggr1, Integer aggr2) { + return aggr1 + aggr2; + } + }; + } + }, new KeyValueMapper() { + @Override + public String apply(String key, String value) { + return value; + } + }); + + // stream-table join + KStream stream2 = stream1.leftJoin(table2, new ValueJoiner() { + @Override + public Integer apply(Integer value1, Integer value2) { + if (value2 == null) + return 0; + else + return value1 * value2; + } + }); + + // table-table join + KTable table3 = table1.outerJoin(table2, new ValueJoiner() { + @Override + public String apply(String value1, Integer value2) { + if (value2 == null) + return value1 + "-null"; + else if (value1 == null) + return "null-" + value2; + else + return value1 + "-" + value2; + } + }); + + // windowed table self join + KWindowedTable wtable2 = wtable1.leftJoin(wtable1, 1000 * 60 * 60 * 24 * 7 /* a week ago*/, new ValueJoiner() { + @Override + public Integer apply(Integer value1, Integer value2) { + if (value2 == null) + return value1; + else + return value1 - value2; + } + }); + + KStream stream3 = wtable2.toStream(new WindowMapper() { + @Override + public KeyValue apply(String key, Integer value, Window window) { + return new KeyValue<>(key + window.start(), value.toString()); + } + }); + + stream3.to("topic3"); + + KafkaStreaming kstream = new KafkaStreaming(builder, config); + kstream.start(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java new file mode 100644 index 0000000000000..d715fbdb3df75 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream; + +public interface Aggregator { + /** + * Set the initial aggregate value + */ + T initialValue(); + + /** + * When a new record with the aggregate key is added, + * updating the aggregate value for this key + */ + T add(K aggKey, V value, T aggregate); + + /** + * when an old record with the aggregate key is removed, + * updating the aggregate value for this key + */ + T remove(K aggKey, V value, T aggregate); + + /** + * Merge two aggregate values + */ + T merge(T aggr1, T aggr2); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java new file mode 100644 index 0000000000000..457c2026e5a83 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.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.streams.kstream; + +import org.apache.kafka.streams.kstream.internals.AbstractWindows; + +import java.util.Collection; +import java.util.Collections; + +public class HoppingWindows extends AbstractWindows implements WindowDef { + + private long size; + + private long period; + + private HoppingWindows(long size) { + super(); + + this.size = size; + this.period = size; + } + + /** + * Returns a half-interval hopping window definition with the window size in milliseconds + * of the form [ N * size, N * size + size ) + */ + public static HoppingWindows of(long size) { + return new HoppingWindows(size); + } + + /** + * Returns a new hopping window definition with the original size but reassign the window + * period in milliseconds of the form [ N * period, N * period + size ) + */ + public HoppingWindows every(long period) { + this.period = period; + + return this; + } + + @Override + public Collection windowsFor(long timestamp) { + // TODO + return Collections.emptyList(); + } + + @Override + public boolean equalTo(HoppingWindows other) { + return this.size == other.size && this.period == other.period; + } +} 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 357f143a3700d..06f3ed5dfc87d 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 @@ -182,9 +182,8 @@ public interface KStream { * @param windowDef the window definition * @param the value type of the other stream * @param the value type of the new stream - * @param the window definition type */ - > KStream join(KStream kstream, ValueJoiner joiner, W windowDef); + KStream join(KStream kstream, ValueJoiner joiner, WindowDef windowDef); /** * Aggregate values of this stream by key on a window basis. @@ -192,7 +191,6 @@ public interface KStream { * @param aggregateSupplier the class of aggregateSupplier * @param windowDef the window definition * @param the value type of the aggregated table - * @param the window definition type */ - > KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, W windowDef); + KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, WindowDef windowDef); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java index 0ed514424a3ca..4140e30b5100a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -46,8 +46,8 @@ public KStreamBuilder() { * @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 KStream stream(String... topics) { + return stream(null, null, topics); } /** @@ -60,7 +60,7 @@ public KStream from(String... topics) { * @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 stream(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { String name = newName(KStreamImpl.SOURCE_NAME); addSource(name, keyDeserializer, valDeserializer, topics); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index c6e7975955abc..9a1b0f6f73f95 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -138,4 +138,13 @@ public interface KTable { */ KTable leftJoin(KTable other, ValueJoiner joiner); + /** + * Aggregate values of this table by the selected key. + * + * @param aggregateSupplier the class of aggregateSupplier + * @param selector the KeyValue mapper that select the aggregate key + * @param the key type of the aggregated table + * @param the value type of the aggregated table + */ + KTable aggregate(AggregateSupplier, V1> aggregateSupplier, KeyValueMapper selector); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KWindowedTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KWindowedTable.java new file mode 100644 index 0000000000000..ee67faf55d7f9 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KWindowedTable.java @@ -0,0 +1,111 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +/** + * KWindowedTable is an abstraction of a change log stream on a window basis. + * + * + * @param the type of keys + * @param the type of values + * @param the type of window definition + */ +public interface KWindowedTable { + + /** + * Creates a new instance of KTable consists of all elements of this stream which satisfy a predicate + * + * @param predicate the instance of Predicate + * @return the instance of KTable with only those elements that satisfy the predicate + */ + KWindowedTable filter(Predicate predicate); + + /** + * Creates a new instance of KTable consists all elements of this stream which do not satisfy a predicate + * + * @param predicate the instance of Predicate + * @return the instance of KTable with only those elements that do not satisfy the predicate + */ + KWindowedTable filterOut(Predicate predicate); + + /** + * Creates a new instance of KTable by transforming each value in this stream into a different value in the new stream. + * + * @param mapper the instance of ValueMapper + * @param the value type of the new stream + * @return the instance of KTable + */ + KWindowedTable mapValues(ValueMapper mapper); + + /** + * Transform this windowed table into a stream + * + * @param windowMapper the class of WindowMapper + * @return the instance of KStream that contains transformed keys and values + */ + KStream toStream(WindowMapper windowMapper); + + /** + * Combines values of this KWindowedTable with another KWindowedTable using Inner Join. + * The window definition of the other KWindowedTable must be the same. + * + * @param other the instance of KTable joined with this stream + * @param joiner ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream + * @return the instance of the KWindowedTable + */ + KWindowedTable join(KWindowedTable other, ValueJoiner joiner); + + /** + * Combines values of this KWindowedTable with another KWindowedTable using Outer Join. + * The window definition of the other WindowedKtable must be the same. + * + * @param other the instance of KTable joined with this stream + * @param joiner ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream + * @return the instance of KStream + */ + KWindowedTable outerJoin(KWindowedTable other, ValueJoiner joiner); + + /** + * Combines values of this KWindowedTable with another KWindowedTable using Left Join. + * The window definition of the other KWindowedTable must be the same. + * + * @param other the instance of KTable joined with this stream + * @param joiner ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream + * @return the instance of KStream + */ + KWindowedTable leftJoin(KWindowedTable other, ValueJoiner joiner); + + /** + * Combines values of this KWindowedTable with another KWindowedTable using Left Join. + * The window type of the other KWindowedTable must be the same. + * + * @param other the instance of KTable joined with this stream + * @param timeDifference difference in window definition + * @param joiner ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream + * @return the instance of KStream + */ + KWindowedTable leftJoin(KWindowedTable other, long timeDifference, ValueJoiner joiner); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java similarity index 72% rename from streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java index 33c2e35e7892a..047ad09399283 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java @@ -18,38 +18,34 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.streams.kstream.internals.AbstractWindows; + import java.util.Collection; import java.util.Collections; -public class SlidingWindow extends WindowDef { +public class SlidingWindows extends AbstractWindows implements WindowDef { private long size; - private SlidingWindow(long size) { + private SlidingWindows(long size) { this.size = size; } /** * Returns a half-interval sliding window definition with the window size in milliseconds */ - public static SlidingWindow of(long size) { - return new SlidingWindow(size); + public static SlidingWindows of(long size) { + return new SlidingWindows(size); } @Override public Collection windowsFor(long timestamp) { // TODO - return Collections.EMPTY_LIST; + return Collections.emptyList(); } - @Override - public boolean equals(WindowDef other) { - if (!other.getClass().equals(SlidingWindow.class)) - return false; - - SlidingWindow otherWindow = (SlidingWindow) other; - - return this.size == otherWindow.size; + public boolean equalTo(SlidingWindows other) { + return this.size == other.size; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java new file mode 100644 index 0000000000000..f651a6a2a3997 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream; + +import org.apache.kafka.streams.kstream.internals.AbstractWindows; + +import java.util.Collection; +import java.util.Collections; + +public class UnlimitedWindow extends AbstractWindows implements WindowDef { + + private long start; + + private UnlimitedWindow(long start) { + super(); + + this.start = start; + } + + /** + * Returns an unlimited window definition + */ + public static UnlimitedWindow on(long start) { + return new UnlimitedWindow(start); + } + + @Override + public Collection windowsFor(long timestamp) { + // TODO + return Collections.emptyList(); + } + + @Override + public boolean equalTo(UnlimitedWindow other) { + return true; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index b1827f9a4e59f..33f9b0a94402d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -17,18 +17,29 @@ package org.apache.kafka.streams.kstream; +public class Window implements Comparable { -public abstract class Window implements Comparable { + private long start; + private long end; + + public Window(long start, long end) { + this.start = start; + this.end = end; + } /** * Returns the start timestamp of this window, inclusive */ - public abstract long start(); + public long start() { + return start; + } /** * Returns the end timestamp of this window, exclusive */ - public abstract long end(); + public long end() { + return end; + } @Override public int compareTo(Window other) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java index b3c0cfab7c7e2..f8f155af6496b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java @@ -19,40 +19,9 @@ import java.util.Collection; -public abstract class WindowDef { +public interface WindowDef { - private static long DEFAULT_EMIT_DURATION = 1000L; + boolean equalTo(W other); - private static long DEFAULT_MAINTAIN_DURATION = 24 * 60 * 60 * 1000L; // one day - - private long emitDuration; - - private long maintainDuration; - - abstract public boolean equals(WindowDef other); - - abstract public Collection windowsFor(long timestamp); - - protected WindowDef() { - this.emitDuration = DEFAULT_EMIT_DURATION; - this.maintainDuration = DEFAULT_MAINTAIN_DURATION; - } - - /** - * Set the window emit duration in milliseconds of system time - */ - public WindowDef emit(long duration) { - this.emitDuration = duration; - - return this; - } - - /** - * Set the window maintain duration in milliseconds of system time - */ - WindowDef until(long duration) { - this.maintainDuration = duration; - - return this; - } -} \ No newline at end of file + Collection windowsFor(long timestamp); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowMapper.java new file mode 100644 index 0000000000000..00f803b88bea2 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowMapper.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream; + +public interface WindowMapper { + + KeyValue apply(K key, V value, Window window); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractWindows.java new file mode 100644 index 0000000000000..fcbd2c9afc3cf --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractWindows.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream.internals; + +public abstract class AbstractWindows { + + private static final long DEFAULT_EMIT_DURATION = 1000L; + + private static final long DEFAULT_MAINTAIN_DURATION = 24 * 60 * 60 * 1000L; // one day + + private long emitDuration; + + private long maintainDuration; + + protected AbstractWindows() { + this.emitDuration = DEFAULT_EMIT_DURATION; + this.maintainDuration = DEFAULT_MAINTAIN_DURATION; + } + + /** + * Set the window emit duration in milliseconds of system time + */ + public AbstractWindows emit(long duration) { + this.emitDuration = duration; + + return this; + } + + /** + * Set the window maintain duration in milliseconds of system time + */ + public AbstractWindows until(long duration) { + this.maintainDuration = duration; + + return this; + } +} \ No newline at end of file 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 604bf295ad0cd..0a89d1f8ca8d9 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 @@ -31,8 +31,8 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.ValueMapper; -import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.WindowDef; +import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorSupplier; import java.lang.reflect.Array; @@ -61,7 +61,7 @@ public class KStreamImpl extends AbstractStream implements KStream through(String topic, Deserializer valDeserializer) { to(topic, keySerializer, valSerializer); - return topology.from(keyDeserializer, valDeserializer, topic); + return topology.stream(keyDeserializer, valDeserializer, topic); } @Override @@ -245,13 +245,22 @@ public KStream leftJoin(KTable other, ValueJoiner } @Override - public > KStream join(KStream kstream, ValueJoiner joiner, W windowDef) { + public KStream join(KStream kstream, ValueJoiner joiner, WindowDef windowDef) { // TODO - return null; + String name = topology.newName(JOINTHIS_NAME); + + topology.addProcessor(name, new ProcessorSupplier() { + @Override + public Processor get() { + return null; + } + }, this.name); + + return new KStreamImpl<>(topology, name, sourceNodes); } @Override - public > KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, W windowDef) { + public KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, WindowDef windowDef) { // TODO return null; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 308e4f5e50364..3a9508a92b70a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -19,9 +19,12 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.AggregateSupplier; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; @@ -233,4 +236,9 @@ public KTable leftJoin(KTable other, ValueJoiner return new KTableImpl<>(topology, name, leftJoin, allSourceNodes); } + @Override + public KTable aggregate(AggregateSupplier, V1> aggregateSupplier, KeyValueMapper selector) { + // TODO + return null; + } } 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 index d6994a9b997a6..f79063f0d4070 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java @@ -32,7 +32,7 @@ public class KStreamBuilderTest { public void testFrom() { final KStreamBuilder builder = new KStreamBuilder(); - builder.from("topic-1", "topic-2"); + builder.stream("topic-1", "topic-2"); builder.addSource(KStreamImpl.SOURCE_NAME + "0000000000", "topic-3"); } @@ -59,8 +59,8 @@ public void testMerge() { KStreamBuilder builder = new KStreamBuilder(); - KStream source1 = builder.from(topic1); - KStream source2 = builder.from(topic2); + KStream source1 = builder.stream(topic1); + KStream source2 = builder.stream(topic2); KStream merged = builder.merge(source1, source2); MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index 40eba2f103a5f..88366fac3dd24 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -67,7 +67,7 @@ public boolean test(Integer key, String value) { KStream[] branches; MockProcessorSupplier[] processors; - stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(keyDeserializer, valDeserializer, topicName); branches = stream.branch(isEven, isMultipleOfThree, isOdd); assertEquals(3, branches.length); 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 d1e5d38a987f7..3bad0416152b1 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 @@ -52,7 +52,7 @@ public void testFilter() { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(keyDeserializer, valDeserializer, topicName); stream.filter(isMultipleOfThree).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); @@ -72,7 +72,7 @@ public void testFilterOut() { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(keyDeserializer, valDeserializer, topicName); stream.filterOut(isMultipleOfThree).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); 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 61b5ccd5a1dc4..a55fd30e5c191 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 @@ -60,7 +60,7 @@ public Iterable> apply(Integer key, String value) { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(keyDeserializer, valDeserializer, topicName); stream.flatMap(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); 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 66faf077885c7..eef7933c627f8 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 @@ -58,7 +58,7 @@ public Iterable apply(String value) { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(keyDeserializer, valDeserializer, topicName); stream.flatMapValues(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); 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 1e775b8aeba2f..05ddc18409c41 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 @@ -25,7 +25,7 @@ import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.test.MockProcessorSupplier; -import org.apache.kafka.test.UnlimitedWindowDef; +import org.apache.kafka.streams.kstream.UnlimitedWindow; import org.junit.Test; import java.util.Collections; @@ -40,9 +40,9 @@ public void testNumProcesses() { final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); - KStream source1 = builder.from(deserializer, deserializer, "topic-1", "topic-2"); + KStream source1 = builder.stream(deserializer, deserializer, "topic-1", "topic-2"); - KStream source2 = builder.from(deserializer, deserializer, "topic-3", "topic-4"); + KStream source2 = builder.stream(deserializer, deserializer, "topic-3", "topic-4"); KStream stream1 = source1.filter(new Predicate() { @@ -101,21 +101,19 @@ public boolean test(String key, Integer value) { } ); - 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 stream4 = streams2[0].join(streams3[0], new ValueJoiner() { + @Override + public Integer apply(Integer value1, Integer value2) { + return value1 + value2; + } + }, UnlimitedWindow.on(0)); - 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; - } - }); + KStream stream5 = streams2[1].join(streams3[1], new ValueJoiner() { + @Override + public Integer apply(Integer value1, Integer value2) { + return value1 + value2; + } + }, UnlimitedWindow.on(0)); stream4.to("topic-5"); 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 2ae8a97515dcb..0f7cb6a43e4e0 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 @@ -54,7 +54,7 @@ public KeyValue apply(Integer key, String value) { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(keyDeserializer, valDeserializer, topicName); stream.map(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); 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 f830c0010b338..68fd285870f71 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 @@ -51,7 +51,7 @@ public Integer apply(String value) { KStream stream; MockProcessorSupplier processor = new MockProcessorSupplier<>(); - stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(keyDeserializer, valDeserializer, topicName); stream.mapValues(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index e397dd1fe20dd..0b7b1e788a756 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -73,7 +73,7 @@ public void close() { KStream stream; MockProcessorSupplier processor = new MockProcessorSupplier<>(); - stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(keyDeserializer, valDeserializer, topicName); stream.transform(transformerSupplier).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java index c5c9b39893c70..7def9db45bca0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -72,7 +72,7 @@ public void close() { KStream stream; MockProcessorSupplier processor = new MockProcessorSupplier<>(); - stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(keyDeserializer, valDeserializer, topicName); stream.transformValues(valueTransformerSupplier).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java deleted file mode 100644 index b788819319d9d..0000000000000 --- a/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.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.test; - -import org.apache.kafka.streams.kstream.Window; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.kstream.KeyValue; -import org.apache.kafka.streams.kstream.WindowDef; -import org.apache.kafka.streams.kstream.internals.FilteredIterator; -import org.apache.kafka.streams.processor.internals.Stamped; - -import java.util.Iterator; -import java.util.LinkedList; - -public class UnlimitedWindowDef implements WindowDef { - - private final String name; - - public UnlimitedWindowDef(String name) { - this.name = name; - } - - public String name() { - return name; - } - - public Window get() { - return new UnlimitedWindow(); - } - - public class UnlimitedWindow implements Window { - - private final LinkedList>> list = new LinkedList<>(); - - @Override - public void init(ProcessorContext context) { - context.register(this, true, null); - } - - @Override - public Iterator find(final K key, long timestamp) { - return find(key, Long.MIN_VALUE, timestamp); - } - - @Override - public Iterator findAfter(final K key, long timestamp) { - return find(key, timestamp, Long.MAX_VALUE); - } - - @Override - public Iterator findBefore(final K key, long timestamp) { - return find(key, Long.MIN_VALUE, Long.MAX_VALUE); - } - - private Iterator find(final K key, final long startTime, final long endTime) { - return new FilteredIterator>>(list.iterator()) { - protected V filter(Stamped> item) { - if (item.value.key.equals(key) && startTime <= item.timestamp && item.timestamp <= endTime) - return item.value.value; - else - return null; - } - }; - } - - @Override - public void put(K key, V value, long timestamp) { - list.add(new Stamped<>(KeyValue.pair(key, value), timestamp)); - } - - @Override - public String name() { - return name; - } - - @Override - public void flush() { - } - - @Override - public void close() { - } - - @Override - public boolean persistent() { - return false; - } - } -} From 9558891bdaccc0b8861f882b957b5131556f896c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 14 Dec 2015 16:30:20 -0800 Subject: [PATCH 05/14] address Yasu's comments --- .../kafka/streams/examples/KTableJob.java | 11 ++-- .../kafka/streams/kstream/HoppingWindow.java | 25 +++++++++ .../kafka/streams/kstream/HoppingWindows.java | 13 +++-- .../apache/kafka/streams/kstream/KStream.java | 8 +-- .../apache/kafka/streams/kstream/KTable.java | 3 +- .../kafka/streams/kstream/KWindowedTable.java | 4 +- .../{WindowDef.java => SlidingWindow.java} | 9 ++- .../kafka/streams/kstream/SlidingWindows.java | 13 +++-- .../streams/kstream/UnlimitedWindow.java | 33 +---------- .../streams/kstream/UnlimitedWindows.java | 55 +++++++++++++++++++ .../apache/kafka/streams/kstream/Window.java | 2 +- .../kafka/streams/kstream/WindowMapper.java | 4 +- .../AbstractWindows.java => Windows.java} | 26 +++++++-- .../kstream/internals/KStreamImpl.java | 7 ++- .../kstream/internals/KStreamImplTest.java | 6 +- 15 files changed, 147 insertions(+), 72 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindow.java rename streams/src/main/java/org/apache/kafka/streams/kstream/{WindowDef.java => SlidingWindow.java} (84%) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java rename streams/src/main/java/org/apache/kafka/streams/kstream/{internals/AbstractWindows.java => Windows.java} (74%) diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java index 9b1f4a32b9a35..393f591f83350 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.streams.kstream.AggregateSupplier; import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.HoppingWindow; import org.apache.kafka.streams.kstream.HoppingWindows; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.StreamingConfig; @@ -55,7 +56,7 @@ public static void main(String[] args) throws Exception { // stream aggregate KStream stream1 = builder.stream("topic1"); - KWindowedTable wtable1 = stream1.aggregateByKey(new AggregateSupplier() { + KWindowedTable wtable1 = stream1.aggregateByKey(new AggregateSupplier() { @Override public Aggregator get() { return new Aggregator() { @@ -80,7 +81,7 @@ public Integer merge(Integer aggr1, Integer aggr2) { } }; } - }, HoppingWindows.of(1000).every(500)); + }, HoppingWindows.of(1000).every(500).emit(1000).until(1000 * 60 * 60 * 24 /* one day */)); // table aggregation KTable table1 = builder.table("topic2"); @@ -142,7 +143,7 @@ else if (value1 == null) }); // windowed table self join - KWindowedTable wtable2 = wtable1.leftJoin(wtable1, 1000 * 60 * 60 * 24 * 7 /* a week ago*/, new ValueJoiner() { + KWindowedTable wtable2 = wtable1.leftJoin(wtable1, 1000 * 60 * 60 * 24 * 7 /* a week ago*/, new ValueJoiner() { @Override public Integer apply(Integer value1, Integer value2) { if (value2 == null) @@ -152,9 +153,9 @@ public Integer apply(Integer value1, Integer value2) { } }); - KStream stream3 = wtable2.toStream(new WindowMapper() { + KStream stream3 = wtable2.toStream(new WindowMapper() { @Override - public KeyValue apply(String key, Integer value, Window window) { + public KeyValue apply(String key, Integer value, HoppingWindow window) { return new KeyValue<>(key + window.start(), value.toString()); } }); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindow.java new file mode 100644 index 0000000000000..fcfe017db36cd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindow.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream; + +public class HoppingWindow extends Window { + + public HoppingWindow(long start, long end) { + super(start, end); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java index 457c2026e5a83..671289794f0c7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java @@ -17,12 +17,10 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.streams.kstream.internals.AbstractWindows; - import java.util.Collection; import java.util.Collections; -public class HoppingWindows extends AbstractWindows implements WindowDef { +public class HoppingWindows extends Windows { private long size; @@ -60,7 +58,12 @@ public Collection windowsFor(long timestamp) { } @Override - public boolean equalTo(HoppingWindows other) { - return this.size == other.size && this.period == other.period; + public boolean equalTo(Windows other) { + if (!other.getClass().equals(HoppingWindows.class)) + return false; + + HoppingWindows otherWindows = (HoppingWindows) other; + + return this.size == otherWindows.size && this.period == otherWindows.period; } } 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 06f3ed5dfc87d..76c5edcbb6910 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 @@ -179,18 +179,18 @@ public interface KStream { * * @param kstream the instance of KStream joined with this stream * @param joiner ValueJoiner - * @param windowDef the window definition + * @param windows the window definition * @param the value type of the other stream * @param the value type of the new stream */ - KStream join(KStream kstream, ValueJoiner joiner, WindowDef windowDef); + KStream join(KStream kstream, ValueJoiner joiner, Windows windows); /** * Aggregate values of this stream by key on a window basis. * * @param aggregateSupplier the class of aggregateSupplier - * @param windowDef the window definition + * @param windows the window definition * @param the value type of the aggregated table */ - KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, WindowDef windowDef); + KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, Windows windows); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 9a1b0f6f73f95..a9ab7317d53f7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -143,8 +143,9 @@ public interface KTable { * * @param aggregateSupplier the class of aggregateSupplier * @param selector the KeyValue mapper that select the aggregate key + * @param name the name of the resulted table * @param the key type of the aggregated table * @param the value type of the aggregated table */ - KTable aggregate(AggregateSupplier, V1> aggregateSupplier, KeyValueMapper selector); + KTable aggregate(AggregateSupplier, V1> aggregateSupplier, KeyValueMapper selector, String name); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KWindowedTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KWindowedTable.java index ee67faf55d7f9..80dc5dc6e29d1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KWindowedTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KWindowedTable.java @@ -25,7 +25,7 @@ * @param the type of values * @param the type of window definition */ -public interface KWindowedTable { +public interface KWindowedTable { /** * Creates a new instance of KTable consists of all elements of this stream which satisfy a predicate @@ -58,7 +58,7 @@ public interface KWindowedTable { * @param windowMapper the class of WindowMapper * @return the instance of KStream that contains transformed keys and values */ - KStream toStream(WindowMapper windowMapper); + KStream toStream(WindowMapper windowMapper); /** * Combines values of this KWindowedTable with another KWindowedTable using Inner Join. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java similarity index 84% rename from streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java index f8f155af6496b..e0a79040ac74a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java @@ -17,11 +17,10 @@ package org.apache.kafka.streams.kstream; -import java.util.Collection; -public interface WindowDef { +public class SlidingWindow extends Window { - boolean equalTo(W other); - - Collection windowsFor(long timestamp); + public SlidingWindow(long start, long end) { + super(start, end); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java index 047ad09399283..e19ee4fd66ab8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java @@ -18,12 +18,10 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.streams.kstream.internals.AbstractWindows; - import java.util.Collection; import java.util.Collections; -public class SlidingWindows extends AbstractWindows implements WindowDef { +public class SlidingWindows extends Windows { private long size; @@ -45,7 +43,12 @@ public Collection windowsFor(long timestamp) { } @Override - public boolean equalTo(SlidingWindows other) { - return this.size == other.size; + public boolean equalTo(Windows other) { + if (!other.getClass().equals(SlidingWindows.class)) + return false; + + SlidingWindows otherWindows = (SlidingWindows) other; + + return this.size == otherWindows.size; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java index f651a6a2a3997..5e2bb07871147 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java @@ -17,36 +17,9 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.streams.kstream.internals.AbstractWindows; +public class UnlimitedWindow extends Window { -import java.util.Collection; -import java.util.Collections; - -public class UnlimitedWindow extends AbstractWindows implements WindowDef { - - private long start; - - private UnlimitedWindow(long start) { - super(); - - this.start = start; - } - - /** - * Returns an unlimited window definition - */ - public static UnlimitedWindow on(long start) { - return new UnlimitedWindow(start); - } - - @Override - public Collection windowsFor(long timestamp) { - // TODO - return Collections.emptyList(); - } - - @Override - public boolean equalTo(UnlimitedWindow other) { - return true; + public UnlimitedWindow(long start) { + super(start, Long.MAX_VALUE); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java new file mode 100644 index 0000000000000..5d35a7eeea0dc --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.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.streams.kstream; + +import java.util.Collection; +import java.util.Collections; + +public class UnlimitedWindows extends Windows { + + private long start; + + private UnlimitedWindows(long start) { + super(); + + this.start = start; + } + + /** + * Returns an unlimited window definition + */ + public static UnlimitedWindows on(long start) { + return new UnlimitedWindows(start); + } + + @Override + public Collection windowsFor(long timestamp) { + // TODO + return Collections.emptyList(); + } + + @Override + public boolean equalTo(Windows other) { + if (!other.getClass().equals(UnlimitedWindows.class)) + return false; + + UnlimitedWindows otherWindows = (UnlimitedWindows) other; + + return this.start == otherWindows.start; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index 33f9b0a94402d..5ebe6ab7b595e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.kstream; -public class Window implements Comparable { +public abstract class Window implements Comparable { private long start; private long end; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowMapper.java index 00f803b88bea2..27d3885129374 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowMapper.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.kstream; -public interface WindowMapper { +public interface WindowMapper { - KeyValue apply(K key, V value, Window window); + KeyValue apply(K key, V value, W window); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java similarity index 74% rename from streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractWindows.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java index fcbd2c9afc3cf..3c1ed367ce111 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java @@ -15,9 +15,11 @@ * limitations under the License. */ -package org.apache.kafka.streams.kstream.internals; +package org.apache.kafka.streams.kstream; -public abstract class AbstractWindows { +import java.util.Collection; + +public abstract class Windows { private static final long DEFAULT_EMIT_DURATION = 1000L; @@ -27,7 +29,7 @@ public abstract class AbstractWindows { private long maintainDuration; - protected AbstractWindows() { + protected Windows() { this.emitDuration = DEFAULT_EMIT_DURATION; this.maintainDuration = DEFAULT_MAINTAIN_DURATION; } @@ -35,7 +37,7 @@ protected AbstractWindows() { /** * Set the window emit duration in milliseconds of system time */ - public AbstractWindows emit(long duration) { + public Windows emit(long duration) { this.emitDuration = duration; return this; @@ -44,9 +46,21 @@ public AbstractWindows emit(long duration) { /** * Set the window maintain duration in milliseconds of system time */ - public AbstractWindows until(long duration) { + public Windows until(long duration) { this.maintainDuration = duration; return this; } -} \ No newline at end of file + + public long emitEveryMs() { + return this.emitDuration; + } + + public long maintainMs() { + return this.maintainDuration; + } + + abstract boolean equalTo(Windows other); + + abstract Collection windowsFor(long timestamp); +} 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 0a89d1f8ca8d9..8ceca6ff608de 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 @@ -31,7 +31,8 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.ValueMapper; -import org.apache.kafka.streams.kstream.WindowDef; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.Windows; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorSupplier; @@ -245,7 +246,7 @@ public KStream leftJoin(KTable other, ValueJoiner } @Override - public KStream join(KStream kstream, ValueJoiner joiner, WindowDef windowDef) { + public KStream join(KStream kstream, ValueJoiner joiner, Windows windows) { // TODO String name = topology.newName(JOINTHIS_NAME); @@ -260,7 +261,7 @@ public Processor get() { } @Override - public KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, WindowDef windowDef) { + public KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, Windows windows) { // TODO return null; } 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 05ddc18409c41..94a3afa734786 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 @@ -25,7 +25,7 @@ import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.test.MockProcessorSupplier; -import org.apache.kafka.streams.kstream.UnlimitedWindow; +import org.apache.kafka.streams.kstream.UnlimitedWindows; import org.junit.Test; import java.util.Collections; @@ -106,14 +106,14 @@ public boolean test(String key, Integer value) { public Integer apply(Integer value1, Integer value2) { return value1 + value2; } - }, UnlimitedWindow.on(0)); + }, UnlimitedWindows.on(0)); KStream stream5 = streams2[1].join(streams3[1], new ValueJoiner() { @Override public Integer apply(Integer value1, Integer value2) { return value1 + value2; } - }, UnlimitedWindow.on(0)); + }, UnlimitedWindows.on(0)); stream4.to("topic-5"); From e6373cbc4229637100c97bbb440555c2f0719d03 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 14 Dec 2015 17:50:17 -0800 Subject: [PATCH 06/14] add built-in aggregates --- .../streams/kstream/AggregateSupplier.java | 2 +- .../apache/kafka/streams/kstream/KStream.java | 17 +++++++++++++++++ .../kstream/internals/KStreamImpl.java | 19 +++++++++++++++++++ 3 files changed, 37 insertions(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java index 3ee80756adbb4..cd6d317c99acc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java @@ -19,5 +19,5 @@ public interface AggregateSupplier { - Aggregator get(); + Aggregator get(Class keyClass, Class valueClass, Class aggClass); } 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 76c5edcbb6910..caa5fc5c46b08 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 @@ -21,6 +21,8 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.ProcessorSupplier; +import java.util.Collection; + /** * KStream is an abstraction of a stream of key-value pairs. * @@ -193,4 +195,19 @@ public interface KStream { * @param the value type of the aggregated table */ KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, Windows windows); + + /** + * Sum values of this stream by key on a window basis. + */ + KWindowedTable sumByKey(Windows windows); + + /** + * Count number of records of this stream by key on a window basis. + */ + KWindowedTable countByKey(Windows windows); + + /** + * Get the top-k records of this stream by key on a window basis. + */ + KWindowedTable, W> topKByKey(int k, Windows windows); } 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 8ceca6ff608de..266280c1b0248 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 @@ -37,6 +37,7 @@ import org.apache.kafka.streams.processor.ProcessorSupplier; import java.lang.reflect.Array; +import java.util.Collection; import java.util.HashSet; import java.util.Set; @@ -265,4 +266,22 @@ public KWindowedTable aggregateByKey(AggregateSup // TODO return null; } + + @Override + public KWindowedTable sumByKey(Windows windows) { + + return this.aggregateByKey(new SumSupplier<>(), windows); + } + + @Override + public KWindowedTable countByKey(Windows windows) { + + return this.aggregateByKey(new CountSupplier<>(), windows); + } + + @Override + public KWindowedTable, W> topKByKey(int k, Windows windows) { + + return this.aggregateByKey(new TopKSupplier<>(k), windows); + } } From 66e122adc8911334e924921bc7fa67275445bd71 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 14 Dec 2015 19:17:59 -0800 Subject: [PATCH 07/14] add built-in aggregates in KTable --- .../kafka/streams/examples/KTableJob.java | 60 ++----------------- .../apache/kafka/streams/kstream/KTable.java | 19 +++++- .../streams/kstream/internals/KTableImpl.java | 21 ++++++- 3 files changed, 43 insertions(+), 57 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java index 393f591f83350..a68bbe216897c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java @@ -56,67 +56,17 @@ public static void main(String[] args) throws Exception { // stream aggregate KStream stream1 = builder.stream("topic1"); - KWindowedTable wtable1 = stream1.aggregateByKey(new AggregateSupplier() { - @Override - public Aggregator get() { - return new Aggregator() { - @Override - public Integer initialValue() { - return 0; - } - - @Override - public Integer add(String aggKey, Integer value, Integer aggregate) { - return aggregate + value; - } - - @Override - public Integer remove(String aggKey, Integer value, Integer aggregate) { - return aggregate - value; - } - - @Override - public Integer merge(Integer aggr1, Integer aggr2) { - return aggr1 + aggr2; - } - }; - } - }, HoppingWindows.of(1000).every(500).emit(1000).until(1000 * 60 * 60 * 24 /* one day */)); + KWindowedTable wtable1 = stream1.sumByKey(HoppingWindows.of(1000).every(500).emit(1000).until(1000 * 60 * 60 * 24 /* one day */)); // table aggregation KTable table1 = builder.table("topic2"); - KTable table2 = table1.aggregate(new AggregateSupplier, Integer>() { + KTable table2 = table1.sum(new KeyValueMapper>() { @Override - public Aggregator, Integer> get() { - return new Aggregator, Integer>() { - @Override - public Integer initialValue() { - return 0; - } - - @Override - public Integer add(String aggKey, KeyValue value, Integer aggregate) { - return aggregate + new Integer(value.value); - } - - @Override - public Integer remove(String aggKey, KeyValue value, Integer aggregate) { - return aggregate - new Integer(value.value); - } - - @Override - public Integer merge(Integer aggr1, Integer aggr2) { - return aggr1 + aggr2; - } - }; + public KeyValue apply(String key, String value) { + return new KeyValue<>(value, new Integer(key)); } - }, new KeyValueMapper() { - @Override - public String apply(String key, String value) { - return value; - } - }); + }, "table2"); // stream-table join KStream stream2 = stream1.leftJoin(table2, new ValueJoiner() { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index a9ab7317d53f7..d08e547746257 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -20,6 +20,8 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import java.util.Collection; + /** * KTable is an abstraction of a change log stream. * @@ -147,5 +149,20 @@ public interface KTable { * @param the key type of the aggregated table * @param the value type of the aggregated table */ - KTable aggregate(AggregateSupplier, V1> aggregateSupplier, KeyValueMapper selector, String name); + KTable aggregate(AggregateSupplier aggregateSupplier, KeyValueMapper> selector, String name); + + /** + * Sum values of this table by the selected key on a window basis. + */ + KTablesum(KeyValueMapper> selector, String name); + + /** + * Count number of records of this table by the selected key on a window basis. + */ + KTable count(KeyValueMapper> selector, String name); + + /** + * Get the top-k records of this table by the selected key on a window basis. + */ + KTable> topK(int k, KeyValueMapper> selector, String name); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 3a9508a92b70a..16a490d1f3085 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -30,6 +30,7 @@ import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.processor.StateStoreSupplier; +import java.util.Collection; import java.util.Set; /** @@ -237,8 +238,26 @@ public KTable leftJoin(KTable other, ValueJoiner } @Override - public KTable aggregate(AggregateSupplier, V1> aggregateSupplier, KeyValueMapper selector) { + public KTable aggregate(AggregateSupplier aggregateSupplier, KeyValueMapper> selector, String name) { // TODO return null; } + + @Override + public KTablesum(KeyValueMapper> selector, String name) { + + return this.aggregate(new SumSupplier<>(), selector, name); + } + + @Override + public KTable count(KeyValueMapper> selector, String name) { + + return this.aggregate(new CountSupplier<>(), selector, name); + } + + @Override + public KTable> topK(int k, KeyValueMapper> selector, String name) { + + return this.aggregate(new TopKSupplier<>(k), selector, name); + } } From 1f360a25022d0286f6ebbf1a6735201ba8fdab53 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 15 Dec 2015 11:43:53 -0800 Subject: [PATCH 08/14] address Yasu's comments --- .../kafka/streams/kstream/HoppingWindow.java | 10 ++++++++++ .../kafka/streams/kstream/HoppingWindows.java | 4 ++++ .../kafka/streams/kstream/SlidingWindow.java | 10 ++++++++++ .../kafka/streams/kstream/SlidingWindows.java | 6 ++++++ .../streams/kstream/UnlimitedWindow.java | 10 ++++++++++ .../streams/kstream/UnlimitedWindows.java | 4 ++++ .../apache/kafka/streams/kstream/Window.java | 16 +++++++-------- .../apache/kafka/streams/kstream/Windows.java | 20 +++++++++++++++++++ 8 files changed, 71 insertions(+), 9 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindow.java index fcfe017db36cd..795d58188225e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindow.java @@ -22,4 +22,14 @@ public class HoppingWindow extends Window { public HoppingWindow(long start, long end) { super(start, end); } + + @Override + public boolean overlap(Window other) { + return super.overlap(other) && other.getClass().equals(HoppingWindow.class); + } + + @Override + public boolean equalsTo(Window other) { + return super.equalsTo(other) && other.getClass().equals(HoppingWindow.class); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java index 671289794f0c7..6a60e405b3e51 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java @@ -22,6 +22,8 @@ public class HoppingWindows extends Windows { + private static final String NAME = "HOPPING-WINDOWS-"; + private long size; private long period; @@ -29,6 +31,8 @@ public class HoppingWindows extends Windows { private HoppingWindows(long size) { super(); + setName(newName(NAME)); + this.size = size; this.period = size; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java index e0a79040ac74a..1f38ad84b4b0d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindow.java @@ -23,4 +23,14 @@ public class SlidingWindow extends Window { public SlidingWindow(long start, long end) { super(start, end); } + + @Override + public boolean overlap(Window other) { + return super.overlap(other) && other.getClass().equals(SlidingWindow.class); + } + + @Override + public boolean equalsTo(Window other) { + return super.equalsTo(other) && other.getClass().equals(SlidingWindow.class); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java index e19ee4fd66ab8..f07c908efb8a9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java @@ -23,9 +23,15 @@ public class SlidingWindows extends Windows { + private static final String NAME = "SLIDING-WINDOWS-"; + private long size; private SlidingWindows(long size) { + super(); + + setName(newName(NAME)); + this.size = size; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java index 5e2bb07871147..b2638fedbab00 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindow.java @@ -22,4 +22,14 @@ public class UnlimitedWindow extends Window { public UnlimitedWindow(long start) { super(start, Long.MAX_VALUE); } + + @Override + public boolean overlap(Window other) { + return super.overlap(other) && other.getClass().equals(UnlimitedWindow.class); + } + + @Override + public boolean equalsTo(Window other) { + return super.equalsTo(other) && other.getClass().equals(UnlimitedWindow.class); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java index 5d35a7eeea0dc..1d347c2d77b5d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java @@ -22,11 +22,15 @@ public class UnlimitedWindows extends Windows { + private static final String NAME = "UNLIMITED-WINDOWS-"; + private long start; private UnlimitedWindows(long start) { super(); + setName(newName(NAME)); + this.start = start; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index 5ebe6ab7b595e..63e0a35725f6f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.kstream; -public abstract class Window implements Comparable { +public abstract class Window { private long start; private long end; @@ -41,13 +41,11 @@ public long end() { return end; } - @Override - public int compareTo(Window other) { - if (this.start() > other.start()) - return 1; - else if (this.start() < other.start()) - return -1; - else - return 0; + public boolean overlap(Window other) { + return this.start() < other.end() || other.start() < this.end(); + } + + public boolean equalsTo(Window other) { + return this.start() == other.start() && this.end() == other.end(); } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java index 3c1ed367ce111..ff504e89e9b09 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream; import java.util.Collection; +import java.util.concurrent.atomic.AtomicInteger; public abstract class Windows { @@ -25,15 +26,30 @@ public abstract class Windows { private static final long DEFAULT_MAINTAIN_DURATION = 24 * 60 * 60 * 1000L; // one day + private static final AtomicInteger NAME_INDEX = new AtomicInteger(0); + private long emitDuration; private long maintainDuration; + private String name; + protected Windows() { this.emitDuration = DEFAULT_EMIT_DURATION; this.maintainDuration = DEFAULT_MAINTAIN_DURATION; } + public String name() { + return name; + } + + /** + * Set the window name + */ + protected void setName(String name) { + this.name = name; + } + /** * Set the window emit duration in milliseconds of system time */ @@ -60,6 +76,10 @@ public long maintainMs() { return this.maintainDuration; } + protected String newName(String prefix) { + return prefix + String.format("%010d", NAME_INDEX.getAndIncrement()); + } + abstract boolean equalTo(Windows other); abstract Collection windowsFor(long timestamp); From 2b027bf8614026cbec05404dffd5e9c2598db6f4 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 15 Dec 2015 12:58:11 -0800 Subject: [PATCH 09/14] add missing files --- .../kstream/internals/CountSupplier.java | 51 +++++ .../kstream/internals/SumSupplier.java | 190 ++++++++++++++++++ .../kstream/internals/TopKSupplier.java | 102 ++++++++++ 3 files changed, 343 insertions(+) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java new file mode 100644 index 0000000000000..b28cf1eee96c6 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.AggregateSupplier; +import org.apache.kafka.streams.kstream.Aggregator; + +public class CountSupplier implements AggregateSupplier { + + private class Count implements Aggregator { + @Override + public Long initialValue(){ + return 0L; + } + + public @Override + Long add(K aggKey, V value, Long aggregate) { + return aggregate + 1L; + } + + public @Override + Long remove(K aggKey, V value, Long aggregate) { + return aggregate - 1L; + } + + public @Override + Long merge(Long aggr1, Long aggr2) { + return aggr1 + aggr2; + } + } + + @Override + public Aggregator get(Class keyClass, Class valueClass, Class aggClass) { + return new Count(); + } +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java new file mode 100644 index 0000000000000..886516b8adfb3 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java @@ -0,0 +1,190 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.KafkaException; +import org.apache.kafka.streams.kstream.AggregateSupplier; +import org.apache.kafka.streams.kstream.Aggregator; + +public class SumSupplier implements AggregateSupplier { + + private class LongSum implements Aggregator { + + @Override + public Long initialValue(){ + return 0L; + } + + @Override + public Long add(K aggKey, Long value, Long aggregate) { + return aggregate + value; + } + + @Override + public Long remove(K aggKey, Long value, Long aggregate) { + return aggregate - value; + } + + @Override + public Long merge(Long aggr1, Long aggr2) { + return aggr1 + aggr2; + } + } + + private class IntSum implements Aggregator { + + @Override + public Integer initialValue(){ + return 0; + } + + @Override + public Integer add(K aggKey, Integer value, Integer aggregate) { + return aggregate + value; + } + + @Override + public Integer remove(K aggKey, Integer value, Integer aggregate) { + return aggregate - value; + } + + @Override + public Integer merge(Integer aggr1, Integer aggr2) { + return aggr1 + aggr2; + } + } + + private class ShortSum implements Aggregator { + + @Override + public Short initialValue(){ + return (short) 0; + } + + @Override + public Short add(K aggKey, Short value, Short aggregate) { + return (short) (aggregate + value); + } + + @Override + public Short remove(K aggKey, Short value, Short aggregate) { + return (short) (aggregate - value); + } + + @Override + public Short merge(Short aggr1, Short aggr2) { + return (short) (aggr1 + aggr2); + } + } + + private class DoubleSum implements Aggregator { + + @Override + public Double initialValue(){ + return 0.0d; + } + + @Override + public Double add(K aggKey, Double value, Double aggregate) { + return aggregate + value; + } + + @Override + public Double remove(K aggKey, Double value, Double aggregate) { + return aggregate - value; + } + + @Override + public Double merge(Double aggr1, Double aggr2) { + return aggr1 + aggr2; + } + } + + private class FloatSum implements Aggregator { + + @Override + public Float initialValue(){ + return 0.0f; + } + + @Override + public Float add(K aggKey, Float value, Float aggregate) { + return aggregate + value; + } + + @Override + public Float remove(K aggKey, Float value, Float aggregate) { + return aggregate - value; + } + + @Override + public Float merge(Float aggr1, Float aggr2) { + return aggr1 + aggr2; + } + } + + private class ByteSum implements Aggregator { + + @Override + public Byte initialValue(){ + return (byte) 0L; + } + + @Override + public Byte add(K aggKey, Byte value, Byte aggregate) { + long aggValue = aggregate; + + return (byte) (aggValue + value); + } + + @Override + public Byte remove(K aggKey, Byte value, Byte aggregate) { + long aggValue = aggregate; + + return (byte) (aggValue - value); + } + + @Override + public Byte merge(Byte aggr1, Byte aggr2) { + long aggValue1 = aggr1; + + return (byte) (aggValue1 + aggr2); + } + } + + @SuppressWarnings("unchecked") + @Override + public Aggregator get(Class keyClass, Class valueClass, Class aggClass) { + if (valueClass == Long.class) { + return (Aggregator) new LongSum(); + } else if (valueClass == Integer.class) { + return (Aggregator) new IntSum(); + } else if (valueClass == Short.class) { + return (Aggregator) new ShortSum(); + } else if (valueClass == Double.class) { + return (Aggregator) new DoubleSum(); + } else if (valueClass == Float.class) { + return (Aggregator) new FloatSum(); + } else if (valueClass == Byte.class) { + return (Aggregator) new ByteSum(); + } else { + throw new KafkaException("The value type " + valueClass.getName() + + " is not supported for built-in sum aggregations."); + } + } +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java new file mode 100644 index 0000000000000..d35096476bf70 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.AggregateSupplier; +import org.apache.kafka.streams.kstream.Aggregator; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Set; + +public class TopKSupplier implements AggregateSupplier> { + + private final int k; + + public TopKSupplier(int k) { + this.k = k; + } + + private class TopK implements Aggregator> { + + private final Map> sorted = new HashMap<>(); + + @Override + public Collection initialValue(){ + return Collections.emptySet(); + } + + @Override + public Collection add(K aggKey, V value, Collection aggregate) { + PriorityQueue queue = sorted.get(aggKey); + if (queue == null) { + queue = new PriorityQueue<>(); + sorted.put(aggKey, queue); + } + + queue.add(value); + + PriorityQueue copy = new PriorityQueue<>(queue); + + Set ret = new HashSet<>(); + for (int i = 1; i <= k; i++) + ret.add(copy.poll()); + + return ret; + } + + @Override + public Collection remove(K aggKey, V value, Collection aggregate) { + PriorityQueue queue = sorted.get(aggKey); + + if (queue == null) + throw new IllegalStateException("This should not happen."); + + queue.remove(value); + + PriorityQueue copy = new PriorityQueue<>(queue); + + Set ret = new HashSet<>(); + for (int i = 1; i <= k; i++) + ret.add(copy.poll()); + + return ret; + } + + @Override + public Collection merge(Collection aggr1, Collection aggr2) { + PriorityQueue copy = new PriorityQueue<>(aggr1); + copy.addAll(aggr2); + + Set ret = new HashSet<>(); + for (int i = 1; i <= k; i++) + ret.add(copy.poll()); + + return ret; + } + } + + @Override + public Aggregator> get(Class keyClass, Class valueClass, Class> aggClass) { + return new TopK<>(); + } +} \ No newline at end of file From 5214b12fcd66eb4cfa9af4258ca2146c11aa2e89 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 15 Dec 2015 15:11:27 -0800 Subject: [PATCH 10/14] address Yasu's comments --- .../kafka/streams/examples/KTableJob.java | 34 ++-- .../streams/kstream/AggregateSupplier.java | 2 +- .../apache/kafka/streams/kstream/KStream.java | 2 +- .../apache/kafka/streams/kstream/KTable.java | 2 +- .../kstream/internals/CountSupplier.java | 2 +- .../kstream/internals/KStreamImpl.java | 4 +- .../streams/kstream/internals/KTableImpl.java | 4 +- .../kstream/internals/SumSupplier.java | 160 ++---------------- .../kstream/internals/TopKSupplier.java | 6 +- 9 files changed, 49 insertions(+), 167 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java index a68bbe216897c..a60263675b69b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java @@ -33,6 +33,7 @@ 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.streams.kstream.Window; import org.apache.kafka.streams.kstream.WindowMapper; @@ -56,33 +57,44 @@ public static void main(String[] args) throws Exception { // stream aggregate KStream stream1 = builder.stream("topic1"); - KWindowedTable wtable1 = stream1.sumByKey(HoppingWindows.of(1000).every(500).emit(1000).until(1000 * 60 * 60 * 24 /* one day */)); + @SuppressWarnings("unchecked") + KWindowedTable wtable1 = stream1.sumByKey(new ValueMapper() { + @Override + public Long apply(Integer value) { + return (long) value; + } + }, HoppingWindows.of(1000).every(500).emit(1000).until(1000 * 60 * 60 * 24 /* one day */)); // table aggregation KTable table1 = builder.table("topic2"); - KTable table2 = table1.sum(new KeyValueMapper>() { + KTable table2 = table1.sum(new KeyValueMapper>() { @Override public KeyValue apply(String key, String value) { return new KeyValue<>(value, new Integer(key)); } + }, new ValueMapper() { + @Override + public Long apply(Integer value) { + return (long) value; + } }, "table2"); // stream-table join - KStream stream2 = stream1.leftJoin(table2, new ValueJoiner() { + KStream stream2 = stream1.leftJoin(table2, new ValueJoiner() { @Override - public Integer apply(Integer value1, Integer value2) { + public Long apply(Integer value1, Long value2) { if (value2 == null) - return 0; + return 0L; else return value1 * value2; } }); // table-table join - KTable table3 = table1.outerJoin(table2, new ValueJoiner() { + KTable table3 = table1.outerJoin(table2, new ValueJoiner() { @Override - public String apply(String value1, Integer value2) { + public String apply(String value1, Long value2) { if (value2 == null) return value1 + "-null"; else if (value1 == null) @@ -93,9 +105,9 @@ else if (value1 == null) }); // windowed table self join - KWindowedTable wtable2 = wtable1.leftJoin(wtable1, 1000 * 60 * 60 * 24 * 7 /* a week ago*/, new ValueJoiner() { + KWindowedTable wtable2 = wtable1.leftJoin(wtable1, 1000 * 60 * 60 * 24 * 7 /* a week ago*/, new ValueJoiner() { @Override - public Integer apply(Integer value1, Integer value2) { + public Long apply(Long value1, Long value2) { if (value2 == null) return value1; else @@ -103,9 +115,9 @@ public Integer apply(Integer value1, Integer value2) { } }); - KStream stream3 = wtable2.toStream(new WindowMapper() { + KStream stream3 = wtable2.toStream(new WindowMapper() { @Override - public KeyValue apply(String key, Integer value, HoppingWindow window) { + public KeyValue apply(String key, Long value, HoppingWindow window) { return new KeyValue<>(key + window.start(), value.toString()); } }); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java index cd6d317c99acc..3ee80756adbb4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java @@ -19,5 +19,5 @@ public interface AggregateSupplier { - Aggregator get(Class keyClass, Class valueClass, Class aggClass); + Aggregator get(); } 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 caa5fc5c46b08..32a0ceec0af23 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 @@ -199,7 +199,7 @@ public interface KStream { /** * Sum values of this stream by key on a window basis. */ - KWindowedTable sumByKey(Windows windows); + KWindowedTable sumByKey(ValueMapper valueMapper, Windows windows); /** * Count number of records of this stream by key on a window basis. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index d08e547746257..6ac0064986613 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -154,7 +154,7 @@ public interface KTable { /** * Sum values of this table by the selected key on a window basis. */ - KTablesum(KeyValueMapper> selector, String name); + KTablesum(KeyValueMapper> selector, ValueMapper valueMapper, String name); /** * Count number of records of this table by the selected key on a window basis. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java index b28cf1eee96c6..39e4188d4913d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java @@ -45,7 +45,7 @@ Long merge(Long aggr1, Long aggr2) { } @Override - public Aggregator get(Class keyClass, Class valueClass, Class aggClass) { + public Aggregator get() { return new Count(); } } \ No newline at end of file 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 266280c1b0248..72651c2819f38 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 @@ -268,9 +268,9 @@ public KWindowedTable aggregateByKey(AggregateSup } @Override - public KWindowedTable sumByKey(Windows windows) { + public KWindowedTable sumByKey(ValueMapper valueMapper, Windows windows) { - return this.aggregateByKey(new SumSupplier<>(), windows); + return this.aggregateByKey(new SumSupplier<>(valueMapper), windows); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 1495f7f735d8b..646e0d6757b0d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -254,9 +254,9 @@ public KTable aggregate(AggregateSupplier aggre } @Override - public KTablesum(KeyValueMapper> selector, String name) { + public KTable sum(KeyValueMapper> selector, ValueMapper valueMapper, String name) { - return this.aggregate(new SumSupplier<>(), selector, name); + return this.aggregate(new SumSupplier<>(valueMapper), selector, name); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java index 886516b8adfb3..bd2b99e8365f7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java @@ -17,174 +17,44 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.streams.kstream.AggregateSupplier; import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.ValueMapper; -public class SumSupplier implements AggregateSupplier { +public class SumSupplier implements AggregateSupplier { - private class LongSum implements Aggregator { + private final ValueMapper valueMapper; - @Override - public Long initialValue(){ - return 0L; - } - - @Override - public Long add(K aggKey, Long value, Long aggregate) { - return aggregate + value; - } - - @Override - public Long remove(K aggKey, Long value, Long aggregate) { - return aggregate - value; - } - - @Override - public Long merge(Long aggr1, Long aggr2) { - return aggr1 + aggr2; - } - } - - private class IntSum implements Aggregator { - - @Override - public Integer initialValue(){ - return 0; - } - - @Override - public Integer add(K aggKey, Integer value, Integer aggregate) { - return aggregate + value; - } - - @Override - public Integer remove(K aggKey, Integer value, Integer aggregate) { - return aggregate - value; - } - - @Override - public Integer merge(Integer aggr1, Integer aggr2) { - return aggr1 + aggr2; - } - } - - private class ShortSum implements Aggregator { - - @Override - public Short initialValue(){ - return (short) 0; - } - - @Override - public Short add(K aggKey, Short value, Short aggregate) { - return (short) (aggregate + value); - } - - @Override - public Short remove(K aggKey, Short value, Short aggregate) { - return (short) (aggregate - value); - } - - @Override - public Short merge(Short aggr1, Short aggr2) { - return (short) (aggr1 + aggr2); - } - } - - private class DoubleSum implements Aggregator { - - @Override - public Double initialValue(){ - return 0.0d; - } - - @Override - public Double add(K aggKey, Double value, Double aggregate) { - return aggregate + value; - } - - @Override - public Double remove(K aggKey, Double value, Double aggregate) { - return aggregate - value; - } - - @Override - public Double merge(Double aggr1, Double aggr2) { - return aggr1 + aggr2; - } + public SumSupplier(ValueMapper valueMapper) { + this.valueMapper = valueMapper; } - private class FloatSum implements Aggregator { + private class LongSum implements Aggregator { @Override - public Float initialValue(){ - return 0.0f; + public Long initialValue(){ + return 0L; } @Override - public Float add(K aggKey, Float value, Float aggregate) { - return aggregate + value; + public Long add(K aggKey, V value, Long aggregate) { + return aggregate + valueMapper.apply(value); } @Override - public Float remove(K aggKey, Float value, Float aggregate) { - return aggregate - value; + public Long remove(K aggKey, V value, Long aggregate) { + return aggregate - valueMapper.apply(value); } @Override - public Float merge(Float aggr1, Float aggr2) { + public Long merge(Long aggr1, Long aggr2) { return aggr1 + aggr2; } } - private class ByteSum implements Aggregator { - - @Override - public Byte initialValue(){ - return (byte) 0L; - } - - @Override - public Byte add(K aggKey, Byte value, Byte aggregate) { - long aggValue = aggregate; - - return (byte) (aggValue + value); - } - - @Override - public Byte remove(K aggKey, Byte value, Byte aggregate) { - long aggValue = aggregate; - - return (byte) (aggValue - value); - } - - @Override - public Byte merge(Byte aggr1, Byte aggr2) { - long aggValue1 = aggr1; - - return (byte) (aggValue1 + aggr2); - } - } - @SuppressWarnings("unchecked") @Override - public Aggregator get(Class keyClass, Class valueClass, Class aggClass) { - if (valueClass == Long.class) { - return (Aggregator) new LongSum(); - } else if (valueClass == Integer.class) { - return (Aggregator) new IntSum(); - } else if (valueClass == Short.class) { - return (Aggregator) new ShortSum(); - } else if (valueClass == Double.class) { - return (Aggregator) new DoubleSum(); - } else if (valueClass == Float.class) { - return (Aggregator) new FloatSum(); - } else if (valueClass == Byte.class) { - return (Aggregator) new ByteSum(); - } else { - throw new KafkaException("The value type " + valueClass.getName() + - " is not supported for built-in sum aggregations."); - } + public Aggregator get() { + return new LongSum(); } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java index d35096476bf70..e09ddbe2004f4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java @@ -36,7 +36,7 @@ public TopKSupplier(int k) { this.k = k; } - private class TopK implements Aggregator> { + private class TopK implements Aggregator> { private final Map> sorted = new HashMap<>(); @@ -96,7 +96,7 @@ public Collection merge(Collection aggr1, Collection aggr2) { } @Override - public Aggregator> get(Class keyClass, Class valueClass, Class> aggClass) { - return new TopK<>(); + public Aggregator> get() { + return new TopK(); } } \ No newline at end of file From a603a9afde8a86906d085b6cf942df67d2082fb9 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 15 Dec 2015 15:15:29 -0800 Subject: [PATCH 11/14] rename aggregateSupplier to aggregatorSupplier --- .../java/org/apache/kafka/streams/examples/KTableJob.java | 3 --- .../{AggregateSupplier.java => AggregatorSupplier.java} | 2 +- .../main/java/org/apache/kafka/streams/kstream/KStream.java | 4 ++-- .../main/java/org/apache/kafka/streams/kstream/KTable.java | 4 ++-- .../apache/kafka/streams/kstream/internals/CountSupplier.java | 4 ++-- .../apache/kafka/streams/kstream/internals/KStreamImpl.java | 4 ++-- .../apache/kafka/streams/kstream/internals/KTableImpl.java | 4 ++-- .../apache/kafka/streams/kstream/internals/SumSupplier.java | 4 ++-- .../apache/kafka/streams/kstream/internals/TopKSupplier.java | 4 ++-- 9 files changed, 15 insertions(+), 18 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/kstream/{AggregateSupplier.java => AggregatorSupplier.java} (94%) diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java index a60263675b69b..e296f01a5bc6e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java @@ -20,8 +20,6 @@ import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.streams.kstream.AggregateSupplier; -import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.HoppingWindow; import org.apache.kafka.streams.kstream.HoppingWindows; import org.apache.kafka.streams.kstream.KStreamBuilder; @@ -34,7 +32,6 @@ 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.streams.kstream.Window; import org.apache.kafka.streams.kstream.WindowMapper; import java.util.Properties; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/AggregatorSupplier.java similarity index 94% rename from streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/AggregatorSupplier.java index 3ee80756adbb4..6ed9125a888bf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/AggregateSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/AggregatorSupplier.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.kstream; -public interface AggregateSupplier { +public interface AggregatorSupplier { Aggregator get(); } 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 32a0ceec0af23..b4cf253576ffa 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 @@ -190,11 +190,11 @@ public interface KStream { /** * Aggregate values of this stream by key on a window basis. * - * @param aggregateSupplier the class of aggregateSupplier + * @param aggregatorSupplier the class of aggregatorSupplier * @param windows the window definition * @param the value type of the aggregated table */ - KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, Windows windows); + KWindowedTable aggregateByKey(AggregatorSupplier aggregatorSupplier, Windows windows); /** * Sum values of this stream by key on a window basis. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 6ac0064986613..010881ed0cfeb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -143,13 +143,13 @@ public interface KTable { /** * Aggregate values of this table by the selected key. * - * @param aggregateSupplier the class of aggregateSupplier + * @param aggregatorSupplier the class of aggregatorSupplier * @param selector the KeyValue mapper that select the aggregate key * @param name the name of the resulted table * @param the key type of the aggregated table * @param the value type of the aggregated table */ - KTable aggregate(AggregateSupplier aggregateSupplier, KeyValueMapper> selector, String name); + KTable aggregate(AggregatorSupplier aggregatorSupplier, KeyValueMapper> selector, String name); /** * Sum values of this table by the selected key on a window basis. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java index 39e4188d4913d..6203afa2305c2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java @@ -17,10 +17,10 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.kstream.AggregateSupplier; +import org.apache.kafka.streams.kstream.AggregatorSupplier; import org.apache.kafka.streams.kstream.Aggregator; -public class CountSupplier implements AggregateSupplier { +public class CountSupplier implements AggregatorSupplier { private class Count implements Aggregator { @Override 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 72651c2819f38..ac090b8d81338 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 @@ -19,7 +19,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streams.kstream.AggregateSupplier; +import org.apache.kafka.streams.kstream.AggregatorSupplier; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KWindowedTable; @@ -262,7 +262,7 @@ public Processor get() { } @Override - public KWindowedTable aggregateByKey(AggregateSupplier aggregateSupplier, Windows windows) { + public KWindowedTable aggregateByKey(AggregatorSupplier aggregatorSupplier, Windows windows) { // TODO return null; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 646e0d6757b0d..a8208e877df0c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streams.kstream.AggregateSupplier; +import org.apache.kafka.streams.kstream.AggregatorSupplier; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -248,7 +248,7 @@ public KTable leftJoin(KTable other, ValueJoiner } @Override - public KTable aggregate(AggregateSupplier aggregateSupplier, KeyValueMapper> selector, String name) { + public KTable aggregate(AggregatorSupplier aggregatorSupplier, KeyValueMapper> selector, String name) { // TODO return null; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java index bd2b99e8365f7..fd31802327fe4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java @@ -17,11 +17,11 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.kstream.AggregateSupplier; +import org.apache.kafka.streams.kstream.AggregatorSupplier; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.ValueMapper; -public class SumSupplier implements AggregateSupplier { +public class SumSupplier implements AggregatorSupplier { private final ValueMapper valueMapper; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java index e09ddbe2004f4..5e5b7781bb0cc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.kstream.AggregateSupplier; +import org.apache.kafka.streams.kstream.AggregatorSupplier; import org.apache.kafka.streams.kstream.Aggregator; import java.util.Collection; @@ -28,7 +28,7 @@ import java.util.PriorityQueue; import java.util.Set; -public class TopKSupplier implements AggregateSupplier> { +public class TopKSupplier implements AggregatorSupplier> { private final int k; From e186710bc3b66e88148ab81087276cedffa2bad3 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 16 Dec 2015 14:20:59 -0800 Subject: [PATCH 12/14] modify built-in aggregates --- .../kafka/streams/examples/KTableJob.java | 25 ++++++++-------- .../apache/kafka/streams/kstream/KStream.java | 8 +++-- .../apache/kafka/streams/kstream/KTable.java | 10 +++++-- .../streams/kstream/KeyValueToIntMapper.java | 23 +++++++++++++++ .../kstream/internals/KStreamImpl.java | 27 ++++++++++++----- .../streams/kstream/internals/KTableImpl.java | 29 ++++++++++++++----- .../kstream/internals/TopKSupplier.java | 2 +- 7 files changed, 92 insertions(+), 32 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToIntMapper.java diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java index e296f01a5bc6e..a4a004137db24 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java @@ -30,6 +30,7 @@ import org.apache.kafka.streams.kstream.KWindowedTable; import org.apache.kafka.streams.kstream.KeyValue; import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.KeyValueToLongMapper; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.kstream.WindowMapper; @@ -52,35 +53,35 @@ public static void main(String[] args) throws Exception { KStreamBuilder builder = new KStreamBuilder(); // stream aggregate - KStream stream1 = builder.stream("topic1"); + KStream stream1 = builder.stream("topic1"); @SuppressWarnings("unchecked") - KWindowedTable wtable1 = stream1.sumByKey(new ValueMapper() { + KWindowedTable wtable1 = stream1.sumByKey(new KeyValueToLongMapper() { @Override - public Long apply(Integer value) { - return (long) value; + public long apply(String key, Long value) { + return value; } }, HoppingWindows.of(1000).every(500).emit(1000).until(1000 * 60 * 60 * 24 /* one day */)); // table aggregation KTable table1 = builder.table("topic2"); - KTable table2 = table1.sum(new KeyValueMapper>() { + KTable table2 = table1.sum(new KeyValueMapper() { @Override - public KeyValue apply(String key, String value) { - return new KeyValue<>(value, new Integer(key)); + public String apply(String key, String value) { + return value; } - }, new ValueMapper() { + }, new KeyValueToLongMapper() { @Override - public Long apply(Integer value) { - return (long) value; + public long apply(String key, String value) { + return Long.parseLong(value); } }, "table2"); // stream-table join - KStream stream2 = stream1.leftJoin(table2, new ValueJoiner() { + KStream stream2 = stream1.leftJoin(table2, new ValueJoiner() { @Override - public Long apply(Integer value1, Long value2) { + public Long apply(Long value1, Long value2) { if (value2 == null) return 0L; else 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 b4cf253576ffa..b8dd9b5d71d37 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 @@ -199,7 +199,11 @@ public interface KStream { /** * Sum values of this stream by key on a window basis. */ - KWindowedTable sumByKey(ValueMapper valueMapper, Windows windows); + KWindowedTable sumByKey(KeyValueToLongMapper valueSelector, Windows windows); + + KWindowedTable sumByKey(KeyValueToIntMapper valueSelector, Windows windows); + + KWindowedTable sumByKey(KeyValueToDoubleMapper valueSelector, Windows windows); /** * Count number of records of this stream by key on a window basis. @@ -209,5 +213,5 @@ public interface KStream { /** * Get the top-k records of this stream by key on a window basis. */ - KWindowedTable, W> topKByKey(int k, Windows windows); + > KWindowedTable, W> topKByKey(int k, KeyValueMapper valueSelector, Windows windows); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 010881ed0cfeb..7f70fbd096230 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -154,15 +154,19 @@ public interface KTable { /** * Sum values of this table by the selected key on a window basis. */ - KTablesum(KeyValueMapper> selector, ValueMapper valueMapper, String name); + KTable sum(KeyValueMapper keySelector, KeyValueToLongMapper valueSelector, String name); + + KTable sum(KeyValueMapper keySelector, KeyValueToIntMapper valueSelector, String name); + + KTable sum(KeyValueMapper keySelector, KeyValueToDoubleMapper valueSelector, String name); /** * Count number of records of this table by the selected key on a window basis. */ - KTable count(KeyValueMapper> selector, String name); + KTable count(KeyValueMapper keySelector, String name); /** * Get the top-k records of this table by the selected key on a window basis. */ - KTable> topK(int k, KeyValueMapper> selector, String name); + > KTable> topK(int k, KeyValueMapper selector, String name); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToIntMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToIntMapper.java new file mode 100644 index 0000000000000..72e5ee935a1c2 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToIntMapper.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream; + +public interface KeyValueToIntMapper { + + int apply(K key, V value); +} 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 ac090b8d81338..5956d2872f8c8 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 @@ -24,6 +24,9 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KWindowedTable; import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueToDoubleMapper; +import org.apache.kafka.streams.kstream.KeyValueToIntMapper; +import org.apache.kafka.streams.kstream.KeyValueToLongMapper; import org.apache.kafka.streams.kstream.TransformerSupplier; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueTransformerSupplier; @@ -268,20 +271,30 @@ public KWindowedTable aggregateByKey(AggregatorSu } @Override - public KWindowedTable sumByKey(ValueMapper valueMapper, Windows windows) { + public KWindowedTable sumByKey(KeyValueToLongMapper valueSelector, Windows windows) { + // TODO + return null; + } - return this.aggregateByKey(new SumSupplier<>(valueMapper), windows); + public KWindowedTable sumByKey(KeyValueToIntMapper valueSelector, Windows windows) { + // TODO + return null; + } + + public KWindowedTable sumByKey(KeyValueToDoubleMapper valueSelector, Windows windows) { + // TODO + return null; } @Override public KWindowedTable countByKey(Windows windows) { - - return this.aggregateByKey(new CountSupplier<>(), windows); + // TODO + return null; } @Override - public KWindowedTable, W> topKByKey(int k, Windows windows) { - - return this.aggregateByKey(new TopKSupplier<>(k), windows); + public > KWindowedTable, W> topKByKey(int k, KeyValueMapper valueSelector, Windows windows) { + // TODO + return null; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index a8208e877df0c..08301e630a970 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -25,6 +25,9 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValue; import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.KeyValueToDoubleMapper; +import org.apache.kafka.streams.kstream.KeyValueToIntMapper; +import org.apache.kafka.streams.kstream.KeyValueToLongMapper; import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; @@ -254,20 +257,32 @@ public KTable aggregate(AggregatorSupplier aggr } @Override - public KTable sum(KeyValueMapper> selector, ValueMapper valueMapper, String name) { - - return this.aggregate(new SumSupplier<>(valueMapper), selector, name); + public KTable sum(KeyValueMapper keySelector, KeyValueToLongMapper valueSelector, String name) { + // TODO + return null; } @Override - public KTable count(KeyValueMapper> selector, String name) { + public KTable sum(KeyValueMapper keySelector, KeyValueToIntMapper valueSelector, String name) { + // TODO + return null; + } - return this.aggregate(new CountSupplier<>(), selector, name); + @Override + public KTable sum(KeyValueMapper keySelector, KeyValueToDoubleMapper valueSelector, String name) { + // TODO + return null; } @Override - public KTable> topK(int k, KeyValueMapper> selector, String name) { + public KTable count(KeyValueMapper keySelector, String name) { + // TODO + return null; + } - return this.aggregate(new TopKSupplier<>(k), selector, name); + @Override + public > KTable> topK(int k, KeyValueMapper selector, String name) { + // TODO + return null; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java index 5e5b7781bb0cc..becdeceba7d07 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java @@ -28,7 +28,7 @@ import java.util.PriorityQueue; import java.util.Set; -public class TopKSupplier implements AggregatorSupplier> { +public class TopKSupplier> implements AggregatorSupplier> { private final int k; From 5bb1e8c95e0c1ab131d5212d1a7d793ce8b49414 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 16 Dec 2015 14:24:10 -0800 Subject: [PATCH 13/14] add missing files --- .../kstream/KeyValueToDoubleMapper.java | 23 +++++++++++++++++++ .../streams/kstream/KeyValueToLongMapper.java | 23 +++++++++++++++++++ 2 files changed, 46 insertions(+) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToDoubleMapper.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToLongMapper.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToDoubleMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToDoubleMapper.java new file mode 100644 index 0000000000000..ae3b85817f9fe --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToDoubleMapper.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream; + +public interface KeyValueToDoubleMapper { + + double apply(K key, V value); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToLongMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToLongMapper.java new file mode 100644 index 0000000000000..3a8d8a89d1814 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueToLongMapper.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.kstream; + +public interface KeyValueToLongMapper { + + long apply(K key, V value); +} From 4570dd0d98526f8388c13ef5fe4af12d372f73c6 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 16 Dec 2015 16:01:58 -0800 Subject: [PATCH 14/14] further comments addressed --- .../kafka/streams/examples/KTableJob.java | 2 +- .../kafka/streams/kstream/HoppingWindows.java | 28 +++++---- .../apache/kafka/streams/kstream/KStream.java | 25 +++++++- .../apache/kafka/streams/kstream/KTable.java | 42 ++++++++++--- .../kafka/streams/kstream/SlidingWindows.java | 21 ++++--- .../streams/kstream/UnlimitedWindows.java | 20 ++++--- .../apache/kafka/streams/kstream/Windows.java | 9 +-- .../kstream/internals/CountSupplier.java | 51 ---------------- .../streams/kstream/internals/KTableImpl.java | 2 +- .../kstream/internals/SumSupplier.java | 60 ------------------- .../kstream/internals/TopKSupplier.java | 4 ++ 11 files changed, 108 insertions(+), 156 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java index a4a004137db24..c58f3b3f4f62c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KTableJob.java @@ -61,7 +61,7 @@ public static void main(String[] args) throws Exception { public long apply(String key, Long value) { return value; } - }, HoppingWindows.of(1000).every(500).emit(1000).until(1000 * 60 * 60 * 24 /* one day */)); + }, HoppingWindows.of("window1").with(500L).every(500L).emit(1000L).until(1000L * 60 * 60 * 24 /* one day */)); // table aggregation KTable table1 = builder.table("topic2"); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java index 6a60e405b3e51..289d054c32855 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java @@ -22,27 +22,35 @@ public class HoppingWindows extends Windows { - private static final String NAME = "HOPPING-WINDOWS-"; + private static final long DEFAULT_SIZE_MS = 1000L; private long size; private long period; - private HoppingWindows(long size) { - super(); + private HoppingWindows(String name) { + super(name); - setName(newName(NAME)); - - this.size = size; - this.period = size; + this.size = DEFAULT_SIZE_MS; + this.period = this.size; } /** * Returns a half-interval hopping window definition with the window size in milliseconds - * of the form [ N * size, N * size + size ) + * of the form [ N * default_size, N * default_size + default_size ) */ - public static HoppingWindows of(long size) { - return new HoppingWindows(size); + public static HoppingWindows of(String name) { + return new HoppingWindows(name); + } + + /** + * Returns a new hopping window definition with the original size but reassign the window + * period in milliseconds of the form [ N * period, N * period + size ) + */ + public HoppingWindows with(long size) { + this.size = size; + + return this; } /** 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 b8dd9b5d71d37..3ffb69bef3c96 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 @@ -197,21 +197,42 @@ public interface KStream { KWindowedTable aggregateByKey(AggregatorSupplier aggregatorSupplier, Windows windows); /** - * Sum values of this stream by key on a window basis. + * Sum extracted long integer values of this stream by key on a window basis. + * + * @param valueSelector the class of KeyValueToLongMapper to extract the long integer from value + * @param windows the window definition */ KWindowedTable sumByKey(KeyValueToLongMapper valueSelector, Windows windows); + /** + * Sum extracted integer values of this stream by key on a window basis. + * + * @param valueSelector the class of KeyValueToIntMapper to extract the long integer from value + * @param windows the window definition + */ KWindowedTable sumByKey(KeyValueToIntMapper valueSelector, Windows windows); + /** + * Sum extracted double decimal values of this stream by key on a window basis. + * + * @param valueSelector the class of KeyValueToDoubleMapper to extract the long integer from value + * @param windows the window definition + */ KWindowedTable sumByKey(KeyValueToDoubleMapper valueSelector, Windows windows); /** * Count number of records of this stream by key on a window basis. + * + * @param windows the window definition */ KWindowedTable countByKey(Windows windows); /** - * Get the top-k records of this stream by key on a window basis. + * Get the top-k values of this stream by key on a window basis. + * + * @param k parameter of the top-k computation + * @param valueSelector the class of KeyValueMapper to extract the comparable value + * @param windows the window definition */ > KWindowedTable, W> topKByKey(int k, KeyValueMapper valueSelector, Windows windows); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 7f70fbd096230..71a3486fd4dd0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -114,7 +114,7 @@ public interface KTable { * @param joiner ValueJoiner * @param the value type of the other stream * @param the value type of the new stream - * @return the instance of KStream + * @return the instance of KTable */ KTable join(KTable other, ValueJoiner joiner); @@ -125,7 +125,7 @@ public interface KTable { * @param joiner ValueJoiner * @param the value type of the other stream * @param the value type of the new stream - * @return the instance of KStream + * @return the instance of KTable */ KTable outerJoin(KTable other, ValueJoiner joiner); @@ -136,37 +136,63 @@ public interface KTable { * @param joiner ValueJoiner * @param the value type of the other stream * @param the value type of the new stream - * @return the instance of KStream + * @return the instance of KTable */ KTable leftJoin(KTable other, ValueJoiner joiner); /** * Aggregate values of this table by the selected key. * - * @param aggregatorSupplier the class of aggregatorSupplier + * @param aggregatorSupplier the class of AggregatorSupplier * @param selector the KeyValue mapper that select the aggregate key * @param name the name of the resulted table * @param the key type of the aggregated table * @param the value type of the aggregated table + * @return the instance of KTable */ KTable aggregate(AggregatorSupplier aggregatorSupplier, KeyValueMapper> selector, String name); /** - * Sum values of this table by the selected key on a window basis. + * Sum extracted long integer values of this table by the selected aggregation key + * + * @param keySelector the class of KeyValueMapper to select the aggregation key + * @param valueSelector the class of KeyValueToLongMapper to extract the long integer from value + * @param name the name of the resulted table */ KTable sum(KeyValueMapper keySelector, KeyValueToLongMapper valueSelector, String name); + /** + * Sum extracted integer values of this table by the selected aggregation key + * + * @param keySelector the class of KeyValueMapper to select the aggregation key + * @param valueSelector the class of KeyValueToIntMapper to extract the long integer from value + * @param name the name of the resulted table + */ KTable sum(KeyValueMapper keySelector, KeyValueToIntMapper valueSelector, String name); + /** + * Sum extracted double decimal values of this table by the selected aggregation key + * + * @param keySelector the class of KeyValueMapper to select the aggregation key + * @param valueSelector the class of KeyValueToDoubleMapper to extract the long integer from value + * @param name the name of the resulted table + */ KTable sum(KeyValueMapper keySelector, KeyValueToDoubleMapper valueSelector, String name); /** - * Count number of records of this table by the selected key on a window basis. + * Count number of records of this table by the selected aggregation key + * + * @param keySelector the class of KeyValueMapper to select the aggregation key + * @param name the name of the resulted table */ KTable count(KeyValueMapper keySelector, String name); /** - * Get the top-k records of this table by the selected key on a window basis. + * Get the top-k values of this table by the selected aggregation key + * + * @param k parameter of the top-k computation + * @param keySelector the class of KeyValueMapper to select the aggregation key + * @param name the name of the resulted table */ - > KTable> topK(int k, KeyValueMapper selector, String name); + > KTable> topK(int k, KeyValueMapper keySelector, String name); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java index f07c908efb8a9..eb98dfdfbfb7b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindows.java @@ -23,23 +23,30 @@ public class SlidingWindows extends Windows { - private static final String NAME = "SLIDING-WINDOWS-"; + private static final long DEFAULT_SIZE_MS = 1000L; private long size; - private SlidingWindows(long size) { - super(); + private SlidingWindows(String name) { + super(name); - setName(newName(NAME)); + this.size = DEFAULT_SIZE_MS; + } - this.size = size; + /** + * Returns a half-interval sliding window definition with the default window size + */ + public static SlidingWindows of(String name) { + return new SlidingWindows(name); } /** * Returns a half-interval sliding window definition with the window size in milliseconds */ - public static SlidingWindows of(long size) { - return new SlidingWindows(size); + public SlidingWindows with(long size) { + this.size = size; + + return this; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java index 1d347c2d77b5d..c4dff697b06ef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java @@ -22,23 +22,27 @@ public class UnlimitedWindows extends Windows { - private static final String NAME = "UNLIMITED-WINDOWS-"; + private static final long DEFAULT_START_TIMESTAMP = 0L; private long start; - private UnlimitedWindows(long start) { - super(); + private UnlimitedWindows(String name) { + super(name); - setName(newName(NAME)); - - this.start = start; + this.start = DEFAULT_START_TIMESTAMP; } /** * Returns an unlimited window definition */ - public static UnlimitedWindows on(long start) { - return new UnlimitedWindows(start); + public static UnlimitedWindows of(String name) { + return new UnlimitedWindows(name); + } + + public UnlimitedWindows startOn(long start) { + this.start = start; + + return this; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java index ff504e89e9b09..9a4a6ed45f084 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java @@ -34,7 +34,7 @@ public abstract class Windows { private String name; - protected Windows() { + protected Windows(String name) { this.emitDuration = DEFAULT_EMIT_DURATION; this.maintainDuration = DEFAULT_MAINTAIN_DURATION; } @@ -43,13 +43,6 @@ public String name() { return name; } - /** - * Set the window name - */ - protected void setName(String name) { - this.name = name; - } - /** * Set the window emit duration in milliseconds of system time */ diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java deleted file mode 100644 index 6203afa2305c2..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CountSupplier.java +++ /dev/null @@ -1,51 +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.kstream.AggregatorSupplier; -import org.apache.kafka.streams.kstream.Aggregator; - -public class CountSupplier implements AggregatorSupplier { - - private class Count implements Aggregator { - @Override - public Long initialValue(){ - return 0L; - } - - public @Override - Long add(K aggKey, V value, Long aggregate) { - return aggregate + 1L; - } - - public @Override - Long remove(K aggKey, V value, Long aggregate) { - return aggregate - 1L; - } - - public @Override - Long merge(Long aggr1, Long aggr2) { - return aggr1 + aggr2; - } - } - - @Override - public Aggregator get() { - return new Count(); - } -} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 08301e630a970..8e5ec8054ea4d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -281,7 +281,7 @@ public KTable count(KeyValueMapper keySelector, String } @Override - public > KTable> topK(int k, KeyValueMapper selector, String name) { + public > KTable> topK(int k, KeyValueMapper keySelector, String name) { // TODO return null; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.java deleted file mode 100644 index fd31802327fe4..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SumSupplier.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.streams.kstream.internals; - -import org.apache.kafka.streams.kstream.AggregatorSupplier; -import org.apache.kafka.streams.kstream.Aggregator; -import org.apache.kafka.streams.kstream.ValueMapper; - -public class SumSupplier implements AggregatorSupplier { - - private final ValueMapper valueMapper; - - public SumSupplier(ValueMapper valueMapper) { - this.valueMapper = valueMapper; - } - - private class LongSum implements Aggregator { - - @Override - public Long initialValue(){ - return 0L; - } - - @Override - public Long add(K aggKey, V value, Long aggregate) { - return aggregate + valueMapper.apply(value); - } - - @Override - public Long remove(K aggKey, V value, Long aggregate) { - return aggregate - valueMapper.apply(value); - } - - @Override - public Long merge(Long aggr1, Long aggr2) { - return aggr1 + aggr2; - } - } - - @SuppressWarnings("unchecked") - @Override - public Aggregator get() { - return new LongSum(); - } -} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java index becdeceba7d07..2d765cb74b96a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TopKSupplier.java @@ -28,6 +28,10 @@ import java.util.PriorityQueue; import java.util.Set; +/** + * NOTE: This is just a demo aggregate supplier that can be implemented by users to add their own built-in aggregates. + * It is highly in-efficient and is not supposed to be merged in. + */ public class TopKSupplier> implements AggregatorSupplier> { private final int k;