SlideShare une entreprise Scribd logo
1  sur  32
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

Contenu connexe

Tendances

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 intelligenceWei 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 streamsRadu Tudoran
 
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 pipelinesLars Albertsson
 
KliqObjects Overview
KliqObjects OverviewKliqObjects Overview
KliqObjects OverviewKT-Labs
 
KliqMap for Esri: Actionable Location Analytics
KliqMap for Esri: Actionable Location AnalyticsKliqMap for Esri: Actionable Location Analytics
KliqMap for Esri: Actionable Location AnalyticsKT-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 AnalyticsItzhak 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 QiuSpark Summit
 
KliqPlan Overview
KliqPlan OverviewKliqPlan Overview
KliqPlan OverviewKT-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 FlinkFabian 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 FlinkTill 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 SparkDatabricks
 
Building real time data-driven products
Building real time data-driven productsBuilding real time data-driven products
Building real time data-driven productsLars Albertsson
 

Tendances (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
 

Similaire à 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 dayVadym 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 microservicesAndrá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 StepsStephen Mallette
 
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 StoryDatabricks
 
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 StoryDatabricks
 
Java8: Language Enhancements
Java8: Language EnhancementsJava8: Language Enhancements
Java8: Language EnhancementsYuriy 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 TalkNikolaos Tsantalis
 
Chapter 1 Basic Concepts
Chapter 1 Basic ConceptsChapter 1 Basic Concepts
Chapter 1 Basic ConceptsHareem Aslam
 
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 codeShriKant 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 PatternsHao 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 HooksSoluto
 

Similaire à 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
 

Plus de 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 FlinkFlink 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 OperatorFlink Forward
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeFlink 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 SinkFlink 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.pptxFlink 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 PinterestFlink 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 EraFlink 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 FlinkFlink 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 DeploymentFlink 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 2022Flink Forward
 
Flink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink 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 AlertsFlink 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 PinotFlink 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 ServicesFlink 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 & IcebergFlink Forward
 

Plus de 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
 

Dernier

Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...Neo4j
 
Presentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreterPresentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreternaman860154
 
A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)Gabriella Davis
 
Breaking the Kubernetes Kill Chain: Host Path Mount
Breaking the Kubernetes Kill Chain: Host Path MountBreaking the Kubernetes Kill Chain: Host Path Mount
Breaking the Kubernetes Kill Chain: Host Path MountPuma Security, LLC
 
08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking MenDelhi Call girls
 
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Drew Madelung
 
Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024The Digital Insurer
 
Developing An App To Navigate The Roads of Brazil
Developing An App To Navigate The Roads of BrazilDeveloping An App To Navigate The Roads of Brazil
Developing An App To Navigate The Roads of BrazilV3cube
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...Martijn de Jong
 
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...apidays
 
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationSafe Software
 
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking MenDelhi Call girls
 
Histor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slideHistor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slidevu2urc
 
Kalyanpur ) Call Girls in Lucknow Finest Escorts Service 🍸 8923113531 🎰 Avail...
Kalyanpur ) Call Girls in Lucknow Finest Escorts Service 🍸 8923113531 🎰 Avail...Kalyanpur ) Call Girls in Lucknow Finest Escorts Service 🍸 8923113531 🎰 Avail...
Kalyanpur ) Call Girls in Lucknow Finest Escorts Service 🍸 8923113531 🎰 Avail...gurkirankumar98700
 
Exploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone ProcessorsExploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone Processorsdebabhi2
 
Partners Life - Insurer Innovation Award 2024
Partners Life - Insurer Innovation Award 2024Partners Life - Insurer Innovation Award 2024
Partners Life - Insurer Innovation Award 2024The Digital Insurer
 
Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024The Digital Insurer
 
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure serviceWhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure servicePooja Nehwal
 
Data Cloud, More than a CDP by Matt Robison
Data Cloud, More than a CDP by Matt RobisonData Cloud, More than a CDP by Matt Robison
Data Cloud, More than a CDP by Matt RobisonAnna Loughnan Colquhoun
 
CNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of ServiceCNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of Servicegiselly40
 

Dernier (20)

Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
 
Presentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreterPresentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreter
 
A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)
 
Breaking the Kubernetes Kill Chain: Host Path Mount
Breaking the Kubernetes Kill Chain: Host Path MountBreaking the Kubernetes Kill Chain: Host Path Mount
Breaking the Kubernetes Kill Chain: Host Path Mount
 
08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men
 
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
 
Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024
 
Developing An App To Navigate The Roads of Brazil
Developing An App To Navigate The Roads of BrazilDeveloping An App To Navigate The Roads of Brazil
Developing An App To Navigate The Roads of Brazil
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...
 
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
 
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
 
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
 
Histor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slideHistor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slide
 
Kalyanpur ) Call Girls in Lucknow Finest Escorts Service 🍸 8923113531 🎰 Avail...
Kalyanpur ) Call Girls in Lucknow Finest Escorts Service 🍸 8923113531 🎰 Avail...Kalyanpur ) Call Girls in Lucknow Finest Escorts Service 🍸 8923113531 🎰 Avail...
Kalyanpur ) Call Girls in Lucknow Finest Escorts Service 🍸 8923113531 🎰 Avail...
 
Exploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone ProcessorsExploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone Processors
 
Partners Life - Insurer Innovation Award 2024
Partners Life - Insurer Innovation Award 2024Partners Life - Insurer Innovation Award 2024
Partners Life - Insurer Innovation Award 2024
 
Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024
 
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure serviceWhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
 
Data Cloud, More than a CDP by Matt Robison
Data Cloud, More than a CDP by Matt RobisonData Cloud, More than a CDP by Matt Robison
Data Cloud, More than a CDP by Matt Robison
 
CNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of ServiceCNv6 Instructor Chapter 6 Quality of Service
CNv6 Instructor Chapter 6 Quality of Service
 

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