SlideShare a Scribd company logo
April 10, 2018
Seth Wiesman
Testing Stateful Streaming Applications
©2018 MediaMath Inc. 2
val words: DataStream[String] = ???
words
.keyBy(word ⇒ word)
.map(new CountingMap)
Basic Stateful Word Count
“hello” → (“hello”, 1)
“hello” → (“hello”, 2)
“world” → (“world”, 1)
“hello” → (“hello”, 3)
©2018 MediaMath Inc. 3
• Stateless jobs are easy to reason about
• The same input always results in the same output
• Referentially Transparent
• That doesn’t mean no state at all!
• Stateless:: (IN, STATE) → (OUT, STATE)
vs Stateless Jobs
©2018 MediaMath Inc. 4
Operator State
©2018 MediaMath Inc. 5
import java.util.{List ⇒ JList}
class ResourceToStream(path: String)
extends RichSourceFunction[String] with ListCheckpointed[Integer] {
var numLines: Int = 0
override def cancel(): Unit = ???
override def run(ctx: SourceFunction.SourceContext[String]): Unit = ???
override def restoreState(state: JList[Integer]): Unit = ???
override def snapshotState(checkpointId: Long, timestamp: Long): JList[Integer] = ???
}
©2018 MediaMath Inc. 6
import java.util.{List ⇒ JList}
class ResourceToStream(path: String)
extends RichSourceFunction[String] with ListCheckpointed[Integer] {
var numLines: Int = 0
override def cancel(): Unit = ???
override def run(ctx: SourceFunction.SourceContext[String]): Unit = ???
override def restoreState(state: JList[Integer]): Unit = ???
override def snapshotState(checkpointId: Long, timestamp: Long): JList[Integer] = ???
}
Business Logic
State Management
©2018 MediaMath Inc. 7
it should "process a resource file" in {
val runner = new ResourceToStream("/testfile.txt")
val ctx = new MockSourceContext
runner.run(ctx)
ctx.output.size should be (4)
ctx.output should be (ListBuffer("hello", "world", "good", "day"))
}
Validate Business Logic
©2018 MediaMath Inc. 8
it should "restore from state" in {
val runner = new ResourceToStream("/testfile.txt")
val ctx = new MockSourceContext
runner.restoreState(Arrays.asList(2))
runner.run(ctx)
ctx.output.size should be (2)
ctx.output should be (ListBuffer("good", "day"))
}
Consider State Management
©2018 MediaMath Inc. 9
val mainStream: DataStream[String] = ???
val filterStream: DataStream[String] = env
.addSource(new ResourceToStream(“”))
mainStream
.connect(filterStream.broadcast)
.flatMap(new BroadcastFilter)
Broadcast State
©2018 MediaMath Inc. 10
class BroadcastFilter extends RichCoFlatMapFunction[String, String, String] with ??? {
val filters = mutable.Set.empty[String]
override def flatMap1(value: String, out: Collector[String]): Unit = {
if (!filters.contains(value)) {
out.collect(value)
}
}
override def flatMap2(value: String, out: Collector[String]): Unit = {
filters += value
}
©2018 MediaMath Inc. 11
override def initializeState(context: FunctionInitializationContext): Unit = {
state = context.getOperatorStateStore.getUnionListState(
new ListStateDescriptor[String]("filters", BasicTypeInfo.STRING_TYPE_INFO)
)
???
}
CheckpointedFunction
©2018 MediaMath Inc. 12
override def snapshotState(context: FunctionSnapshotContext): Unit = {
state.clear()
if (getRuntimeContext.getIndexOfThisSubtask == 0) {
for (filter ← filters) {
state.add(filter)
}
}
}
CheckpointedFunction
©2018 MediaMath Inc. 13
• Testing business logic is still straightforward
• Testing checkpoint methods is less clear
• Mocking out operator state is not fun
CheckpointedFunction
it should "filter out blacklisted words" in {
val filter = new BroadcastFilter
val out = new MockCollector
filter.flatMap2("hello", out)
out.output.size should be (0)
filter.flatMap1("hello", out)
out.output.size should be (0)
filter.flatMap1("world", out)
out.output.size should be (1)
}
©2018 MediaMath Inc. 14
• Used as part of Flink’s internal test suite
• ⚠️ not part of the public api ⚠️
• Not recommended as a first option
• Try and get by with integration tests first
• Be careful to test your code, not Flink’s!
• Used sparingly, can be very powerful
OperatorTestHarness
©2018 MediaMath Inc. 15
it should "restore from state" in {
val initialFilter = new BroadcastFilter
val initialOperator = new CoStreamFlatMap(initialFilter)
val initialTestHarness = new TwoInputStreamOperatorTestHarness(initialOperator)
initialTestHarness.initializeState(new OperatorStateHandles(0, null, null, null, null))
initialTestHarness.open()
initialTestHarness.processElement2(new StreamRecord[String]("hello"))
val snapshot = initialTestHarness.snapshot(0L, 0L)
initialTestHarness.close()
©2018 MediaMath Inc. 16
val restoreFilter = new BroadcastFilter
val restoreOperator = new CoStreamFlatMap(restoreFilter)
val restoreTestHarness = new TwoInputStreamOperatorTestHarness(restoreOperator)
restoreTestHarness.initializeState(snapshot)
restoreTestHarness.setup()
restoreTestHarness.open()
restoreTestHarness.processElement1(new StreamRecord[String]("hello"))
restoreTestHarness.getOutput.size() should be (0)
restoreTestHarness.close()
}
©2018 MediaMath Inc. 17
Keyed State
©2018 MediaMath Inc. 18
• Many keyed state operations can be handled “statelessly”
Keyed State
words
.keyBy(word ⇒ word)
.countWindow(3)
.reduce((word, _) ⇒ word)
• Not always an option
©2018 MediaMath Inc. 19
class EveryNthWord(n: Int) extends RichFlatMapFunction[String, String] {
@transient private lazy val countDesc = new ValueStateDescriptor("count", classOf[Int])
override def flatMap(value: String, out: Collector[String]): Unit = {
val countState = getRuntimeContext.getState(countDesc)
val count = Option(countState.value()).map(_.intValue()).getOrElse(1)
if (count == n) {
countState.clear()
out.collect(value)
} else {
countState.update(count + 1)
}
}
}
©2018 MediaMath Inc. 20
object StatelessEveryNthWord {
type StatefulFunction = (String, Option[Int]) ⇒ (TraversableOnce[String], Option[Int])
}
FlatMapWithState
©2018 MediaMath Inc. 21
class StatelessEveryNthWord(n: Int) extends StatelessEveryNthWord.StatefulFunction {
override def apply(word: String, state: Option[Int]): (TraversableOnce[String], Option[Int])= {
state match {
case Some(count) if count + 1 == n ⇒ (Some(word), None)
case Some(count) ⇒ (None, Some(count + 1))
case None ⇒ (None, Some(1))
}
}
}
FlatMapWithState
©2018 MediaMath Inc. 22
it should "initialize state the first time it sees a word" in {
val function = new StatelessEveryNthWord(3)
val (output, state) = function("hello", None)
output should be (None)
state should be (Some(1))
}
FlatMapWithState Test
it should "modify state in the middle of a run" in {
val function = new StatelessEveryNthWord(3)
val (output, state) = function("hello", Some(1))
output should be (None)
state should be (Some(2))
}
©2018 MediaMath Inc. 23
override def flatMap(value: String, out: Collector[String]): Unit = {
val state = Option(getRuntimeContext.getState(countDesc).value())
f(value, state) match {
case (output, None) ⇒
output.foreach(out.collect)
getRuntimeContext.getState(countDesc).clear()
case (output, Some(x)) ⇒
output.foreach(out.collect)
getRuntimeContext.getState(countDesc).update(x)
}
Possible FlatMapWithState Implemention
react to results
initialize
context
©2018 MediaMath Inc. 24
• Group all elements of the same key that occur within 30 seconds of each other
• similar to a session window
• Output the third instance of every word immediately
• When more than 30 seconds have passed since the last word:
• clear state
• If more than 3 instances of a word have been observed, output to heavy hitter side output
• If less than 3 instances of a word have been observed, output to infrequent side output
A more complex example
©2018 MediaMath Inc. 25
object Algebra {
final case class Context(count: Long, watermark: Long)
final case class Result(word: Option[String], timer: Long)
}
trait Algebra {
def evaluateElem(word: String, ctx: Algebra.Context): Algebra.Result
def evaluateTimer(word: String, count: Long): Option[Either[String, String]]
}
Specialized Context
©2018 MediaMath Inc. 26
class StatelessAlgebra extends Algebra {
final val private[this] thirty_seconds: Long = 30 * 1000L
override def evaluateElem(word: String, ctx: Algebra.Context): Algebra.Result = {
val output = if (ctx.count == 3) {
Some(word)
} else {
None
}
val timer = ctx.watermark + thirty_seconds
Algebra.Result(output, timer)
}
Stateless Business Logic
©2018 MediaMath Inc. 27
override def evaluateTimer(word: String, count: Long): Option[Either[String, String]] = {
if (count < 3) {
Some(Left(word))
} else if (count > 3) {
Some(Right(word))
} else {
None
}
}
}
Stateless Business Logic
©2018 MediaMath Inc. 28
getRuntimeContext.getState(wordDesc).update(value)
getRuntimeContext.getReducingState(countDesc).add(1L)
val count = getRuntimeContext.getReducingState(countDesc).get()
val context = Algebra.Context(count, ctx.timerService().currentWatermark())
val Algebra.Result(output, time) = program.evaluateElem(value, context)
output.foreach(out.collect)
getRuntimeContext.getState(timerDesc).update(time)
ctx.timerService().registerEventTimeTimer(time)
Program Runner
react to results
initialize
context
©2018 MediaMath Inc. 29
• Business Logic
• Business logic can be tested in isolation of Flink
• Simple to simulate various scenarios
• Runner
• Does not require knowledge of business domain to validate
• Isolates the more complex parts of writing streaming applications
Testing our Implementation
©2018 MediaMath Inc. 30
Integration Testing
©2018 MediaMath Inc. 31
• Test the full End-to-End pipeline
• More closely simulate production than unit tests
• StreamingMultipleProgramsTestBase
• Runs a local version of Flink
• Beware only testing the happy path
• Consider running each integration test twice – both with and without failure
• See BucketingSink test suite for an example of failing the stream [1]
Integration Testing
[1] https://github.com/apache/flink/blob/7b07fa5a09279a55c99d80db92ebf98a7dcd9644/flink-connectors/flink-connector-
filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java#L200
©2018 MediaMath Inc. 32
Full Code Examples
github.com/sjwiesman/ff-sf-2018
Thank You

More Related Content

What's hot

Spark summit 2017- Transforming B2B sales with Spark powered sales intelligence
Spark summit 2017- Transforming B2B sales with Spark powered sales intelligenceSpark summit 2017- Transforming B2B sales with Spark powered sales intelligence
Spark summit 2017- Transforming B2B sales with Spark powered sales intelligence
Wei Di
 
Flink Forward Berlin 2018: Xiaowei Jiang - Keynote: "Unified Engine for Data ...
Flink Forward Berlin 2018: Xiaowei Jiang - Keynote: "Unified Engine for Data ...Flink Forward Berlin 2018: Xiaowei Jiang - Keynote: "Unified Engine for Data ...
Flink Forward Berlin 2018: Xiaowei Jiang - Keynote: "Unified Engine for Data ...
Flink Forward
 
Towards sql for streams
Towards sql for streamsTowards sql for streams
Towards sql for streams
Radu Tudoran
 
Flink SQL in Action
Flink SQL in ActionFlink SQL in Action
Flink SQL in Action
Fabian Hueske
 
Apache Beam (incubating)
Apache Beam (incubating)Apache Beam (incubating)
Apache Beam (incubating)
Apache Apex
 
Test strategies for data processing pipelines
Test strategies for data processing pipelinesTest strategies for data processing pipelines
Test strategies for data processing pipelines
Lars Albertsson
 
KliqObjects Overview
KliqObjects OverviewKliqObjects Overview
KliqObjects Overview
KT-Labs
 
KliqMap for Esri: Actionable Location Analytics
KliqMap for Esri: Actionable Location AnalyticsKliqMap for Esri: Actionable Location Analytics
KliqMap for Esri: Actionable Location Analytics
KT-Labs
 
Flink Forward San Francisco 2018: David Reniz & Dahyr Vergara - "Real-time m...
Flink Forward San Francisco 2018:  David Reniz & Dahyr Vergara - "Real-time m...Flink Forward San Francisco 2018:  David Reniz & Dahyr Vergara - "Real-time m...
Flink Forward San Francisco 2018: David Reniz & Dahyr Vergara - "Real-time m...
Flink Forward
 
Big Data, Bigger Analytics
Big Data, Bigger AnalyticsBig Data, Bigger Analytics
Big Data, Bigger Analytics
Itzhak Kameli
 
Flink Forward SF 2017: Shaoxuan Wang_Xiaowei Jiang - Blinks Improvements to F...
Flink Forward SF 2017: Shaoxuan Wang_Xiaowei Jiang - Blinks Improvements to F...Flink Forward SF 2017: Shaoxuan Wang_Xiaowei Jiang - Blinks Improvements to F...
Flink Forward SF 2017: Shaoxuan Wang_Xiaowei Jiang - Blinks Improvements to F...
Flink Forward
 
Enhancements on Spark SQL optimizer by Min Qiu
Enhancements on Spark SQL optimizer by Min QiuEnhancements on Spark SQL optimizer by Min Qiu
Enhancements on Spark SQL optimizer by Min Qiu
Spark Summit
 
KliqPlan Overview
KliqPlan OverviewKliqPlan Overview
KliqPlan Overview
KT-Labs
 
Flink Forward Berlin 2017 Keynote: Ferd Scheepers - Taking away customer fric...
Flink Forward Berlin 2017 Keynote: Ferd Scheepers - Taking away customer fric...Flink Forward Berlin 2017 Keynote: Ferd Scheepers - Taking away customer fric...
Flink Forward Berlin 2017 Keynote: Ferd Scheepers - Taking away customer fric...
Flink Forward
 
Stream Analytics with SQL on Apache Flink
 Stream Analytics with SQL on Apache Flink Stream Analytics with SQL on Apache Flink
Stream Analytics with SQL on Apache Flink
Fabian Hueske
 
Flink Forward Berlin 2018: Aljoscha Krettek & Till Rohrmann - Keynote: "A Yea...
Flink Forward Berlin 2018: Aljoscha Krettek & Till Rohrmann - Keynote: "A Yea...Flink Forward Berlin 2018: Aljoscha Krettek & Till Rohrmann - Keynote: "A Yea...
Flink Forward Berlin 2018: Aljoscha Krettek & Till Rohrmann - Keynote: "A Yea...
Flink Forward
 
 On Track with Apache Kafka: Building a Streaming ETL solution with Rail Dat...
 On Track with Apache Kafka: Building a Streaming ETL solution with Rail Dat... On Track with Apache Kafka: Building a Streaming ETL solution with Rail Dat...
 On Track with Apache Kafka: Building a Streaming ETL solution with Rail Dat...
HostedbyConfluent
 
Scaling stream data pipelines with Pravega and Apache Flink
Scaling stream data pipelines with Pravega and Apache FlinkScaling stream data pipelines with Pravega and Apache Flink
Scaling stream data pipelines with Pravega and Apache Flink
Till Rohrmann
 
Keeping Identity Graphs In Sync With Apache Spark
Keeping Identity Graphs In Sync With Apache SparkKeeping Identity Graphs In Sync With Apache Spark
Keeping Identity Graphs In Sync With Apache Spark
Databricks
 
Building real time data-driven products
Building real time data-driven productsBuilding real time data-driven products
Building real time data-driven products
Lars Albertsson
 

What's hot (20)

Spark summit 2017- Transforming B2B sales with Spark powered sales intelligence
Spark summit 2017- Transforming B2B sales with Spark powered sales intelligenceSpark summit 2017- Transforming B2B sales with Spark powered sales intelligence
Spark summit 2017- Transforming B2B sales with Spark powered sales intelligence
 
Flink Forward Berlin 2018: Xiaowei Jiang - Keynote: "Unified Engine for Data ...
Flink Forward Berlin 2018: Xiaowei Jiang - Keynote: "Unified Engine for Data ...Flink Forward Berlin 2018: Xiaowei Jiang - Keynote: "Unified Engine for Data ...
Flink Forward Berlin 2018: Xiaowei Jiang - Keynote: "Unified Engine for Data ...
 
Towards sql for streams
Towards sql for streamsTowards sql for streams
Towards sql for streams
 
Flink SQL in Action
Flink SQL in ActionFlink SQL in Action
Flink SQL in Action
 
Apache Beam (incubating)
Apache Beam (incubating)Apache Beam (incubating)
Apache Beam (incubating)
 
Test strategies for data processing pipelines
Test strategies for data processing pipelinesTest strategies for data processing pipelines
Test strategies for data processing pipelines
 
KliqObjects Overview
KliqObjects OverviewKliqObjects Overview
KliqObjects Overview
 
KliqMap for Esri: Actionable Location Analytics
KliqMap for Esri: Actionable Location AnalyticsKliqMap for Esri: Actionable Location Analytics
KliqMap for Esri: Actionable Location Analytics
 
Flink Forward San Francisco 2018: David Reniz & Dahyr Vergara - "Real-time m...
Flink Forward San Francisco 2018:  David Reniz & Dahyr Vergara - "Real-time m...Flink Forward San Francisco 2018:  David Reniz & Dahyr Vergara - "Real-time m...
Flink Forward San Francisco 2018: David Reniz & Dahyr Vergara - "Real-time m...
 
Big Data, Bigger Analytics
Big Data, Bigger AnalyticsBig Data, Bigger Analytics
Big Data, Bigger Analytics
 
Flink Forward SF 2017: Shaoxuan Wang_Xiaowei Jiang - Blinks Improvements to F...
Flink Forward SF 2017: Shaoxuan Wang_Xiaowei Jiang - Blinks Improvements to F...Flink Forward SF 2017: Shaoxuan Wang_Xiaowei Jiang - Blinks Improvements to F...
Flink Forward SF 2017: Shaoxuan Wang_Xiaowei Jiang - Blinks Improvements to F...
 
Enhancements on Spark SQL optimizer by Min Qiu
Enhancements on Spark SQL optimizer by Min QiuEnhancements on Spark SQL optimizer by Min Qiu
Enhancements on Spark SQL optimizer by Min Qiu
 
KliqPlan Overview
KliqPlan OverviewKliqPlan Overview
KliqPlan Overview
 
Flink Forward Berlin 2017 Keynote: Ferd Scheepers - Taking away customer fric...
Flink Forward Berlin 2017 Keynote: Ferd Scheepers - Taking away customer fric...Flink Forward Berlin 2017 Keynote: Ferd Scheepers - Taking away customer fric...
Flink Forward Berlin 2017 Keynote: Ferd Scheepers - Taking away customer fric...
 
Stream Analytics with SQL on Apache Flink
 Stream Analytics with SQL on Apache Flink Stream Analytics with SQL on Apache Flink
Stream Analytics with SQL on Apache Flink
 
Flink Forward Berlin 2018: Aljoscha Krettek & Till Rohrmann - Keynote: "A Yea...
Flink Forward Berlin 2018: Aljoscha Krettek & Till Rohrmann - Keynote: "A Yea...Flink Forward Berlin 2018: Aljoscha Krettek & Till Rohrmann - Keynote: "A Yea...
Flink Forward Berlin 2018: Aljoscha Krettek & Till Rohrmann - Keynote: "A Yea...
 
 On Track with Apache Kafka: Building a Streaming ETL solution with Rail Dat...
 On Track with Apache Kafka: Building a Streaming ETL solution with Rail Dat... On Track with Apache Kafka: Building a Streaming ETL solution with Rail Dat...
 On Track with Apache Kafka: Building a Streaming ETL solution with Rail Dat...
 
Scaling stream data pipelines with Pravega and Apache Flink
Scaling stream data pipelines with Pravega and Apache FlinkScaling stream data pipelines with Pravega and Apache Flink
Scaling stream data pipelines with Pravega and Apache Flink
 
Keeping Identity Graphs In Sync With Apache Spark
Keeping Identity Graphs In Sync With Apache SparkKeeping Identity Graphs In Sync With Apache Spark
Keeping Identity Graphs In Sync With Apache Spark
 
Building real time data-driven products
Building real time data-driven productsBuilding real time data-driven products
Building real time data-driven products
 

Similar to Flink Forward San Francisco 2018: Seth Wiesman - "Testing Stateful Streaming Applications"

Stratosphere Intro (Java and Scala Interface)
Stratosphere Intro (Java and Scala Interface)Stratosphere Intro (Java and Scala Interface)
Stratosphere Intro (Java and Scala Interface)
Robert Metzger
 
Lo Mejor Del Pdc2008 El Futrode C#
Lo Mejor Del Pdc2008 El Futrode C#Lo Mejor Del Pdc2008 El Futrode C#
Lo Mejor Del Pdc2008 El Futrode C#
Juan Pablo
 
Reactive programming every day
Reactive programming every dayReactive programming every day
Reactive programming every day
Vadym Khondar
 
Functional programming techniques in real-world microservices
Functional programming techniques in real-world microservicesFunctional programming techniques in real-world microservices
Functional programming techniques in real-world microservices
András Papp
 
Lec 9 05_sept [compatibility mode]
Lec 9 05_sept [compatibility mode]Lec 9 05_sept [compatibility mode]
Lec 9 05_sept [compatibility mode]
Palak Sanghani
 
Extending Gremlin with Foundational Steps
Extending Gremlin with Foundational StepsExtending Gremlin with Foundational Steps
Extending Gremlin with Foundational Steps
Stephen Mallette
 
mobl
moblmobl
Anti patterns
Anti patternsAnti patterns
Anti patterns
Alex Tumanoff
 
User Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love StoryUser Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love Story
Databricks
 
User Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love StoryUser Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love Story
Databricks
 
Java8: Language Enhancements
Java8: Language EnhancementsJava8: Language Enhancements
Java8: Language Enhancements
Yuriy Bondaruk
 
Developing Your Own Flux Packages by David McKay | Head of Developer Relation...
Developing Your Own Flux Packages by David McKay | Head of Developer Relation...Developing Your Own Flux Packages by David McKay | Head of Developer Relation...
Developing Your Own Flux Packages by David McKay | Head of Developer Relation...
InfluxData
 
SANER 2019 Most Influential Paper Talk
SANER 2019 Most Influential Paper TalkSANER 2019 Most Influential Paper Talk
SANER 2019 Most Influential Paper Talk
Nikolaos Tsantalis
 
Chapter 1 Basic Concepts
Chapter 1 Basic ConceptsChapter 1 Basic Concepts
Chapter 1 Basic Concepts
Hareem Aslam
 
Bar graph
Bar graphBar graph
Bar graph
John Reinhardt
 
Functions In Scala
Functions In Scala Functions In Scala
Functions In Scala
Knoldus Inc.
 
Working effectively with legacy code
Working effectively with legacy codeWorking effectively with legacy code
Working effectively with legacy code
ShriKant Vashishtha
 
Apache Flink Training: DataStream API Part 2 Advanced
Apache Flink Training: DataStream API Part 2 Advanced Apache Flink Training: DataStream API Part 2 Advanced
Apache Flink Training: DataStream API Part 2 Advanced
Flink Forward
 
Go Programming Patterns
Go Programming PatternsGo Programming Patterns
Go Programming Patterns
Hao Chen
 
React new features and intro to Hooks
React new features and intro to HooksReact new features and intro to Hooks
React new features and intro to Hooks
Soluto
 

Similar to Flink Forward San Francisco 2018: Seth Wiesman - "Testing Stateful Streaming Applications" (20)

Stratosphere Intro (Java and Scala Interface)
Stratosphere Intro (Java and Scala Interface)Stratosphere Intro (Java and Scala Interface)
Stratosphere Intro (Java and Scala Interface)
 
Lo Mejor Del Pdc2008 El Futrode C#
Lo Mejor Del Pdc2008 El Futrode C#Lo Mejor Del Pdc2008 El Futrode C#
Lo Mejor Del Pdc2008 El Futrode C#
 
Reactive programming every day
Reactive programming every dayReactive programming every day
Reactive programming every day
 
Functional programming techniques in real-world microservices
Functional programming techniques in real-world microservicesFunctional programming techniques in real-world microservices
Functional programming techniques in real-world microservices
 
Lec 9 05_sept [compatibility mode]
Lec 9 05_sept [compatibility mode]Lec 9 05_sept [compatibility mode]
Lec 9 05_sept [compatibility mode]
 
Extending Gremlin with Foundational Steps
Extending Gremlin with Foundational StepsExtending Gremlin with Foundational Steps
Extending Gremlin with Foundational Steps
 
mobl
moblmobl
mobl
 
Anti patterns
Anti patternsAnti patterns
Anti patterns
 
User Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love StoryUser Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love Story
 
User Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love StoryUser Defined Aggregation in Apache Spark: A Love Story
User Defined Aggregation in Apache Spark: A Love Story
 
Java8: Language Enhancements
Java8: Language EnhancementsJava8: Language Enhancements
Java8: Language Enhancements
 
Developing Your Own Flux Packages by David McKay | Head of Developer Relation...
Developing Your Own Flux Packages by David McKay | Head of Developer Relation...Developing Your Own Flux Packages by David McKay | Head of Developer Relation...
Developing Your Own Flux Packages by David McKay | Head of Developer Relation...
 
SANER 2019 Most Influential Paper Talk
SANER 2019 Most Influential Paper TalkSANER 2019 Most Influential Paper Talk
SANER 2019 Most Influential Paper Talk
 
Chapter 1 Basic Concepts
Chapter 1 Basic ConceptsChapter 1 Basic Concepts
Chapter 1 Basic Concepts
 
Bar graph
Bar graphBar graph
Bar graph
 
Functions In Scala
Functions In Scala Functions In Scala
Functions In Scala
 
Working effectively with legacy code
Working effectively with legacy codeWorking effectively with legacy code
Working effectively with legacy code
 
Apache Flink Training: DataStream API Part 2 Advanced
Apache Flink Training: DataStream API Part 2 Advanced Apache Flink Training: DataStream API Part 2 Advanced
Apache Flink Training: DataStream API Part 2 Advanced
 
Go Programming Patterns
Go Programming PatternsGo Programming Patterns
Go Programming Patterns
 
React new features and intro to Hooks
React new features and intro to HooksReact new features and intro to Hooks
React new features and intro to Hooks
 

More from Flink Forward

Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...
Flink Forward
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
Flink Forward
 
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
Flink Forward
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Flink Forward
 
Introducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorIntroducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes Operator
Flink Forward
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive Mode
Flink Forward
 
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Flink Forward
 
One sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkOne sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async Sink
Flink Forward
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptx
Flink Forward
 
Flink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink powered stream processing platform at Pinterest
Flink powered stream processing platform at Pinterest
Flink Forward
 
Apache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraApache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native Era
Flink Forward
 
Where is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in FlinkWhere is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in Flink
Flink Forward
 
Using the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentUsing the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production Deployment
Flink Forward
 
The Current State of Table API in 2022
The Current State of Table API in 2022The Current State of Table API in 2022
The Current State of Table API in 2022
Flink Forward
 
Flink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink SQL on Pulsar made easy
Flink SQL on Pulsar made easy
Flink Forward
 
Dynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsDynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data Alerts
Flink Forward
 
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotExactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Flink Forward
 
Processing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesProcessing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial Services
Flink Forward
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...
Flink Forward
 
Batch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergBatch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & Iceberg
Flink Forward
 

More from Flink Forward (20)

Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
 
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
 
Introducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorIntroducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes Operator
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive Mode
 
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
 
One sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkOne sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async Sink
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptx
 
Flink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink powered stream processing platform at Pinterest
Flink powered stream processing platform at Pinterest
 
Apache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraApache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native Era
 
Where is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in FlinkWhere is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in Flink
 
Using the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentUsing the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production Deployment
 
The Current State of Table API in 2022
The Current State of Table API in 2022The Current State of Table API in 2022
The Current State of Table API in 2022
 
Flink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink SQL on Pulsar made easy
Flink SQL on Pulsar made easy
 
Dynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsDynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data Alerts
 
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotExactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
 
Processing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesProcessing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial Services
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...
 
Batch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergBatch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & Iceberg
 

Recently uploaded

Essentials of Automations: Exploring Attributes & Automation Parameters
Essentials of Automations: Exploring Attributes & Automation ParametersEssentials of Automations: Exploring Attributes & Automation Parameters
Essentials of Automations: Exploring Attributes & Automation Parameters
Safe Software
 
Astute Business Solutions | Oracle Cloud Partner |
Astute Business Solutions | Oracle Cloud Partner |Astute Business Solutions | Oracle Cloud Partner |
Astute Business Solutions | Oracle Cloud Partner |
AstuteBusiness
 
Y-Combinator seed pitch deck template PP
Y-Combinator seed pitch deck template PPY-Combinator seed pitch deck template PP
Y-Combinator seed pitch deck template PP
c5vrf27qcz
 
Nordic Marketo Engage User Group_June 13_ 2024.pptx
Nordic Marketo Engage User Group_June 13_ 2024.pptxNordic Marketo Engage User Group_June 13_ 2024.pptx
Nordic Marketo Engage User Group_June 13_ 2024.pptx
MichaelKnudsen27
 
The Microsoft 365 Migration Tutorial For Beginner.pptx
The Microsoft 365 Migration Tutorial For Beginner.pptxThe Microsoft 365 Migration Tutorial For Beginner.pptx
The Microsoft 365 Migration Tutorial For Beginner.pptx
operationspcvita
 
HCL Notes and Domino License Cost Reduction in the World of DLAU
HCL Notes and Domino License Cost Reduction in the World of DLAUHCL Notes and Domino License Cost Reduction in the World of DLAU
HCL Notes and Domino License Cost Reduction in the World of DLAU
panagenda
 
Programming Foundation Models with DSPy - Meetup Slides
Programming Foundation Models with DSPy - Meetup SlidesProgramming Foundation Models with DSPy - Meetup Slides
Programming Foundation Models with DSPy - Meetup Slides
Zilliz
 
Driving Business Innovation: Latest Generative AI Advancements & Success Story
Driving Business Innovation: Latest Generative AI Advancements & Success StoryDriving Business Innovation: Latest Generative AI Advancements & Success Story
Driving Business Innovation: Latest Generative AI Advancements & Success Story
Safe Software
 
“How Axelera AI Uses Digital Compute-in-memory to Deliver Fast and Energy-eff...
“How Axelera AI Uses Digital Compute-in-memory to Deliver Fast and Energy-eff...“How Axelera AI Uses Digital Compute-in-memory to Deliver Fast and Energy-eff...
“How Axelera AI Uses Digital Compute-in-memory to Deliver Fast and Energy-eff...
Edge AI and Vision Alliance
 
Monitoring and Managing Anomaly Detection on OpenShift.pdf
Monitoring and Managing Anomaly Detection on OpenShift.pdfMonitoring and Managing Anomaly Detection on OpenShift.pdf
Monitoring and Managing Anomaly Detection on OpenShift.pdf
Tosin Akinosho
 
Fueling AI with Great Data with Airbyte Webinar
Fueling AI with Great Data with Airbyte WebinarFueling AI with Great Data with Airbyte Webinar
Fueling AI with Great Data with Airbyte Webinar
Zilliz
 
Your One-Stop Shop for Python Success: Top 10 US Python Development Providers
Your One-Stop Shop for Python Success: Top 10 US Python Development ProvidersYour One-Stop Shop for Python Success: Top 10 US Python Development Providers
Your One-Stop Shop for Python Success: Top 10 US Python Development Providers
akankshawande
 
AppSec PNW: Android and iOS Application Security with MobSF
AppSec PNW: Android and iOS Application Security with MobSFAppSec PNW: Android and iOS Application Security with MobSF
AppSec PNW: Android and iOS Application Security with MobSF
Ajin Abraham
 
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
Alex Pruden
 
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
Edge AI and Vision Alliance
 
Introduction of Cybersecurity with OSS at Code Europe 2024
Introduction of Cybersecurity with OSS  at Code Europe 2024Introduction of Cybersecurity with OSS  at Code Europe 2024
Introduction of Cybersecurity with OSS at Code Europe 2024
Hiroshi SHIBATA
 
Apps Break Data
Apps Break DataApps Break Data
Apps Break Data
Ivo Velitchkov
 
5th LF Energy Power Grid Model Meet-up Slides
5th LF Energy Power Grid Model Meet-up Slides5th LF Energy Power Grid Model Meet-up Slides
5th LF Energy Power Grid Model Meet-up Slides
DanBrown980551
 
Harnessing the Power of NLP and Knowledge Graphs for Opioid Research
Harnessing the Power of NLP and Knowledge Graphs for Opioid ResearchHarnessing the Power of NLP and Knowledge Graphs for Opioid Research
Harnessing the Power of NLP and Knowledge Graphs for Opioid Research
Neo4j
 
Principle of conventional tomography-Bibash Shahi ppt..pptx
Principle of conventional tomography-Bibash Shahi ppt..pptxPrinciple of conventional tomography-Bibash Shahi ppt..pptx
Principle of conventional tomography-Bibash Shahi ppt..pptx
BibashShahi
 

Recently uploaded (20)

Essentials of Automations: Exploring Attributes & Automation Parameters
Essentials of Automations: Exploring Attributes & Automation ParametersEssentials of Automations: Exploring Attributes & Automation Parameters
Essentials of Automations: Exploring Attributes & Automation Parameters
 
Astute Business Solutions | Oracle Cloud Partner |
Astute Business Solutions | Oracle Cloud Partner |Astute Business Solutions | Oracle Cloud Partner |
Astute Business Solutions | Oracle Cloud Partner |
 
Y-Combinator seed pitch deck template PP
Y-Combinator seed pitch deck template PPY-Combinator seed pitch deck template PP
Y-Combinator seed pitch deck template PP
 
Nordic Marketo Engage User Group_June 13_ 2024.pptx
Nordic Marketo Engage User Group_June 13_ 2024.pptxNordic Marketo Engage User Group_June 13_ 2024.pptx
Nordic Marketo Engage User Group_June 13_ 2024.pptx
 
The Microsoft 365 Migration Tutorial For Beginner.pptx
The Microsoft 365 Migration Tutorial For Beginner.pptxThe Microsoft 365 Migration Tutorial For Beginner.pptx
The Microsoft 365 Migration Tutorial For Beginner.pptx
 
HCL Notes and Domino License Cost Reduction in the World of DLAU
HCL Notes and Domino License Cost Reduction in the World of DLAUHCL Notes and Domino License Cost Reduction in the World of DLAU
HCL Notes and Domino License Cost Reduction in the World of DLAU
 
Programming Foundation Models with DSPy - Meetup Slides
Programming Foundation Models with DSPy - Meetup SlidesProgramming Foundation Models with DSPy - Meetup Slides
Programming Foundation Models with DSPy - Meetup Slides
 
Driving Business Innovation: Latest Generative AI Advancements & Success Story
Driving Business Innovation: Latest Generative AI Advancements & Success StoryDriving Business Innovation: Latest Generative AI Advancements & Success Story
Driving Business Innovation: Latest Generative AI Advancements & Success Story
 
“How Axelera AI Uses Digital Compute-in-memory to Deliver Fast and Energy-eff...
“How Axelera AI Uses Digital Compute-in-memory to Deliver Fast and Energy-eff...“How Axelera AI Uses Digital Compute-in-memory to Deliver Fast and Energy-eff...
“How Axelera AI Uses Digital Compute-in-memory to Deliver Fast and Energy-eff...
 
Monitoring and Managing Anomaly Detection on OpenShift.pdf
Monitoring and Managing Anomaly Detection on OpenShift.pdfMonitoring and Managing Anomaly Detection on OpenShift.pdf
Monitoring and Managing Anomaly Detection on OpenShift.pdf
 
Fueling AI with Great Data with Airbyte Webinar
Fueling AI with Great Data with Airbyte WebinarFueling AI with Great Data with Airbyte Webinar
Fueling AI with Great Data with Airbyte Webinar
 
Your One-Stop Shop for Python Success: Top 10 US Python Development Providers
Your One-Stop Shop for Python Success: Top 10 US Python Development ProvidersYour One-Stop Shop for Python Success: Top 10 US Python Development Providers
Your One-Stop Shop for Python Success: Top 10 US Python Development Providers
 
AppSec PNW: Android and iOS Application Security with MobSF
AppSec PNW: Android and iOS Application Security with MobSFAppSec PNW: Android and iOS Application Security with MobSF
AppSec PNW: Android and iOS Application Security with MobSF
 
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
zkStudyClub - LatticeFold: A Lattice-based Folding Scheme and its Application...
 
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
“Temporal Event Neural Networks: A More Efficient Alternative to the Transfor...
 
Introduction of Cybersecurity with OSS at Code Europe 2024
Introduction of Cybersecurity with OSS  at Code Europe 2024Introduction of Cybersecurity with OSS  at Code Europe 2024
Introduction of Cybersecurity with OSS at Code Europe 2024
 
Apps Break Data
Apps Break DataApps Break Data
Apps Break Data
 
5th LF Energy Power Grid Model Meet-up Slides
5th LF Energy Power Grid Model Meet-up Slides5th LF Energy Power Grid Model Meet-up Slides
5th LF Energy Power Grid Model Meet-up Slides
 
Harnessing the Power of NLP and Knowledge Graphs for Opioid Research
Harnessing the Power of NLP and Knowledge Graphs for Opioid ResearchHarnessing the Power of NLP and Knowledge Graphs for Opioid Research
Harnessing the Power of NLP and Knowledge Graphs for Opioid Research
 
Principle of conventional tomography-Bibash Shahi ppt..pptx
Principle of conventional tomography-Bibash Shahi ppt..pptxPrinciple of conventional tomography-Bibash Shahi ppt..pptx
Principle of conventional tomography-Bibash Shahi ppt..pptx
 

Flink Forward San Francisco 2018: Seth Wiesman - "Testing Stateful Streaming Applications"

  • 1. April 10, 2018 Seth Wiesman Testing Stateful Streaming Applications
  • 2. ©2018 MediaMath Inc. 2 val words: DataStream[String] = ??? words .keyBy(word ⇒ word) .map(new CountingMap) Basic Stateful Word Count “hello” → (“hello”, 1) “hello” → (“hello”, 2) “world” → (“world”, 1) “hello” → (“hello”, 3)
  • 3. ©2018 MediaMath Inc. 3 • Stateless jobs are easy to reason about • The same input always results in the same output • Referentially Transparent • That doesn’t mean no state at all! • Stateless:: (IN, STATE) → (OUT, STATE) vs Stateless Jobs
  • 4. ©2018 MediaMath Inc. 4 Operator State
  • 5. ©2018 MediaMath Inc. 5 import java.util.{List ⇒ JList} class ResourceToStream(path: String) extends RichSourceFunction[String] with ListCheckpointed[Integer] { var numLines: Int = 0 override def cancel(): Unit = ??? override def run(ctx: SourceFunction.SourceContext[String]): Unit = ??? override def restoreState(state: JList[Integer]): Unit = ??? override def snapshotState(checkpointId: Long, timestamp: Long): JList[Integer] = ??? }
  • 6. ©2018 MediaMath Inc. 6 import java.util.{List ⇒ JList} class ResourceToStream(path: String) extends RichSourceFunction[String] with ListCheckpointed[Integer] { var numLines: Int = 0 override def cancel(): Unit = ??? override def run(ctx: SourceFunction.SourceContext[String]): Unit = ??? override def restoreState(state: JList[Integer]): Unit = ??? override def snapshotState(checkpointId: Long, timestamp: Long): JList[Integer] = ??? } Business Logic State Management
  • 7. ©2018 MediaMath Inc. 7 it should "process a resource file" in { val runner = new ResourceToStream("/testfile.txt") val ctx = new MockSourceContext runner.run(ctx) ctx.output.size should be (4) ctx.output should be (ListBuffer("hello", "world", "good", "day")) } Validate Business Logic
  • 8. ©2018 MediaMath Inc. 8 it should "restore from state" in { val runner = new ResourceToStream("/testfile.txt") val ctx = new MockSourceContext runner.restoreState(Arrays.asList(2)) runner.run(ctx) ctx.output.size should be (2) ctx.output should be (ListBuffer("good", "day")) } Consider State Management
  • 9. ©2018 MediaMath Inc. 9 val mainStream: DataStream[String] = ??? val filterStream: DataStream[String] = env .addSource(new ResourceToStream(“”)) mainStream .connect(filterStream.broadcast) .flatMap(new BroadcastFilter) Broadcast State
  • 10. ©2018 MediaMath Inc. 10 class BroadcastFilter extends RichCoFlatMapFunction[String, String, String] with ??? { val filters = mutable.Set.empty[String] override def flatMap1(value: String, out: Collector[String]): Unit = { if (!filters.contains(value)) { out.collect(value) } } override def flatMap2(value: String, out: Collector[String]): Unit = { filters += value }
  • 11. ©2018 MediaMath Inc. 11 override def initializeState(context: FunctionInitializationContext): Unit = { state = context.getOperatorStateStore.getUnionListState( new ListStateDescriptor[String]("filters", BasicTypeInfo.STRING_TYPE_INFO) ) ??? } CheckpointedFunction
  • 12. ©2018 MediaMath Inc. 12 override def snapshotState(context: FunctionSnapshotContext): Unit = { state.clear() if (getRuntimeContext.getIndexOfThisSubtask == 0) { for (filter ← filters) { state.add(filter) } } } CheckpointedFunction
  • 13. ©2018 MediaMath Inc. 13 • Testing business logic is still straightforward • Testing checkpoint methods is less clear • Mocking out operator state is not fun CheckpointedFunction it should "filter out blacklisted words" in { val filter = new BroadcastFilter val out = new MockCollector filter.flatMap2("hello", out) out.output.size should be (0) filter.flatMap1("hello", out) out.output.size should be (0) filter.flatMap1("world", out) out.output.size should be (1) }
  • 14. ©2018 MediaMath Inc. 14 • Used as part of Flink’s internal test suite • ⚠️ not part of the public api ⚠️ • Not recommended as a first option • Try and get by with integration tests first • Be careful to test your code, not Flink’s! • Used sparingly, can be very powerful OperatorTestHarness
  • 15. ©2018 MediaMath Inc. 15 it should "restore from state" in { val initialFilter = new BroadcastFilter val initialOperator = new CoStreamFlatMap(initialFilter) val initialTestHarness = new TwoInputStreamOperatorTestHarness(initialOperator) initialTestHarness.initializeState(new OperatorStateHandles(0, null, null, null, null)) initialTestHarness.open() initialTestHarness.processElement2(new StreamRecord[String]("hello")) val snapshot = initialTestHarness.snapshot(0L, 0L) initialTestHarness.close()
  • 16. ©2018 MediaMath Inc. 16 val restoreFilter = new BroadcastFilter val restoreOperator = new CoStreamFlatMap(restoreFilter) val restoreTestHarness = new TwoInputStreamOperatorTestHarness(restoreOperator) restoreTestHarness.initializeState(snapshot) restoreTestHarness.setup() restoreTestHarness.open() restoreTestHarness.processElement1(new StreamRecord[String]("hello")) restoreTestHarness.getOutput.size() should be (0) restoreTestHarness.close() }
  • 17. ©2018 MediaMath Inc. 17 Keyed State
  • 18. ©2018 MediaMath Inc. 18 • Many keyed state operations can be handled “statelessly” Keyed State words .keyBy(word ⇒ word) .countWindow(3) .reduce((word, _) ⇒ word) • Not always an option
  • 19. ©2018 MediaMath Inc. 19 class EveryNthWord(n: Int) extends RichFlatMapFunction[String, String] { @transient private lazy val countDesc = new ValueStateDescriptor("count", classOf[Int]) override def flatMap(value: String, out: Collector[String]): Unit = { val countState = getRuntimeContext.getState(countDesc) val count = Option(countState.value()).map(_.intValue()).getOrElse(1) if (count == n) { countState.clear() out.collect(value) } else { countState.update(count + 1) } } }
  • 20. ©2018 MediaMath Inc. 20 object StatelessEveryNthWord { type StatefulFunction = (String, Option[Int]) ⇒ (TraversableOnce[String], Option[Int]) } FlatMapWithState
  • 21. ©2018 MediaMath Inc. 21 class StatelessEveryNthWord(n: Int) extends StatelessEveryNthWord.StatefulFunction { override def apply(word: String, state: Option[Int]): (TraversableOnce[String], Option[Int])= { state match { case Some(count) if count + 1 == n ⇒ (Some(word), None) case Some(count) ⇒ (None, Some(count + 1)) case None ⇒ (None, Some(1)) } } } FlatMapWithState
  • 22. ©2018 MediaMath Inc. 22 it should "initialize state the first time it sees a word" in { val function = new StatelessEveryNthWord(3) val (output, state) = function("hello", None) output should be (None) state should be (Some(1)) } FlatMapWithState Test it should "modify state in the middle of a run" in { val function = new StatelessEveryNthWord(3) val (output, state) = function("hello", Some(1)) output should be (None) state should be (Some(2)) }
  • 23. ©2018 MediaMath Inc. 23 override def flatMap(value: String, out: Collector[String]): Unit = { val state = Option(getRuntimeContext.getState(countDesc).value()) f(value, state) match { case (output, None) ⇒ output.foreach(out.collect) getRuntimeContext.getState(countDesc).clear() case (output, Some(x)) ⇒ output.foreach(out.collect) getRuntimeContext.getState(countDesc).update(x) } Possible FlatMapWithState Implemention react to results initialize context
  • 24. ©2018 MediaMath Inc. 24 • Group all elements of the same key that occur within 30 seconds of each other • similar to a session window • Output the third instance of every word immediately • When more than 30 seconds have passed since the last word: • clear state • If more than 3 instances of a word have been observed, output to heavy hitter side output • If less than 3 instances of a word have been observed, output to infrequent side output A more complex example
  • 25. ©2018 MediaMath Inc. 25 object Algebra { final case class Context(count: Long, watermark: Long) final case class Result(word: Option[String], timer: Long) } trait Algebra { def evaluateElem(word: String, ctx: Algebra.Context): Algebra.Result def evaluateTimer(word: String, count: Long): Option[Either[String, String]] } Specialized Context
  • 26. ©2018 MediaMath Inc. 26 class StatelessAlgebra extends Algebra { final val private[this] thirty_seconds: Long = 30 * 1000L override def evaluateElem(word: String, ctx: Algebra.Context): Algebra.Result = { val output = if (ctx.count == 3) { Some(word) } else { None } val timer = ctx.watermark + thirty_seconds Algebra.Result(output, timer) } Stateless Business Logic
  • 27. ©2018 MediaMath Inc. 27 override def evaluateTimer(word: String, count: Long): Option[Either[String, String]] = { if (count < 3) { Some(Left(word)) } else if (count > 3) { Some(Right(word)) } else { None } } } Stateless Business Logic
  • 28. ©2018 MediaMath Inc. 28 getRuntimeContext.getState(wordDesc).update(value) getRuntimeContext.getReducingState(countDesc).add(1L) val count = getRuntimeContext.getReducingState(countDesc).get() val context = Algebra.Context(count, ctx.timerService().currentWatermark()) val Algebra.Result(output, time) = program.evaluateElem(value, context) output.foreach(out.collect) getRuntimeContext.getState(timerDesc).update(time) ctx.timerService().registerEventTimeTimer(time) Program Runner react to results initialize context
  • 29. ©2018 MediaMath Inc. 29 • Business Logic • Business logic can be tested in isolation of Flink • Simple to simulate various scenarios • Runner • Does not require knowledge of business domain to validate • Isolates the more complex parts of writing streaming applications Testing our Implementation
  • 30. ©2018 MediaMath Inc. 30 Integration Testing
  • 31. ©2018 MediaMath Inc. 31 • Test the full End-to-End pipeline • More closely simulate production than unit tests • StreamingMultipleProgramsTestBase • Runs a local version of Flink • Beware only testing the happy path • Consider running each integration test twice – both with and without failure • See BucketingSink test suite for an example of failing the stream [1] Integration Testing [1] https://github.com/apache/flink/blob/7b07fa5a09279a55c99d80db92ebf98a7dcd9644/flink-connectors/flink-connector- filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java#L200
  • 32. ©2018 MediaMath Inc. 32 Full Code Examples github.com/sjwiesman/ff-sf-2018 Thank You