├── .travis.yml ├── _config.yml ├── .gitignore ├── charts ├── SlidingWindow.png └── ConcurrentTumblingWindows.png ├── core ├── src │ └── main │ │ └── java │ │ └── stream │ │ └── scotty │ │ └── core │ │ ├── windowType │ │ ├── WindowMeasure.java │ │ ├── windowContext │ │ │ ├── WindowModifications.java │ │ │ ├── AddModification.java │ │ │ ├── DeleteModification.java │ │ │ └── ShiftModification.java │ │ ├── Window.java │ │ ├── ForwardContextFree.java │ │ ├── TupleContext.java │ │ ├── ForwardContextAware.java │ │ ├── ContextFreeWindow.java │ │ ├── TumblingWindow.java │ │ ├── SlidingWindow.java │ │ └── FixedBandWindow.java │ │ ├── WindowCollector.java │ │ ├── windowFunction │ │ ├── InvertibleReduceAggregateFunction.java │ │ ├── CommutativeAggregateFunction.java │ │ ├── CloneablePartialStateFunction.java │ │ ├── ReduceAggregateFunction.java │ │ ├── InvertibleAggregateFunction.java │ │ └── AggregateFunction.java │ │ ├── AggregateWindow.java │ │ ├── WindowOperator.java │ │ └── TimeMeasure.java └── pom.xml ├── logs └── workers-artifacts │ └── testTopology-1-1613327334 │ └── 1024 │ └── worker.yaml ├── state ├── src │ └── main │ │ └── java │ │ └── stream │ │ └── scotty │ │ └── state │ │ ├── ValueState.java │ │ ├── State.java │ │ ├── ListState.java │ │ ├── StateFactory.java │ │ ├── SetState.java │ │ └── memory │ │ ├── MemoryStateFactory.java │ │ ├── MemoryListState.java │ │ ├── MemorySetState.java │ │ └── MemoryValueState.java └── pom.xml ├── pom.xml.asc ├── slicing ├── src │ ├── main │ │ └── java │ │ │ └── stream │ │ │ └── scotty │ │ │ └── slicing │ │ │ ├── aggregationstore │ │ │ ├── AggregationStoreFactory.java │ │ │ └── AggregationStore.java │ │ │ ├── slice │ │ │ ├── StreamRecord.java │ │ │ ├── EagerSlice.java │ │ │ ├── SliceFactory.java │ │ │ ├── LazySlice.java │ │ │ ├── Slice.java │ │ │ └── AbstractSlice.java │ │ │ ├── state │ │ │ ├── AggregateWindowState.java │ │ │ ├── AggregateState.java │ │ │ └── AggregateValueState.java │ │ │ └── SlicingWindowOperator.java │ └── test │ │ └── java │ │ └── stream │ │ └── scotty │ │ └── slicing │ │ └── aggregationstore │ │ └── test │ │ ├── windowTest │ │ ├── SumWindowFunctionTest.java │ │ └── WindowAssert.java │ │ ├── StateFactoryMock.java │ │ └── OutOfOrderArchitecture.md └── pom.xml ├── samza-connector ├── README.md ├── src │ └── main │ │ └── Properties │ │ └── config.properties └── pom.xml ├── spark-connector ├── README.md ├── src │ └── main │ │ └── java │ │ └── stream │ │ └── scotty │ │ └── sparkconnector │ │ └── demo │ │ └── DemoEvent.java └── pom.xml ├── kafkaStreams-connector ├── README.md ├── src │ └── main │ │ └── java │ │ └── stream │ │ └── scotty │ │ └── kafkastreamsconnector │ │ ├── KeyedScottyWindowTransformer.java │ │ ├── KeyedScottyWindowOperatorSupplier.java │ │ └── KeyedScottyWindowTransformerSupplier.java └── pom.xml ├── demo ├── storm-demo │ ├── src │ │ └── main │ │ │ └── java │ │ │ └── stream │ │ │ └── scotty │ │ │ └── demo │ │ │ └── storm │ │ │ ├── windowFunctions │ │ │ ├── Max.java │ │ │ ├── Min.java │ │ │ ├── Sum.java │ │ │ ├── Count.java │ │ │ ├── Mean.java │ │ │ └── Quantile.java │ │ │ ├── PrinterBolt.java │ │ │ ├── ScottyDemoTopology.java │ │ │ └── DataGeneratorSpout.java │ └── pom.xml ├── samza-demo │ ├── src │ │ └── main │ │ │ └── java │ │ │ └── stream │ │ │ └── scotty │ │ │ └── demo │ │ │ └── samza │ │ │ ├── windowFunctions │ │ │ ├── MaxWindowFunction.java │ │ │ ├── MinWindowFunction.java │ │ │ ├── SumWindowFunction.java │ │ │ └── QuantileWindowFunction.java │ │ │ ├── DemoTaskFactory.java │ │ │ ├── DemoKafkaProducer.java │ │ │ └── SamzaSumDemo.java │ └── pom.xml ├── spark-demo │ ├── src │ │ └── main │ │ │ └── java │ │ │ └── stream │ │ │ └── scotty │ │ │ └── demo │ │ │ └── spark │ │ │ ├── windowFunctions │ │ │ ├── MaxWindowFunction.java │ │ │ ├── MinWindowFunction.java │ │ │ ├── SumWindowFunction.java │ │ │ └── QuantileWindowFunction.java │ │ │ └── DemoKafkaProducer.java │ └── pom.xml ├── kafka-demo │ ├── src │ │ └── main │ │ │ └── java │ │ │ └── stream │ │ │ └── scotty │ │ │ └── demo │ │ │ └── kafkaStreams │ │ │ ├── windowFunctions │ │ │ ├── MaxWindowFunction.java │ │ │ ├── MinWindowFunction.java │ │ │ ├── SumWindowFunction.java │ │ │ └── QuantileWindowFunction.java │ │ │ ├── DemoPrinter.java │ │ │ ├── DemoKafkaProducer.java │ │ │ └── KafkaStreamsSumDemo.java │ └── pom.xml ├── beam-demo │ ├── src │ │ └── main │ │ │ └── java │ │ │ └── stream │ │ │ └── scotty │ │ │ └── demo │ │ │ └── beam │ │ │ ├── windowFunctions │ │ │ ├── Max.java │ │ │ ├── Min.java │ │ │ ├── Sum.java │ │ │ ├── Count.java │ │ │ ├── Mean.java │ │ │ └── Quantile.java │ │ │ ├── dataGenerator │ │ │ ├── Checkpoint.java │ │ │ ├── DataGeneratorSource.java │ │ │ └── TimeStampGenerator.java │ │ │ └── BeamSumDemo.java │ └── pom.xml ├── flink-demo │ ├── src │ │ └── main │ │ │ └── java │ │ │ └── stream │ │ │ └── scotty │ │ │ └── demo │ │ │ └── flink │ │ │ ├── windowFunctions │ │ │ ├── MaxWindowFunction.java │ │ │ ├── MinWindowFunction.java │ │ │ ├── SumWindowFunction.java │ │ │ └── QuantileWindowFunction.java │ │ │ ├── FlinkQuantileDemo.java │ │ │ ├── FlinkSumCountWindowDemo.java │ │ │ ├── FlinkSumDemo.java │ │ │ ├── DemoSource.java │ │ │ └── FlinkPunctuationDemo.java │ └── pom.xml └── pom.xml ├── benchmark ├── configurations │ ├── random_tumbling_benchmark_flink.json │ ├── sliding_benchmark_flink.json │ ├── sliding_benchmark_Scotty.json │ └── random_tumbling_benchmark.json ├── src │ └── main │ │ └── java │ │ └── stream │ │ └── scotty │ │ ├── flinkBenchmark │ │ ├── BenchmarkConfig.java │ │ ├── ThroughputStatistics.java │ │ ├── aggregations │ │ │ └── SumAggregation.java │ │ ├── ThroughputLogger.java │ │ ├── AnalyzeTool.java │ │ ├── LoadGeneratorSource.java │ │ ├── FlinkBenchmarkJob.java │ │ └── BenchmarkJob.java │ │ └── microbenchmark │ │ ├── AggregationStateInline.java │ │ ├── AggregationStoreBenchmark.java │ │ └── SlicingWindowOperatorBenchmark.java └── pom.xml ├── docs ├── package-list ├── script.js ├── de │ └── tub │ │ └── dima │ │ └── scotty │ │ ├── flinkBenchmark │ │ ├── aggregations │ │ │ └── package-frame.html │ │ └── package-frame.html │ │ ├── flinkconnector │ │ ├── package-frame.html │ │ └── demo │ │ │ ├── package-frame.html │ │ │ └── windowFunctions │ │ │ └── package-frame.html │ │ ├── slicing │ │ ├── aggregationstore │ │ │ ├── test │ │ │ │ ├── package-frame.html │ │ │ │ └── windowTest │ │ │ │ │ └── package-frame.html │ │ │ └── package-frame.html │ │ ├── package-frame.html │ │ └── slice │ │ │ └── package-frame.html │ │ ├── state │ │ ├── memory │ │ │ └── package-frame.html │ │ └── package-frame.html │ │ ├── microbenchmark │ │ └── package-frame.html │ │ └── core │ │ ├── package-frame.html │ │ ├── windowType │ │ ├── windowContext │ │ │ └── package-frame.html │ │ └── package-frame.html │ │ └── windowFunction │ │ └── package-frame.html ├── overview-frame.html └── index.html └── flink-connector ├── pom.xml └── src └── main └── java └── stream └── scotty └── flinkconnector └── GlobalScottyWindowOperator.java /.travis.yml: -------------------------------------------------------------------------------- 1 | language: java 2 | -------------------------------------------------------------------------------- /_config.yml: -------------------------------------------------------------------------------- 1 | theme: jekyll-theme-tactile -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *.iml 2 | *.idea 3 | *target 4 | *.class -------------------------------------------------------------------------------- /charts/SlidingWindow.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/TU-Berlin-DIMA/scotty-window-processor/HEAD/charts/SlidingWindow.png -------------------------------------------------------------------------------- /charts/ConcurrentTumblingWindows.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/TU-Berlin-DIMA/scotty-window-processor/HEAD/charts/ConcurrentTumblingWindows.png -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/WindowMeasure.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType; 2 | 3 | public enum WindowMeasure { 4 | Time, Count 5 | } 6 | -------------------------------------------------------------------------------- /logs/workers-artifacts/testTopology-1-1613327334/1024/worker.yaml: -------------------------------------------------------------------------------- 1 | worker-id: c3f3c11f-7aa2-4550-829b-31b0e3cc386d 2 | logs.users: [] 3 | logs.groups: [] 4 | topology.submitter.user: Jule 5 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/windowContext/WindowModifications.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType.windowContext; 2 | 3 | public interface WindowModifications { 4 | } 5 | -------------------------------------------------------------------------------- /state/src/main/java/stream/scotty/state/ValueState.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.state; 2 | 3 | public interface ValueState extends State { 4 | 5 | ValueType get(); 6 | 7 | void set(final ValueType value); 8 | 9 | } 10 | -------------------------------------------------------------------------------- /state/src/main/java/stream/scotty/state/State.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.state; 2 | 3 | import java.io.Serializable; 4 | 5 | public interface State extends Serializable { 6 | 7 | void clean(); 8 | 9 | boolean isEmpty(); 10 | 11 | } 12 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/WindowCollector.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core; 2 | 3 | import stream.scotty.core.windowType.*; 4 | 5 | public interface WindowCollector { 6 | 7 | public void trigger(long start, long end, WindowMeasure measure); 8 | } 9 | -------------------------------------------------------------------------------- /pom.xml.asc: -------------------------------------------------------------------------------- 1 | -----BEGIN PGP SIGNATURE----- 2 | 3 | iHUEABYKAB0WIQRHQaOwm1nt6mdV7T3vPRZ2ysnnZwUCY/4TKAAKCRDvPRZ2ysnn 4 | Z0rMAP92duqB+YNPDF4Nlpe3BJ2BSTZgBFs+z5YUp3Om3JKMowD/fMy9HeoSSUmg 5 | 9+2jnwQR74eGlNgwPX5uHc7fMbmM+wI= 6 | =S11Y 7 | -----END PGP SIGNATURE----- 8 | -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/aggregationstore/AggregationStoreFactory.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.aggregationstore; 2 | 3 | public interface AggregationStoreFactory { 4 | 5 | AggregationStore createAggregationStore(); 6 | } 7 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/Window.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType; 2 | 3 | import stream.scotty.core.*; 4 | 5 | import java.io.*; 6 | 7 | public interface Window extends Serializable { 8 | WindowMeasure getWindowMeasure(); 9 | } 10 | -------------------------------------------------------------------------------- /samza-connector/README.md: -------------------------------------------------------------------------------- 1 | #Samza Connector 2 | 3 | ### Demo Requirements: 4 | * Apache Zookeeper host:localhost , port:2181 5 | * Apache Kafka Server host:localhost , port:9092 6 | 7 | ### Demo Usage: 8 | * Start Zookeeper 9 | * Start Kafka Server 10 | * Start SamzaSumDemo.java -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/ForwardContextFree.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType; 2 | 3 | import stream.scotty.core.windowType.windowContext.*; 4 | 5 | public interface ForwardContextFree extends Window { 6 | 7 | WindowContext createContext(); 8 | } 9 | -------------------------------------------------------------------------------- /spark-connector/README.md: -------------------------------------------------------------------------------- 1 | # Spark Connector 2 | 3 | ### Demo Requirements: 4 | * Apache Zookeeper host:localhost , port:2181 5 | * Apache Kafka Server host:localhost , port:9092 6 | 7 | ### Demo Usage: 8 | * Start Zookeeper 9 | * Start Kafka Server 10 | * Start SparkSumDemo.java -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowFunction/InvertibleReduceAggregateFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowFunction; 2 | 3 | public interface InvertibleReduceAggregateFunction extends ReduceAggregateFunction, InvertibleAggregateFunction { 4 | 5 | 6 | } 7 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/TupleContext.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType; 2 | 3 | public interface TupleContext { 4 | 5 | Iterable iterator(); 6 | Iterable iterator(int start, int end); 7 | Tuple lookup(int position); 8 | 9 | } 10 | -------------------------------------------------------------------------------- /kafkaStreams-connector/README.md: -------------------------------------------------------------------------------- 1 | # Kafka Streams Connector 2 | 3 | ### Demo Requirements: 4 | * Apache Zookeeper host:localhost , port:2181 5 | * Apache Kafka Server host:localhost , port:9092 6 | 7 | ### Demo Usage: 8 | * Start Zookeeper 9 | * Start Kafka Server 10 | * Start KafkaStreamsSumDemo.java -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowFunction/CommutativeAggregateFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowFunction; 2 | 3 | public interface CommutativeAggregateFunction extends AggregateFunction { 4 | } 5 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/windowContext/AddModification.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType.windowContext; 2 | 3 | public class AddModification implements WindowModifications { 4 | public final long post; 5 | 6 | public AddModification(long post) { 7 | this.post = post; 8 | } 9 | } -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/windowContext/DeleteModification.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType.windowContext; 2 | 3 | public class DeleteModification implements WindowModifications { 4 | public final long pre; 5 | 6 | public DeleteModification(long pre) { 7 | this.pre = pre; 8 | } 9 | } 10 | -------------------------------------------------------------------------------- /state/src/main/java/stream/scotty/state/ListState.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.state; 2 | 3 | import java.util.List; 4 | 5 | public interface ListState extends State { 6 | 7 | List get(); 8 | 9 | void set(final int i, final ItemType value); 10 | 11 | void add(final int i, final ItemType value); 12 | 13 | } 14 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/ForwardContextAware.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType; 2 | 3 | import stream.scotty.core.windowType.windowContext.*; 4 | import stream.scotty.core.windowType.windowContext.*; 5 | 6 | public interface ForwardContextAware extends Window { 7 | 8 | WindowContext createContext(); 9 | } 10 | -------------------------------------------------------------------------------- /state/src/main/java/stream/scotty/state/StateFactory.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.state; 2 | 3 | import java.io.*; 4 | 5 | public interface StateFactory extends Serializable { 6 | 7 | ValueState createValueState(); 8 | 9 | ListState createListState(); 10 | 11 | > SetState createSetState(); 12 | } 13 | -------------------------------------------------------------------------------- /state/src/main/java/stream/scotty/state/SetState.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.state; 2 | 3 | public interface SetState> extends State, Iterable { 4 | 5 | public Type getLast(); 6 | 7 | public Type getFirst(); 8 | 9 | public Type dropLast(); 10 | 11 | public Type dropFrist(); 12 | 13 | public void add(Type record); 14 | 15 | } 16 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/windowContext/ShiftModification.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType.windowContext; 2 | 3 | public class ShiftModification implements WindowModifications { 4 | public final long pre; 5 | public final long post; 6 | 7 | public ShiftModification(long pre, long post) { 8 | this.pre = pre; 9 | this.post = post; 10 | } 11 | } -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/ContextFreeWindow.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType; 2 | 3 | import stream.scotty.core.*; 4 | import stream.scotty.core.*; 5 | 6 | public interface ContextFreeWindow extends Window { 7 | 8 | long assignNextWindowStart(long position); 9 | 10 | void triggerWindows(WindowCollector aggregateWindows, long lastWatermark, long currentWatermark); 11 | 12 | long clearDelay(); 13 | } 14 | -------------------------------------------------------------------------------- /demo/storm-demo/src/main/java/stream/scotty/demo/storm/windowFunctions/Max.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.storm.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 4 | 5 | public class Max implements ReduceAggregateFunction { 6 | 7 | @Override 8 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 9 | return Math.max(partialAggregate1,partialAggregate2); 10 | } 11 | } 12 | -------------------------------------------------------------------------------- /demo/storm-demo/src/main/java/stream/scotty/demo/storm/windowFunctions/Min.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.storm.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 4 | 5 | public class Min implements ReduceAggregateFunction { 6 | 7 | @Override 8 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 9 | return Math.min(partialAggregate1,partialAggregate2); 10 | } 11 | } 12 | -------------------------------------------------------------------------------- /benchmark/configurations/random_tumbling_benchmark_flink.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "tumbling_window_benchmark", 3 | "runtime" : 30, 4 | "throughput": 1600, 5 | "windowConfigurations": [ 6 | [ 7 | "randomTumbling(1,1,20)" 8 | ] 9 | ], 10 | "configurations": [ 11 | "Flink" 12 | ], 13 | "sessionConfig": { 14 | "gapCount": 10, 15 | "minGapTime": 1000, 16 | "maxGapTime": 1001 17 | }, 18 | "aggFunctions":[ 19 | "Sum" 20 | ] 21 | } 22 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowFunction/CloneablePartialStateFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowFunction; 2 | 3 | public interface CloneablePartialStateFunction { 4 | 5 | /** 6 | * Method that returns a deep copy of the partial state. 7 | * @param partialAggregate original object 8 | * @return cloned object 9 | */ 10 | public PartialAggregateType clone(PartialAggregateType partialAggregate); 11 | 12 | } 13 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/AggregateWindow.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core; 2 | 3 | import stream.scotty.core.windowType.*; 4 | 5 | import java.io.*; 6 | import java.util.List; 7 | 8 | public interface AggregateWindow extends Serializable { 9 | 10 | 11 | public WindowMeasure getMeasure(); 12 | 13 | public long getStart(); 14 | 15 | public long getEnd(); 16 | 17 | public List getAggValues(); 18 | 19 | public boolean hasValue(); 20 | 21 | } 22 | -------------------------------------------------------------------------------- /demo/samza-demo/src/main/java/stream/scotty/demo/samza/windowFunctions/MaxWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.samza.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 4 | 5 | public class MaxWindowFunction implements ReduceAggregateFunction { 6 | @Override 7 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 8 | return Math.max(partialAggregate1,partialAggregate2); 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /demo/samza-demo/src/main/java/stream/scotty/demo/samza/windowFunctions/MinWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.samza.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 4 | 5 | public class MinWindowFunction implements ReduceAggregateFunction { 6 | @Override 7 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 8 | return Math.min(partialAggregate1, partialAggregate2); 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /demo/spark-demo/src/main/java/stream/scotty/demo/spark/windowFunctions/MaxWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.spark.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 4 | 5 | public class MaxWindowFunction implements ReduceAggregateFunction { 6 | @Override 7 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 8 | return Math.max(partialAggregate1,partialAggregate2); 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /demo/spark-demo/src/main/java/stream/scotty/demo/spark/windowFunctions/MinWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.spark.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 4 | 5 | public class MinWindowFunction implements ReduceAggregateFunction { 6 | @Override 7 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 8 | return Math.min(partialAggregate1, partialAggregate2); 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /demo/kafka-demo/src/main/java/stream/scotty/demo/kafkaStreams/windowFunctions/MaxWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.kafkaStreams.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 4 | 5 | public class MaxWindowFunction implements ReduceAggregateFunction { 6 | @Override 7 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 8 | return Math.max(partialAggregate1,partialAggregate2); 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /demo/kafka-demo/src/main/java/stream/scotty/demo/kafkaStreams/windowFunctions/MinWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.kafkaStreams.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 4 | 5 | public class MinWindowFunction implements ReduceAggregateFunction { 6 | @Override 7 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 8 | return Math.min(partialAggregate1, partialAggregate2); 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowFunction/ReduceAggregateFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowFunction; 2 | 3 | 4 | @FunctionalInterface 5 | public interface ReduceAggregateFunction extends AggregateFunction { 6 | 7 | @Override 8 | default InputType lift(InputType inputTuple) { 9 | return inputTuple; 10 | } 11 | 12 | @Override 13 | default InputType lower(InputType aggregate) { 14 | return aggregate; 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /state/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | scotty 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | state 13 | 14 | 15 | -------------------------------------------------------------------------------- /state/src/main/java/stream/scotty/state/memory/MemoryStateFactory.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.state.memory; 2 | 3 | import stream.scotty.state.*; 4 | 5 | public class MemoryStateFactory implements StateFactory { 6 | @Override 7 | public ValueState createValueState() { 8 | return new MemoryValueState<>(); 9 | } 10 | 11 | @Override 12 | public ListState createListState() { 13 | return new MemoryListState<>(); 14 | } 15 | 16 | @Override 17 | public > SetState createSetState() { 18 | return new MemorySetState<>(); 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /demo/storm-demo/src/main/java/stream/scotty/demo/storm/windowFunctions/Sum.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.storm.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.InvertibleReduceAggregateFunction; 4 | 5 | public class Sum implements InvertibleReduceAggregateFunction { 6 | @Override 7 | public Integer invert(Integer currentAggregate, Integer toRemove) { 8 | return currentAggregate - toRemove; 9 | } 10 | 11 | @Override 12 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 13 | return partialAggregate1 + partialAggregate2; 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /docs/package-list: -------------------------------------------------------------------------------- 1 | stream.scotty.core 2 | stream.scotty.core.windowFunction 3 | stream.scotty.core.windowType 4 | stream.scotty.core.windowType.windowContext 5 | stream.scotty.flinkBenchmark 6 | stream.scotty.flinkBenchmark.aggregations 7 | stream.scotty.flinkconnector 8 | stream.scotty.flinkconnector.demo 9 | stream.scotty.demo.flink.windowFunctions 10 | stream.scotty.microbenchmark 11 | stream.scotty.slicing 12 | stream.scotty.slicing.aggregationstore 13 | stream.scotty.slicing.aggregationstore.test 14 | stream.scotty.slicing.aggregationstore.test.windowTest 15 | stream.scotty.slicing.slice 16 | stream.scotty.state 17 | stream.scotty.state.memory 18 | -------------------------------------------------------------------------------- /demo/samza-demo/src/main/java/stream/scotty/demo/samza/windowFunctions/SumWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.samza.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.InvertibleReduceAggregateFunction; 4 | 5 | public class SumWindowFunction implements InvertibleReduceAggregateFunction { 6 | @Override 7 | public Integer invert(Integer currentAggregate, Integer toRemove) { 8 | return currentAggregate-toRemove; 9 | } 10 | 11 | @Override 12 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 13 | return partialAggregate1+partialAggregate2; 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /demo/spark-demo/src/main/java/stream/scotty/demo/spark/windowFunctions/SumWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.spark.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.InvertibleReduceAggregateFunction; 4 | 5 | public class SumWindowFunction implements InvertibleReduceAggregateFunction { 6 | @Override 7 | public Integer invert(Integer currentAggregate, Integer toRemove) { 8 | return currentAggregate-toRemove; 9 | } 10 | 11 | @Override 12 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 13 | return partialAggregate1+partialAggregate2; 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /demo/beam-demo/src/main/java/stream/scotty/demo/beam/windowFunctions/Max.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.beam.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 4 | import org.apache.beam.sdk.values.KV; 5 | 6 | import java.io.Serializable; 7 | 8 | public class Max implements ReduceAggregateFunction>, Serializable { 9 | 10 | @Override 11 | public KV combine(KV partialAggregate1, KV partialAggregate2) { 12 | return KV.of(partialAggregate1.getKey(), Math.max(partialAggregate1.getValue(), partialAggregate2.getValue())); 13 | } 14 | } -------------------------------------------------------------------------------- /demo/beam-demo/src/main/java/stream/scotty/demo/beam/windowFunctions/Min.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.beam.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 4 | import org.apache.beam.sdk.values.KV; 5 | 6 | import java.io.Serializable; 7 | 8 | public class Min implements ReduceAggregateFunction>, Serializable { 9 | 10 | @Override 11 | public KV combine(KV partialAggregate1, KV partialAggregate2) { 12 | return KV.of(partialAggregate1.getKey(), Math.min(partialAggregate1.getValue(), partialAggregate2.getValue())); 13 | } 14 | } -------------------------------------------------------------------------------- /samza-connector/src/main/Properties/config.properties: -------------------------------------------------------------------------------- 1 | job.name=SamzaSumDemo 2 | # Use a PassthroughJobCoordinator since there is no coordination needed 3 | job.coordinator.factory=org.apache.samza.standalone.PassthroughJobCoordinatorFactory 4 | job.coordination.utils.factory=org.apache.samza.standalone.PassthroughCoordinationUtilsFactory 5 | 6 | job.changelog.system=testSystem 7 | 8 | # Use a single container to process all of the data 9 | task.name.grouper.factory=org.apache.samza.container.grouper.task.SingleContainerGrouperFactory 10 | processor.id=0 11 | 12 | # Read from the beginning of the topic 13 | systems.kafka.default.stream.samza.offset.default=oldest -------------------------------------------------------------------------------- /demo/flink-demo/src/main/java/stream/scotty/demo/flink/windowFunctions/MaxWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.flink.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.*; 4 | import org.apache.flink.api.java.tuple.*; 5 | 6 | import java.io.*; 7 | 8 | public class MaxWindowFunction implements ReduceAggregateFunction>, Serializable { 9 | 10 | @Override 11 | public Tuple2 combine(Tuple2 partialAggregate1, Tuple2 partialAggregate2) { 12 | return new Tuple2<>(partialAggregate1.f0, Math.max(partialAggregate1.f1, partialAggregate2.f1)); 13 | } 14 | } -------------------------------------------------------------------------------- /demo/flink-demo/src/main/java/stream/scotty/demo/flink/windowFunctions/MinWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.flink.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.*; 4 | import org.apache.flink.api.java.tuple.*; 5 | 6 | import java.io.*; 7 | 8 | public class MinWindowFunction implements ReduceAggregateFunction>, Serializable { 9 | 10 | @Override 11 | public Tuple2 combine(Tuple2 partialAggregate1, Tuple2 partialAggregate2) { 12 | return new Tuple2<>(partialAggregate1.f0, Math.min(partialAggregate1.f1, partialAggregate2.f1)); 13 | } 14 | } -------------------------------------------------------------------------------- /demo/kafka-demo/src/main/java/stream/scotty/demo/kafkaStreams/windowFunctions/SumWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.kafkaStreams.windowFunctions; 2 | import stream.scotty.core.windowFunction.InvertibleReduceAggregateFunction; 3 | 4 | public class SumWindowFunction implements InvertibleReduceAggregateFunction { 5 | @Override 6 | public Integer invert(Integer currentAggregate, Integer toRemove) { 7 | return currentAggregate-toRemove; 8 | } 9 | 10 | @Override 11 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 12 | return partialAggregate1+partialAggregate2; 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /demo/storm-demo/src/main/java/stream/scotty/demo/storm/windowFunctions/Count.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.storm.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.InvertibleReduceAggregateFunction; 4 | 5 | public class Count implements InvertibleReduceAggregateFunction { 6 | 7 | @Override 8 | public Integer lift(Integer inputTuple) { 9 | return 1; 10 | } 11 | 12 | @Override 13 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 14 | return partialAggregate1+partialAggregate2; 15 | } 16 | 17 | @Override 18 | public Integer invert(Integer currentAggregate, Integer toRemove) { 19 | return currentAggregate-1; 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/flinkBenchmark/BenchmarkConfig.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.flinkBenchmark; 2 | 3 | import java.util.List; 4 | 5 | /** 6 | * Created by philipp on 5/29/17. 7 | */ 8 | public class BenchmarkConfig { 9 | 10 | public int throughput; 11 | public long runtime; 12 | public String name; 13 | 14 | // [Sliding(1,2), Tumbling(1), Session(2)] 15 | public List> windowConfigurations; 16 | 17 | // [Bucket, Naive, Slicing_Lazy, Slicing_Heap] 18 | public List configurations; 19 | // [sum, quantil] 20 | public List aggFunctions; 21 | 22 | public SessionConfig sessionConfig; 23 | 24 | 25 | public class SessionConfig { 26 | int gapCount = 0; 27 | int minGapTime = 0; 28 | int maxGapTime = 0; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /demo/kafka-demo/src/main/java/stream/scotty/demo/kafkaStreams/DemoPrinter.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.kafkaStreams; 2 | 3 | import org.apache.kafka.streams.processor.Processor; 4 | import org.apache.kafka.streams.processor.ProcessorContext; 5 | 6 | public class DemoPrinter implements Processor { 7 | private ProcessorContext context; 8 | 9 | @Override 10 | public void init(ProcessorContext context) { 11 | this.context = context; 12 | } 13 | 14 | @Override 15 | public void process(Key key, Value value) { 16 | System.out.println("Processing result: Key: "+key+" Value: "+value); 17 | 18 | context.forward(key,value); 19 | } 20 | 21 | @Override 22 | public void close() { 23 | 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/microbenchmark/AggregationStateInline.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.microbenchmark; 2 | 3 | import stream.scotty.state.StateFactory; 4 | import stream.scotty.state.ValueState; 5 | 6 | import java.util.ArrayList; 7 | 8 | public class AggregationStateInline { 9 | 10 | private boolean empty; 11 | private int value; 12 | 13 | private ArrayList> stateList = new ArrayList<>(); 14 | private ArrayList v = new ArrayList<>(); 15 | 16 | public AggregationStateInline(StateFactory stateFactory) { 17 | stateList.add(stateFactory.createValueState()); 18 | stateList.get(0).set(0); 19 | v.add(0); 20 | } 21 | 22 | public void addElement(Integer state) { 23 | v.set(0,v.get(0)); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /demo/beam-demo/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | demo 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | beam-demo 13 | 14 | 15 | stream.scotty 16 | beam-connector 17 | 0.4 18 | 19 | 20 | 21 | 22 | -------------------------------------------------------------------------------- /demo/flink-demo/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | demo 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | flink-demo 13 | 14 | 15 | stream.scotty 16 | flink-connector 17 | 0.4 18 | 19 | 20 | 21 | -------------------------------------------------------------------------------- /demo/samza-demo/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | demo 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | samza-demo 13 | 14 | 15 | stream.scotty 16 | samza-connector 17 | 0.4 18 | 19 | 20 | 21 | -------------------------------------------------------------------------------- /demo/spark-demo/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | demo 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | spark-demo 13 | 14 | 15 | stream.scotty 16 | spark-connector 17 | 0.4 18 | 19 | 20 | 21 | -------------------------------------------------------------------------------- /demo/storm-demo/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | demo 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | storm-demo 13 | 14 | 15 | stream.scotty 16 | storm-connector 17 | 0.4 18 | 19 | 20 | 21 | -------------------------------------------------------------------------------- /demo/kafka-demo/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | demo 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | kafka-demo 13 | 14 | 15 | stream.scotty 16 | kafkaStreams-connector 17 | 0.4 18 | 19 | 20 | 21 | -------------------------------------------------------------------------------- /core/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | scotty 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | core 13 | 14 | 15 | 16 | org.apache.flink 17 | flink-core 18 | 1.9.2 19 | compile 20 | 21 | 22 | 23 | -------------------------------------------------------------------------------- /benchmark/configurations/sliding_benchmark_flink.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "tumbling_window_benchmark", 3 | "runtime" : 30, 4 | "throughput": 1600000, 5 | "windowConfigurations": [ 6 | [ 7 | "Sliding(60000,60000)" 8 | ], 9 | [ 10 | "Sliding(60000,60000)" 11 | ], 12 | [ 13 | "Sliding(60000,30000)" 14 | ], 15 | [ 16 | "Sliding(60000,10000)" 17 | ], 18 | [ 19 | "Sliding(60000,5000)" 20 | ], 21 | [ 22 | "Sliding(60000,1000)" 23 | ], 24 | [ 25 | "Sliding(60000,500)" 26 | ], 27 | [ 28 | "Sliding(60000,250)" 29 | ] 30 | ], 31 | "configurations": [ 32 | "Flink" 33 | ], 34 | "sessionConfig": { 35 | "gapCount": 10, 36 | "minGapTime": 1000, 37 | "maxGapTime": 1001 38 | }, 39 | "aggFunctions":[ 40 | "Sum" 41 | ] 42 | } 43 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowFunction/InvertibleAggregateFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowFunction; 2 | 3 | public interface InvertibleAggregateFunction extends AggregateFunction { 4 | /** 5 | * Removes one partial aggregate from another with an incremental operation. 6 | * @param currentAggregate 7 | * @param toRemove 8 | * @return 9 | */ 10 | PartialAggregateType invert(PartialAggregateType currentAggregate, PartialAggregateType toRemove); 11 | 12 | default PartialAggregateType liftAndInvert(PartialAggregateType partialAggregate, InputType toRemove){ 13 | PartialAggregateType lifted = lift(toRemove); 14 | return invert(partialAggregate, lifted); 15 | }; 16 | } 17 | -------------------------------------------------------------------------------- /benchmark/configurations/sliding_benchmark_Scotty.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "tumbling_window_benchmark", 3 | "runtime" : 30, 4 | "throughput": 1700000, 5 | "windowConfigurations": [ 6 | [ 7 | "Sliding(60000,60000)" 8 | ], 9 | [ 10 | "Sliding(60000,60000)" 11 | ], 12 | [ 13 | "Sliding(60000,30000)" 14 | ], 15 | [ 16 | "Sliding(60000,10000)" 17 | ], 18 | [ 19 | "Sliding(60000,5000)" 20 | ], 21 | [ 22 | "Sliding(60000,1000)" 23 | ], 24 | [ 25 | "Sliding(60000,500)" 26 | ], 27 | [ 28 | "Sliding(60000,250)" 29 | ] 30 | ], 31 | "configurations": [ 32 | "Slicing" 33 | ], 34 | "sessionConfig": { 35 | "gapCount": 10, 36 | "minGapTime": 1000, 37 | "maxGapTime": 1001 38 | }, 39 | "aggFunctions":[ 40 | "Sum" 41 | ] 42 | } 43 | -------------------------------------------------------------------------------- /demo/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | scotty 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | demo 13 | pom 14 | 15 | flink-demo 16 | spark-demo 17 | storm-demo 18 | beam-demo 19 | kafka-demo 20 | samza-demo 21 | 22 | 23 | 24 | -------------------------------------------------------------------------------- /benchmark/configurations/random_tumbling_benchmark.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "tumbling_window_benchmark", 3 | "runtime" : 30, 4 | "throughput": 2000000, 5 | "windowConfigurations": [ 6 | [ 7 | "randomTumbling(1,1,20)" 8 | ], 9 | [ 10 | "randomTumbling(5,1,20)" 11 | ], 12 | [ 13 | "randomTumbling(10,1,20)" 14 | ], 15 | [ 16 | "randomTumbling(20,1,20)" 17 | ], 18 | [ 19 | "randomTumbling(40,1,20)" 20 | ], 21 | [ 22 | "randomTumbling(80,1,20)" 23 | ], 24 | [ 25 | "randomTumbling(500,1,20)" 26 | ], 27 | [ 28 | "randomTumbling(1000,1,20)" 29 | ] 30 | ], 31 | "configurations": [ 32 | "Slicing" 33 | ], 34 | "sessionConfig": { 35 | "gapCount": 10, 36 | "minGapTime": 1000, 37 | "maxGapTime": 1001 38 | }, 39 | "aggFunctions":[ 40 | "Sum" 41 | ] 42 | } 43 | -------------------------------------------------------------------------------- /state/src/main/java/stream/scotty/state/memory/MemoryListState.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.state.memory; 2 | 3 | import stream.scotty.state.ListState; 4 | 5 | import java.util.ArrayList; 6 | import java.util.List; 7 | 8 | public class MemoryListState implements ListState { 9 | 10 | private ArrayList innerList = new ArrayList<>(); 11 | 12 | @Override 13 | public List get() { 14 | return innerList; 15 | } 16 | 17 | @Override 18 | public void set(int i, T value) { 19 | innerList.set(i, value); 20 | } 21 | 22 | @Override 23 | public void add(int i, T value) { 24 | innerList.add(i, value); 25 | } 26 | 27 | @Override 28 | public void clean() { 29 | innerList.clear(); 30 | } 31 | 32 | @Override 33 | public boolean isEmpty() { 34 | return innerList.isEmpty(); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /demo/flink-demo/src/main/java/stream/scotty/demo/flink/windowFunctions/SumWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.flink.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.*; 4 | import org.apache.flink.api.java.tuple.*; 5 | 6 | import java.io.*; 7 | 8 | public class SumWindowFunction implements InvertibleReduceAggregateFunction>, Serializable { 9 | 10 | @Override 11 | public Tuple2 invert(Tuple2 currentAggregate, Tuple2 toRemove) { 12 | return new Tuple2<>(currentAggregate.f0, currentAggregate.f1 - toRemove.f1); 13 | } 14 | 15 | @Override 16 | public Tuple2 combine(Tuple2 partialAggregate1, Tuple2 partialAggregate2) { 17 | return new Tuple2<>(partialAggregate1.f0, partialAggregate1.f1 + partialAggregate2.f1); 18 | } 19 | } -------------------------------------------------------------------------------- /demo/beam-demo/src/main/java/stream/scotty/demo/beam/windowFunctions/Sum.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.beam.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.InvertibleReduceAggregateFunction; 4 | import org.apache.beam.sdk.values.KV; 5 | 6 | import java.io.Serializable; 7 | 8 | public class Sum implements InvertibleReduceAggregateFunction>, Serializable { 9 | 10 | @Override 11 | public KV invert( KV currentAggregate, KV toRemove) { 12 | return KV.of(currentAggregate.getKey(),currentAggregate.getValue()-toRemove.getValue()); 13 | } 14 | 15 | @Override 16 | public KV combine( KV partialAggregate1, KV partialAggregate2) { 17 | return KV.of(partialAggregate1.getKey(), partialAggregate1.getValue()+ partialAggregate2.getValue()); 18 | } 19 | } -------------------------------------------------------------------------------- /docs/script.js: -------------------------------------------------------------------------------- 1 | function show(type) 2 | { 3 | count = 0; 4 | for (var key in methods) { 5 | var row = document.getElementById(key); 6 | if ((methods[key] & type) != 0) { 7 | row.style.display = ''; 8 | row.className = (count++ % 2) ? rowColor : altColor; 9 | } 10 | else 11 | row.style.display = 'none'; 12 | } 13 | updateTabs(type); 14 | } 15 | 16 | function updateTabs(type) 17 | { 18 | for (var value in tabs) { 19 | var sNode = document.getElementById(tabs[value][0]); 20 | var spanNode = sNode.firstChild; 21 | if (value == type) { 22 | sNode.className = activeTableTab; 23 | spanNode.innerHTML = tabs[value][1]; 24 | } 25 | else { 26 | sNode.className = tableTab; 27 | spanNode.innerHTML = "" + tabs[value][1] + ""; 28 | } 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/slice/StreamRecord.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.slice; 2 | 3 | import org.jetbrains.annotations.*; 4 | 5 | public class StreamRecord implements Comparable> { 6 | public final long ts; 7 | public final Type record; 8 | 9 | public StreamRecord(long ts, Type type) { 10 | this.ts = ts; 11 | this.record = type; 12 | } 13 | 14 | @Override 15 | public boolean equals(Object obj) { 16 | if (obj instanceof StreamRecord) { 17 | if (((StreamRecord) obj).record.equals(this.record)) { 18 | return true; 19 | } 20 | } 21 | return false; 22 | } 23 | 24 | @Override 25 | public int compareTo(@NotNull StreamRecord o) { 26 | return Long.compare(ts, o.ts); 27 | } 28 | 29 | @Override 30 | public String toString() { 31 | return "(ts="+ ts + ", value=" + record +")"; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /slicing/src/test/java/stream/scotty/slicing/aggregationstore/test/windowTest/SumWindowFunctionTest.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.aggregationstore.test.windowTest; 2 | 3 | import stream.scotty.core.windowFunction.InvertibleReduceAggregateFunction; 4 | import org.apache.flink.api.java.tuple.Tuple2; 5 | 6 | import java.io.Serializable; 7 | 8 | public class SumWindowFunctionTest implements InvertibleReduceAggregateFunction>, Serializable{ 9 | 10 | @Override 11 | public Tuple2 invert(Tuple2 currentAggregate, Tuple2 toRemove) { 12 | return new Tuple2<>(currentAggregate.f0, currentAggregate.f1 - toRemove.f1); 13 | } 14 | 15 | @Override 16 | public Tuple2 combine(Tuple2 partialAggregate1, Tuple2 partialAggregate2) { 17 | return new Tuple2<>(partialAggregate2.f0, partialAggregate1.f1 + partialAggregate2.f1); 18 | } 19 | } -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/slice/EagerSlice.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.slice; 2 | 3 | 4 | import stream.scotty.slicing.state.AggregateState; 5 | import stream.scotty.slicing.WindowManager; 6 | import stream.scotty.state.StateFactory; 7 | 8 | public class EagerSlice extends AbstractSlice { 9 | 10 | private final AggregateState state; 11 | 12 | public EagerSlice(StateFactory stateFactory, WindowManager windowManager, long startTs, long endTs, long startC, long endC, Type type) { 13 | super(startTs, endTs,startC, endC, type); 14 | this.state = new AggregateState(stateFactory, windowManager.getAggregations(), null); 15 | } 16 | 17 | @Override 18 | public AggregateState getAggState() { 19 | return state; 20 | } 21 | 22 | @Override 23 | public void addElement(InputType element, long ts) { 24 | super.addElement(element, ts); 25 | state.addElement(element); 26 | } 27 | 28 | 29 | } 30 | -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/flinkBenchmark/ThroughputStatistics.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.flinkBenchmark; 2 | 3 | public class ThroughputStatistics { 4 | 5 | private static ThroughputStatistics statistics; 6 | private boolean pause; 7 | 8 | private ThroughputStatistics() { 9 | } 10 | 11 | private double counter = 0; 12 | private double sum = 0; 13 | public static ThroughputStatistics getInstance() { 14 | if (statistics == null) 15 | statistics = new ThroughputStatistics(); 16 | return statistics; 17 | } 18 | 19 | 20 | public void addThrouputResult(double throuputPerS) { 21 | if (this.pause) 22 | return; 23 | counter++; 24 | sum += throuputPerS; 25 | } 26 | 27 | public void clean() { 28 | counter = 0; 29 | sum = 0; 30 | } 31 | 32 | public double mean() { 33 | return sum / counter; 34 | } 35 | 36 | @Override 37 | public String toString() { 38 | return "\nThroughput Mean: " + (sum / counter); 39 | } 40 | 41 | public void pause(final boolean pause) { 42 | this.pause = pause; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/flinkBenchmark/aggregations/SumAggregation.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.flinkBenchmark.aggregations; 2 | 3 | import stream.scotty.core.windowFunction.*; 4 | import org.apache.flink.api.java.tuple.*; 5 | 6 | import java.io.*; 7 | 8 | public class SumAggregation implements InvertibleReduceAggregateFunction>, Serializable { 9 | 10 | @Override 11 | public Tuple4 invert(Tuple4 currentAggregate, Tuple4 toRemove) { 12 | return new Tuple4<>(currentAggregate.f0, currentAggregate.f1 - toRemove.f1, currentAggregate.f2, currentAggregate.f3); 13 | } 14 | 15 | @Override 16 | public Tuple4 combine(Tuple4 partialAggregate1, Tuple4 partialAggregate2) { 17 | return new Tuple4<>(partialAggregate1.f0, partialAggregate1.f1 + partialAggregate2.f1, partialAggregate1.f2, partialAggregate1.f3); 18 | } 19 | } -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/flinkBenchmark/aggregations/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.flinkBenchmark.aggregations 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.flinkBenchmark.aggregations

