├── .gitignore ├── README.md ├── pom.xml └── src ├── main ├── java │ └── com │ │ └── dataartisans │ │ └── cookbook │ │ ├── multiaggregate │ │ ├── HostMetric.java │ │ ├── HostMetricGenerator.java │ │ ├── Hostname.java │ │ └── MultiLevelAggregation.java │ │ └── windows │ │ ├── CountWindows.java │ │ └── ProcessingTimeWindows.java ├── resources │ └── log4j.properties └── scala │ └── com │ └── dataartisans │ └── cookbook │ ├── dynamicstreaming │ ├── data │ │ ├── CompositeKey.scala │ │ ├── Impression.scala │ │ ├── Keyed.scala │ │ └── Program.scala │ ├── functions │ │ ├── DynamicCountWindowFunction.scala │ │ └── DynamicKeyFunction.scala │ └── jobs │ │ └── DynamicStreamingJob.scala │ ├── expiringstate │ └── ExpiringStateJob.scala │ └── misc │ └── StateTracking.scala └── test └── scala └── exampletest └── ExampleTest.scala /.gitignore: -------------------------------------------------------------------------------- 1 | .idea 2 | cookbook.iml 3 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | ## Flink Cookbook 2 | 3 | This repository contains common patterns for (Flink) streaming applications. Each pattern should be runnable and contain a description of the use case. The patterns maybe in Scala or Java. -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 19 | 22 | 4.0.0 23 | 24 | com.dataartisans 25 | flink-cookbook 26 | 0.1 27 | jar 28 | 29 | Flink Cookbook 30 | http://www.data-artisans.com 31 | 32 | 33 | 34 | apache.snapshots 35 | Apache Development Snapshot Repository 36 | https://repository.apache.org/content/repositories/snapshots/ 37 | 38 | false 39 | 40 | 41 | true 42 | 43 | 44 | 45 | 46 | 47 | UTF-8 48 | 1.5.2 49 | 2.11 50 | 2.11.12 51 | 52 | 53 | 54 | 55 | 56 | 57 | org.apache.flink 58 | flink-scala_${scala.binary.version} 59 | ${flink.version} 60 | provided 61 | 62 | 63 | org.apache.flink 64 | flink-streaming-scala_${scala.binary.version} 65 | ${flink.version} 66 | provided 67 | 68 | 69 | 70 | 71 | org.scala-lang 72 | scala-library 73 | ${scala.version} 74 | provided 75 | 76 | 77 | 78 | 79 | 87 | 88 | 89 | 90 | 91 | org.slf4j 92 | slf4j-log4j12 93 | 1.7.7 94 | runtime 95 | 96 | 97 | log4j 98 | log4j 99 | 1.2.17 100 | runtime 101 | 102 | 103 | 104 | 105 | org.apache.flink 106 | flink-test-utils_${scala.binary.version} 107 | ${flink.version} 108 | test 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | org.apache.maven.plugins 118 | maven-shade-plugin 119 | 3.0.0 120 | 121 | 122 | 123 | package 124 | 125 | shade 126 | 127 | 128 | 129 | 130 | org.apache.flink:force-shading 131 | com.google.code.findbugs:jsr305 132 | org.slf4j:* 133 | log4j:* 134 | 135 | 136 | 137 | 138 | 140 | *:* 141 | 142 | META-INF/*.SF 143 | META-INF/*.DSA 144 | META-INF/*.RSA 145 | 146 | 147 | 148 | 149 | 150 | org.myorg.quickstart.StreamingJob 151 | 152 | 153 | 154 | 155 | 156 | 157 | 158 | 159 | 160 | org.apache.maven.plugins 161 | maven-compiler-plugin 162 | 3.1 163 | 164 | 1.8 165 | 1.8 166 | 167 | 168 | 169 | 170 | 171 | net.alchim31.maven 172 | scala-maven-plugin 173 | 3.2.2 174 | 175 | 176 | 177 | compile 178 | testCompile 179 | 180 | 181 | 182 | 183 | 184 | 185 | 186 | org.apache.maven.plugins 187 | maven-eclipse-plugin 188 | 2.8 189 | 190 | true 191 | 192 | org.scala-ide.sdt.core.scalanature 193 | org.eclipse.jdt.core.javanature 194 | 195 | 196 | org.scala-ide.sdt.core.scalabuilder 197 | 198 | 199 | org.scala-ide.sdt.launching.SCALA_CONTAINER 200 | org.eclipse.jdt.launching.JRE_CONTAINER 201 | 202 | 203 | org.scala-lang:scala-library 204 | org.scala-lang:scala-compiler 205 | 206 | 207 | **/*.scala 208 | **/*.java 209 | 210 | 211 | 212 | 213 | org.codehaus.mojo 214 | build-helper-maven-plugin 215 | 1.7 216 | 217 | 218 | 219 | add-source 220 | generate-sources 221 | 222 | add-source 223 | 224 | 225 | 226 | src/main/scala 227 | 228 | 229 | 230 | 231 | 232 | add-test-source 233 | generate-test-sources 234 | 235 | add-test-source 236 | 237 | 238 | 239 | src/test/scala 240 | 241 | 242 | 243 | 244 | 245 | 246 | 247 | 248 | 249 | 250 | 251 | 252 | 253 | add-dependencies-for-IDEA 254 | 255 | 256 | 257 | idea.version 258 | 259 | 260 | 261 | 262 | 263 | org.apache.flink 264 | flink-scala_${scala.binary.version} 265 | ${flink.version} 266 | compile 267 | 268 | 269 | org.apache.flink 270 | flink-streaming-scala_${scala.binary.version} 271 | ${flink.version} 272 | compile 273 | 274 | 275 | org.scala-lang 276 | scala-library 277 | ${scala.version} 278 | compile 279 | 280 | 281 | 282 | 283 | 284 | 285 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/cookbook/multiaggregate/HostMetric.java: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.multiaggregate; 2 | 3 | public class HostMetric { 4 | private long timestamp = 0; 5 | private Hostname hostname = new Hostname(); 6 | private String metricName = ""; 7 | private double value = 0.0; 8 | 9 | public HostMetric() { 10 | 11 | } 12 | 13 | public HostMetric(long timestamp, Hostname hostname, String metricName, double value) { 14 | this.timestamp = timestamp; 15 | this.hostname = hostname; 16 | this.metricName = metricName; 17 | this.value = value; 18 | } 19 | 20 | public long getTimestamp() { 21 | return timestamp; 22 | } 23 | 24 | public void setTimestamp(long timestamp) { 25 | this.timestamp = timestamp; 26 | } 27 | 28 | public Hostname getHostname() { 29 | return hostname; 30 | } 31 | 32 | public void setHostname(Hostname hostname) { 33 | this.hostname = hostname; 34 | } 35 | 36 | public String getMetricName() { 37 | return metricName; 38 | } 39 | 40 | public void setMetricName(String metricName) { 41 | this.metricName = metricName; 42 | } 43 | 44 | public double getValue() { 45 | return value; 46 | } 47 | 48 | public void setValue(double value) { 49 | this.value = value; 50 | } 51 | 52 | @Override 53 | public String toString() { 54 | return "HostMetric{" + 55 | "timestamp=" + timestamp + 56 | ", hostname=" + hostname + 57 | ", metricName='" + metricName + '\'' + 58 | ", value=" + value + 59 | '}'; 60 | } 61 | 62 | } 63 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/cookbook/multiaggregate/HostMetricGenerator.java: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.multiaggregate; 2 | 3 | import org.apache.flink.streaming.api.functions.source.SourceFunction; 4 | import org.apache.flink.streaming.api.watermark.Watermark; 5 | 6 | public class HostMetricGenerator implements SourceFunction { 7 | private volatile boolean isRunning = true; 8 | 9 | private String[] hostnames = { 10 | "vader", 11 | "luke", 12 | "hans", 13 | "leia", 14 | "r2-d2", 15 | "c-3po", 16 | "obi-wan", 17 | "lando", 18 | "yoda", 19 | "chewbacca" 20 | }; 21 | 22 | private String[] domains = { 23 | "wikipedia.org", 24 | "google.com", 25 | "flink.org", 26 | "starwars.com", 27 | "gamespot", 28 | "timeout.com", 29 | "galactic-voyage.com", 30 | "screenrant.com", 31 | "zimbio.com", 32 | "telegraph.co.uk" 33 | }; 34 | 35 | private String[] metrics = { 36 | "cpu_usage", 37 | "memory_usage", 38 | "jvm_heap_usage", 39 | "gc_time", 40 | "gc_collections", 41 | "disk_read_time", 42 | "disk_write_time", 43 | "network_read_time", 44 | "network_write_time", 45 | "uptime" 46 | }; 47 | 48 | @Override 49 | public void run(SourceContext ctx) throws Exception { 50 | 51 | long timestamp = 0; 52 | while(isRunning){ 53 | for(int i=0; i sourceStream = env.addSource(new HostMetricGenerator()); 23 | 24 | DataStream hostStream = sourceStream 25 | //.filter(hm -> hm.getMetricName().equals("memory_usage")) 26 | .keyBy("hostname", "metricName") 27 | .timeWindow(Time.seconds(1)) 28 | .sum("value"); 29 | 30 | DataStream domainStream = hostStream 31 | .keyBy("hostname.domain", "metricName") 32 | .timeWindow(Time.seconds(1)) 33 | .sum("value") 34 | .map(elideHost()); 35 | 36 | DataStream metricStream = domainStream 37 | .keyBy("metricName") 38 | .timeWindow(Time.seconds(1)) 39 | .sum("value") 40 | .map(elideHostAndDomain()); 41 | 42 | boolean mergedOutput = false; 43 | if (mergedOutput) { 44 | union(hostStream, domainStream, metricStream).addSink(new PrintSinkFunction<>(false)); 45 | } else { 46 | hostStream.addSink(new PrintSinkFunction(false)); 47 | domainStream.addSink(new PrintSinkFunction<>(false)); 48 | metricStream.addSink(new PrintSinkFunction<>(true)); 49 | } 50 | 51 | // execute program 52 | env.execute("Name"); 53 | } 54 | 55 | private static DataStream union(DataStream hostStream, DataStream domainStream, DataStream metricStream) { 56 | return hostStream 57 | .union(domainStream) 58 | .union(metricStream); 59 | } 60 | 61 | private static MapFunction elideHost() { 62 | return hostMetric -> { 63 | hostMetric.getHostname().setHost("*"); 64 | return hostMetric; 65 | }; 66 | } 67 | 68 | private static MapFunction elideHostAndDomain() { 69 | return hostMetric -> { 70 | hostMetric.getHostname().setHost("*"); 71 | hostMetric.getHostname().setDomain("*"); 72 | return hostMetric; 73 | }; 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/cookbook/windows/CountWindows.java: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.windows; 2 | 3 | import org.apache.flink.api.java.tuple.Tuple2; 4 | import org.apache.flink.streaming.api.TimeCharacteristic; 5 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 6 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 7 | 8 | public class CountWindows { 9 | public static void main(String[] args) throws Exception { 10 | 11 | // set up the execution environment 12 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 13 | 14 | DataStreamSource> sourceStream = env.fromElements( 15 | new Tuple2("a", 1), 16 | new Tuple2("a", 1), 17 | new Tuple2("a", 1), 18 | new Tuple2("a", 1), 19 | new Tuple2("a", 1), 20 | new Tuple2("b", 2), 21 | new Tuple2("b", 2), 22 | new Tuple2("b", 2), 23 | new Tuple2("b", 2), 24 | new Tuple2("b", 2)); 25 | 26 | sourceStream 27 | .keyBy(0) 28 | .countWindow(5) 29 | .sum(1) 30 | .print(); 31 | 32 | // execute program 33 | env.execute("Count Windows"); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/cookbook/windows/ProcessingTimeWindows.java: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.windows; 2 | 3 | import org.apache.flink.api.common.typeinfo.TypeHint; 4 | import org.apache.flink.api.java.tuple.Tuple2; 5 | import org.apache.flink.streaming.api.TimeCharacteristic; 6 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 7 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 8 | import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; 9 | import org.apache.flink.streaming.api.windowing.time.Time; 10 | 11 | public class ProcessingTimeWindows { 12 | public static void main(String[] args) throws Exception { 13 | 14 | // set up the execution environment 15 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 16 | env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime); 17 | env.setParallelism(1); 18 | 19 | DataStreamSource> sourceStream = env.fromElements( 20 | new Tuple2("a", 1), 21 | new Tuple2("a", 1), 22 | new Tuple2("a", 1), 23 | new Tuple2("a", 1), 24 | new Tuple2("a", 1), 25 | new Tuple2("b", 2), 26 | new Tuple2("b", 2), 27 | new Tuple2("b", 2), 28 | new Tuple2("b", 2), 29 | new Tuple2("b", 2)); 30 | 31 | sourceStream 32 | .map( t -> {Thread.sleep(500); return t; }) 33 | .returns(new TypeHint>(){}) 34 | .keyBy(0) 35 | .timeWindow(Time.seconds(1)) 36 | .sum(1) 37 | .addSink(new PrintSinkFunction<>(true)); 38 | 39 | // execute program 40 | env.execute("Processing Time Windows"); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 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 | 19 | log4j.rootLogger=INFO, console 20 | 21 | log4j.appender.console=org.apache.log4j.ConsoleAppender 22 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 23 | log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n 24 | -------------------------------------------------------------------------------- /src/main/scala/com/dataartisans/cookbook/dynamicstreaming/data/CompositeKey.scala: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.dynamicstreaming.data 2 | 3 | case class CompositeKey[KEY](name: String, value: KEY) 4 | -------------------------------------------------------------------------------- /src/main/scala/com/dataartisans/cookbook/dynamicstreaming/data/Impression.scala: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.dynamicstreaming.data 2 | 3 | case class Impression(tweetId: Long, userId: Long, count: Long) 4 | -------------------------------------------------------------------------------- /src/main/scala/com/dataartisans/cookbook/dynamicstreaming/data/Keyed.scala: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.dynamicstreaming.data 2 | 3 | case class Keyed[IN, KEY](wrapped: IN, key: CompositeKey[KEY]) 4 | -------------------------------------------------------------------------------- /src/main/scala/com/dataartisans/cookbook/dynamicstreaming/data/Program.scala: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.dynamicstreaming.data 2 | 3 | import org.apache.flink.api.java.functions.KeySelector 4 | 5 | case class Program[IN, KEY, OUT]( 6 | keySelector: KeySelector[IN, KEY], 7 | keyName: String, 8 | windowLength: Long, 9 | windowFunction: Iterable[IN] => OUT) -------------------------------------------------------------------------------- /src/main/scala/com/dataartisans/cookbook/dynamicstreaming/functions/DynamicCountWindowFunction.scala: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.dynamicstreaming.functions 2 | 3 | import com.dataartisans.cookbook.dynamicstreaming.data.{CompositeKey, Program, Keyed} 4 | import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction 5 | import org.apache.flink.util.Collector 6 | 7 | import scala.collection.mutable 8 | 9 | class DynamicCountWindowFunction[IN, KEY, OUT] extends RichCoFlatMapFunction[Keyed[IN, KEY], Program[IN, KEY, OUT], (CompositeKey[KEY], OUT)] { 10 | 11 | private case class KeyState(var count: Long, val items: mutable.Buffer[IN]) 12 | 13 | // The program to run for each different keyed stream (extend to many) 14 | private val programs = mutable.Map[String, Program[IN, KEY, OUT]]() 15 | private val keyState = mutable.Map[CompositeKey[KEY], KeyState]() 16 | 17 | override def flatMap1(data: Keyed[IN, KEY], out: Collector[(CompositeKey[KEY], OUT)]): Unit = { 18 | 19 | // Get the program to run 20 | val program = programs(data.key.name) 21 | 22 | // Retrieve the state for the current key 23 | val state = stateForKey(data.key) 24 | 25 | // Fire the window if the program-specified count has been reached 26 | if (state.count < program.windowLength) { 27 | state.items += data.wrapped 28 | state.count += 1 29 | } 30 | else { 31 | keyState.remove(data.key) 32 | // Fire the window function 33 | out.collect(data.key, program.windowFunction(state.items)) 34 | } 35 | } 36 | 37 | override def flatMap2(program: Program[IN, KEY, OUT], out: Collector[(CompositeKey[KEY], OUT)]): Unit = { 38 | programs.put(program.keyName, program) 39 | } 40 | 41 | private def stateForKey(key: CompositeKey[KEY]): KeyState = { 42 | keyState.getOrElseUpdate(key, KeyState(0, emptyBuffer)) 43 | } 44 | 45 | private def emptyBuffer = mutable.Buffer.empty[IN] 46 | } 47 | 48 | -------------------------------------------------------------------------------- /src/main/scala/com/dataartisans/cookbook/dynamicstreaming/functions/DynamicKeyFunction.scala: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.dynamicstreaming.functions 2 | 3 | import com.dataartisans.cookbook.dynamicstreaming.data.{CompositeKey, Keyed, Program} 4 | import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction 5 | import org.apache.flink.util.Collector 6 | 7 | import scala.collection.mutable 8 | 9 | class DynamicKeyFunction[IN, KEY, OUT] extends RichCoFlatMapFunction[IN, Program[IN, KEY, OUT], Keyed[IN, KEY]] { 10 | 11 | private val programs = mutable.Set[Program[IN, KEY, OUT]]() 12 | 13 | override def flatMap1(data: IN, out: Collector[Keyed[IN, KEY]]): Unit = { 14 | // for each program output a message with keyed according to the needs of the program 15 | programs 16 | .foreach(p => { 17 | out.collect(Keyed(data, CompositeKey(p.keyName, p.keySelector.getKey(data)))) 18 | }) 19 | } 20 | 21 | override def flatMap2(program: Program[IN, KEY, OUT], out: Collector[Keyed[IN, KEY]]): Unit = { 22 | programs.add(program) 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/scala/com/dataartisans/cookbook/dynamicstreaming/jobs/DynamicStreamingJob.scala: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.dynamicstreaming.jobs 2 | 3 | import com.dataartisans.cookbook.dynamicstreaming.data.{Impression, Program} 4 | import com.dataartisans.cookbook.dynamicstreaming.functions.{DynamicKeyFunction, DynamicCountWindowFunction} 5 | import org.apache.flink.api.java.functions.KeySelector 6 | import org.apache.flink.api.scala._ 7 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext 8 | import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment 9 | 10 | 11 | /** 12 | * This program demonstrates how we can do "dynamic" streaming. In this case 13 | * data is read from one input source and programs are read from another. A program 14 | * publishes how it needs the input stream *keyed*, as well as a window length, and a 15 | * window function to run. This is just an example. You would make this more dynamic than 16 | * just simple window functions. You'd expose all of TimelyFlatMap, for example. 17 | * 18 | * The idea here is this is how you would support many different users simultaneously that 19 | * all want to run simple programs over the same input data stream. 20 | */ 21 | object DynamicStreamingJob { 22 | def main(args: Array[String]) { 23 | val env = StreamExecutionEnvironment.getExecutionEnvironment 24 | env.setParallelism(1) 25 | 26 | val stream = env.addSource(dataSource) 27 | 28 | val programStream = env.addSource(programSource).broadcast 29 | 30 | stream 31 | .connect(programStream) 32 | .flatMap(new DynamicKeyFunction) 33 | .keyBy(_.key) 34 | .connect(programStream) 35 | .flatMap(new DynamicCountWindowFunction) 36 | .print() 37 | 38 | env.execute() 39 | } 40 | 41 | /** 42 | * Just some test data 43 | */ 44 | def dataSource = { 45 | (sc: SourceContext[Impression]) => { 46 | while (true) { 47 | Thread.sleep(100) 48 | sc.collect(Impression(123456789, 24, 10)) 49 | sc.collect(Impression(123456789, 25, 20)) 50 | } 51 | } 52 | } 53 | 54 | /** 55 | * This is an example of a stream of programs. They do different things 56 | * and they provide a keySelector that indicates how they want the input stream 57 | * keyed. These are generated at runtime! In a real system they would be programs 58 | * streamed in -- maybe groovy or other dynamic JVM language. 59 | */ 60 | def programSource = { 61 | (sc: SourceContext[Program[Impression, Long, Long]]) => { 62 | 63 | /** 64 | * Key by TweetId, window into blocks of 10, and sum 65 | */ 66 | sc.collect( 67 | Program( 68 | keySelector = keySelector(_.tweetId), 69 | keyName = "tweetId", 70 | windowLength = 10, 71 | windowFunction = (itor) => { 72 | itor.map(_.count).sum // compute sum -- expect 150 73 | } 74 | ) 75 | ) 76 | 77 | /** 78 | * Key by userId, window into blocks of 5, and average 79 | */ 80 | sc.collect( 81 | Program( 82 | keySelector = keySelector(_.userId), 83 | keyName = "userId", 84 | windowLength = 5, 85 | windowFunction = (itor) => { 86 | itor.map(_.count).sum / itor.size // compute average -- expect 10 & 20 87 | } 88 | ) 89 | ) 90 | 91 | while (true) { 92 | Thread.sleep(500) 93 | } 94 | } 95 | } 96 | 97 | private def keySelector[IN, KEY](function: IN => KEY): KeySelector[IN, KEY] = { 98 | new KeySelector[IN, KEY]() { 99 | override def getKey(value: IN): KEY = { 100 | function(value) 101 | } 102 | } 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /src/main/scala/com/dataartisans/cookbook/expiringstate/ExpiringStateJob.scala: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.expiringstate 2 | 3 | import org.apache.flink.api.common.state.ValueStateDescriptor 4 | import org.apache.flink.api.common.typeinfo.{TypeHint, TypeInformation} 5 | import org.apache.flink.streaming.api.TimeCharacteristic 6 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext 7 | import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, _} 8 | import org.apache.flink.streaming.api.watermark.Watermark 9 | import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows 10 | import org.apache.flink.streaming.api.windowing.triggers.Trigger.TriggerContext 11 | import org.apache.flink.streaming.api.windowing.triggers.{Trigger, TriggerResult} 12 | import org.apache.flink.streaming.api.windowing.windows.GlobalWindow 13 | 14 | object ExpiringStateJob { 15 | def main(args: Array[String]) { 16 | val env = StreamExecutionEnvironment.getExecutionEnvironment 17 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) 18 | env.setParallelism(1) 19 | 20 | val stream = env.addSource((ctx: SourceContext[Data]) => { 21 | val MessageIntervalMs = 100 22 | var key = 0 23 | var timestamp = 0L 24 | while (true) { 25 | for (i <- 0 until 10) { 26 | ctx.collectWithTimestamp(Data(timestamp, key, 1), timestamp) 27 | ctx.emitWatermark(new Watermark(timestamp)) 28 | Thread.sleep(MessageIntervalMs) 29 | timestamp += MessageIntervalMs 30 | } 31 | // keyspace is always growing but we only see each key for 1 second and then never again 32 | key += 1 33 | } 34 | }) 35 | 36 | // Doing this checkpoints will grow larger and larger as the keyspace grows forever and will 37 | // never get cleaned up. 38 | // stream 39 | // .keyBy("key") 40 | // .reduce((a, b) => Data(a.timestamp, a.key, a.value + b.value)) 41 | // .print() 42 | 43 | // Here's the same thing with a custom trigger that cleans up state for each key after 2 seconds 44 | stream 45 | .keyBy("key") 46 | .window(GlobalWindows.create()) 47 | .trigger(new StateTtlTrigger(2000)) 48 | .reduce((a, b) => Data(a.timestamp, a.key, a.value + b.value)) 49 | .print 50 | 51 | env.execute() 52 | } 53 | 54 | case class Data(timestamp: Long, key: Long, value: Long) 55 | 56 | /** 57 | * This trigger fires the window on every element and registers a timer when it's first created 58 | * that will purge/destory the window after `expireTimeMs`. Since the state kept is scoped by window 59 | * and key the state will be cleaned up when the window is purged. 60 | * 61 | * This essentially has the effect of creating state with a TTL. 62 | * 63 | * @param expireTimeMs 64 | */ 65 | class StateTtlTrigger(expireTimeMs: Long) extends Trigger[Data, GlobalWindow] { 66 | 67 | val stateDesc = new ValueStateDescriptor[Boolean]("timer_created", TypeInformation.of(new TypeHint[Boolean]() {}), false) 68 | 69 | override def onElement(t: Data, l: Long, w: GlobalWindow, triggerContext: TriggerContext): TriggerResult = { 70 | val timerCreated = triggerContext.getPartitionedState(stateDesc) 71 | if (!timerCreated.value()) { 72 | println(s"Registering purge timer for key ${t.key}") 73 | triggerContext.registerEventTimeTimer(t.timestamp + expireTimeMs) 74 | timerCreated.update(true) 75 | } 76 | TriggerResult.FIRE 77 | } 78 | 79 | override def onEventTime(l: Long, w: GlobalWindow, triggerContext: TriggerContext): TriggerResult = { 80 | println(s"Purging state for key ${(l - expireTimeMs) / 1000} @ time ${l}") 81 | triggerContext.getPartitionedState(stateDesc).clear() 82 | triggerContext.deleteEventTimeTimer(l) 83 | TriggerResult.PURGE 84 | } 85 | 86 | override def onProcessingTime(l: Long, w: GlobalWindow, triggerContext: TriggerContext): TriggerResult = { 87 | ??? 88 | } 89 | 90 | override def clear(w: GlobalWindow, triggerContext: TriggerContext): Unit = ??? 91 | } 92 | } 93 | -------------------------------------------------------------------------------- /src/main/scala/com/dataartisans/cookbook/misc/StateTracking.scala: -------------------------------------------------------------------------------- 1 | package com.dataartisans.cookbook.misc 2 | 3 | import org.apache.flink.api.common.state.ValueStateDescriptor 4 | import org.apache.flink.api.scala._ 5 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext 6 | import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment 7 | import org.apache.flink.streaming.api.windowing.time.Time 8 | import org.apache.flink.streaming.api.windowing.triggers.Trigger.TriggerContext 9 | import org.apache.flink.streaming.api.windowing.triggers.{Trigger, TriggerResult} 10 | import org.apache.flink.streaming.api.windowing.windows.TimeWindow 11 | import org.apache.flink.util.Collector 12 | 13 | case class Event(guid: String, state: String) 14 | 15 | object ProcessingTimeTriggerWithPeriodicFirings { 16 | def apply(intervalMs: Long) = { 17 | new ProcessingTimeTriggerWithPeriodicFirings(intervalMs) 18 | } 19 | } 20 | 21 | class ProcessingTimeTriggerWithPeriodicFirings(intervalMs: Long) extends Trigger[Event, TimeWindow] { 22 | private val startTimeDesc = new ValueStateDescriptor[Long]("start-time", classOf[Long], 0L) 23 | 24 | override def onElement(element: Event, timestamp: Long, window: TimeWindow, ctx: TriggerContext): TriggerResult = { 25 | val startTime = ctx.getPartitionedState(startTimeDesc) 26 | if (startTime.value == 0) { 27 | startTime.update(window.getStart) 28 | ctx.registerProcessingTimeTimer(window.getEnd) 29 | ctx.registerProcessingTimeTimer(System.currentTimeMillis() + intervalMs) 30 | } 31 | TriggerResult.CONTINUE 32 | } 33 | 34 | override def onProcessingTime(time: Long, window: TimeWindow, ctx: TriggerContext): TriggerResult = { 35 | if (time == window.getEnd) { 36 | TriggerResult.PURGE 37 | } 38 | else { 39 | ctx.registerProcessingTimeTimer(time + intervalMs) 40 | TriggerResult.FIRE 41 | } 42 | } 43 | 44 | override def onEventTime(time: Long, window: TimeWindow, ctx: TriggerContext): TriggerResult = { 45 | TriggerResult.CONTINUE 46 | } 47 | 48 | override def clear(w: TimeWindow, triggerContext: TriggerContext): Unit = ??? 49 | } 50 | 51 | object StateTracking { 52 | def main(args: Array[String]) { 53 | 54 | val env: StreamExecutionEnvironment = StreamExecutionEnvironment.getExecutionEnvironment 55 | 56 | // TODO: Remove 57 | env.setParallelism(1) 58 | 59 | val stream = env.addSource((ctx: SourceContext[Event]) => { 60 | var guid = 1 61 | while (true) { 62 | for (i <- 1 to 1) { 63 | val state = s"state-$i" 64 | ctx.collect(Event(s"guid-$guid", state)) 65 | Thread.sleep(1000) 66 | } 67 | guid = guid + 1 68 | } 69 | }) 70 | 71 | val results = stream 72 | .keyBy(_.guid) 73 | .timeWindow(Time.seconds(30)) 74 | .trigger(ProcessingTimeTriggerWithPeriodicFirings(1000)) 75 | .apply( 76 | (e1, e2) => e2, 77 | (k, w, i, c: Collector[(String, Long)]) => { 78 | if (i.head != null) c.collect((i.head.state, 1)) 79 | } 80 | ) 81 | .keyBy(0) 82 | .timeWindow(Time.seconds(1)) 83 | .sum(1) 84 | 85 | 86 | results.printToErr() 87 | 88 | env.execute("Job") 89 | } 90 | 91 | } 92 | -------------------------------------------------------------------------------- /src/test/scala/exampletest/ExampleTest.scala: -------------------------------------------------------------------------------- 1 | package exampletest 2 | 3 | import java.util.concurrent.TimeUnit 4 | 5 | import org.apache.flink.api.scala._ 6 | import org.apache.flink.streaming.api.TimeCharacteristic 7 | import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks 8 | import org.apache.flink.streaming.api.functions.sink.SinkFunction 9 | import org.apache.flink.streaming.api.functions.source.SourceFunction 10 | import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment 11 | import org.apache.flink.streaming.api.watermark.Watermark 12 | import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows 13 | import org.apache.flink.streaming.api.windowing.time.Time 14 | import org.apache.flink.test.util.AbstractTestBase 15 | import org.junit.Assert._ 16 | import org.junit.Test 17 | 18 | import scala.collection.mutable 19 | 20 | /** 21 | * Tests for Folds over windows. These also test whether OutputTypeConfigurable functions 22 | * work for windows, because FoldWindowFunction is OutputTypeConfigurable. 23 | */ 24 | class ExampleTest extends AbstractTestBase { 25 | 26 | @Test 27 | def testReduceWindow(): Unit = { 28 | val env = StreamExecutionEnvironment.getExecutionEnvironment 29 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) 30 | env.setParallelism(1) 31 | 32 | val source1 = env.addSource(new SourceFunction[(String, Int)]() { 33 | def run(ctx: SourceFunction.SourceContext[(String, Int)]) { 34 | ctx.collect(("a", 0)) 35 | ctx.collect(("a", 1)) 36 | ctx.collect(("a", 2)) 37 | ctx.collect(("b", 3)) 38 | ctx.collect(("b", 4)) 39 | ctx.collect(("b", 5)) 40 | ctx.collect(("a", 6)) 41 | ctx.collect(("a", 7)) 42 | ctx.collect(("a", 8)) 43 | 44 | // source is finite, so it will have an implicit MAX watermark when it finishes 45 | } 46 | 47 | override def cancel(): Unit = {} 48 | }).assignTimestampsAndWatermarks(new ExampleTest.Tuple2TimestampExtractor) 49 | 50 | source1 51 | .keyBy(0) 52 | .window(TumblingEventTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS))) 53 | .reduce((a, b) => (a._1 + b._1, a._2 + b._2)) 54 | .addSink(new SinkFunction[(String, Int)]() { 55 | def invoke(value: (String, Int)) { 56 | ExampleTest.testResults += value.toString 57 | } 58 | }) 59 | 60 | env.execute("Reduce Window Test") 61 | 62 | val expectedResult = mutable.MutableList( 63 | "(aaa,3)", 64 | "(aaa,21)", 65 | "(bbb,12)") 66 | 67 | assertEquals(expectedResult.sorted, ExampleTest.testResults.sorted) 68 | } 69 | } 70 | 71 | object ExampleTest { 72 | 73 | val testResults = mutable.MutableList[String]() 74 | 75 | private class Tuple2TimestampExtractor extends AssignerWithPunctuatedWatermarks[(String, Int)] { 76 | 77 | private var currentTimestamp = -1L 78 | 79 | override def extractTimestamp(element: (String, Int), previousTimestamp: Long): Long = { 80 | currentTimestamp = element._2 81 | currentTimestamp 82 | } 83 | 84 | def checkAndGetNextWatermark(lastElement: (String, Int), extractedTimestamp: Long): Watermark = { 85 | new Watermark(lastElement._2 - 1) 86 | } 87 | } 88 | } 89 | 90 | 91 | --------------------------------------------------------------------------------