├── .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 |
--------------------------------------------------------------------------------