13 |
14 |

Classes

15 | 18 |
19 | 20 | 21 | -------------------------------------------------------------------------------- /slicing/src/test/java/stream/scotty/slicing/aggregationstore/test/windowTest/WindowAssert.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.aggregationstore.test.windowTest; 2 | 3 | import stream.scotty.core.*; 4 | import org.junit.*; 5 | 6 | import java.util.*; 7 | 8 | public class WindowAssert { 9 | 10 | public static void assertEquals(AggregateWindow aggregateWindow, long start, long end, Object value){ 11 | Assert.assertEquals(start, aggregateWindow.getStart()); 12 | Assert.assertEquals(end, aggregateWindow.getEnd()); 13 | Assert.assertEquals(value, aggregateWindow.getAggValues().get(0)); 14 | } 15 | 16 | public static void assertContains(List resultWindows, int start, int end, Object value) { 17 | for (AggregateWindow window: resultWindows){ 18 | if(window.getStart() == start && window.getEnd() == end && window.getAggValues().get(0).equals(value)) { 19 | Assert.assertTrue(true); 20 | return; 21 | } 22 | } 23 | Assert.assertTrue(false); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /demo/beam-demo/src/main/java/stream/scotty/demo/beam/windowFunctions/Count.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.beam.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.InvertibleReduceAggregateFunction; 4 | import org.apache.beam.sdk.values.KV; 5 | 6 | import java.io.Serializable; 7 | 8 | public class Count implements InvertibleReduceAggregateFunction>, Serializable { 9 | 10 | @Override 11 | public KV lift(KV inputTuple) { 12 | return KV.of(inputTuple.getKey(), 1); 13 | } 14 | 15 | @Override 16 | public KV combine(KV partialAggregate1, KV partialAggregate2) { 17 | return KV.of(partialAggregate1.getKey(), partialAggregate1.getValue() + partialAggregate2.getValue()); 18 | } 19 | 20 | @Override 21 | public KV invert(KV currentAggregate, KV toRemove) { 22 | return KV.of(currentAggregate.getKey(), currentAggregate.getValue() - toRemove.getValue()); 23 | } 24 | } -------------------------------------------------------------------------------- /kafkaStreams-connector/src/main/java/stream/scotty/kafkastreamsconnector/KeyedScottyWindowTransformer.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.kafkastreamsconnector; 2 | 3 | import stream.scotty.core.windowFunction.AggregateFunction; 4 | import org.apache.kafka.streams.KeyValue; 5 | import org.apache.kafka.streams.kstream.Transformer; 6 | 7 | 8 | /** 9 | * 10 | * Use the {@link KeyedScottyWindowOperator} as a transformer step in the Streams DSL. 11 | * 12 | * @author bjoernv 13 | * 14 | * @param key type 15 | * @param value type 16 | * @param {@link KeyValue} return type (both key and value type can be set arbitrarily) 17 | */ 18 | public class KeyedScottyWindowTransformer extends KeyedScottyWindowOperator implements Transformer{ 19 | 20 | public KeyedScottyWindowTransformer(AggregateFunction windowFunction, long allowedLateness) { 21 | super(windowFunction, allowedLateness); 22 | } 23 | 24 | @Override 25 | public Result transform(Key key, Value value) { 26 | this.process(key, value); 27 | return null; 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/flinkconnector/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.flinkconnector 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.flinkconnector

13 |
14 |

Classes

15 | 19 |
20 | 21 | 22 | -------------------------------------------------------------------------------- /slicing/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | scotty 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | slicing 13 | 14 | 15 | stream.scotty 16 | core 17 | 0.4 18 | 19 | 20 | stream.scotty 21 | state 22 | 0.4 23 | 24 | 25 | org.jetbrains 26 | annotations 27 | RELEASE 28 | 29 | 30 | 31 | 32 | 33 | -------------------------------------------------------------------------------- /state/src/main/java/stream/scotty/state/memory/MemorySetState.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.state.memory; 2 | 3 | import stream.scotty.state.*; 4 | 5 | import java.util.*; 6 | 7 | public class MemorySetState> implements SetState { 8 | 9 | private TreeSet values = new TreeSet<>(); 10 | 11 | public Type getLast() { 12 | return values.last(); 13 | } 14 | 15 | public Type getFirst() { 16 | return values.first(); 17 | } 18 | 19 | public Type dropLast() { 20 | return values.pollLast(); 21 | } 22 | 23 | public Type dropFrist() { 24 | return values.pollFirst(); 25 | } 26 | 27 | public void add(Type record) { 28 | values.add(record); 29 | } 30 | 31 | @Override 32 | public Iterator iterator() { 33 | return values.iterator(); 34 | } 35 | 36 | @Override 37 | public void clean() { 38 | values.clear(); 39 | } 40 | 41 | @Override 42 | public boolean isEmpty() { 43 | return values.isEmpty(); 44 | } 45 | 46 | @Override 47 | public String toString() { 48 | return values.toString(); 49 | } 50 | } -------------------------------------------------------------------------------- /demo/storm-demo/src/main/java/stream/scotty/demo/storm/windowFunctions/Mean.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.storm.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.AggregateFunction; 4 | 5 | public class Mean implements AggregateFunction { 6 | 7 | @Override 8 | public Pair lift(Integer inputTuple) { 9 | return new Pair(inputTuple); 10 | } 11 | 12 | @Override 13 | public Pair combine(Pair partialAggregate1, Pair partialAggregate2) { 14 | return new Pair(partialAggregate1.sum + partialAggregate2.sum, partialAggregate1.count + partialAggregate2.count); 15 | } 16 | 17 | @Override 18 | public Integer lower(Pair aggregate) { 19 | return aggregate.getResult(); 20 | } 21 | } 22 | 23 | class Pair { 24 | int sum; 25 | int count; 26 | 27 | public Pair(int sum) { 28 | this.sum = sum; 29 | this.count = 1; 30 | } 31 | 32 | public Pair(int sum, int count) { 33 | this.sum = sum; 34 | this.count = count; 35 | } 36 | 37 | public int getResult() { 38 | return sum / count; 39 | } 40 | 41 | @Override 42 | public String toString() { 43 | return getResult()+""; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/slicing/aggregationstore/test/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.slicing.aggregationstore.test 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.slicing.aggregationstore.test

13 |
14 |

Classes

15 | 19 |
20 | 21 | 22 | -------------------------------------------------------------------------------- /state/src/main/java/stream/scotty/state/memory/MemoryValueState.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.state.memory; 2 | 3 | import stream.scotty.state.ValueState; 4 | 5 | import java.util.*; 6 | 7 | public class MemoryValueState implements ValueState { 8 | 9 | private T value; 10 | 11 | @Override 12 | public T get() { 13 | return value; 14 | } 15 | 16 | @Override 17 | public void set(T value) { 18 | this.value = value; 19 | } 20 | 21 | @Override 22 | public void clean() { 23 | value = null; 24 | } 25 | 26 | @Override 27 | public boolean isEmpty() { 28 | return this.value == null; 29 | } 30 | 31 | @Override 32 | public String toString() { 33 | if(isEmpty()) 34 | return ""; 35 | return value.toString(); 36 | } 37 | 38 | @Override 39 | public boolean equals(Object o) { 40 | if (this == o) return true; 41 | if (o == null || getClass() != o.getClass()) return false; 42 | MemoryValueState that = (MemoryValueState) o; 43 | return Objects.equals(value, that.value); 44 | } 45 | 46 | @Override 47 | public int hashCode() { 48 | return Objects.hash(value); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/state/memory/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.state.memory 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.state.memory

13 |
14 |

Classes

15 | 20 |
21 | 22 | 23 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/WindowOperator.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core; 2 | 3 | import stream.scotty.core.windowType.*; 4 | import stream.scotty.core.windowFunction.AggregateFunction; 5 | 6 | import java.io.*; 7 | import java.util.List; 8 | 9 | public interface WindowOperator extends Serializable { 10 | 11 | /** 12 | * Process a new element of the stream 13 | */ 14 | void processElement(InputType element, long ts); 15 | 16 | /** 17 | * Process a watermark at a specific timestamp 18 | */ 19 | List processWatermark(long watermarkTs); 20 | 21 | /** 22 | * Add a window assigner to the window operator. 23 | */ 24 | void addWindowAssigner(Window window); 25 | 26 | /** 27 | * Add a aggregation 28 | * @param windowFunction 29 | */ 30 | void addAggregation(AggregateFunction windowFunction); 31 | 32 | /** 33 | * Set the max lateness for the window operator. 34 | * LastWatermark - maxLateness is the point in time where slices get garbage collected and no further late elements are processed. 35 | * @param maxLateness 36 | */ 37 | void setMaxLateness(long maxLateness); 38 | 39 | 40 | } 41 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/flinkconnector/demo/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.flinkconnector.demo 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.flinkconnector.demo

13 |
14 |

Classes

15 | 20 |
21 | 22 | 23 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/microbenchmark/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.microbenchmark 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.microbenchmark

13 | 21 | 22 | 23 | -------------------------------------------------------------------------------- /slicing/src/test/java/stream/scotty/slicing/aggregationstore/test/StateFactoryMock.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.aggregationstore.test; 2 | 3 | import stream.scotty.state.*; 4 | import stream.scotty.state.memory.*; 5 | 6 | public class StateFactoryMock implements StateFactory { 7 | @Override 8 | public ValueState createValueState() { 9 | return new ValueState() { 10 | T value; 11 | 12 | @Override 13 | public T get() { 14 | return value; 15 | } 16 | 17 | @Override 18 | public void set(T value) { 19 | this.value = value; 20 | } 21 | 22 | @Override 23 | public void clean() { 24 | 25 | } 26 | 27 | @Override 28 | public boolean isEmpty() { 29 | return false; 30 | } 31 | 32 | @Override 33 | public String toString() { 34 | return "" + value; 35 | } 36 | }; 37 | } 38 | 39 | @Override 40 | public ListState createListState() { 41 | return null; 42 | } 43 | 44 | @Override 45 | public > SetState createSetState() { 46 | return new MemorySetState<>(); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /spark-connector/src/main/java/stream/scotty/sparkconnector/demo/DemoEvent.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.sparkconnector.demo; 2 | import java.io.Serializable; 3 | 4 | /** 5 | * User-defined data type representing the input events 6 | */ 7 | public class DemoEvent implements Serializable { 8 | private Integer key; 9 | private Integer value; 10 | private long timestamp; 11 | 12 | public DemoEvent() { 13 | } 14 | 15 | public DemoEvent(Integer key, Integer value, long timestamp) { 16 | this.key = key; 17 | this.value = value; 18 | this.timestamp = timestamp; 19 | } 20 | 21 | public Integer getValue() { 22 | return value; 23 | } 24 | 25 | public long getTimestamp() { 26 | return timestamp; 27 | } 28 | 29 | public Integer getKey() { 30 | return key; 31 | } 32 | 33 | public void setTimestamp(long timestamp) { 34 | this.timestamp = timestamp; 35 | } 36 | 37 | public void setValue(Integer value) { 38 | this.value = value; 39 | } 40 | 41 | public void setKey(Integer key) { 42 | this.key = key; 43 | } 44 | 45 | @Override 46 | public String toString() { 47 | return "DemoEvent{" + 48 | "value=" + value + 49 | ", timestamp=" + timestamp + 50 | '}'; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/slice/SliceFactory.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.slice; 2 | 3 | import stream.scotty.slicing.WindowManager; 4 | import stream.scotty.state.StateFactory; 5 | 6 | 7 | public class SliceFactory { 8 | 9 | private final WindowManager windowManager; 10 | private StateFactory stateFactory; 11 | 12 | public SliceFactory(WindowManager windowManager, StateFactory stateFactory) { 13 | this.windowManager = windowManager; 14 | this.stateFactory = stateFactory; 15 | } 16 | 17 | public Slice createSlice(long startTs, long maxValue, long startCount, long endCount, Slice.Type type) { 18 | if(!windowManager.hasCountMeasure() && ((!windowManager.hasContextAwareWindow() || windowManager.isSessionWindowCase()) && windowManager.getMaxLateness()>0)){ 19 | return new EagerSlice<>(stateFactory, windowManager, startTs, maxValue, startCount, endCount, type); 20 | } 21 | return new LazySlice<>(stateFactory, windowManager, startTs, maxValue, startCount, endCount, type); 22 | } 23 | public Slice createSlice(long startTs, long maxValue, Slice.Type type) { 24 | return createSlice(startTs, maxValue, windowManager.getCurrentCount(), windowManager.getCurrentCount(), type); 25 | } 26 | 27 | 28 | } 29 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/state/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.state 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.state

13 |
14 |

Interfaces

15 | 21 |
22 | 23 | 24 | -------------------------------------------------------------------------------- /demo/flink-demo/src/main/java/stream/scotty/demo/flink/FlinkQuantileDemo.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.flink; 2 | 3 | import stream.scotty.core.windowType.*; 4 | import stream.scotty.flinkconnector.*; 5 | import stream.scotty.demo.flink.windowFunctions.*; 6 | import org.apache.flink.api.java.tuple.*; 7 | import org.apache.flink.streaming.api.*; 8 | import org.apache.flink.streaming.api.datastream.*; 9 | import org.apache.flink.streaming.api.environment.*; 10 | 11 | import java.io.*; 12 | 13 | public class FlinkQuantileDemo implements Serializable { 14 | 15 | public static void main(String[] args) throws Exception { 16 | LocalStreamEnvironment sev = StreamExecutionEnvironment.createLocalEnvironment(); 17 | sev.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 18 | 19 | DataStream> stream = sev.addSource(new DemoSource()); 20 | 21 | KeyedScottyWindowOperator, Tuple2> windowOperator = 22 | new KeyedScottyWindowOperator<>(new QuantileWindowFunction(0.5)); 23 | 24 | windowOperator.addWindow(new TumblingWindow(WindowMeasure.Time, 1000)); 25 | 26 | stream 27 | .keyBy(0) 28 | .process(windowOperator) 29 | .map(x -> x.getAggValues().get(0).f1) 30 | .print(); 31 | 32 | sev.execute("demo"); 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /demo/storm-demo/src/main/java/stream/scotty/demo/storm/windowFunctions/Quantile.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.storm.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.AggregateFunction; 4 | import stream.scotty.core.windowFunction.CloneablePartialStateFunction; 5 | 6 | public class Quantile implements AggregateFunction, CloneablePartialStateFunction { 7 | private final double quantile; 8 | 9 | public Quantile(double quantile) { 10 | this.quantile = quantile; 11 | } 12 | 13 | @Override 14 | public QuantileTreeMap lift(Integer inputTuple) { 15 | return new QuantileTreeMap(inputTuple,quantile); 16 | } 17 | 18 | @Override 19 | public QuantileTreeMap combine(QuantileTreeMap partialAggregate1, QuantileTreeMap partialAggregate2) { 20 | return partialAggregate1.merge(partialAggregate2); 21 | } 22 | 23 | @Override 24 | public QuantileTreeMap liftAndCombine(QuantileTreeMap partialAggregate, Integer inputTuple) { 25 | partialAggregate.addValue(inputTuple); 26 | return partialAggregate; 27 | } 28 | 29 | @Override 30 | public Integer lower(QuantileTreeMap aggregate) { 31 | return aggregate.getQuantile(); 32 | } 33 | 34 | @Override 35 | public QuantileTreeMap clone(QuantileTreeMap partialAggregate) { 36 | return partialAggregate.clone(); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /demo/flink-demo/src/main/java/stream/scotty/demo/flink/FlinkSumCountWindowDemo.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.flink; 2 | 3 | import stream.scotty.core.windowType.*; 4 | import stream.scotty.flinkconnector.*; 5 | import stream.scotty.demo.flink.windowFunctions.*; 6 | import org.apache.flink.api.java.tuple.*; 7 | import org.apache.flink.streaming.api.*; 8 | import org.apache.flink.streaming.api.datastream.*; 9 | import org.apache.flink.streaming.api.environment.*; 10 | 11 | import java.io.*; 12 | 13 | public class FlinkSumCountWindowDemo implements Serializable { 14 | 15 | 16 | public static void main(String[] args) throws Exception { 17 | LocalStreamEnvironment sev = StreamExecutionEnvironment.createLocalEnvironment(); 18 | sev.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 19 | sev.setParallelism(1); 20 | sev.setMaxParallelism(1); 21 | 22 | DataStream> stream = sev.addSource(new DemoSource()); 23 | 24 | KeyedScottyWindowOperator, Tuple2> processingFunction = 25 | new KeyedScottyWindowOperator<>(new SumWindowFunction()); 26 | 27 | processingFunction.addWindow(new TumblingWindow(WindowMeasure.Count, 1000)); 28 | 29 | stream 30 | .keyBy(0) 31 | .process(processingFunction) 32 | .print(); 33 | 34 | sev.execute("demo"); 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /demo/samza-demo/src/main/java/stream/scotty/demo/samza/windowFunctions/QuantileWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.samza.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.AggregateFunction; 4 | import stream.scotty.core.windowFunction.CloneablePartialStateFunction; 5 | 6 | public class QuantileWindowFunction implements AggregateFunction, CloneablePartialStateFunction { 7 | private final double quantile; 8 | 9 | public QuantileWindowFunction(final double quantile) { 10 | this.quantile = quantile; 11 | } 12 | 13 | @Override 14 | public QuantileTreeMap lift(Integer inputTuple) { 15 | return new QuantileTreeMap(inputTuple, quantile); 16 | } 17 | 18 | @Override 19 | public Integer lower(QuantileTreeMap aggregate) { 20 | return aggregate.getQuantile(); 21 | } 22 | 23 | @Override 24 | public QuantileTreeMap combine(QuantileTreeMap partialAggregate1, QuantileTreeMap partialAggregate2) { 25 | return partialAggregate1.merge(partialAggregate2); 26 | } 27 | 28 | @Override 29 | public QuantileTreeMap liftAndCombine(QuantileTreeMap partialAggregate, Integer inputTuple) { 30 | partialAggregate.addValue(inputTuple); 31 | return partialAggregate; 32 | } 33 | 34 | @Override 35 | public QuantileTreeMap clone(QuantileTreeMap partialAggregate) { 36 | return partialAggregate.clone(); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /demo/spark-demo/src/main/java/stream/scotty/demo/spark/windowFunctions/QuantileWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.spark.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.AggregateFunction; 4 | import stream.scotty.core.windowFunction.CloneablePartialStateFunction; 5 | 6 | public class QuantileWindowFunction implements AggregateFunction, CloneablePartialStateFunction { 7 | private final double quantile; 8 | 9 | public QuantileWindowFunction(final double quantile) { 10 | this.quantile = quantile; 11 | } 12 | 13 | @Override 14 | public QuantileTreeMap lift(Integer inputTuple) { 15 | return new QuantileTreeMap(inputTuple, quantile); 16 | } 17 | 18 | @Override 19 | public Integer lower(QuantileTreeMap aggregate) { 20 | return aggregate.getQuantile(); 21 | } 22 | 23 | @Override 24 | public QuantileTreeMap combine(QuantileTreeMap partialAggregate1, QuantileTreeMap partialAggregate2) { 25 | return partialAggregate1.merge(partialAggregate2); 26 | } 27 | 28 | @Override 29 | public QuantileTreeMap liftAndCombine(QuantileTreeMap partialAggregate, Integer inputTuple) { 30 | partialAggregate.addValue(inputTuple); 31 | return partialAggregate; 32 | } 33 | 34 | @Override 35 | public QuantileTreeMap clone(QuantileTreeMap partialAggregate) { 36 | return partialAggregate.clone(); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /flink-connector/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | scotty 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | flink-connector 13 | 14 | 15 | stream.scotty 16 | core 17 | 0.4 18 | 19 | 20 | stream.scotty 21 | slicing 22 | 0.4 23 | 24 | 25 | stream.scotty 26 | state 27 | 0.4 28 | 29 | 30 | 31 | org.apache.flink 32 | flink-streaming-java_2.11 33 | 1.9.2 34 | 35 | 36 | 37 | 38 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/core/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.core 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.core

13 |
14 |

Interfaces

15 | 20 |

Classes

21 | 24 |
25 | 26 | 27 | -------------------------------------------------------------------------------- /demo/kafka-demo/src/main/java/stream/scotty/demo/kafkaStreams/windowFunctions/QuantileWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.kafkaStreams.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.AggregateFunction; 4 | import stream.scotty.core.windowFunction.CloneablePartialStateFunction; 5 | 6 | public class QuantileWindowFunction implements AggregateFunction, CloneablePartialStateFunction { 7 | private final double quantile; 8 | 9 | public QuantileWindowFunction(final double quantile) { 10 | this.quantile = quantile; 11 | } 12 | 13 | @Override 14 | public QuantileTreeMap lift(Integer inputTuple) { 15 | return new QuantileTreeMap(inputTuple, quantile); 16 | } 17 | 18 | @Override 19 | public Integer lower(QuantileTreeMap aggregate) { 20 | return aggregate.getQuantile(); 21 | } 22 | 23 | @Override 24 | public QuantileTreeMap combine(QuantileTreeMap partialAggregate1, QuantileTreeMap partialAggregate2) { 25 | return partialAggregate1.merge(partialAggregate2); 26 | } 27 | 28 | @Override 29 | public QuantileTreeMap liftAndCombine(QuantileTreeMap partialAggregate, Integer inputTuple) { 30 | partialAggregate.addValue(inputTuple); 31 | return partialAggregate; 32 | } 33 | 34 | @Override 35 | public QuantileTreeMap clone(QuantileTreeMap partialAggregate) { 36 | return partialAggregate.clone(); 37 | } 38 | } 39 | 40 | -------------------------------------------------------------------------------- /demo/samza-demo/src/main/java/stream/scotty/demo/samza/DemoTaskFactory.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.samza; 2 | 3 | import stream.scotty.core.windowType.SlidingWindow; 4 | import stream.scotty.core.windowType.TumblingWindow; 5 | import stream.scotty.core.windowType.WindowMeasure; 6 | import stream.scotty.samzaconnector.KeyedScottyWindowOperator; 7 | import stream.scotty.demo.samza.windowFunctions.SumWindowFunction; 8 | import org.apache.samza.system.SystemStream; 9 | import org.apache.samza.task.StreamTask; 10 | import org.apache.samza.task.StreamTaskFactory; 11 | 12 | public class DemoTaskFactory implements StreamTaskFactory { 13 | private String SYSTEM_DESCRIPTOR_NAME; 14 | private String OUTPUT_DESCRIPTOR_NAME; 15 | 16 | public DemoTaskFactory(String SYSTEM_DESCRIPTOR_NAME, String OUTPUT_DESCRIPTOR_NAME) { 17 | this.SYSTEM_DESCRIPTOR_NAME = SYSTEM_DESCRIPTOR_NAME; 18 | this.OUTPUT_DESCRIPTOR_NAME = OUTPUT_DESCRIPTOR_NAME; 19 | } 20 | 21 | @Override 22 | public StreamTask createInstance() { 23 | SystemStream stream = new SystemStream(SYSTEM_DESCRIPTOR_NAME, OUTPUT_DESCRIPTOR_NAME); 24 | KeyedScottyWindowOperator operator = new KeyedScottyWindowOperator 25 | (new SumWindowFunction(), 100, stream); 26 | operator.addWindow(new SlidingWindow(WindowMeasure.Time, 5000, 1000)); 27 | operator.addWindow(new TumblingWindow(WindowMeasure.Time, 2000)); 28 | 29 | return operator; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/slicing/aggregationstore/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.slicing.aggregationstore 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.slicing.aggregationstore

13 |
14 |

Interfaces

15 | 19 |

Classes

20 | 23 |
24 | 25 | 26 | -------------------------------------------------------------------------------- /demo/storm-demo/src/main/java/stream/scotty/demo/storm/PrinterBolt.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with 3 | * this work for additional information regarding copyright ownership. The ASF licenses this file to you under the Apache License, Version 4 | * 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, 9 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions 10 | * and limitations under the License. 11 | */ 12 | 13 | package stream.scotty.demo.storm; 14 | 15 | import org.apache.storm.topology.BasicOutputCollector; 16 | import org.apache.storm.topology.OutputFieldsDeclarer; 17 | import org.apache.storm.topology.base.BaseBasicBolt; 18 | import org.apache.storm.tuple.Tuple; 19 | 20 | 21 | public class PrinterBolt extends BaseBasicBolt { 22 | 23 | @Override 24 | public void execute(Tuple tuple, BasicOutputCollector collector) { 25 | System.out.printf("\nResult Tuple: "+ tuple.getValues().toString()); 26 | } 27 | 28 | @Override 29 | public void declareOutputFields(OutputFieldsDeclarer declarer) { 30 | } 31 | 32 | } -------------------------------------------------------------------------------- /demo/flink-demo/src/main/java/stream/scotty/demo/flink/FlinkSumDemo.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.flink; 2 | 3 | import stream.scotty.core.windowType.*; 4 | import stream.scotty.flinkconnector.*; 5 | import stream.scotty.demo.flink.windowFunctions.*; 6 | import org.apache.flink.api.java.tuple.*; 7 | import org.apache.flink.streaming.api.*; 8 | import org.apache.flink.streaming.api.datastream.*; 9 | import org.apache.flink.streaming.api.environment.*; 10 | 11 | import java.io.*; 12 | 13 | public class FlinkSumDemo implements Serializable { 14 | 15 | 16 | public static void main(String[] args) throws Exception { 17 | LocalStreamEnvironment sev = StreamExecutionEnvironment.createLocalEnvironment(); 18 | sev.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 19 | sev.setParallelism(1); 20 | sev.setMaxParallelism(1); 21 | 22 | DataStream> stream = sev.addSource(new DemoSource()); 23 | 24 | KeyedScottyWindowOperator, Tuple2> processingFunction = 25 | new KeyedScottyWindowOperator<>(new SumWindowFunction()); 26 | 27 | processingFunction 28 | .addWindow(new TumblingWindow(WindowMeasure.Time, 2000)) 29 | .addWindow(new SlidingWindow(WindowMeasure.Time, 5000,1000)); 30 | 31 | stream 32 | .keyBy(0) 33 | .process(processingFunction) 34 | .print(); 35 | 36 | sev.execute("demo"); 37 | } 38 | 39 | } 40 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/slicing/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.slicing 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.slicing

13 |
14 |

Classes

15 | 23 |
24 | 25 | 26 | -------------------------------------------------------------------------------- /benchmark/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | scotty 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | benchmark 13 | pom 14 | 15 | 16 | stream.scotty 17 | slicing 18 | 0.4 19 | 20 | 21 | stream.scotty 22 | flink-connector 23 | 0.4 24 | 25 | 26 | org.openjdk.jmh 27 | jmh-core 28 | 1.19 29 | 30 | 31 | org.openjdk.jmh 32 | jmh-generator-annprocess 33 | 1.19 34 | 35 | 36 | com.google.code.gson 37 | gson 38 | 2.8.9 39 | 40 | 41 | 42 | -------------------------------------------------------------------------------- /demo/beam-demo/src/main/java/stream/scotty/demo/beam/windowFunctions/Mean.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.beam.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.AggregateFunction; 4 | import org.apache.beam.sdk.values.KV; 5 | 6 | import java.io.Serializable; 7 | 8 | public class Mean implements AggregateFunction, KV, KV>, Serializable { 9 | @Override 10 | public KV lift(KV inputTuple) { 11 | return KV.of(inputTuple.getKey(), new Pair(inputTuple.getValue())); 12 | } 13 | 14 | @Override 15 | public KV combine(KV partialAggregate1, KV partialAggregate2) { 16 | return KV.of(partialAggregate1.getKey(), new Pair(partialAggregate1.getValue().sum + partialAggregate2.getValue().sum, 17 | partialAggregate1.getValue().count + partialAggregate2.getValue().count)); 18 | } 19 | 20 | @Override 21 | public KV lower(KV aggregate) { 22 | return KV.of(aggregate.getKey(), aggregate.getValue().getResult()); 23 | } 24 | } 25 | 26 | class Pair { 27 | int sum; 28 | int count; 29 | 30 | public Pair(int sum) { 31 | this.sum = sum; 32 | this.count = 1; 33 | } 34 | 35 | public Pair(int sum, int count) { 36 | this.sum = sum; 37 | this.count = count; 38 | } 39 | 40 | public int getResult() { 41 | return sum / count; 42 | } 43 | 44 | @Override 45 | public String toString() { 46 | return getResult() + ""; 47 | } 48 | } -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/TumblingWindow.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType; 2 | 3 | import stream.scotty.core.*; 4 | import stream.scotty.core.*; 5 | 6 | public class TumblingWindow implements ContextFreeWindow { 7 | 8 | private final WindowMeasure measure; 9 | /** 10 | * Size of the tumbling window 11 | */ 12 | private final long size; 13 | 14 | public TumblingWindow(WindowMeasure measure, long size) { 15 | this.measure = measure; 16 | this.size = size; 17 | } 18 | 19 | public long getSize() { 20 | return size; 21 | } 22 | 23 | @Override 24 | public WindowMeasure getWindowMeasure() { 25 | return measure; 26 | } 27 | 28 | @Override 29 | public long assignNextWindowStart(long recordStamp) { 30 | return recordStamp + getSize() - (recordStamp) % getSize(); 31 | } 32 | 33 | @Override 34 | public void triggerWindows(WindowCollector aggregateWindows, long lastWatermark, long currentWatermark) { 35 | long lastStart = lastWatermark - ((lastWatermark + size) % size); 36 | for (long windowStart = lastStart; windowStart + size <= currentWatermark; windowStart += size) { 37 | aggregateWindows.trigger(windowStart, windowStart + size, measure); 38 | } 39 | } 40 | 41 | @Override 42 | public long clearDelay() { 43 | return size; 44 | } 45 | 46 | @Override 47 | public String toString() { 48 | return "TumblingWindow{" + 49 | "measure=" + measure + 50 | ", size=" + size + 51 | '}'; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/slicing/aggregationstore/test/windowTest/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.slicing.aggregationstore.test.windowTest 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.slicing.aggregationstore.test.windowTest

13 | 22 | 23 | 24 | -------------------------------------------------------------------------------- /demo/samza-demo/src/main/java/stream/scotty/demo/samza/DemoKafkaProducer.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.samza; 2 | 3 | import org.apache.kafka.clients.producer.KafkaProducer; 4 | import org.apache.kafka.clients.producer.Producer; 5 | import org.apache.kafka.clients.producer.ProducerRecord; 6 | 7 | import java.util.Properties; 8 | import java.util.Random; 9 | 10 | public class DemoKafkaProducer extends Thread { 11 | private Properties properties; 12 | private Producer producer; 13 | private Random key; 14 | private Random value; 15 | private String inputTopicName; 16 | 17 | public DemoKafkaProducer(String inputTopicName) { 18 | properties = new Properties(); 19 | properties.put("bootstrap.servers", "localhost:9092"); 20 | properties.put("acks", "all"); 21 | properties.put("key.serializer", "org.apache.kafka.common.serialization.IntegerSerializer"); 22 | properties.put("value.serializer", "org.apache.kafka.common.serialization.IntegerSerializer"); 23 | producer = new KafkaProducer<>(properties); 24 | this.key = new Random(42); 25 | this.value = new Random(43); 26 | this.inputTopicName = inputTopicName; 27 | setDaemon(true); 28 | } 29 | 30 | @Override 31 | public void run() { 32 | while (!isInterrupted()) { 33 | try { 34 | producer.send(new ProducerRecord<>(inputTopicName, 1, value.nextInt(10))); 35 | Thread.sleep(1); 36 | } catch (InterruptedException e) { 37 | producer.close(); 38 | interrupt(); 39 | } 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/flinkconnector/demo/windowFunctions/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.demo.flink.windowFunctions 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.demo.flink.windowFunctions

13 |
14 |

Classes

15 | 22 |
23 | 24 | 25 | -------------------------------------------------------------------------------- /demo/kafka-demo/src/main/java/stream/scotty/demo/kafkaStreams/DemoKafkaProducer.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.kafkaStreams; 2 | 3 | import org.apache.kafka.clients.producer.KafkaProducer; 4 | import org.apache.kafka.clients.producer.Producer; 5 | import org.apache.kafka.clients.producer.ProducerRecord; 6 | 7 | import java.util.Properties; 8 | import java.util.Random; 9 | 10 | public class DemoKafkaProducer extends Thread { 11 | private Properties properties; 12 | private Producer producer; 13 | private Random key; 14 | private Random value; 15 | private String inputTopicName; 16 | 17 | public DemoKafkaProducer(String inputTopicName) { 18 | properties = new Properties(); 19 | properties.put("bootstrap.servers", "localhost:9092"); 20 | properties.put("acks", "all"); 21 | properties.put("key.serializer", "org.apache.kafka.common.serialization.IntegerSerializer"); 22 | properties.put("value.serializer", "org.apache.kafka.common.serialization.IntegerSerializer"); 23 | producer = new KafkaProducer<>(properties); 24 | this.key = new Random(42); 25 | this.value = new Random(43); 26 | this.inputTopicName = inputTopicName; 27 | setDaemon(true); 28 | } 29 | 30 | @Override 31 | public void run() { 32 | while (!isInterrupted()) { 33 | try { 34 | producer.send(new ProducerRecord<>(inputTopicName, 1, value.nextInt(10))); 35 | Thread.sleep(1); 36 | } catch (InterruptedException e) { 37 | producer.close(); 38 | interrupt(); 39 | } 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /demo/spark-demo/src/main/java/stream/scotty/demo/spark/DemoKafkaProducer.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.spark; 2 | 3 | import org.apache.kafka.clients.producer.KafkaProducer; 4 | import org.apache.kafka.clients.producer.Producer; 5 | import org.apache.kafka.clients.producer.ProducerRecord; 6 | 7 | import java.util.Properties; 8 | import java.util.Random; 9 | 10 | public class DemoKafkaProducer extends Thread { 11 | private Properties properties; 12 | private Producer producer; 13 | private Random key; 14 | private Random value; 15 | private String inputTopicName; 16 | 17 | public DemoKafkaProducer(String inputTopicName) { 18 | properties = new Properties(); 19 | properties.put("bootstrap.servers", "localhost:9092"); 20 | properties.put("acks", "all"); 21 | properties.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 22 | properties.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 23 | producer = new KafkaProducer<>(properties); 24 | this.key = new Random(42); 25 | this.value = new Random(43); 26 | this.inputTopicName = inputTopicName; 27 | setDaemon(true); 28 | } 29 | 30 | @Override 31 | public void run() { 32 | while (!isInterrupted()) { 33 | try { 34 | producer.send(new ProducerRecord<>(inputTopicName, "1", Integer.toString(value.nextInt(10)))); 35 | Thread.sleep(1); 36 | } catch (InterruptedException e) { 37 | producer.close(); 38 | interrupt(); 39 | } 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /demo/beam-demo/src/main/java/stream/scotty/demo/beam/dataGenerator/Checkpoint.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.beam.dataGenerator; 2 | 3 | import org.apache.beam.sdk.coders.AvroCoder; 4 | import org.apache.beam.sdk.coders.DefaultCoder; 5 | import org.apache.beam.sdk.io.UnboundedSource; 6 | import org.joda.time.Instant; 7 | 8 | import java.io.IOException; 9 | 10 | /** 11 | * The checkpoint is simply the last value produced. 12 | */ 13 | @DefaultCoder(AvroCoder.class) 14 | public class Checkpoint implements UnboundedSource.CheckpointMark { 15 | private final int lastEmittedKey; 16 | private final int lastEmittedValue; 17 | 18 | private final Instant startTime; 19 | 20 | @SuppressWarnings("unused") // For AvroCoder 21 | private Checkpoint() { 22 | this.lastEmittedKey = 1; 23 | this.lastEmittedValue = 1; 24 | this.startTime = Instant.now(); 25 | } 26 | 27 | 28 | /** Creates a checkpoint mark reflecting the last emitted value. */ 29 | public Checkpoint(int lastEmittedKey, int lastEmittedValue, Instant startTime) { 30 | this.lastEmittedKey = lastEmittedKey; 31 | this.lastEmittedValue = lastEmittedValue; 32 | this.startTime = startTime; 33 | } 34 | 35 | /** Returns the last value emitted by the reader. */ 36 | public Integer getLastEmittedKey() { 37 | return lastEmittedKey; 38 | } 39 | 40 | public Integer getLastEmittedValue() { 41 | return lastEmittedValue; 42 | } 43 | 44 | /** Returns the time the reader was started. */ 45 | public Instant getStartTime() { 46 | return startTime; 47 | } 48 | 49 | 50 | @Override 51 | public void finalizeCheckpoint() throws IOException {} 52 | } -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/core/windowType/windowContext/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.core.windowType.windowContext 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.core.windowType.windowContext

13 |
14 |

Interfaces

15 | 18 |

Classes

19 | 25 |
26 | 27 | 28 | -------------------------------------------------------------------------------- /demo/flink-demo/src/main/java/stream/scotty/demo/flink/DemoSource.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.flink; 2 | 3 | import org.apache.flink.api.java.tuple.*; 4 | import org.apache.flink.configuration.*; 5 | import org.apache.flink.streaming.api.functions.source.*; 6 | import org.apache.flink.streaming.api.watermark.*; 7 | 8 | import java.io.*; 9 | import java.util.*; 10 | 11 | public class DemoSource extends RichSourceFunction> implements Serializable { 12 | 13 | private Random key; 14 | private Random value; 15 | private boolean canceled = false; 16 | /** 17 | * This parameter configures the watermark delay. 18 | */ 19 | private long watermarkDelay = 1000; 20 | 21 | public DemoSource(){} 22 | 23 | public DemoSource(long watermarkDelay){ 24 | this.watermarkDelay = watermarkDelay; 25 | } 26 | 27 | @Override 28 | public void open(Configuration parameters) throws Exception { 29 | super.open(parameters); 30 | this.key = new Random(42); 31 | this.value = new Random(43); 32 | } 33 | 34 | public long lastWaterMarkTs = 0; 35 | 36 | @Override 37 | public void run(SourceContext> ctx) throws Exception { 38 | while (!canceled) { 39 | 40 | ctx.collectWithTimestamp(new Tuple2<>(1, value.nextInt(10)), System.currentTimeMillis()); 41 | if (lastWaterMarkTs + 1000 < System.currentTimeMillis()) { 42 | long watermark = System.currentTimeMillis() - watermarkDelay; 43 | ctx.emitWatermark(new Watermark(watermark)); 44 | lastWaterMarkTs = System.currentTimeMillis(); 45 | } 46 | Thread.sleep(1); 47 | } 48 | } 49 | 50 | @Override 51 | public void cancel() { 52 | canceled = true; 53 | } 54 | } -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/flinkBenchmark/ThroughputLogger.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.flinkBenchmark; 2 | 3 | import org.apache.flink.api.common.functions.FlatMapFunction; 4 | import org.apache.flink.util.Collector; 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | 8 | public class ThroughputLogger implements FlatMapFunction { 9 | 10 | private static final Logger LOG = LoggerFactory.getLogger(ThroughputLogger.class); 11 | 12 | private long totalReceived = 0; 13 | private long lastTotalReceived = 0; 14 | private long lastLogTimeMs = -1; 15 | private int elementSize; 16 | private long logfreq; 17 | 18 | public ThroughputLogger(int elementSize, long logfreq) { 19 | this.elementSize = elementSize; 20 | this.logfreq = logfreq; 21 | } 22 | 23 | @Override 24 | public void flatMap(T element, Collector collector) throws Exception { 25 | totalReceived++; 26 | if (totalReceived % logfreq == 0) { 27 | // throughput over entire time 28 | long now = System.currentTimeMillis(); 29 | 30 | // throughput for the last "logfreq" elements 31 | if (lastLogTimeMs == -1) { 32 | // init (the first) 33 | lastLogTimeMs = now; 34 | lastTotalReceived = totalReceived; 35 | } else { 36 | long timeDiff = now - lastLogTimeMs; 37 | long elementDiff = totalReceived - lastTotalReceived; 38 | double ex = (1000 / (double) timeDiff); 39 | LOG.error("During the last {} ms, we received {} elements. That's {} elements/second/core. ", 40 | timeDiff, elementDiff, elementDiff * ex); 41 | 42 | ThroughputStatistics.getInstance().addThrouputResult(elementDiff * ex); 43 | //System.out.println(ThroughputStatistics.getInstance().toString()); 44 | // reinit 45 | lastLogTimeMs = now; 46 | lastTotalReceived = totalReceived; 47 | } 48 | } 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /kafkaStreams-connector/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | scotty 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | kafkaStreams-connector 13 | 14 | 15 | 0.4 16 | 1.2.0 17 | 2.4.0 18 | 19 | 20 | 21 | 22 | stream.scotty 23 | core 24 | ${scotty.version} 25 | 26 | 27 | stream.scotty 28 | slicing 29 | ${scotty.version} 30 | 31 | 32 | stream.scotty 33 | state 34 | ${scotty.version} 35 | 36 | 37 | 38 | org.apache.kafka 39 | kafka-streams 40 | ${kafka.version} 41 | 42 | 43 | org.apache.kafka 44 | kafka-clients 45 | ${kafka.version} 46 | 47 | 48 | 49 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/core/windowFunction/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.core.windowFunction 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.core.windowFunction

13 | 23 | 24 | 25 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/slicing/slice/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.slicing.slice 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.slicing.slice

13 |
14 |

Interfaces

15 | 19 |

Classes

20 | 27 |
28 | 29 | 30 | -------------------------------------------------------------------------------- /kafkaStreams-connector/src/main/java/stream/scotty/kafkastreamsconnector/KeyedScottyWindowOperatorSupplier.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.kafkastreamsconnector; 2 | 3 | import stream.scotty.core.windowFunction.AggregateFunction; 4 | import stream.scotty.core.windowType.SlidingWindow; 5 | import stream.scotty.core.windowType.TumblingWindow; 6 | import stream.scotty.core.windowType.Window; 7 | import java.util.ArrayList; 8 | import java.util.List; 9 | import org.apache.kafka.streams.processor.Processor; 10 | import org.apache.kafka.streams.processor.ProcessorSupplier; 11 | 12 | public class KeyedScottyWindowOperatorSupplier implements ProcessorSupplier { 13 | 14 | private final AggregateFunction windowFunction; 15 | private final long allowedLateness; 16 | private final List windows; 17 | 18 | public KeyedScottyWindowOperatorSupplier(AggregateFunction windowFunction, long allowedLateness) { 19 | this.windowFunction = windowFunction; 20 | this.windows = new ArrayList<>(); 21 | this.allowedLateness = allowedLateness; 22 | } 23 | 24 | /** 25 | * Register a new @{@link Window} definition that should be added to the Window Operator. 26 | * For example {@link SlidingWindow} or {@link TumblingWindow} 27 | * 28 | * @param window the new window definition 29 | */ 30 | public KeyedScottyWindowOperatorSupplier addWindow(Window window) { 31 | windows.add(window); 32 | return this; 33 | } 34 | 35 | @Override 36 | public Processor get() { 37 | final KeyedScottyWindowOperator processor = 38 | new KeyedScottyWindowOperator<>(this.windowFunction, this.allowedLateness); 39 | 40 | for (Window window : this.windows) { 41 | processor.addWindow(window); 42 | } 43 | 44 | return processor; 45 | } 46 | 47 | } 48 | -------------------------------------------------------------------------------- /demo/flink-demo/src/main/java/stream/scotty/demo/flink/FlinkPunctuationDemo.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.flink; 2 | 3 | import stream.scotty.core.windowType.PunctuationWindow; 4 | import stream.scotty.demo.flink.windowFunctions.SumWindowFunction; 5 | import stream.scotty.flinkconnector.KeyedScottyWindowOperator; 6 | import org.apache.flink.api.java.tuple.Tuple; 7 | import org.apache.flink.api.java.tuple.Tuple2; 8 | import org.apache.flink.streaming.api.TimeCharacteristic; 9 | import org.apache.flink.streaming.api.datastream.DataStream; 10 | import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; 11 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 12 | 13 | import java.io.Serializable; 14 | 15 | public class FlinkPunctuationDemo implements Serializable { 16 | public static void main(String[] args) throws Exception { 17 | 18 | LocalStreamEnvironment sev = StreamExecutionEnvironment.createLocalEnvironment(); 19 | sev.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 20 | sev.setParallelism(1); 21 | sev.setMaxParallelism(1); 22 | 23 | DataStream> stream = sev.addSource(new DemoSource()); 24 | 25 | KeyedScottyWindowOperator, Tuple2> processingFunction = 26 | new KeyedScottyWindowOperator<>(new SumWindowFunction()); 27 | 28 | //define the punctuation, can have different values 29 | Tuple2 p = new Tuple2(1, 9); //every time a tuple with the value 9 is emitted, a new window starts 30 | 31 | processingFunction 32 | .addWindow(new PunctuationWindow(p)); 33 | 34 | stream 35 | .keyBy(0) 36 | .process(processingFunction) 37 | .print(); 38 | 39 | sev.execute("demo"); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /demo/flink-demo/src/main/java/stream/scotty/demo/flink/windowFunctions/QuantileWindowFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.flink.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.*; 4 | import org.apache.flink.api.java.tuple.*; 5 | 6 | public class QuantileWindowFunction implements AggregateFunction, 7 | Tuple2, 8 | Tuple2>, 9 | CloneablePartialStateFunction> { 10 | 11 | private final double quantile; 12 | 13 | public QuantileWindowFunction(final double quantile) { 14 | this.quantile = quantile; 15 | } 16 | 17 | 18 | @Override 19 | public Tuple2 lift(Tuple2 inputTuple) { 20 | return new Tuple2<>(inputTuple.f0, new QuantileTreeMap(inputTuple.f1, quantile)); 21 | } 22 | 23 | @Override 24 | public Tuple2 lower(Tuple2 aggregate) { 25 | return new Tuple2<>(aggregate.f0, aggregate.f1.getQuantile()); 26 | } 27 | 28 | @Override 29 | public Tuple2 combine(Tuple2 partialAggregate1, Tuple2 partialAggregate2) { 30 | return new Tuple2<>(partialAggregate1.f0, partialAggregate1.f1.merge(partialAggregate2.f1)); 31 | } 32 | 33 | @Override 34 | public Tuple2 liftAndCombine(Tuple2 partialAggregate, Tuple2 inputTuple) { 35 | partialAggregate.f1.addValue(inputTuple.f1); 36 | return partialAggregate; 37 | } 38 | 39 | @Override 40 | public Tuple2 clone(Tuple2 partialAggregate) { 41 | return new Tuple2<>(partialAggregate.f0, partialAggregate.f1.clone()); 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /kafkaStreams-connector/src/main/java/stream/scotty/kafkastreamsconnector/KeyedScottyWindowTransformerSupplier.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.kafkastreamsconnector; 2 | 3 | import stream.scotty.core.windowFunction.AggregateFunction; 4 | import stream.scotty.core.windowType.SlidingWindow; 5 | import stream.scotty.core.windowType.TumblingWindow; 6 | import stream.scotty.core.windowType.Window; 7 | import java.util.ArrayList; 8 | import java.util.List; 9 | import org.apache.kafka.streams.kstream.Transformer; 10 | import org.apache.kafka.streams.kstream.TransformerSupplier; 11 | 12 | public class KeyedScottyWindowTransformerSupplier implements TransformerSupplier { 13 | 14 | private final AggregateFunction windowFunction; 15 | private final long allowedLateness; 16 | private final List windows; 17 | 18 | public KeyedScottyWindowTransformerSupplier(AggregateFunction windowFunction, long allowedLateness) { 19 | this.windowFunction = windowFunction; 20 | this.windows = new ArrayList<>(); 21 | this.allowedLateness = allowedLateness; 22 | } 23 | 24 | /** 25 | * Register a new @{@link Window} definition that should be added to the Window Operator. 26 | * For example {@link SlidingWindow} or {@link TumblingWindow} 27 | * 28 | * @param window the new window definition 29 | */ 30 | public KeyedScottyWindowTransformerSupplier addWindow(Window window) { 31 | windows.add(window); 32 | return this; 33 | } 34 | 35 | @Override 36 | public Transformer get() { 37 | final KeyedScottyWindowTransformer processor = 38 | new KeyedScottyWindowTransformer<>(this.windowFunction, this.allowedLateness); 39 | 40 | for (Window window : this.windows) { 41 | processor.addWindow(window); 42 | } 43 | 44 | return processor; 45 | } 46 | 47 | } 48 | -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/flinkBenchmark/AnalyzeTool.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.flinkBenchmark; 2 | 3 | import org.apache.commons.math3.stat.descriptive.SummaryStatistics; 4 | 5 | import java.io.File; 6 | import java.io.FileNotFoundException; 7 | import java.util.List; 8 | import java.util.Scanner; 9 | import java.util.regex.Matcher; 10 | import java.util.regex.Pattern; 11 | 12 | public class AnalyzeTool { 13 | 14 | public static class Result { 15 | 16 | 17 | SummaryStatistics throughputs; 18 | 19 | 20 | public Result(SummaryStatistics throughputs) { 21 | 22 | this.throughputs = throughputs; 23 | 24 | } 25 | } 26 | 27 | public static Result analyze(String file, List toIgnore) throws FileNotFoundException { 28 | Scanner sc = new Scanner(new File(file)); 29 | 30 | String l; 31 | 32 | Pattern throughputPattern = Pattern.compile(".*That's ([0-9.]+) elements\\/second\\/core.*"); 33 | 34 | 35 | SummaryStatistics throughputs = new SummaryStatistics(); 36 | String currentHost = null; 37 | 38 | 39 | while (sc.hasNextLine()) { 40 | l = sc.nextLine(); 41 | 42 | // ---------- throughput --------------- 43 | Matcher tpMatcher = throughputPattern.matcher(l); 44 | if (tpMatcher.matches()) { 45 | double eps = Double.valueOf(tpMatcher.group(1)); 46 | throughputs.addValue(eps); 47 | // System.out.println("epts = "+eps); 48 | 49 | } 50 | } 51 | 52 | return new Result(throughputs); 53 | } 54 | 55 | public static void main(String[] args) throws FileNotFoundException { 56 | Result r1 = analyze(args[0], null); 57 | 58 | SummaryStatistics throughputs = r1.throughputs; 59 | // System.out.println("lat-mean;lat-median;lat-90percentile;lat-95percentile;lat-99percentile;throughput-mean;throughput-max;latencies;throughputs;"); 60 | System.out.println("all-machines;" + throughputs.getMean() + ";" + throughputs.getMax() + ";" + throughputs.getN()); 61 | 62 | 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /demo/beam-demo/src/main/java/stream/scotty/demo/beam/windowFunctions/Quantile.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.beam.windowFunctions; 2 | 3 | import stream.scotty.core.windowFunction.AggregateFunction; 4 | import stream.scotty.core.windowFunction.CloneablePartialStateFunction; 5 | import org.apache.beam.sdk.values.KV; 6 | 7 | import java.io.Serializable; 8 | 9 | public class Quantile implements AggregateFunction< 10 | KV, 11 | KV, 12 | KV>, 13 | CloneablePartialStateFunction>, Serializable { 14 | private final double quantile; 15 | 16 | public Quantile(double quantile) { 17 | this.quantile = quantile; 18 | } 19 | 20 | @Override 21 | public KV lower(KV aggregate) { 22 | return KV.of(aggregate.getKey(),aggregate.getValue().getQuantile()); 23 | } 24 | 25 | @Override 26 | public KV lift(KV inputTuple) { 27 | return KV.of(inputTuple.getKey(),new QuantileTreeMap(Math.toIntExact(inputTuple.getValue()),quantile)); 28 | } 29 | 30 | @Override 31 | public KV combine(KV partialAggregate1, KV partialAggregate2) { 32 | return KV.of(partialAggregate1.getKey(),partialAggregate1.getValue().merge(partialAggregate2.getValue())); 33 | } 34 | 35 | @Override 36 | public KV liftAndCombine(KV partialAggregate, KV inputTuple) { 37 | partialAggregate.getValue().addValue(Math.toIntExact(inputTuple.getValue())); 38 | return partialAggregate; 39 | } 40 | 41 | @Override 42 | public KV clone(KV partialAggregate) { 43 | return KV.of(partialAggregate.getKey(),partialAggregate.getValue().clone()); 44 | } 45 | } -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/microbenchmark/AggregationStoreBenchmark.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.microbenchmark; 2 | 3 | import stream.scotty.slicing.state.AggregateState; 4 | import stream.scotty.state.memory.MemoryStateFactory; 5 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 6 | import org.openjdk.jmh.annotations.*; 7 | import org.openjdk.jmh.runner.Runner; 8 | import org.openjdk.jmh.runner.RunnerException; 9 | import org.openjdk.jmh.runner.options.Options; 10 | import org.openjdk.jmh.runner.options.OptionsBuilder; 11 | 12 | import java.util.Collections; 13 | 14 | @State(Scope.Benchmark) 15 | public class AggregationStoreBenchmark { 16 | 17 | 18 | private long ts; 19 | private long n; 20 | private AggregateState aggregationState; 21 | private AggregationStateInline aggregationStateInline; 22 | 23 | @Setup(Level.Iteration) 24 | public void setupIteration() throws Exception { 25 | n = 0; 26 | ts = 0; 27 | MemoryStateFactory memoryStateFactory = new MemoryStateFactory(); 28 | ReduceAggregateFunction wf = new ReduceAggregateFunction() { 29 | @Override 30 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 31 | return partialAggregate1 + partialAggregate2; 32 | } 33 | }; 34 | this.aggregationState = new AggregateState(memoryStateFactory, Collections.singletonList(wf)); 35 | this.aggregationStateInline = new AggregationStateInline(memoryStateFactory); 36 | } 37 | 38 | 39 | 40 | @Benchmark() 41 | public void benchmarkInlineAggStore() throws Exception { 42 | aggregationStateInline.addElement(10); 43 | } 44 | 45 | 46 | public static void main(String[] args) throws RunnerException { 47 | 48 | Options opt = new OptionsBuilder() 49 | .include(AggregationStoreBenchmark.class.getName()) 50 | .forks(1) 51 | .build(); 52 | 53 | new Runner(opt).run(); 54 | } 55 | 56 | 57 | } 58 | -------------------------------------------------------------------------------- /spark-connector/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | scotty 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | spark-connector 13 | 14 | 15 | 0.4 16 | 2.4.4 17 | 2.4.0 18 | 19 | 20 | 21 | 22 | stream.scotty 23 | core 24 | ${scotty.version} 25 | 26 | 27 | stream.scotty 28 | slicing 29 | ${scotty.version} 30 | 31 | 32 | stream.scotty 33 | state 34 | ${scotty.version} 35 | 36 | 37 | 38 | org.apache.spark 39 | spark-sql-kafka-0-10_2.12 40 | ${spark.version} 41 | 42 | 43 | org.apache.spark 44 | spark-sql_2.12 45 | ${spark.version} 46 | 47 | 48 | 49 | org.apache.kafka 50 | kafka-clients 51 | ${kafka.version} 52 | 53 | 54 | 55 | -------------------------------------------------------------------------------- /demo/storm-demo/src/main/java/stream/scotty/demo/storm/ScottyDemoTopology.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.storm; 2 | 3 | import stream.scotty.core.windowType.SessionWindow; 4 | import stream.scotty.core.windowType.SlidingWindow; 5 | import stream.scotty.core.windowType.TumblingWindow; 6 | import stream.scotty.core.windowType.WindowMeasure; 7 | import stream.scotty.stormconnector.KeyedScottyWindowOperator; 8 | import stream.scotty.demo.storm.windowFunctions.Sum; 9 | import org.apache.storm.Config; 10 | import org.apache.storm.LocalCluster; 11 | import org.apache.storm.topology.TopologyBuilder; 12 | import org.apache.storm.tuple.Fields; 13 | 14 | /* 15 | * @author Batuhan Tüter 16 | * Runner class for Scotty on Storm 17 | * 18 | * 19 | * Input data should be in the following format: 20 | * , timeStamp is type Long 21 | * */ 22 | 23 | public class ScottyDemoTopology { 24 | 25 | public static void main(String[] args) throws Exception { 26 | LocalCluster cluster = new LocalCluster(); 27 | TopologyBuilder builder = new TopologyBuilder(); 28 | 29 | Config conf = new Config(); 30 | conf.setDebug(false); 31 | conf.setNumWorkers(1); 32 | conf.setMaxTaskParallelism(1); 33 | //Disable Acking 34 | conf.setNumAckers(0); 35 | 36 | KeyedScottyWindowOperator scottyBolt = new KeyedScottyWindowOperator(new Sum(), 0); 37 | scottyBolt.addWindow(new TumblingWindow(WindowMeasure.Time, 1000)); 38 | scottyBolt.addWindow(new SlidingWindow(WindowMeasure.Time, 1000, 250)); 39 | scottyBolt.addWindow(new SessionWindow(WindowMeasure.Time, 1000)); 40 | 41 | builder.setSpout("spout", new DataGeneratorSpout()); 42 | builder.setBolt("scottyWindow", scottyBolt).fieldsGrouping("spout", new Fields("key")); 43 | builder.setBolt("printer", new PrinterBolt()).shuffleGrouping("scottyWindow"); 44 | 45 | cluster.submitTopology("testTopology", conf, builder.createTopology()); 46 | //cluster.killTopology("testTopology"); 47 | //cluster.shutdown(); 48 | } 49 | } -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/flinkBenchmark/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.flinkBenchmark 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.flinkBenchmark

13 | 28 | 29 | 30 | -------------------------------------------------------------------------------- /demo/beam-demo/src/main/java/stream/scotty/demo/beam/BeamSumDemo.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.beam; 2 | 3 | import stream.scotty.beamconnector.KeyedScottyWindowOperator; 4 | import stream.scotty.demo.beam.dataGenerator.DataGeneratorSource; 5 | import stream.scotty.demo.beam.dataGenerator.TimeStampGenerator; 6 | import stream.scotty.demo.beam.windowFunctions.Sum; 7 | import stream.scotty.core.windowType.TumblingWindow; 8 | import stream.scotty.core.windowType.WindowMeasure; 9 | import org.apache.beam.sdk.Pipeline; 10 | import org.apache.beam.sdk.io.Read; 11 | import org.apache.beam.sdk.options.PipelineOptions; 12 | import org.apache.beam.sdk.options.PipelineOptionsFactory; 13 | import org.apache.beam.sdk.transforms.DoFn; 14 | import org.apache.beam.sdk.transforms.ParDo; 15 | import org.apache.beam.sdk.values.KV; 16 | import org.apache.beam.sdk.values.PCollection; 17 | 18 | public class BeamSumDemo { 19 | public static void main(String[] args) { 20 | 21 | PipelineOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().create(); 22 | Pipeline p = Pipeline.create(options); 23 | System.out.println("Running Pipeline\n " + p.getOptions()); 24 | 25 | PCollection> data = p.begin().apply(Read.from(new DataGeneratorSource(0, new TimeStampGenerator()))); 26 | 27 | KeyedScottyWindowOperator scottyWindowDoFn = new KeyedScottyWindowOperator(0, new Sum()); 28 | scottyWindowDoFn.addWindow(new TumblingWindow(WindowMeasure.Time, 5000)); 29 | //scottyWindowDoFn.addWindow(new SlidingWindow(WindowMeasure.Time, 2000, 1000)); 30 | //scottyWindowDoFn.addWindow(new SessionWindow(WindowMeasure.Time, 2000)); 31 | 32 | //Apply Scotty Windowing 33 | PCollection result = data.apply(ParDo.of(scottyWindowDoFn)); 34 | 35 | //Print window results 36 | result.apply(ParDo.of(new printObject())); 37 | p.run().waitUntilFinish(); 38 | 39 | } 40 | 41 | static class printObject extends DoFn { 42 | @ProcessElement 43 | public void processElement(@Element Object input) { 44 | System.out.println(input.toString()); 45 | } 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /samza-connector/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | scotty 7 | stream.scotty 8 | 0.4 9 | 10 | 4.0.0 11 | 12 | samza-connector 13 | 14 | 15 | 0.4 16 | 1.2.0 17 | 2.3.0 18 | 19 | 20 | 21 | 22 | stream.scotty 23 | core 24 | ${scotty.version} 25 | 26 | 27 | stream.scotty 28 | slicing 29 | ${scotty.version} 30 | 31 | 32 | stream.scotty 33 | state 34 | ${scotty.version} 35 | 36 | 37 | 38 | org.apache.samza 39 | samza-api 40 | ${samza.version} 41 | 42 | 43 | org.apache.samza 44 | samza-core_2.11 45 | ${samza.version} 46 | 47 | 48 | org.apache.samza 49 | samza-kafka_2.11 50 | ${samza.version} 51 | 52 | 53 | 54 | org.apache.kafka 55 | kafka-clients 56 | ${kafka.version} 57 | 58 | 59 | 60 | 61 | 62 | -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/microbenchmark/SlicingWindowOperatorBenchmark.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.microbenchmark; 2 | 3 | import stream.scotty.slicing.SlicingWindowOperator; 4 | import stream.scotty.state.memory.MemoryStateFactory; 5 | import stream.scotty.core.windowType.TumblingWindow; 6 | import stream.scotty.core.windowType.WindowMeasure; 7 | import stream.scotty.core.windowFunction.ReduceAggregateFunction; 8 | import org.openjdk.jmh.annotations.*; 9 | import org.openjdk.jmh.runner.Runner; 10 | import org.openjdk.jmh.runner.RunnerException; 11 | import org.openjdk.jmh.runner.options.Options; 12 | import org.openjdk.jmh.runner.options.OptionsBuilder; 13 | 14 | @State(Scope.Benchmark) 15 | public class SlicingWindowOperatorBenchmark { 16 | 17 | private SlicingWindowOperator windowOperator; 18 | 19 | private long ts; 20 | private long n; 21 | 22 | @Setup(Level.Iteration) 23 | public void setupIteration() throws Exception { 24 | n = 0; 25 | ts = 0; 26 | MemoryStateFactory memoryStateFactory = new MemoryStateFactory(); 27 | this.windowOperator = new SlicingWindowOperator<>(memoryStateFactory); 28 | this.windowOperator.addWindowAssigner(new TumblingWindow(WindowMeasure.Time, 10)); 29 | this.windowOperator.addWindowFunction(new ReduceAggregateFunction() { 30 | @Override 31 | public Integer combine(Integer partialAggregate1, Integer partialAggregate2) { 32 | return partialAggregate1 + partialAggregate2; 33 | } 34 | }); 35 | } 36 | 37 | @Benchmark() 38 | public void benchmarkSameSlice() throws Exception { 39 | windowOperator.processElement(10, 0); 40 | } 41 | 42 | @Benchmark() 43 | public void benchmarkSameSlice1000() throws Exception { 44 | 45 | n++; 46 | if (n == 1000) { 47 | n = 0; 48 | ts += 10; 49 | } 50 | 51 | windowOperator.processElement(10, ts); 52 | } 53 | 54 | 55 | public static void main(String[] args) throws RunnerException { 56 | 57 | Options opt = new OptionsBuilder() 58 | .include(SlicingWindowOperatorBenchmark.class.getName()) 59 | .forks(1) 60 | .build(); 61 | 62 | new Runner(opt).run(); 63 | } 64 | 65 | 66 | } 67 | -------------------------------------------------------------------------------- /docs/de/tub/dima/scotty/core/windowType/package-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | stream.scotty.core.windowType 7 | 8 | 9 | 10 | 11 | 12 |

stream.scotty.core.windowType

13 |
14 |

Interfaces

15 | 22 |

Classes

23 | 28 |

Enums

29 | 32 |
33 | 34 | 35 | -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/slice/LazySlice.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.slice; 2 | 3 | 4 | import stream.scotty.slicing.*; 5 | import stream.scotty.slicing.state.*; 6 | import stream.scotty.state.*; 7 | 8 | public class LazySlice extends AbstractSlice { 9 | 10 | private final AggregateState state; 11 | private final SetState> records; 12 | 13 | public LazySlice(StateFactory stateFactory, WindowManager windowManager, long startTs, long endTs, long startC, long endC, Type type) { 14 | super(startTs, endTs, startC, endC, type); 15 | this.records = stateFactory.createSetState(); 16 | this.state = new AggregateState<>(stateFactory, windowManager.getAggregations(), this.records); 17 | } 18 | 19 | @Override 20 | public void addElement(InputType element, long ts) { 21 | super.addElement(element, ts); 22 | state.addElement(element); 23 | records.add(new StreamRecord(ts, element)); 24 | } 25 | 26 | public void prependElement(StreamRecord newElement) { 27 | super.addElement(newElement.record, newElement.ts); 28 | records.add(newElement); 29 | state.addElement(newElement.record); 30 | } 31 | 32 | public StreamRecord dropLastElement() { 33 | StreamRecord dropRecord = records.dropLast(); 34 | this.setCLast(this.getCLast()-1); 35 | if(!records.isEmpty()) { 36 | StreamRecord currentLast = records.getLast(); 37 | this.setTLast(currentLast.ts); 38 | } 39 | this.state.removeElement(dropRecord); 40 | return dropRecord; 41 | } 42 | 43 | public StreamRecord dropFirstElement() { 44 | StreamRecord dropRecord = records.dropFrist(); 45 | StreamRecord currentFirst = records.getFirst(); 46 | this.setCLast(this.getCLast()-1); 47 | this.setTFirst(currentFirst.ts); 48 | this.state.removeElement(dropRecord); 49 | return dropRecord; 50 | } 51 | 52 | @Override 53 | public AggregateState getAggState() { 54 | return state; 55 | } 56 | 57 | public SetState> getRecords(){ 58 | return this.records; 59 | } 60 | 61 | 62 | } 63 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/SlidingWindow.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType; 2 | 3 | import stream.scotty.core.*; 4 | import stream.scotty.core.*; 5 | 6 | public class SlidingWindow implements ContextFreeWindow { 7 | 8 | private final WindowMeasure measure; 9 | 10 | /** 11 | * Size of the sliding window 12 | */ 13 | private final long size; 14 | 15 | /** 16 | * The window slide step 17 | */ 18 | private final long slide; 19 | 20 | public SlidingWindow(WindowMeasure measure, long size, long slide) { 21 | this.measure = measure; 22 | this.size = size; 23 | this.slide = slide; 24 | } 25 | 26 | public long getSize() { 27 | return size; 28 | } 29 | 30 | public long getSlide() { 31 | return slide; 32 | } 33 | 34 | @Override 35 | public WindowMeasure getWindowMeasure() { 36 | return measure; 37 | } 38 | 39 | 40 | @Override 41 | public long assignNextWindowStart(long recordStamp) { 42 | long nextWindowStart = recordStamp + getSlide() - (recordStamp) % getSlide(); 43 | long nextWindowEnd = recordStamp < getSize() ? getSize() : recordStamp + this.getSlide() - (recordStamp - this.getSize()) % this.getSlide(); 44 | return Math.min(nextWindowStart, nextWindowEnd); 45 | } 46 | 47 | public static long getWindowStartWithOffset(long timestamp, long windowSize) { 48 | return timestamp - (timestamp + windowSize) % windowSize; 49 | } 50 | 51 | @Override 52 | public void triggerWindows(WindowCollector collector, long lastWatermark, long currentWatermark) { 53 | long lastStart = getWindowStartWithOffset(currentWatermark, slide); 54 | 55 | for (long windowStart = lastStart; windowStart + size > lastWatermark; windowStart -= slide) { 56 | if (windowStart>=0 && windowStart + size <= currentWatermark + 1) 57 | collector.trigger(windowStart, windowStart + size, measure); 58 | } 59 | } 60 | 61 | @Override 62 | public long clearDelay() { 63 | return size; 64 | } 65 | 66 | @Override 67 | public String toString() { 68 | return "SlidingWindow{" + 69 | "measure=" + measure + 70 | ", size=" + size + 71 | ", slide=" + slide + 72 | '}'; 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /demo/storm-demo/src/main/java/stream/scotty/demo/storm/DataGeneratorSpout.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.storm; 2 | 3 | import org.apache.storm.spout.SpoutOutputCollector; 4 | import org.apache.storm.task.TopologyContext; 5 | import org.apache.storm.topology.OutputFieldsDeclarer; 6 | import org.apache.storm.topology.base.BaseRichSpout; 7 | import org.apache.storm.tuple.Fields; 8 | import org.apache.storm.tuple.Values; 9 | import org.slf4j.Logger; 10 | import org.slf4j.LoggerFactory; 11 | 12 | import java.util.Map; 13 | import java.util.Random; 14 | 15 | public class DataGeneratorSpout extends BaseRichSpout { 16 | private static final Logger LOG = LoggerFactory.getLogger(DataGeneratorSpout.class); 17 | private SpoutOutputCollector collector; 18 | private long msgId = 0; 19 | private int numberOfKeys; 20 | private long eventTime = -1; 21 | private Random generator; 22 | private int value=0; 23 | private long throughputLimit; 24 | 25 | public DataGeneratorSpout() { 26 | this.numberOfKeys = 1; 27 | this.generator = new Random(); 28 | this.throughputLimit = 1000; 29 | } 30 | 31 | @Override 32 | public void open(Map map, TopologyContext topologyContext, SpoutOutputCollector spoutOutputCollector) { 33 | this.collector = spoutOutputCollector; 34 | } 35 | 36 | @Override 37 | public void nextTuple() { 38 | if (throughputLimit != 0) { 39 | long startTs = System.currentTimeMillis(); 40 | for (int i = 0; i < throughputLimit; i++) { 41 | collector.emit(new Values(generator.nextInt(numberOfKeys), ++value, ++eventTime), ++msgId); 42 | } 43 | while (System.currentTimeMillis() < startTs + 1000) { 44 | // active waiting 45 | } 46 | } 47 | else { 48 | while (true) { 49 | collector.emit(new Values(generator.nextInt(numberOfKeys), ++value, ++eventTime), ++msgId); 50 | } 51 | } 52 | } 53 | 54 | @Override 55 | public void declareOutputFields(OutputFieldsDeclarer declarer) { 56 | declarer.declare(new Fields("key", "value", "ts")); 57 | } 58 | 59 | @Override 60 | public void ack(Object msgId) { 61 | LOG.debug("Got ACK for msgId : " + msgId); 62 | } 63 | 64 | @Override 65 | public void fail(Object msgId) { 66 | LOG.debug("Got FAIL for msgId : " + msgId); 67 | } 68 | } -------------------------------------------------------------------------------- /demo/beam-demo/src/main/java/stream/scotty/demo/beam/dataGenerator/DataGeneratorSource.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.beam.dataGenerator; 2 | 3 | import org.apache.beam.sdk.coders.AvroCoder; 4 | import org.apache.beam.sdk.coders.Coder; 5 | import org.apache.beam.sdk.coders.KvCoder; 6 | import org.apache.beam.sdk.coders.VarIntCoder; 7 | import org.apache.beam.sdk.io.UnboundedSource; 8 | import org.apache.beam.sdk.options.PipelineOptions; 9 | import org.apache.beam.sdk.transforms.SerializableFunction; 10 | import org.apache.beam.sdk.values.KV; 11 | import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList; 12 | import org.joda.time.Instant; 13 | 14 | import javax.annotation.Nullable; 15 | import java.io.IOException; 16 | import java.util.List; 17 | import java.util.Objects; 18 | 19 | public class DataGeneratorSource extends UnboundedSource, Checkpoint> { 20 | 21 | private int throughput; 22 | public final SerializableFunction timestampFn; 23 | 24 | public DataGeneratorSource(int throughput, SerializableFunction timestampFn) { 25 | this.throughput = throughput; 26 | this.timestampFn = timestampFn; 27 | } 28 | 29 | @Override 30 | public List, Checkpoint>> split(int desiredNumSplits, PipelineOptions options){ 31 | ImmutableList.Builder splits = ImmutableList.builder(); 32 | splits.add(new DataGeneratorSource(throughput, timestampFn)); 33 | return splits.build(); 34 | } 35 | 36 | @Override 37 | public UnboundedReader createReader(PipelineOptions options, @Nullable Checkpoint mark) throws IOException { 38 | return new DataGeneratorReader(throughput, this, mark); 39 | } 40 | 41 | @Override 42 | public Coder getCheckpointMarkCoder() { 43 | return AvroCoder.of(Checkpoint.class); 44 | 45 | } 46 | 47 | @Override 48 | public Coder> getOutputCoder() { 49 | return KvCoder.of(VarIntCoder.of(),VarIntCoder.of()); 50 | } 51 | 52 | @Override 53 | public boolean equals(Object other) { 54 | if (!(other instanceof DataGeneratorSource)) { 55 | return false; 56 | } 57 | DataGeneratorSource that = (DataGeneratorSource) other; 58 | return this.timestampFn.equals(that.timestampFn); 59 | } 60 | 61 | @Override 62 | public int hashCode() { 63 | return Objects.hash(timestampFn); 64 | } 65 | } 66 | 67 | 68 | 69 | -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/state/AggregateWindowState.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.state; 2 | 3 | import stream.scotty.core.*; 4 | import stream.scotty.core.windowFunction.*; 5 | import stream.scotty.core.windowType.*; 6 | import stream.scotty.slicing.slice.*; 7 | import stream.scotty.state.*; 8 | 9 | import java.util.*; 10 | 11 | public class AggregateWindowState implements AggregateWindow { 12 | 13 | private final long start; 14 | private final long endTs; 15 | private final WindowMeasure measure; 16 | private final AggregateState windowState; 17 | 18 | public AggregateWindowState(long startTs, long endTs, WindowMeasure measure, StateFactory stateFactory, List windowFunctionList) { 19 | this.start = startTs; 20 | this.endTs = endTs; 21 | this.windowState = new AggregateState(stateFactory, windowFunctionList); 22 | this.measure = measure; 23 | } 24 | 25 | public boolean containsSlice(Slice currentSlice) { 26 | if (measure == WindowMeasure.Time) { 27 | return this.getStart() <= currentSlice.getTStart() && (this.getEnd() > currentSlice.getTLast()); 28 | } else { 29 | return this.getStart() <= currentSlice.getCStart() && (this.getEnd() >= currentSlice.getCLast()); 30 | } 31 | } 32 | 33 | public long getStart() { 34 | return start; 35 | } 36 | 37 | public long getEnd() { 38 | return endTs; 39 | } 40 | 41 | @Override 42 | public List getAggValues() { 43 | return windowState.getValues(); 44 | } 45 | 46 | @Override 47 | public boolean hasValue() { 48 | return windowState.hasValues(); 49 | } 50 | 51 | public void addState(AggregateState aggregationState) { 52 | this.windowState.merge(aggregationState); 53 | } 54 | 55 | public WindowMeasure getMeasure() { 56 | return measure; 57 | } 58 | 59 | 60 | 61 | @Override 62 | public boolean equals(Object o) { 63 | if (this == o) return true; 64 | if (o == null || getClass() != o.getClass()) return false; 65 | AggregateWindowState that = (AggregateWindowState) o; 66 | return start == that.start && 67 | endTs == that.endTs && 68 | Objects.equals(windowState, that.windowState); 69 | } 70 | 71 | @Override 72 | public int hashCode() { 73 | return Objects.hash(start, endTs, windowState); 74 | } 75 | 76 | @Override 77 | public String toString() { 78 | return "WindowResult(" + 79 | measure.toString() + ","+ 80 | start + "-" + endTs + 81 | "," + windowState + 82 | ')'; 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowType/FixedBandWindow.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowType; 2 | 3 | import stream.scotty.core.WindowCollector; 4 | 5 | public class FixedBandWindow implements ContextFreeWindow { 6 | /** 7 | * This window starts at the defined start timestamp and ends at start + size. 8 | * Scotty will continue to add tuples to one big slice, after the result of the window aggregation has been output. 9 | * Therefore, the execution of this window type has to be stopped manually. 10 | */ 11 | private final WindowMeasure measure; 12 | private final long start; 13 | private final long size; 14 | 15 | /** 16 | * @param measure WindowMeasurement, has to be time 17 | * @param start start timestamp of the fixed-band window 18 | * @param size size of the fixed-band window 19 | */ 20 | public FixedBandWindow(WindowMeasure measure, long start, long size) { 21 | this.measure = measure; 22 | this.size = size; 23 | this.start = start; 24 | } 25 | 26 | public long getSize() { 27 | return size; 28 | } 29 | 30 | public long getStart() { 31 | return start; 32 | } 33 | 34 | public WindowMeasure getWindowMeasure() { return measure; } 35 | 36 | @Override 37 | public long assignNextWindowStart(long position) { 38 | if(position == Long.MAX_VALUE || position < getStart()){ 39 | //at the first invocation is position == Long.MAX_VALUE, returns the start timestamp of the window 40 | return getStart(); 41 | }else if(position >= getStart() && position < getStart()+size){ 42 | //returns end timestamp of the window 43 | return getStart()+size; 44 | }else{ 45 | //all tuples arriving after the window end are collected in one big slice -> manual termination required 46 | return Long.MAX_VALUE; 47 | } 48 | } 49 | 50 | @Override 51 | public void triggerWindows(WindowCollector aggregateWindows, long lastWatermark, long currentWatermark) { 52 | //triggers the window, if it started after lastWatermark and ended before currentWatermark 53 | long windowStart = getStart(); 54 | if(lastWatermark <= windowStart+size && windowStart+size <= currentWatermark) { 55 | aggregateWindows.trigger(windowStart, windowStart + size, measure); 56 | } 57 | } 58 | 59 | @Override 60 | public long clearDelay() { 61 | return size; 62 | } 63 | 64 | @Override 65 | public String toString() { 66 | return "Fixed Band Window{" + 67 | "measure=" + measure + 68 | ", start="+ start + 69 | ", size=" + size + 70 | '}'; 71 | } 72 | 73 | } 74 | -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/slice/Slice.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.slice; 2 | 3 | import stream.scotty.slicing.state.*; 4 | 5 | public interface Slice { 6 | 7 | /** 8 | * @return slice start timestamp 9 | */ 10 | long getTStart(); 11 | 12 | void setTStart(long tStart); 13 | 14 | /** 15 | * @return slice end timestamp 16 | */ 17 | long getTEnd(); 18 | 19 | void setTEnd(long tEnd); 20 | 21 | /** 22 | * timestamp of the first record in the slice 23 | * 24 | * @return long 25 | */ 26 | long getTFirst(); 27 | 28 | void merge(Slice otherSlice); 29 | 30 | /** 31 | * timestamp of the last record in the slice 32 | * 33 | * @return long 34 | */ 35 | long getTLast(); 36 | 37 | 38 | /** 39 | * The measure of the slice end. 40 | * 41 | * @return Type 42 | */ 43 | Type getType(); 44 | 45 | /** 46 | * Set the end of the slice 47 | * 48 | * @param type of slice end 49 | */ 50 | void setType(Type type); 51 | 52 | /** 53 | * @return 54 | */ 55 | AggregateState getAggState(); 56 | 57 | /** 58 | * Add a element to the slice. 59 | * 60 | * @param element the element which is added 61 | * @param ts timestamp of the element 62 | */ 63 | void addElement(InputType element, long ts); 64 | 65 | default void removeElement(InputType element){ 66 | 67 | }; 68 | 69 | /** 70 | * Element count of first element 71 | * @return 72 | */ 73 | long getCStart(); 74 | 75 | /** 76 | * Element count of last element 77 | * @return 78 | */ 79 | long getCLast(); 80 | 81 | 82 | interface Type { 83 | boolean isMovable(); 84 | } 85 | 86 | public final class Fixed implements Type { 87 | 88 | @Override 89 | public boolean isMovable() { 90 | return false; 91 | } 92 | } 93 | 94 | public final class Flexible implements Type { 95 | private int counter; 96 | 97 | public Flexible() { 98 | this(1); 99 | } 100 | 101 | public Flexible(int counter) { 102 | this.counter = counter; 103 | } 104 | 105 | public long getCount() { 106 | return counter; 107 | } 108 | 109 | public void decrementCount() { 110 | counter--; 111 | } 112 | 113 | public void incrementCount() { 114 | counter++; 115 | } 116 | 117 | @Override 118 | public boolean isMovable() { 119 | return getCount() == 1; 120 | } 121 | } 122 | } -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/aggregationstore/AggregationStore.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.aggregationstore; 2 | 3 | import stream.scotty.slicing.*; 4 | import stream.scotty.slicing.slice.Slice; 5 | import stream.scotty.slicing.state.*; 6 | 7 | public interface AggregationStore { 8 | 9 | /** 10 | * @return get the newest slice. 11 | */ 12 | Slice getCurrentSlice(); 13 | 14 | /** 15 | * Lookup for a slice which contains this timestamp. 16 | * The timestamp is >= slice start < slice end. 17 | * 18 | * @param ts element timestamp 19 | * @return index of containing slice. 20 | */ 21 | int findSliceIndexByTimestamp(long ts); 22 | 23 | int findSliceIndexByCount(long count); 24 | 25 | /** 26 | * Returns slice for a given index or @{@link IndexOutOfBoundsException} 27 | * 28 | * @param index >= 0 < size 29 | * @return Slice 30 | */ 31 | Slice getSlice(int index); 32 | 33 | /** 34 | * Insert the element to the current slice. 35 | * 36 | * @param element the element 37 | * @param ts timestamp of record 38 | */ 39 | void insertValueToCurrentSlice(InputType element, long ts); 40 | 41 | void insertValueToSlice(int index, InputType element, long ts); 42 | 43 | /** 44 | * Appends a new slice as new current slice 45 | * 46 | * @param newSlice 47 | */ 48 | void appendSlice(Slice newSlice); 49 | 50 | int size(); 51 | 52 | /** 53 | * @return true if AggregationStore contains no slices 54 | */ 55 | boolean isEmpty(); 56 | 57 | /** 58 | * Generates the window aggregates. 59 | * On every @{@link AggregateWindowState} the aggregated value is set. 60 | * @param aggregateWindows definition of the requested window 61 | * @param minTs startTimestamp of the earliest window. 62 | * @param maxTs endTimestamp of the latest window 63 | * @param minCount 64 | * @param maxCount 65 | */ 66 | void aggregate(WindowManager.AggregationWindowCollector aggregateWindows, long minTs, long maxTs, long minCount, long maxCount); 67 | 68 | /** 69 | * Add a new Slice at a specific index 70 | * @param index 71 | * @param newSlice 72 | */ 73 | void addSlice(int index, Slice newSlice); 74 | 75 | /** 76 | * Merging two slices A and B happens in three steps: 77 | * 1. Update the end of A such that t end (A) ← t end (B). 78 | * 2. Update the aggregate of A such that a ← a ⊕ b. 79 | * 3. Delete slice B, which is now merged into A. 80 | * @param sliceIndex index of slice A 81 | */ 82 | void mergeSlice(int sliceIndex); 83 | 84 | int findSliceByEnd(long pre); 85 | 86 | void removeSlices(long maxTimestamp); 87 | } 88 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/windowFunction/AggregateFunction.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.core.windowFunction; 2 | 3 | 4 | import java.io.*; 5 | 6 | public interface AggregateFunction extends Serializable { 7 | 8 | /** 9 | * Transforms a tuple to a partial aggregate. 10 | * For example, consider an average computation. If a tuple ⟨p,v⟩ 11 | * contains its position p and a value v, the lift function will transform 12 | * it to ⟨sum←v, count←1⟩, which is the partial aggregate of 13 | * that one tuple. 14 | * @param inputTuple input Tuple 15 | * @return PartialAggregate 16 | */ 17 | PartialAggregateType lift(InputType inputTuple); 18 | 19 | /** 20 | * Computes the combined aggregate from partial aggregates. 21 | * Each incremental aggregation step results in one call of the combine function. 22 | * It is expected that this method returns new partial aggregate. 23 | * This method is used in two ways. 24 | * 1. To add a single element to the partial aggregate. 25 | * 2. To merge two big partial aggregates (or slices) , for example for emitting a final window. 26 | * For the second step it is needed to emit a completely new partial aggregate. 27 | * This can involve a deep copy of the partial aggregates. 28 | * To prevent this a AggregationFunction can implement {@link CloneablePartialStateFunction}. 29 | * In this case copy provides a cloned partial aggregate as the first argument. 30 | * @param partialAggregate1 the first original partial aggregate or a cloned object iff the AggregationFunction implements {@link CloneablePartialStateFunction} 31 | * @param partialAggregate2 the second ordinal partial aggregate. 32 | * @return combined PartialAggregate 33 | */ 34 | PartialAggregateType combine(PartialAggregateType partialAggregate1, PartialAggregateType partialAggregate2); 35 | 36 | /** 37 | * Default implementation for combining the lift and combine function. 38 | * This is used by Scotty to add a single element to a slice. 39 | * For some AggregationFunctions it can be beneficial to implement a more efficient lift and combination function. 40 | * @param partialAggregate 41 | * @param inputTuple 42 | * @return 43 | */ 44 | default PartialAggregateType liftAndCombine(PartialAggregateType partialAggregate, InputType inputTuple){ 45 | PartialAggregateType lifted = lift(inputTuple); 46 | return combine(partialAggregate, lifted); 47 | }; 48 | 49 | /** 50 | * Transforms a partial aggregate to a final aggregate. 51 | * In our example, the lower function computes the average from sum and count: 52 | * ⟨sum, count⟩ 7→ sum/count. 53 | * @param aggregate 54 | * @return final Aggregate 55 | */ 56 | FinalAggregateType lower(PartialAggregateType aggregate); 57 | 58 | } 59 | -------------------------------------------------------------------------------- /demo/beam-demo/src/main/java/stream/scotty/demo/beam/dataGenerator/TimeStampGenerator.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.beam.dataGenerator; 2 | 3 | import org.apache.beam.sdk.transforms.SerializableFunction; 4 | import org.joda.time.Duration; 5 | import org.joda.time.Instant; 6 | 7 | import java.util.concurrent.ThreadLocalRandom; 8 | 9 | public class TimeStampGenerator implements SerializableFunction { 10 | private Instant eventTime; 11 | private double outOfOrderProbability; 12 | private long sessionPeriod; 13 | private long minLateness; 14 | private long maxLateness; 15 | private long minGap; 16 | private long maxGap; 17 | private long lastSecond; 18 | private Instant lastGap; 19 | 20 | public TimeStampGenerator() { 21 | this.eventTime = new Instant(0); 22 | this.lastGap = new Instant(0); 23 | this.outOfOrderProbability = 0; 24 | this.minLateness = 0; 25 | this.maxLateness = 0; 26 | this.sessionPeriod = 0; 27 | this.minGap = 0; 28 | this.maxGap = 0; 29 | } 30 | 31 | @Override 32 | public Instant apply(Integer input) { 33 | //Increase the event-time at every 1 ms in processing-time 34 | long currentTime = System.currentTimeMillis(); 35 | if(currentTime> lastSecond){ 36 | eventTime = eventTime.plus(Duration.millis(1)); 37 | lastSecond = currentTime; 38 | } 39 | 40 | // With given probability data is out-of-order 41 | // We make sure the tuple does not have negative timestamp 42 | if (ThreadLocalRandom.current().nextDouble() <= (outOfOrderProbability / 100)) { 43 | //We make sure the tuple does not have negative or 0 timestamp 44 | long bigger = Math.max(eventTime.minus(Duration.millis(ThreadLocalRandom.current().nextLong(minLateness, maxLateness+1))).getMillis(), 1); 45 | return new Instant(bigger); 46 | } 47 | 48 | //A session gap is created every sessionPeriod 49 | //Event time is shifted to future to simulate session gap 50 | if (sessionPeriod!=0 && eventTime.isAfter(lastGap.plus(Duration.millis(sessionPeriod)))) { 51 | long sessionGap = ThreadLocalRandom.current().nextLong(minGap, maxGap + 1); 52 | try { 53 | System.out.printf("\nWaiting for session gap"); 54 | Thread.sleep(sessionGap); 55 | } catch (InterruptedException e) { 56 | e.printStackTrace(); 57 | } 58 | eventTime = eventTime.plus(Duration.millis(sessionGap)); 59 | lastGap = eventTime; 60 | } 61 | 62 | return eventTime; 63 | } 64 | 65 | @Override 66 | public boolean equals(Object other) { 67 | if (!(other instanceof TimeStampGenerator)) { 68 | return false; 69 | } 70 | TimeStampGenerator that = (TimeStampGenerator) other; 71 | return that.lastGap.equals(that.lastGap); 72 | } 73 | } -------------------------------------------------------------------------------- /docs/overview-frame.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | Overview List 7 | 8 | 9 | 10 | 11 | 12 | 13 | 35 |

 

36 | 37 | 38 | -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/flinkBenchmark/LoadGeneratorSource.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.flinkBenchmark; 2 | 3 | import org.apache.flink.api.java.tuple.*; 4 | import org.apache.flink.streaming.api.functions.source.*; 5 | import org.slf4j.*; 6 | 7 | import java.util.*; 8 | 9 | 10 | public class LoadGeneratorSource implements SourceFunction> { 11 | 12 | private static int maxBackpressure = 5000; 13 | private final long runtime; 14 | 15 | private static final Logger LOG = LoggerFactory.getLogger(LoadGeneratorSource.class); 16 | 17 | private final int throughput; 18 | private boolean running = true; 19 | 20 | private final List> gaps; 21 | private int currentGapIndex; 22 | 23 | private long nextGapStart = 0; 24 | private long nextGapEnd; 25 | 26 | private long timeOffset; 27 | private Random random; 28 | 29 | public LoadGeneratorSource(long runtime, int throughput, final List> gaps) { 30 | 31 | this.throughput = throughput; 32 | this.gaps = gaps; 33 | this.random = new Random(); 34 | this.runtime = runtime; 35 | } 36 | 37 | private int backpressureCounter = 0; 38 | 39 | @Override 40 | public void run(final SourceContext> ctx) throws Exception { 41 | 42 | ThroughputStatistics.getInstance().pause(false); 43 | 44 | long endTime = System.currentTimeMillis() + runtime; 45 | while (running) { 46 | long startTs = System.currentTimeMillis(); 47 | 48 | for (int i = 0; i < throughput; i++) { 49 | emitValue(readNextTuple(), ctx); 50 | } 51 | while (System.currentTimeMillis() < startTs + 1000) { 52 | // active waiting 53 | } 54 | 55 | if(endTime <= System.currentTimeMillis()) 56 | running = false; 57 | } 58 | } 59 | 60 | private void emitValue(final Tuple4 tuple3, final SourceContext> ctx) { 61 | 62 | if (tuple3.f3 > nextGapStart) { 63 | ThroughputStatistics.getInstance().pause(true); 64 | //System.out.println("in Gap"); 65 | if (tuple3.f3 > this.nextGapEnd) { 66 | ThroughputStatistics.getInstance().pause(false); 67 | this.currentGapIndex++; 68 | if (currentGapIndex < gaps.size()) { 69 | this.nextGapStart = this.gaps.get(currentGapIndex).f0 + this.timeOffset; 70 | this.nextGapEnd = this.nextGapStart + this.gaps.get(currentGapIndex).f1; 71 | } 72 | } else 73 | return; 74 | } 75 | ctx.collect(tuple3); 76 | } 77 | 78 | private Tuple4 readNextTuple() throws Exception { 79 | return new Tuple4<>("key", random.nextInt(), random.nextLong(), System.currentTimeMillis()); 80 | 81 | } 82 | 83 | @Override 84 | public void cancel() { 85 | running = false; 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /demo/kafka-demo/src/main/java/stream/scotty/demo/kafkaStreams/KafkaStreamsSumDemo.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.kafkaStreams; 2 | 3 | import stream.scotty.core.windowType.SlidingWindow; 4 | import stream.scotty.core.windowType.TumblingWindow; 5 | import stream.scotty.core.windowType.WindowMeasure; 6 | import stream.scotty.kafkastreamsconnector.KeyedScottyWindowOperatorSupplier; 7 | import stream.scotty.demo.kafkaStreams.windowFunctions.SumWindowFunction; 8 | import org.apache.kafka.common.serialization.Serdes; 9 | import org.apache.kafka.streams.KafkaStreams; 10 | import org.apache.kafka.streams.StreamsConfig; 11 | import org.apache.kafka.streams.Topology; 12 | 13 | import java.util.Properties; 14 | 15 | public class KafkaStreamsSumDemo { 16 | 17 | public static void main(String[] args) { 18 | Properties props = new Properties(); 19 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "SumDemo"); 20 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 21 | props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); 22 | props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); 23 | String INPUT_DESCRIPTOR_NAME = "testInput"; 24 | 25 | /** 26 | * Topology approach 27 | */ 28 | 29 | KeyedScottyWindowOperatorSupplier processorSupplier = new KeyedScottyWindowOperatorSupplier<>(new SumWindowFunction(), 100); 30 | processorSupplier 31 | .addWindow(new TumblingWindow(WindowMeasure.Time, 2000)) 32 | .addWindow(new SlidingWindow(WindowMeasure.Time, 5000,1000)); 33 | 34 | 35 | Topology demoTopology = new Topology(); 36 | demoTopology.addSource("TestSource","testInput") 37 | .addProcessor("ScottyProcess", processorSupplier, "TestSource") 38 | .addProcessor("ResultPrinter", DemoPrinter::new, "ScottyProcess") 39 | .addSink("TestSink","testOutput","ResultPrinter"); 40 | 41 | System.out.println(demoTopology.describe()); 42 | Thread demoSource = new DemoKafkaProducer(INPUT_DESCRIPTOR_NAME); 43 | KafkaStreams scottyProcessing = new KafkaStreams(demoTopology,props); 44 | demoSource.start(); 45 | scottyProcessing.start(); 46 | 47 | /** 48 | * StreamBuilder approach 49 | */ 50 | /* 51 | StreamsBuilder builder = new StreamsBuilder(); 52 | KStream input = builder.stream("testInput"); 53 | KeyedScottyWindowOperatorSupplier processorSupplier = new KeyedScottyWindowOperatorSupplier<>(new SumWindowFunction(), 100); 54 | processorSupplier 55 | .addWindow(new TumblingWindow(WindowMeasure.Time, 2000)) 56 | .addWindow(new SlidingWindow(WindowMeasure.Time, 5000,1000)); 57 | input.process(processorSupplier); 58 | Thread demoSource = new DemoKafkaProducer(INPUT_DESCRIPTOR_NAME); 59 | KafkaStreams streams = new KafkaStreams(builder.build(),props); 60 | demoSource.start(); 61 | streams.start(); 62 | */ 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/flinkBenchmark/FlinkBenchmarkJob.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.flinkBenchmark; 2 | 3 | import stream.scotty.core.windowType.*; 4 | import stream.scotty.core.windowType.Window; 5 | import org.apache.flink.api.java.tuple.*; 6 | import org.apache.flink.api.java.utils.*; 7 | import org.apache.flink.streaming.api.*; 8 | import org.apache.flink.streaming.api.datastream.*; 9 | import org.apache.flink.streaming.api.environment.*; 10 | import org.apache.flink.streaming.api.functions.sink.*; 11 | import org.apache.flink.streaming.api.windowing.time.*; 12 | import org.apache.flink.streaming.api.windowing.windows.*; 13 | 14 | import java.util.*; 15 | 16 | public class FlinkBenchmarkJob { 17 | public FlinkBenchmarkJob(List assigners, StreamExecutionEnvironment env, long runtime, int throughput, List> gaps) { 18 | Map configMap = new HashMap<>(); 19 | ParameterTool parameters = ParameterTool.fromMap(configMap); 20 | 21 | env.getConfig().setGlobalJobParameters(parameters); 22 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 23 | env.setParallelism(1); 24 | env.setMaxParallelism(1); 25 | 26 | DataStream> messageStream = env 27 | .addSource(new LoadGeneratorSource(runtime, throughput, gaps)); 28 | 29 | messageStream.flatMap(new ThroughputLogger<>(200, throughput)); 30 | 31 | final SingleOutputStreamOperator> timestampsAndWatermarks = messageStream 32 | .assignTimestampsAndWatermarks(new BenchmarkJob.TimestampsAndWatermarks()); 33 | 34 | KeyedStream, Tuple> keyedStream = timestampsAndWatermarks 35 | .keyBy(0); 36 | 37 | for (Window w : assigners) { 38 | 39 | if (w instanceof TumblingWindow) { 40 | WindowedStream, Tuple, TimeWindow> tw = keyedStream.timeWindow(Time.seconds(((TumblingWindow) w).getSize())); 41 | 42 | tw.sum(1).addSink(new SinkFunction() { 43 | 44 | @Override 45 | public void invoke(final Object value) throws Exception { 46 | //System.out.println(value); 47 | } 48 | }); 49 | } 50 | if (w instanceof SlidingWindow) { 51 | WindowedStream, Tuple, TimeWindow> tw = keyedStream.timeWindow(Time.milliseconds(((SlidingWindow) w).getSize()), Time.milliseconds(((SlidingWindow) w).getSlide())); 52 | 53 | tw.sum(1).addSink(new SinkFunction() { 54 | 55 | @Override 56 | public void invoke(final Object value) throws Exception { 57 | //System.out.println(value); 58 | } 59 | }); 60 | } 61 | 62 | 63 | } 64 | 65 | 66 | try { 67 | env.execute(); 68 | 69 | } catch (Exception e) { 70 | e.printStackTrace(); 71 | } 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /demo/samza-demo/src/main/java/stream/scotty/demo/samza/SamzaSumDemo.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.demo.samza; 2 | 3 | import com.google.common.collect.ImmutableList; 4 | import com.google.common.collect.ImmutableMap; 5 | import joptsimple.OptionSet; 6 | import org.apache.samza.application.TaskApplication; 7 | import org.apache.samza.application.descriptors.TaskApplicationDescriptor; 8 | import org.apache.samza.config.Config; 9 | import org.apache.samza.runtime.LocalApplicationRunner; 10 | import org.apache.samza.serializers.IntegerSerde; 11 | import org.apache.samza.serializers.KVSerde; 12 | import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor; 13 | import org.apache.samza.system.kafka.descriptors.KafkaOutputDescriptor; 14 | import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor; 15 | import org.apache.samza.util.CommandLine; 16 | 17 | import java.util.List; 18 | import java.util.Map; 19 | 20 | public class SamzaSumDemo implements TaskApplication { 21 | private static final List KAFKA_CONSUMER_ZK_CONNECT = ImmutableList.of("localhost:2181"); 22 | private static final List KAFKA_PRODUCER_BOOTSTRAP_SERVERS = ImmutableList.of("localhost:9092"); 23 | private static final Map KAFKA_DEFAULT_STREAM_CONFIGS = ImmutableMap.of("replication.factor", "1"); 24 | private static final String SYSTEM_DESCRIPTOR_NAME = "testSystem"; 25 | private static final String INPUT_DESCRIPTOR_NAME = "testInput"; 26 | private static final String OUTPUT_DESCRIPTOR_NAME = "testOutput"; 27 | 28 | public static void main(String[] args) { 29 | String[] configArgs = {"--config-factory=org.apache.samza.config.factories.PropertiesConfigFactory" 30 | , "--config-path=samza-connector/src/main/Properties/config.properties"}; 31 | CommandLine cmdLine = new CommandLine(); 32 | OptionSet options = cmdLine.parser().parse(configArgs); 33 | Config config = cmdLine.loadConfig(options); 34 | LocalApplicationRunner runner = new LocalApplicationRunner(new SamzaSumDemo(), config); 35 | runner.run(); 36 | runner.waitForFinish(); 37 | } 38 | 39 | @Override 40 | public void describe(TaskApplicationDescriptor appDescriptor) { 41 | Thread demoSource = new DemoKafkaProducer(INPUT_DESCRIPTOR_NAME); 42 | demoSource.start(); 43 | KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(SYSTEM_DESCRIPTOR_NAME) 44 | .withConsumerZkConnect(KAFKA_CONSUMER_ZK_CONNECT) 45 | .withProducerBootstrapServers(KAFKA_PRODUCER_BOOTSTRAP_SERVERS) 46 | .withDefaultStreamConfigs(KAFKA_DEFAULT_STREAM_CONFIGS); 47 | KafkaInputDescriptor kid = ksd.getInputDescriptor(INPUT_DESCRIPTOR_NAME, KVSerde.of(new IntegerSerde(), new IntegerSerde())); 48 | KafkaOutputDescriptor kod = ksd.getOutputDescriptor(OUTPUT_DESCRIPTOR_NAME, KVSerde.of(new IntegerSerde(), new IntegerSerde())); 49 | 50 | appDescriptor 51 | .withInputStream(kid) 52 | .withOutputStream(kod); 53 | 54 | appDescriptor.withTaskFactory(new DemoTaskFactory(SYSTEM_DESCRIPTOR_NAME, OUTPUT_DESCRIPTOR_NAME)); 55 | 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /docs/index.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | Generated Documentation (Untitled) 7 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | <noscript> 69 | <div>JavaScript is disabled on your browser.</div> 70 | </noscript> 71 | <h2>Frame Alert</h2> 72 | <p>This document is designed to be viewed using the frames feature. If you see this message, you are using a non-frame-capable web client. Link to <a href="overview-summary.html">Non-frame version</a>.</p> 73 | 74 | 75 | 76 | -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/slice/AbstractSlice.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.slice; 2 | 3 | public abstract class AbstractSlice implements Slice { 4 | 5 | private long tStart; 6 | private long tEnd; 7 | 8 | private Type type; 9 | 10 | private long tLast; 11 | private long tFirst = Long.MAX_VALUE; 12 | 13 | private long cStart; 14 | private long cLast; 15 | 16 | public AbstractSlice(long startTs, long endTs, long cStart, long cLast, Type type) { 17 | this.type = type; 18 | this.tStart = startTs; 19 | this.tEnd = endTs; 20 | this.tLast = startTs; 21 | this.cLast = cLast; 22 | this.cStart = cStart; 23 | } 24 | 25 | 26 | @Override 27 | public void addElement(InputType element, long ts) { 28 | this.tLast = Math.max(this.tLast, ts); 29 | this.tFirst = Math.min(this.tFirst, ts); 30 | this.cLast++; 31 | } 32 | 33 | @Override 34 | public void merge(Slice otherSlice) { 35 | this.tLast = Math.max(this.tLast, otherSlice.getTLast()); 36 | this.tFirst = Math.min(this.tFirst, otherSlice.getTFirst()); 37 | this.setTEnd(Math.max(this.tEnd, otherSlice.getTEnd())); 38 | this.getAggState().merge(otherSlice.getAggState()); 39 | } 40 | 41 | 42 | 43 | @Override 44 | public long getTLast() { 45 | return tLast; 46 | } 47 | 48 | @Override 49 | public long getTFirst() { 50 | return this.tFirst; 51 | } 52 | 53 | @Override 54 | public long getTStart() { 55 | return tStart; 56 | } 57 | 58 | @Override 59 | public void setTStart(long tStart) { 60 | this.tStart = tStart; 61 | } 62 | 63 | @Override 64 | public long getTEnd() { 65 | return tEnd; 66 | } 67 | 68 | @Override 69 | public void setTEnd(long tEnd) { 70 | this.tEnd = tEnd; 71 | } 72 | 73 | @Override 74 | public Type getType() { 75 | return type; 76 | } 77 | 78 | @Override 79 | public void setType(Type type) { 80 | this.type = type; 81 | } 82 | 83 | 84 | public void setTLast(long tLast) { 85 | this.tLast = tLast; 86 | } 87 | 88 | public void setTFirst(long tFirst) { 89 | this.tFirst = tFirst; 90 | } 91 | 92 | public void setCStart(long cStart) { 93 | this.cStart = cStart; 94 | } 95 | 96 | public void setCLast(long cLast) { 97 | this.cLast = cLast; 98 | } 99 | 100 | @Override 101 | public String toString() { 102 | return "Slice{" + 103 | "tStart=" + tStart + 104 | ", tEnd=" + tEnd + 105 | ", tLast=" + tLast + 106 | ", tFirst=" + tFirst + 107 | ", cFirst=" + cStart + 108 | ", cLast=" + cLast + 109 | ", measure=" + type + 110 | '}'; 111 | } 112 | 113 | public long getCStart() { 114 | return cStart; 115 | } 116 | 117 | public long getCLast() { 118 | return cLast; 119 | } 120 | 121 | 122 | } 123 | -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/state/AggregateState.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.state; 2 | 3 | import stream.scotty.core.windowFunction.*; 4 | import stream.scotty.slicing.slice.*; 5 | import stream.scotty.state.*; 6 | 7 | import java.io.*; 8 | import java.util.*; 9 | 10 | public class AggregateState implements Serializable { 11 | 12 | private final List> aggregateValueStates; 13 | 14 | public AggregateState(StateFactory stateFactory, List windowFunctions) { 15 | this(stateFactory, windowFunctions, null); 16 | } 17 | 18 | public AggregateState(StateFactory stateFactory, List windowFunctions, SetState> records) { 19 | this.aggregateValueStates = new ArrayList<>(); 20 | for (int i = 0; i < windowFunctions.size(); i++) { 21 | this.aggregateValueStates.add(new AggregateValueState<>(stateFactory.createValueState(), windowFunctions.get(i), records)); 22 | } 23 | } 24 | 25 | public void addElement(InputType state) { 26 | for(AggregateValueState valueState: aggregateValueStates){ 27 | valueState.addElement(state); 28 | } 29 | } 30 | 31 | public void removeElement(StreamRecord toRemove){ 32 | for(AggregateValueState valueState: aggregateValueStates){ 33 | valueState.removeElement(toRemove); 34 | } 35 | } 36 | 37 | public void clear() { 38 | for(AggregateValueState valueState: aggregateValueStates){ 39 | valueState.clear(); 40 | } 41 | } 42 | 43 | 44 | public void merge(AggregateState otherAggState) { 45 | if (this.isMergeable(otherAggState)) { 46 | for (int i = 0; i < otherAggState.aggregateValueStates.size(); i++) { 47 | this.aggregateValueStates.get(i).merge(otherAggState.aggregateValueStates.get(i)); 48 | } 49 | } 50 | } 51 | 52 | private boolean isMergeable(AggregateState otherAggState) { 53 | return otherAggState.aggregateValueStates.size() <= this.aggregateValueStates.size(); 54 | } 55 | 56 | public boolean hasValues(){ 57 | for(AggregateValueState valueState: aggregateValueStates){ 58 | if(valueState.hasValue()){ 59 | return true; 60 | } 61 | } 62 | return false; 63 | } 64 | 65 | public List getValues() { 66 | List objectList = new ArrayList<>(aggregateValueStates.size()); 67 | for(AggregateValueState valueState: aggregateValueStates){ 68 | if(valueState.hasValue()) 69 | objectList.add(valueState.getValue()); 70 | } 71 | return objectList; 72 | } 73 | 74 | @Override 75 | public boolean equals(Object o) { 76 | if (this == o) return true; 77 | if (o == null || getClass() != o.getClass()) return false; 78 | AggregateState that = (AggregateState) o; 79 | return aggregateValueStates.equals(((AggregateState) o).aggregateValueStates); 80 | } 81 | 82 | @Override 83 | public int hashCode() { 84 | 85 | return Objects.hash(aggregateValueStates); 86 | } 87 | 88 | @Override 89 | public String toString() { 90 | return aggregateValueStates.toString(); 91 | } 92 | 93 | } 94 | -------------------------------------------------------------------------------- /flink-connector/src/main/java/stream/scotty/flinkconnector/GlobalScottyWindowOperator.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.flinkconnector; 2 | 3 | import stream.scotty.core.windowFunction.*; 4 | import stream.scotty.core.windowType.*; 5 | import stream.scotty.slicing.*; 6 | import stream.scotty.state.memory.*; 7 | import stream.scotty.core.*; 8 | import org.apache.flink.api.common.time.Time; 9 | import org.apache.flink.configuration.*; 10 | import org.apache.flink.streaming.api.functions.*; 11 | import org.apache.flink.util.*; 12 | 13 | import java.util.*; 14 | 15 | public class GlobalScottyWindowOperator extends ProcessFunction> { 16 | 17 | 18 | private MemoryStateFactory stateFactory; 19 | private SlicingWindowOperator slicingWindowOperator; 20 | private long lastWatermark; 21 | 22 | private final AggregateFunction windowFunction; 23 | private final List windows; 24 | private long allowedLateness = 1; 25 | 26 | public GlobalScottyWindowOperator(AggregateFunction windowFunction) { 27 | this.windowFunction = windowFunction; 28 | this.windows = new ArrayList<>(); 29 | } 30 | 31 | 32 | @Override 33 | public void open(Configuration parameters) throws Exception { 34 | super.open(parameters); 35 | this.stateFactory = new MemoryStateFactory(); 36 | this.slicingWindowOperator = initWindowOperator(); 37 | } 38 | 39 | public SlicingWindowOperator initWindowOperator(){ 40 | SlicingWindowOperator slicingWindowOperator = new SlicingWindowOperator<>(stateFactory); 41 | for(Window window: windows){ 42 | slicingWindowOperator.addWindowAssigner(window); 43 | } 44 | slicingWindowOperator.addAggregation(windowFunction); 45 | slicingWindowOperator.setMaxLateness(allowedLateness); 46 | return slicingWindowOperator; 47 | } 48 | 49 | @Override 50 | public void processElement(InputType value, Context ctx, Collector> out) throws Exception { 51 | 52 | this.slicingWindowOperator.processElement(value, getTimestamp(ctx)); 53 | 54 | long currentWaterMark = ctx.timerService().currentWatermark()<0?getTimestamp(ctx):ctx.timerService().currentWatermark(); 55 | 56 | if (currentWaterMark > this.lastWatermark) { 57 | List aggregates = this.slicingWindowOperator.processWatermark(currentWaterMark); 58 | for(AggregateWindow aggregateWindow: aggregates){ 59 | out.collect(aggregateWindow); 60 | } 61 | this.lastWatermark = currentWaterMark; 62 | } 63 | } 64 | 65 | private long getTimestamp(Context context){ 66 | return context.timestamp()!=null?context.timestamp():context.timerService().currentProcessingTime(); 67 | } 68 | 69 | /** 70 | * Register a new @{@link Window} definition to the ActiveWindow Operator. 71 | * For example {@link SlidingWindow} or {@link TumblingWindow} 72 | * @param window the new window definition 73 | */ 74 | public GlobalScottyWindowOperator addWindow(Window window) { 75 | windows.add(window); 76 | return this; 77 | } 78 | 79 | 80 | public GlobalScottyWindowOperator allowedLateness(Time time){ 81 | this.allowedLateness = time.toMilliseconds(); 82 | return this; 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/SlicingWindowOperator.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing; 2 | 3 | import stream.scotty.core.AggregateWindow; 4 | import stream.scotty.core.WindowOperator; 5 | import stream.scotty.core.windowFunction.AggregateFunction; 6 | import stream.scotty.core.windowType.PunctuationWindow; 7 | import stream.scotty.core.windowType.Window; 8 | import stream.scotty.core.windowType.windowContext.WindowContext; 9 | import stream.scotty.slicing.aggregationstore.AggregationStore; 10 | import stream.scotty.slicing.aggregationstore.LazyAggregateStore; 11 | import stream.scotty.slicing.slice.SliceFactory; 12 | import stream.scotty.state.StateFactory; 13 | 14 | import java.util.List; 15 | 16 | 17 | /** 18 | * Implementation of the slicing window operator based on the Scotty technique: 19 | * Scotty: Efficient AbstractWindow Aggregation for out-of-order Stream Processing: 20 | * Jonas Traub, Philipp M. Grulich, Alejandro Rodrıguez Cuéllar, Sebastian Breß, Asterios Katsifodimos, Tilmann Rabl, Volker Markl * 21 | * @param 22 | */ 23 | public class SlicingWindowOperator implements WindowOperator { 24 | 25 | private final StateFactory stateFactory; 26 | 27 | private final WindowManager windowManager; 28 | private final SliceFactory sliceFactory; 29 | private final SliceManager sliceManager; 30 | private final StreamSlicer slicer; 31 | 32 | public SlicingWindowOperator(StateFactory stateFactory) { 33 | AggregationStore aggregationStore = new LazyAggregateStore<>(); 34 | this.stateFactory = stateFactory; 35 | this.windowManager = new WindowManager(stateFactory, aggregationStore); 36 | this.sliceFactory = new SliceFactory<>(windowManager, stateFactory); 37 | this.sliceManager = new SliceManager<>(sliceFactory, aggregationStore, windowManager); 38 | this.slicer = new StreamSlicer(sliceManager, windowManager); 39 | } 40 | 41 | @Override 42 | public void processElement(InputType element, long ts) { 43 | /*added for Punctuation Window*/ 44 | for (WindowContext windowContext : this.windowManager.getContextAwareWindows()) { 45 | if(windowContext instanceof PunctuationWindow.PunctuationContext){ 46 | ((PunctuationWindow.PunctuationContext)windowContext).processPunctuation(element, ts); 47 | } 48 | } 49 | slicer.determineSlices(ts); 50 | sliceManager.processElement(element, ts); 51 | } 52 | 53 | @Override 54 | public List processWatermark(long watermarkTs) { 55 | return windowManager.processWatermark(watermarkTs); 56 | } 57 | 58 | @Override 59 | public void addWindowAssigner(Window window) { 60 | windowManager.addWindowAssigner(window); 61 | } 62 | 63 | @Override 64 | public void addAggregation(AggregateFunction windowFunction) { 65 | windowManager.addAggregation(windowFunction); 66 | } 67 | 68 | public void addWindowFunction(AggregateFunction windowFunction) { 69 | windowManager.addAggregation(windowFunction); 70 | } 71 | 72 | @Override 73 | public void setMaxLateness(long maxLateness) { 74 | windowManager.setMaxLateness(maxLateness); 75 | } 76 | 77 | public void setResendWindowsInAllowedLateness(boolean resendWindowsInAllowedLateness) {this.windowManager.setResendWindowsInAllowedLateness(resendWindowsInAllowedLateness);} 78 | } 79 | -------------------------------------------------------------------------------- /slicing/src/main/java/stream/scotty/slicing/state/AggregateValueState.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.slicing.state; 2 | 3 | import stream.scotty.core.windowFunction.*; 4 | import stream.scotty.slicing.slice.*; 5 | import stream.scotty.state.*; 6 | 7 | public class AggregateValueState { 8 | 9 | private final ValueState partialAggregateState; 10 | private final AggregateFunction aggregateFunction; 11 | private final SetState> recordSetState; 12 | 13 | public AggregateValueState(ValueState valueState, AggregateFunction aggregateFunction, SetState> recordSet) { 14 | this.partialAggregateState = valueState; 15 | this.aggregateFunction = aggregateFunction; 16 | this.recordSetState = recordSet; 17 | } 18 | 19 | /** 20 | * Add new element to a ValueState. 21 | * @param element 22 | */ 23 | public void addElement(Input element) { 24 | if (partialAggregateState.isEmpty() || partialAggregateState.get() == null) { 25 | Partial liftedElement = aggregateFunction.lift(element); 26 | partialAggregateState.set(liftedElement); 27 | } else { 28 | Partial combined = aggregateFunction.liftAndCombine(partialAggregateState.get(), element); 29 | partialAggregateState.set(combined); 30 | } 31 | } 32 | 33 | public void removeElement(StreamRecord streamRecord){ 34 | if(aggregateFunction instanceof InvertibleAggregateFunction){ 35 | InvertibleAggregateFunction invertibleAggregateFunction = (InvertibleAggregateFunction) aggregateFunction; 36 | Partial newPartial = invertibleAggregateFunction.liftAndInvert(partialAggregateState.get(), streamRecord.record); 37 | partialAggregateState.set(newPartial); 38 | }else{ 39 | recompute(); 40 | } 41 | } 42 | 43 | public void recompute(){ 44 | assert this.recordSetState != null; 45 | clear(); 46 | for(StreamRecord streamRecord: this.recordSetState){ 47 | addElement(streamRecord.record); 48 | } 49 | } 50 | 51 | public void clear(){ 52 | partialAggregateState.clean(); 53 | } 54 | 55 | public void merge(AggregateValueState otherAggState) { 56 | ValueState otherValueState = otherAggState.partialAggregateState; 57 | if (this.partialAggregateState.isEmpty() && !otherValueState.isEmpty()) { 58 | Partial otherValue = otherValueState.get(); 59 | if (this.aggregateFunction instanceof CloneablePartialStateFunction) { 60 | otherValue = ((CloneablePartialStateFunction) this.aggregateFunction).clone(otherValue); 61 | } 62 | this.partialAggregateState.set(otherValue); 63 | } else if (!otherValueState.isEmpty()) { 64 | Partial merged = this.aggregateFunction.combine(this.partialAggregateState.get(), otherValueState.get()); 65 | this.partialAggregateState.set(merged); 66 | } 67 | 68 | 69 | } 70 | 71 | public boolean hasValue(){ 72 | return !partialAggregateState.isEmpty(); 73 | } 74 | 75 | public Output getValue() { 76 | if(partialAggregateState.get() != null) 77 | return this.aggregateFunction.lower(partialAggregateState.get()); 78 | return null; 79 | } 80 | 81 | @Override 82 | public String toString() { 83 | return aggregateFunction.getClass().getSimpleName() + "->" + this.partialAggregateState.toString(); 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /benchmark/src/main/java/stream/scotty/flinkBenchmark/BenchmarkJob.java: -------------------------------------------------------------------------------- 1 | package stream.scotty.flinkBenchmark; 2 | 3 | import stream.scotty.core.windowType.*; 4 | import stream.scotty.flinkBenchmark.aggregations.SumAggregation; 5 | import stream.scotty.flinkconnector.*; 6 | import org.apache.flink.api.java.tuple.*; 7 | import org.apache.flink.api.java.utils.ParameterTool; 8 | import org.apache.flink.streaming.api.TimeCharacteristic; 9 | import org.apache.flink.streaming.api.datastream.DataStream; 10 | import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; 11 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 12 | import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; 13 | import org.apache.flink.streaming.api.functions.sink.SinkFunction; 14 | import org.apache.flink.streaming.api.watermark.Watermark; 15 | 16 | import javax.annotation.Nullable; 17 | import java.util.HashMap; 18 | import java.util.List; 19 | import java.util.Map; 20 | 21 | import static org.apache.flink.streaming.api.windowing.time.Time.seconds; 22 | 23 | /** 24 | * Created by philipp on 5/28/17. 25 | */ 26 | public class BenchmarkJob { 27 | 28 | public BenchmarkJob(List assigner, StreamExecutionEnvironment env, final long runtime, 29 | final int throughput, final List> gaps) { 30 | 31 | 32 | Map configMap = new HashMap<>(); 33 | ParameterTool parameters = ParameterTool.fromMap(configMap); 34 | 35 | env.getConfig().setGlobalJobParameters(parameters); 36 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 37 | env.setParallelism(1); 38 | env.setMaxParallelism(1); 39 | 40 | 41 | KeyedScottyWindowOperator, Tuple4> windowOperator = 42 | new KeyedScottyWindowOperator<>(new SumAggregation()); 43 | 44 | for(Window w: assigner){ 45 | windowOperator.addWindow(w); 46 | } 47 | 48 | 49 | DataStream> messageStream = env 50 | .addSource(new stream.scotty.flinkBenchmark.LoadGeneratorSource(runtime, throughput, gaps)); 51 | 52 | messageStream.flatMap(new stream.scotty.flinkBenchmark.ThroughputLogger<>(200, throughput)); 53 | 54 | 55 | 56 | final SingleOutputStreamOperator> timestampsAndWatermarks = messageStream 57 | .assignTimestampsAndWatermarks(new TimestampsAndWatermarks()); 58 | 59 | 60 | 61 | timestampsAndWatermarks 62 | .keyBy(0) 63 | .process(windowOperator) 64 | .addSink(new SinkFunction() { 65 | 66 | @Override 67 | public void invoke(final Object value) throws Exception { 68 | //System.out.println(value); 69 | } 70 | }); 71 | 72 | try { 73 | env.execute(); 74 | 75 | } catch (Exception e) { 76 | e.printStackTrace(); 77 | } 78 | 79 | } 80 | 81 | 82 | 83 | 84 | public static class TimestampsAndWatermarks implements AssignerWithPeriodicWatermarks> { 85 | private final long maxOutOfOrderness = seconds(20).toMilliseconds(); // 5 seconds 86 | private long currentMaxTimestamp; 87 | private long startTime = System.currentTimeMillis(); 88 | 89 | @Override 90 | public long extractTimestamp(final Tuple4 element, final long previousElementTimestamp) { 91 | long timestamp = element.f3; 92 | currentMaxTimestamp = Math.max(timestamp, currentMaxTimestamp); 93 | return timestamp; 94 | } 95 | 96 | @Nullable 97 | @Override 98 | public Watermark getCurrentWatermark() { 99 | return new Watermark(currentMaxTimestamp); 100 | } 101 | 102 | } 103 | } 104 | -------------------------------------------------------------------------------- /core/src/main/java/stream/scotty/core/TimeMeasure.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package stream.scotty.core; 19 | 20 | import java.util.concurrent.TimeUnit; 21 | /** 22 | * The definition of a time interval for windowing. 23 | */ 24 | public final class TimeMeasure { 25 | 26 | /** The time unit for this policy's time interval. */ 27 | private final TimeUnit unit; 28 | 29 | /** The size of the windows generated by this policy. */ 30 | private final long size; 31 | 32 | /** Instantiation only via factory method. */ 33 | private TimeMeasure(long size, TimeUnit unit) { 34 | this.unit = unit; 35 | this.size = size; 36 | 37 | } 38 | 39 | // ------------------------------------------------------------------------ 40 | // Properties 41 | // ------------------------------------------------------------------------ 42 | 43 | /** 44 | * Gets the time unit for this policy's time interval. 45 | * @return The time unit for this policy's time interval. 46 | */ 47 | public TimeUnit getUnit() { 48 | return unit; 49 | } 50 | 51 | /** 52 | * Gets the length of this policy's time interval. 53 | * @return The length of this policy's time interval. 54 | */ 55 | public long getSize() { 56 | return size; 57 | } 58 | 59 | /** 60 | * Converts the time interval to milliseconds. 61 | * @return The time interval in milliseconds. 62 | */ 63 | public long toMilliseconds() { 64 | return unit.toMillis(size); 65 | } 66 | 67 | // ------------------------------------------------------------------------ 68 | // Factory 69 | // ------------------------------------------------------------------------ 70 | 71 | public static TimeMeasure of(long size, TimeUnit unit) { 72 | return new TimeMeasure(size, unit); 73 | } 74 | 75 | /** 76 | * Creates a new {@link TimeMeasure} that represents the given number of milliseconds. 77 | */ 78 | public static TimeMeasure milliseconds(long milliseconds) { 79 | return of(milliseconds, TimeUnit.MILLISECONDS); 80 | } 81 | 82 | /** 83 | * Creates a new {@link TimeMeasure} that represents the given number of seconds. 84 | */ 85 | public static TimeMeasure seconds(long seconds) { 86 | return of(seconds, TimeUnit.SECONDS); 87 | } 88 | 89 | /** 90 | * Creates a new {@link TimeMeasure} that represents the given number of minutes. 91 | */ 92 | public static TimeMeasure minutes(long minutes) { 93 | return of(minutes, TimeUnit.MINUTES); 94 | } 95 | 96 | /** 97 | * Creates a new {@link TimeMeasure} that represents the given number of hours. 98 | */ 99 | public static TimeMeasure hours(long hours) { 100 | return of(hours, TimeUnit.HOURS); 101 | } 102 | 103 | /** 104 | * Creates a new {@link TimeMeasure} that represents the given number of days. 105 | */ 106 | public static TimeMeasure days(long days) { 107 | return of(days, TimeUnit.DAYS); 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /slicing/src/test/java/stream/scotty/slicing/aggregationstore/test/OutOfOrderArchitecture.md: -------------------------------------------------------------------------------- 1 | # Out-of-order Architecture Document - Allowed Lateness Implementation 2 | This document describes the current implementation of processing out-of-order tuples with regard to the allowed lateness in Scotty and the desired future state. 3 | Test cases are implemented in the [OutOfOrderDemo class](OutOfOrderDemo.java). 4 | 5 | **Allowed lateness** specifies how long a system stores window aggregates. 6 | In Scotty, maxLateness is default 1000, but can be set to an arbitrary value. 7 | 8 | ## Goal: 9 | - When an out-of-order tuple arrives after the watermark but in the allowed lateness arrives, Scotty outputs the updated window aggregates with the next watermark 10 | 11 | ## Current state: 12 | - When tuple in allowed lateness arrives, it is processed and added to the correct slice 13 | - It depends on the type of window, how updated window aggregates are output: 14 | 15 | ### Context-free windows 16 | - Workaround: Function setResendWindowsInAllowedLateness in SlicingWindowOperator can be set to true 17 | - Scotty returns all windows from allowed lateness until lastWatermark again, **if** a tuple in allowed lateness has arrived, and, as always, returns all windows between lastWatermark and currentWatermark 18 | - Problem: all window results from allowed lateness until lastWatermark are returned 19 | → e.g., for tumbling and sliding windows ,many windows are triggered and their results are output again even though they did not change, which leads to performance issues 20 | - only the windows which were affected by the out-of-order tuple in the allowed lateness and have changes in their aggregation result should be output again 21 | - see test case tupleInAllowedLatenessTest 22 | 23 | ### Context-aware windows - Example session window 24 | - With out-of-order tuple, session can be just updated (insert new tuple), session can be changed (extended, split, or merged with other session), or new session can be added (last case was fixed together with out-of-order bug PR #45, see out-of-order test cases in SessionWindowOperatorTest) 25 | - currently, it works when a new session is added, i.e., an out-of-order tuple in allowed lateness arrives that creates a new session window (in Scotty see tupleInAllowedLatenessTestAddSession) 26 | → because a new activeWindow is created and this is then returned whether or not it is between lastWatermark and currentWatermark 27 | → only returns the added window, exactly the target behaviour 28 | - Problem: in triggerWindows, all windows stored in windowContext are returned & then are removed from windowContext 29 | → when session is updated/extended/split/merged, session is already removed from windowContext (see SessionWindow implementation triggerWindows) and can not be correctly updated 30 | - windowContext has to keep all windows in allowed lateness to enable changes, and should only be triggered when they changed due to an out-of-order tuple in allowed lateness 31 | - see tests tupleInAllowedLatenessTestUpdateSession and tupleInAllowedLatenessTestChangeSession 32 | 33 | ## Todo: 34 | - add logic that returns only window aggregations that have been updated due to tuple in allowed lateness 35 | - Need to know: which windows were updated/changed? 36 | - time-based tumbling or sliding window: only the window(s) to which the tuple in allowed lateness belongs 37 | → just trigger these windows again 38 | - count-based tumbling or sliding window: all changed windows 39 | → need to know which windows were changed 40 | - complex for context-aware windows: allowed lateness tuple might change multiple windows 41 | → windowContext has to keep all windows in allowed lateness to enable changes of windows, and needs some indication whether window has been changed in allowed lateness and whether is has to be output again --------------------------------------------------------------------------------