SlideShare a Scribd company logo
1 of 34
Download to read offline
Introducing
BinarySortedState
A new Flink state
primitive to boost
your application
performance
Nico Kruber
–
Software
Engineer
——
David Anderson
–
Community
Engineering
–
Flink Forward
22
About me
Open source
● Apache Flink contributor/committer since 2016
○ Focus on network stack, usability, and performance
Career
● PhD in Computer Science at HU Berlin / Zuse Institute Berlin
● Software engineer -> Solutions architect -> Head of Solutions Architecture
@ DataArtisans/Ververica (acquired by Alibaba)
● Engineering @ Immerok
About Immerok
● Building a fully managed Apache Flink cloud service
for powering real-time systems at any scale
○ immerok.com
2
Agenda
● Motivation
● BinarySortedState
● Results & Future Work
Motivation
Use Case: Stream Sort
20
15
11
30 35
27
21
40
41
35
1
2
5
7 6
9
10
11
13
14
Use Case: Stream Sort - Code
void processElement(Long event, /*...*/) {
TimerService timerSvc = ctx.timerService();
long ts = ctx.timestamp();
if (!isLate(ts, timerSvc)) {
List<Long> listAtTs = events.get(ts);
if (listAtTs == null) {
listAtTs = new ArrayList<>();
}
listAtTs.add(event);
events.put(ts, listAtTs);
timerSvc.registerEventTimeTimer(ts);
}
}
MapState<Long, List<Long>> events;
MapStateDescriptor<Long, List<Long>> desc =
new MapStateDescriptor<>(
"Events",
Types.LONG,
Types.LIST(Types.LONG));
events = getRuntimeContext().getMapState(desc);
void onTimer(long ts, /*...*/) {
events.get(ts).forEach(out::collect);
events.remove(ts);
}
List<Long> listAtTs = events.get(ts);
if (listAtTs == null) {
listAtTs = new ArrayList<>();
}
listAtTs.add(event);
events.put(ts, listAtTs);
Use Case: Stream Sort - What’s
Happening Underneath
State (RocksDB)
De-/Serializer
full list
as byte[]
search memtable
+ sst files
for one entry
lookup
deserialized
(Java) list
List<Long> listAtTs = events.get(ts);
if (listAtTs == null) {
listAtTs = new ArrayList<>();
}
listAtTs.add(event);
events.put(ts, listAtTs);
Use Case: Stream Sort - What’s
Happening Underneath
State (RocksDB)
De-/Serializer
full Java list
serialized list
as byte[]
add new entry
to memtable
(leave old one for
compaction)
Use Case: Stream Sort - Alternative Solutions
● Using MapState<Long, Event> instead of MapState<Long, List<Event>>?
○ Cannot handle multiple events per timestamp
● Using Window API?
○ Efficient event storage per timestamp
○ No really well-matching window types: sliding, tumbling, and session windows
● Using HashMapStateBackend?
○ No de-/serialization overhead
○ state limited by available memory
○ no incremental checkpoints
● Using ListState<Event> and filtering in onTimer()?
○ Reduced overhead in processElement() vs. more to do in onTimer()
{ts: 5, code: GBP,
rate: 1.20}
{ts: 10, code: USD,
rate: 1.00}
{ts: 19, code: USD,
rate: 1.02}
rates
{ts: 15, code: USD,
amount: 1.00}
{ts: 10, code: GBP,
amount: 2.00}
{ts: 25, code: USD,
amount: 3.00}
transactions
Use Case: Event-Time Stream Join
{ts1: 15, ts2: 10,
amount: 1.00}
{ts1: 10, ts2: 5,
amount: 2.40}
{ts1: 25, ts2: 19,
amount: 3.06}
SELECT
t.ts AS ts1, r.ts AS ts2,
t.amount * r.rate AS amount
FROM transactions AS t
LEFT JOIN rates
FOR SYSTEM_TIME AS OF t.ts AS r
ON t.code = r.code;
void onTimer(long ts, /*...*/) {
TreeSet<Entry<Long, Double>> rates =
ratesInRange(ts, rate.entries());
Double myRate = getLast(rates);
transactions.get(ts).forEach(
tx -> out.collect(
new Joined(myRate, tx)));
deleteAllButLast(rates);
transactions.remove(ts);
}
Use Case: Event-Time Stream Join - Code
void processElement1(Transaction value,/*...*/) {
TimerService timerSvc = ctx.timerService();
long ts = ctx.timestamp();
if (!isLate(ts, timerSvc)) {
addTransaction(ts, value);
timerSvc.registerEventTimeTimer(ts);
}
}
// similar for processElement2()
MapState<Long, List<Transaction>>
transactions;
MapState<Long, Double> rate;
void onTimer(long ts, /*...*/) {
TreeSet<Entry<Long, Double>> rates =
ratesInRange(ts, rate.entries());
Double myRate = getLast(rates);
transactions.get(ts).forEach(
tx -> out.collect(
new Joined(myRate, tx)));
deleteAllButLast(rates);
transactions.remove(ts);
}
With RocksDB:
● always fetching all rates’ (key+value) bytes ⚠
● (also need to fit into memory ⚠)
● deserialize all keys keys during iteration ⚠
○ not deserializing values (at least) ✓
Use Case: Event-Time Stream Join - Code
void processElement1(Transaction value,/*...*/) {
TimerService timerSvc = ctx.timerService();
long ts = ctx.timestamp();
if (!isLate(ts, timerSvc)) {
addTransaction(ts, value);
timerSvc.registerEventTimeTimer(ts);
}
}
// similar for processElement2()
Similar to stream sort, with RocksDB:
● always fetch/write full lists ⚠
● always de-/serialize full list ⚠
● additional stress on RocksDB compaction ⚠
MapState<Long, List<Transaction>>
transactions;
MapState<Long, Double> rate;
BinarySortedState
BinarySortedState - History
“Temporal state” Hackathon project (David + Nico + Seth)
● Main primitives: getAt(), getAtOrBefore(), getAtOrAfter(), add(),
addAll(), update()
2020
Nov 2021
April 2022
started as a Hackathon project (David + Nico) on
custom windowing with process functions
Created FLIP-220 and discussed on dev@flink.apache.org
● Extended scope further to allow arbitrary user keys (not just timestamps)
○ Identified further use cases in SQL operators, e.g. min/max with retractions
● Clarified serializer requirements
● Extend proposed API to offer range read and clear operations
● …
● A new keyed-state primitive, built on top of ListState
● Efficiently add to list of values for a user-provided key
● Efficiently iterate user-keys in a well-defined sort order,
with native state-backend support, especially RocksDB
● Efficient operations for time-based functions
(windowing, sorting, event-time joins, custom, ...)
● Operations on subset of the state, based on user-key ranges
● Portable between state backends (RocksDB, HashMap)
BinarySortedState - Goals
BinarySortedState - API (subject to change!)
● Point-operations:
○ valuesAt(key)
○ add(key, value)
○ put(key, values)
○ clearEntryAt(key)
● Lookups:
○ firstEntry(), firstEntry(fromKey)
○ lastEntry(), lastEntry(UK toKey)
● Cleanup:
○ clear()
● Range-operations:
○ readRange(fromKey, toKey,
inclusiveToKey)
○ readRangeUntil(toKey, inclusiveToKey)
○ readRangeFrom(fromKey)
● Range-deletes:
○ clearEntryAt(key)
○ clearRange(
fromKey, toKey, inclusiveToKey)
○ clearRangeUntil(toKey, inclusiveToKey)
○ clearRangeFrom(fromKey)
BinarySortedState<UK, UV>
● RocksDB is a key-value store writing into MemTables → flushing into SST files
● SST files are sorted by the key in lexicographical binary order
(byte-wise unsigned comparison)
BinarySortedState - How does it work with RocksDB?!
● RocksDB offers Prefix Seek and SeekForPrev
● RocksDBMapState.RocksDBMapIterator provides efficient iteration via:
○ Fetching up to 128 RocksDB entries at once
○ RocksDBMapEntry with lazy key/value deserialization
BinarySortedState - LexicographicalTypeSerializer
● “Just” need to provide serializers that are compatible with RocksDB’s sort order
● Based on lexicographical binary order as defined by byte-wise unsigned comparison
● Compatible serializers extend LexicographicalTypeSerializer
public abstract class LexicographicalTypeSerializer<T> extends TypeSerializer<T> {
public Optional<Comparator<T>> findComparator() { return Optional.empty(); }
}
Stream Sort w/out BinarySortedState (1)
private BinarySortedState<Long, Long> events;
BinarySortedStateDescriptor<Long, Long> desc =
new BinarySortedStateDescriptor<>(
"Events",
LexicographicLongSerializer.INSTANCE,
Types.LONG);
events = getRuntimeContext()
.getBinarySortedState(desc);
private MapState<Long, List<Long>> events;
MapStateDescriptor<Long, List<Long>> desc =
new MapStateDescriptor<>(
"Events",
Types.LONG,
Types.LIST(Types.LONG));
events = getRuntimeContext()
.getMapState(desc);
public void onTimer(long ts, /*...*/) {
events.valuesAt(ts).forEach(out::collect);
events.clearEntryAt(ts);
}
public void onTimer(long ts, /*...*/) {
events.get(ts).forEach(out::collect);
events.remove(ts);
}
Stream Sort w/out BinarySortedState (2)
public void processElement(/*...*/) {
// ...
events.add(ts, event);
// ...
}
public void processElement(/*...*/) {
// ...
List<Long> listAtTs = events.get(ts);
if (listAtTs == null) {
listAtTs = new ArrayList<>();
}
listAtTs.add(event);
events.put(ts, listAtTs);
// ...
}
events.add(ts, event);
Stream Sort with
BinarySortedState - What’s
Happening Underneath
State (RocksDB)
De-/Serializer
add new merge
op to memtable
Serialized
entry as byte[]
new list
entry
Stream Sort with
BinarySortedState - What’s
Happening Underneath
State (RocksDB)
De-/Serializer
full list
as byte[]
Lookup
deserialized
(Java) list
search memtable
+ sst files
for all entries
events.valuesAt(ts).forEach(out::collect);
Mark k/v as
deleted
(removal during
compaction)
Stream Sort with
BinarySortedState - What’s
Happening Underneath
State (RocksDB)
De-/Serializer
events.clearEntryAt(ts);
Delete
Event-Time Stream Join w/out BinarySortedState (1)
private BinarySortedState<Long, Transaction>
transactions;
private BinarySortedState<Long, Double> rate;
private MapState<Long, List<Transaction>>
transactions;
private MapState<Long, Double> rate;
public void processElement1(/*...*/) {
// ...
if (!isLate(ts, timerSvc)) {
// append to BinarySortedState:
transactions.add(ts, value);
timerSvc.registerEventTimeTimer(ts);
}
}
// similar for processElement2()
public void processElement1(/*...*/) {
// ...
if (!isLate(ts, timerSvc)) {
// replace list in MapState:
addTransaction(ts, value);
timerSvc.registerEventTimeTimer(ts);
}
}
// similar for processElement2()
Event-Time Stream Join w/out BinarySortedState (2)
public void onTimer(long ts, /*...*/) {
Entry<Long, Iterable<Double>> myRate =
rate.lastEntry(ts);
Double rateVal = Optional
.ofNullable(myRate)
.map(e -> e.getValue().iterator().next())
.orElse(null);
transactions.valuesAt(ts).forEach(
tx -> out.collect(
new Joined(rateVal, tx)));
if (myRate != null) {
rate.clearRangeUntil(myRate.getKey(), false);
}
transactions.clearEntryAt(ts);
}
public void onTimer(long ts, /*...*/) {
TreeSet<Entry<Long, Double>> rates =
ratesInRange(ts, rate.entries());
Double myRate = Optional
.ofNullable(rates.pollLast())
.map(Entry::getValue)
.orElse(null);
transactions.get(ts).forEach(
tx -> out.collect(
new Joined(myRate, tx)));
rates.forEach(this::removeRate);
transactions.remove(ts);
}
Stream Sort with BinarySortedState - Optimized
● Idea:
○ Increase efficiency by processing all events between watermarks
● Challenge:
○ Registering a timer for the next watermark will fire too often
➔ Solution:
○ Register timer for the first unprocessed event
○ When the timer fires:
■ Process all events until the current watermark (not the timer timestamp!)
● events.readRangeUntil(currentWatermark, true)
Event-Time Stream Join with BinarySortedState - Optimized
● Idea:
○ Increase efficiency by processing all events between watermarks
● Challenge:
○ Registering a timer for the next watermark will fire too often
➔ Solution:
○ Same as Stream Sort: Timer for first unprocessed event, processing until watermark, but:
○ When the timer fires:
■ Iterate both, transactions and rate (in the appropriate time range) in event-time order
Results & Future Work
+50%
Stream Sort
+130%
Optimized
Stream Sort
+60%
Stream Join
● (Custom) stream sorting
● Time-based (custom) joins
● Code with range-reads or bulk-deletes
● Custom window implementations
● Min/Max with retractions
● …
● Basically everything maintaining a MapState<?, List<?>> or requiring range operations
BinarySortedState - Who will benefit?
What’s left to do?
● Iron out last bits and pieces + tests
○ Start voting thread for FLIP-220 on dev@flink.apache.org
○ Create a PR and get it merged
● Expected to land in Flink 1.17 (as experimental feature)
● Port Table/SQL/DataStream operators to improve efficiency:
○ TemporalRowTimeJoinOperator (PoC already done for validating the API ✓)
○ RowTimeSortOperator
○ IntervalJoinOperator
○ CepOperator
○ …
● Provide more LexicographicalTypeSerializers
Get ready to ROK!!!
Nico Kruber
linkedin.com/in/nico-kruber
nico@immerok.com

More Related Content

What's hot

Apache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
Apache Spark Data Source V2 with Wenchen Fan and Gengliang WangApache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
Apache Spark Data Source V2 with Wenchen Fan and Gengliang WangDatabricks
 
“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
 
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
 
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
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internalsKostas Tzoumas
 
Extending Flink SQL for stream processing use cases
Extending Flink SQL for stream processing use casesExtending Flink SQL for stream processing use cases
Extending Flink SQL for stream processing use casesFlink Forward
 
Changelog Stream Processing with Apache Flink
Changelog Stream Processing with Apache FlinkChangelog Stream Processing with Apache Flink
Changelog Stream Processing with Apache FlinkFlink Forward
 
Apache Spark At Scale in the Cloud
Apache Spark At Scale in the CloudApache Spark At Scale in the Cloud
Apache Spark At Scale in the CloudDatabricks
 
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...Flink Forward
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Databricks
 
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
 
Enabling Vectorized Engine in Apache Spark
Enabling Vectorized Engine in Apache SparkEnabling Vectorized Engine in Apache Spark
Enabling Vectorized Engine in Apache SparkKazuaki Ishizaki
 
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...Flink Forward
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introductioncolorant
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDatabricks
 
A Deep Dive into Kafka Controller
A Deep Dive into Kafka ControllerA Deep Dive into Kafka Controller
A Deep Dive into Kafka Controllerconfluent
 
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in SparkSpark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in SparkBo Yang
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaJiangjie Qin
 
Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Cloudera, Inc.
 

What's hot (20)

Apache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
Apache Spark Data Source V2 with Wenchen Fan and Gengliang WangApache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
Apache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
 
“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...
 
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
 
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
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
 
Extending Flink SQL for stream processing use cases
Extending Flink SQL for stream processing use casesExtending Flink SQL for stream processing use cases
Extending Flink SQL for stream processing use cases
 
Changelog Stream Processing with Apache Flink
Changelog Stream Processing with Apache FlinkChangelog Stream Processing with Apache Flink
Changelog Stream Processing with Apache Flink
 
Apache Spark At Scale in the Cloud
Apache Spark At Scale in the CloudApache Spark At Scale in the Cloud
Apache Spark At Scale in the Cloud
 
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
 
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
 
Intro to Apache Spark
Intro to Apache SparkIntro to Apache Spark
Intro to Apache Spark
 
Enabling Vectorized Engine in Apache Spark
Enabling Vectorized Engine in Apache SparkEnabling Vectorized Engine in Apache Spark
Enabling Vectorized Engine in Apache Spark
 
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
A Deep Dive into Kafka Controller
A Deep Dive into Kafka ControllerA Deep Dive into Kafka Controller
A Deep Dive into Kafka Controller
 
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in SparkSpark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
Spark Shuffle Deep Dive (Explained In Depth) - How Shuffle Works in Spark
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache Kafka
 
Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0
 

Similar to Introducing BinarySortedMultiMap - A new Flink state primitive to boost your application performance

Lambda at Weather Scale - Cassandra Summit 2015
Lambda at Weather Scale - Cassandra Summit 2015Lambda at Weather Scale - Cassandra Summit 2015
Lambda at Weather Scale - Cassandra Summit 2015Robbie Strickland
 
OpenTSDB 2.0
OpenTSDB 2.0OpenTSDB 2.0
OpenTSDB 2.0HBaseCon
 
Norikra: SQL Stream Processing In Ruby
Norikra: SQL Stream Processing In RubyNorikra: SQL Stream Processing In Ruby
Norikra: SQL Stream Processing In RubySATOSHI TAGOMORI
 
Application Monitoring using Open Source: VictoriaMetrics - ClickHouse
Application Monitoring using Open Source: VictoriaMetrics - ClickHouseApplication Monitoring using Open Source: VictoriaMetrics - ClickHouse
Application Monitoring using Open Source: VictoriaMetrics - ClickHouseVictoriaMetrics
 
Application Monitoring using Open Source - VictoriaMetrics & Altinity ClickHo...
Application Monitoring using Open Source - VictoriaMetrics & Altinity ClickHo...Application Monitoring using Open Source - VictoriaMetrics & Altinity ClickHo...
Application Monitoring using Open Source - VictoriaMetrics & Altinity ClickHo...Altinity Ltd
 
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDKBigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDKnagachika t
 
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...WSO2
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17spark-project
 
RMLL 2013 - Synchronize OpenLDAP and Active Directory with LSC
RMLL 2013 - Synchronize OpenLDAP and Active Directory with LSCRMLL 2013 - Synchronize OpenLDAP and Active Directory with LSC
RMLL 2013 - Synchronize OpenLDAP and Active Directory with LSCClément OUDOT
 
Webinar about Spring Data Neo4j 4
Webinar about Spring Data Neo4j 4Webinar about Spring Data Neo4j 4
Webinar about Spring Data Neo4j 4GraphAware
 
SparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsSparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsDatabricks
 
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...GeeksLab Odessa
 
Introducing TiDB [Delivered: 09/27/18 at NYC SQL Meetup]
Introducing TiDB [Delivered: 09/27/18 at NYC SQL Meetup]Introducing TiDB [Delivered: 09/27/18 at NYC SQL Meetup]
Introducing TiDB [Delivered: 09/27/18 at NYC SQL Meetup]Kevin Xu
 
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...WSO2
 
WSO2 Analytics Platform: The one stop shop for all your data needs
WSO2 Analytics Platform: The one stop shop for all your data needsWSO2 Analytics Platform: The one stop shop for all your data needs
WSO2 Analytics Platform: The one stop shop for all your data needsSriskandarajah Suhothayan
 
BDA403 How Netflix Monitors Applications in Real-time with Amazon Kinesis
BDA403 How Netflix Monitors Applications in Real-time with Amazon KinesisBDA403 How Netflix Monitors Applications in Real-time with Amazon Kinesis
BDA403 How Netflix Monitors Applications in Real-time with Amazon KinesisAmazon Web Services
 
Real-Time ETL in Practice with WSO2 Enterprise Integrator
Real-Time ETL in Practice with WSO2 Enterprise IntegratorReal-Time ETL in Practice with WSO2 Enterprise Integrator
Real-Time ETL in Practice with WSO2 Enterprise IntegratorWSO2
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's comingDatabricks
 

Similar to Introducing BinarySortedMultiMap - A new Flink state primitive to boost your application performance (20)

Lambda at Weather Scale - Cassandra Summit 2015
Lambda at Weather Scale - Cassandra Summit 2015Lambda at Weather Scale - Cassandra Summit 2015
Lambda at Weather Scale - Cassandra Summit 2015
 
So you think you can stream.pptx
So you think you can stream.pptxSo you think you can stream.pptx
So you think you can stream.pptx
 
OpenTSDB 2.0
OpenTSDB 2.0OpenTSDB 2.0
OpenTSDB 2.0
 
Norikra: SQL Stream Processing In Ruby
Norikra: SQL Stream Processing In RubyNorikra: SQL Stream Processing In Ruby
Norikra: SQL Stream Processing In Ruby
 
Application Monitoring using Open Source: VictoriaMetrics - ClickHouse
Application Monitoring using Open Source: VictoriaMetrics - ClickHouseApplication Monitoring using Open Source: VictoriaMetrics - ClickHouse
Application Monitoring using Open Source: VictoriaMetrics - ClickHouse
 
Application Monitoring using Open Source - VictoriaMetrics & Altinity ClickHo...
Application Monitoring using Open Source - VictoriaMetrics & Altinity ClickHo...Application Monitoring using Open Source - VictoriaMetrics & Altinity ClickHo...
Application Monitoring using Open Source - VictoriaMetrics & Altinity ClickHo...
 
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDKBigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
 
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...
WSO2Con USA 2015: WSO2 Analytics Platform - The One Stop Shop for All Your Da...
 
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17Deep Dive with Spark Streaming - Tathagata  Das - Spark Meetup 2013-06-17
Deep Dive with Spark Streaming - Tathagata Das - Spark Meetup 2013-06-17
 
RMLL 2013 - Synchronize OpenLDAP and Active Directory with LSC
RMLL 2013 - Synchronize OpenLDAP and Active Directory with LSCRMLL 2013 - Synchronize OpenLDAP and Active Directory with LSC
RMLL 2013 - Synchronize OpenLDAP and Active Directory with LSC
 
Spark streaming: Best Practices
Spark streaming: Best PracticesSpark streaming: Best Practices
Spark streaming: Best Practices
 
Webinar about Spring Data Neo4j 4
Webinar about Spring Data Neo4j 4Webinar about Spring Data Neo4j 4
Webinar about Spring Data Neo4j 4
 
SparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDsSparkSQL: A Compiler from Queries to RDDs
SparkSQL: A Compiler from Queries to RDDs
 
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...
 
Introducing TiDB [Delivered: 09/27/18 at NYC SQL Meetup]
Introducing TiDB [Delivered: 09/27/18 at NYC SQL Meetup]Introducing TiDB [Delivered: 09/27/18 at NYC SQL Meetup]
Introducing TiDB [Delivered: 09/27/18 at NYC SQL Meetup]
 
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...
WSO2Con ASIA 2016: WSO2 Analytics Platform: The One Stop Shop for All Your Da...
 
WSO2 Analytics Platform: The one stop shop for all your data needs
WSO2 Analytics Platform: The one stop shop for all your data needsWSO2 Analytics Platform: The one stop shop for all your data needs
WSO2 Analytics Platform: The one stop shop for all your data needs
 
BDA403 How Netflix Monitors Applications in Real-time with Amazon Kinesis
BDA403 How Netflix Monitors Applications in Real-time with Amazon KinesisBDA403 How Netflix Monitors Applications in Real-time with Amazon Kinesis
BDA403 How Netflix Monitors Applications in Real-time with Amazon Kinesis
 
Real-Time ETL in Practice with WSO2 Enterprise Integrator
Real-Time ETL in Practice with WSO2 Enterprise IntegratorReal-Time ETL in Practice with WSO2 Enterprise Integrator
Real-Time ETL in Practice with WSO2 Enterprise Integrator
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's coming
 

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
 
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
 
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
 
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
 
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
 
Welcome to the Flink Community!
Welcome to the Flink Community!Welcome to the Flink Community!
Welcome to the Flink Community!Flink Forward
 
Practical learnings from running thousands of Flink jobs
Practical learnings from running thousands of Flink jobsPractical learnings from running thousands of Flink jobs
Practical learnings from running thousands of Flink jobsFlink Forward
 
The top 3 challenges running multi-tenant Flink at scale
The top 3 challenges running multi-tenant Flink at scaleThe top 3 challenges running multi-tenant Flink at scale
The top 3 challenges running multi-tenant Flink at scaleFlink Forward
 
Large Scale Real Time Fraudulent Web Behavior Detection
Large Scale Real Time Fraudulent Web Behavior DetectionLarge Scale Real Time Fraudulent Web Behavior Detection
Large Scale Real Time Fraudulent Web Behavior DetectionFlink Forward
 
Building Reliable Lakehouses with Apache Flink and Delta Lake
Building Reliable Lakehouses with Apache Flink and Delta LakeBuilding Reliable Lakehouses with Apache Flink and Delta Lake
Building Reliable Lakehouses with Apache Flink and Delta LakeFlink Forward
 
Near real-time statistical modeling and anomaly detection using Flink!
Near real-time statistical modeling and anomaly detection using Flink!Near real-time statistical modeling and anomaly detection using Flink!
Near real-time statistical modeling and anomaly detection using Flink!Flink Forward
 
How to build a streaming Lakehouse with Flink, Kafka, and Hudi
How to build a streaming Lakehouse with Flink, Kafka, and HudiHow to build a streaming Lakehouse with Flink, Kafka, and Hudi
How to build a streaming Lakehouse with Flink, Kafka, and HudiFlink Forward
 

More from Flink Forward (15)

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...
 
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 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
 
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
 
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
 
Welcome to the Flink Community!
Welcome to the Flink Community!Welcome to the Flink Community!
Welcome to the Flink Community!
 
Practical learnings from running thousands of Flink jobs
Practical learnings from running thousands of Flink jobsPractical learnings from running thousands of Flink jobs
Practical learnings from running thousands of Flink jobs
 
The top 3 challenges running multi-tenant Flink at scale
The top 3 challenges running multi-tenant Flink at scaleThe top 3 challenges running multi-tenant Flink at scale
The top 3 challenges running multi-tenant Flink at scale
 
Large Scale Real Time Fraudulent Web Behavior Detection
Large Scale Real Time Fraudulent Web Behavior DetectionLarge Scale Real Time Fraudulent Web Behavior Detection
Large Scale Real Time Fraudulent Web Behavior Detection
 
Building Reliable Lakehouses with Apache Flink and Delta Lake
Building Reliable Lakehouses with Apache Flink and Delta LakeBuilding Reliable Lakehouses with Apache Flink and Delta Lake
Building Reliable Lakehouses with Apache Flink and Delta Lake
 
Near real-time statistical modeling and anomaly detection using Flink!
Near real-time statistical modeling and anomaly detection using Flink!Near real-time statistical modeling and anomaly detection using Flink!
Near real-time statistical modeling and anomaly detection using Flink!
 
How to build a streaming Lakehouse with Flink, Kafka, and Hudi
How to build a streaming Lakehouse with Flink, Kafka, and HudiHow to build a streaming Lakehouse with Flink, Kafka, and Hudi
How to build a streaming Lakehouse with Flink, Kafka, and Hudi
 

Recently uploaded

How to write a Business Continuity Plan
How to write a Business Continuity PlanHow to write a Business Continuity Plan
How to write a Business Continuity PlanDatabarracks
 
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptxUse of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptxLoriGlavin3
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxLoriGlavin3
 
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxThe Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxLoriGlavin3
 
SAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxSAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxNavinnSomaal
 
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptx
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptxMerck Moving Beyond Passwords: FIDO Paris Seminar.pptx
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptxLoriGlavin3
 
Streamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupStreamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupFlorian Wilhelm
 
The Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsThe Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsPixlogix Infotech
 
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024BookNet Canada
 
Scanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsScanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsRizwan Syed
 
Developer Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLDeveloper Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLScyllaDB
 
WordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your BrandWordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your Brandgvaughan
 
How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.Curtis Poe
 
DevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenDevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenHervé Boutemy
 
"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr BaganFwdays
 
Commit 2024 - Secret Management made easy
Commit 2024 - Secret Management made easyCommit 2024 - Secret Management made easy
Commit 2024 - Secret Management made easyAlfredo García Lavilla
 
Advanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionAdvanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionDilum Bandara
 
Take control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteTake control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteDianaGray10
 
Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Commit University
 

Recently uploaded (20)

How to write a Business Continuity Plan
How to write a Business Continuity PlanHow to write a Business Continuity Plan
How to write a Business Continuity Plan
 
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptxUse of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
 
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxThe Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
 
SAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxSAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptx
 
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptx
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptxMerck Moving Beyond Passwords: FIDO Paris Seminar.pptx
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptx
 
Streamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupStreamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project Setup
 
The Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsThe Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and Cons
 
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
 
Scanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsScanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL Certs
 
Developer Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQLDeveloper Data Modeling Mistakes: From Postgres to NoSQL
Developer Data Modeling Mistakes: From Postgres to NoSQL
 
WordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your BrandWordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your Brand
 
How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.How AI, OpenAI, and ChatGPT impact business and software.
How AI, OpenAI, and ChatGPT impact business and software.
 
DevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenDevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache Maven
 
"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan
 
Commit 2024 - Secret Management made easy
Commit 2024 - Secret Management made easyCommit 2024 - Secret Management made easy
Commit 2024 - Secret Management made easy
 
Advanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionAdvanced Computer Architecture – An Introduction
Advanced Computer Architecture – An Introduction
 
DMCC Future of Trade Web3 - Special Edition
DMCC Future of Trade Web3 - Special EditionDMCC Future of Trade Web3 - Special Edition
DMCC Future of Trade Web3 - Special Edition
 
Take control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteTake control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test Suite
 
Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!
 

Introducing BinarySortedMultiMap - A new Flink state primitive to boost your application performance

  • 1. Introducing BinarySortedState A new Flink state primitive to boost your application performance Nico Kruber – Software Engineer —— David Anderson – Community Engineering – Flink Forward 22
  • 2. About me Open source ● Apache Flink contributor/committer since 2016 ○ Focus on network stack, usability, and performance Career ● PhD in Computer Science at HU Berlin / Zuse Institute Berlin ● Software engineer -> Solutions architect -> Head of Solutions Architecture @ DataArtisans/Ververica (acquired by Alibaba) ● Engineering @ Immerok About Immerok ● Building a fully managed Apache Flink cloud service for powering real-time systems at any scale ○ immerok.com 2
  • 5. Use Case: Stream Sort 20 15 11 30 35 27 21 40 41 35 1 2 5 7 6 9 10 11 13 14
  • 6. Use Case: Stream Sort - Code void processElement(Long event, /*...*/) { TimerService timerSvc = ctx.timerService(); long ts = ctx.timestamp(); if (!isLate(ts, timerSvc)) { List<Long> listAtTs = events.get(ts); if (listAtTs == null) { listAtTs = new ArrayList<>(); } listAtTs.add(event); events.put(ts, listAtTs); timerSvc.registerEventTimeTimer(ts); } } MapState<Long, List<Long>> events; MapStateDescriptor<Long, List<Long>> desc = new MapStateDescriptor<>( "Events", Types.LONG, Types.LIST(Types.LONG)); events = getRuntimeContext().getMapState(desc); void onTimer(long ts, /*...*/) { events.get(ts).forEach(out::collect); events.remove(ts); }
  • 7. List<Long> listAtTs = events.get(ts); if (listAtTs == null) { listAtTs = new ArrayList<>(); } listAtTs.add(event); events.put(ts, listAtTs); Use Case: Stream Sort - What’s Happening Underneath State (RocksDB) De-/Serializer full list as byte[] search memtable + sst files for one entry lookup deserialized (Java) list
  • 8. List<Long> listAtTs = events.get(ts); if (listAtTs == null) { listAtTs = new ArrayList<>(); } listAtTs.add(event); events.put(ts, listAtTs); Use Case: Stream Sort - What’s Happening Underneath State (RocksDB) De-/Serializer full Java list serialized list as byte[] add new entry to memtable (leave old one for compaction)
  • 9. Use Case: Stream Sort - Alternative Solutions ● Using MapState<Long, Event> instead of MapState<Long, List<Event>>? ○ Cannot handle multiple events per timestamp ● Using Window API? ○ Efficient event storage per timestamp ○ No really well-matching window types: sliding, tumbling, and session windows ● Using HashMapStateBackend? ○ No de-/serialization overhead ○ state limited by available memory ○ no incremental checkpoints ● Using ListState<Event> and filtering in onTimer()? ○ Reduced overhead in processElement() vs. more to do in onTimer()
  • 10. {ts: 5, code: GBP, rate: 1.20} {ts: 10, code: USD, rate: 1.00} {ts: 19, code: USD, rate: 1.02} rates {ts: 15, code: USD, amount: 1.00} {ts: 10, code: GBP, amount: 2.00} {ts: 25, code: USD, amount: 3.00} transactions Use Case: Event-Time Stream Join {ts1: 15, ts2: 10, amount: 1.00} {ts1: 10, ts2: 5, amount: 2.40} {ts1: 25, ts2: 19, amount: 3.06} SELECT t.ts AS ts1, r.ts AS ts2, t.amount * r.rate AS amount FROM transactions AS t LEFT JOIN rates FOR SYSTEM_TIME AS OF t.ts AS r ON t.code = r.code;
  • 11. void onTimer(long ts, /*...*/) { TreeSet<Entry<Long, Double>> rates = ratesInRange(ts, rate.entries()); Double myRate = getLast(rates); transactions.get(ts).forEach( tx -> out.collect( new Joined(myRate, tx))); deleteAllButLast(rates); transactions.remove(ts); } Use Case: Event-Time Stream Join - Code void processElement1(Transaction value,/*...*/) { TimerService timerSvc = ctx.timerService(); long ts = ctx.timestamp(); if (!isLate(ts, timerSvc)) { addTransaction(ts, value); timerSvc.registerEventTimeTimer(ts); } } // similar for processElement2() MapState<Long, List<Transaction>> transactions; MapState<Long, Double> rate;
  • 12. void onTimer(long ts, /*...*/) { TreeSet<Entry<Long, Double>> rates = ratesInRange(ts, rate.entries()); Double myRate = getLast(rates); transactions.get(ts).forEach( tx -> out.collect( new Joined(myRate, tx))); deleteAllButLast(rates); transactions.remove(ts); } With RocksDB: ● always fetching all rates’ (key+value) bytes ⚠ ● (also need to fit into memory ⚠) ● deserialize all keys keys during iteration ⚠ ○ not deserializing values (at least) ✓ Use Case: Event-Time Stream Join - Code void processElement1(Transaction value,/*...*/) { TimerService timerSvc = ctx.timerService(); long ts = ctx.timestamp(); if (!isLate(ts, timerSvc)) { addTransaction(ts, value); timerSvc.registerEventTimeTimer(ts); } } // similar for processElement2() Similar to stream sort, with RocksDB: ● always fetch/write full lists ⚠ ● always de-/serialize full list ⚠ ● additional stress on RocksDB compaction ⚠ MapState<Long, List<Transaction>> transactions; MapState<Long, Double> rate;
  • 14. BinarySortedState - History “Temporal state” Hackathon project (David + Nico + Seth) ● Main primitives: getAt(), getAtOrBefore(), getAtOrAfter(), add(), addAll(), update() 2020 Nov 2021 April 2022 started as a Hackathon project (David + Nico) on custom windowing with process functions Created FLIP-220 and discussed on dev@flink.apache.org ● Extended scope further to allow arbitrary user keys (not just timestamps) ○ Identified further use cases in SQL operators, e.g. min/max with retractions ● Clarified serializer requirements ● Extend proposed API to offer range read and clear operations ● …
  • 15. ● A new keyed-state primitive, built on top of ListState ● Efficiently add to list of values for a user-provided key ● Efficiently iterate user-keys in a well-defined sort order, with native state-backend support, especially RocksDB ● Efficient operations for time-based functions (windowing, sorting, event-time joins, custom, ...) ● Operations on subset of the state, based on user-key ranges ● Portable between state backends (RocksDB, HashMap) BinarySortedState - Goals
  • 16. BinarySortedState - API (subject to change!) ● Point-operations: ○ valuesAt(key) ○ add(key, value) ○ put(key, values) ○ clearEntryAt(key) ● Lookups: ○ firstEntry(), firstEntry(fromKey) ○ lastEntry(), lastEntry(UK toKey) ● Cleanup: ○ clear() ● Range-operations: ○ readRange(fromKey, toKey, inclusiveToKey) ○ readRangeUntil(toKey, inclusiveToKey) ○ readRangeFrom(fromKey) ● Range-deletes: ○ clearEntryAt(key) ○ clearRange( fromKey, toKey, inclusiveToKey) ○ clearRangeUntil(toKey, inclusiveToKey) ○ clearRangeFrom(fromKey) BinarySortedState<UK, UV>
  • 17. ● RocksDB is a key-value store writing into MemTables → flushing into SST files ● SST files are sorted by the key in lexicographical binary order (byte-wise unsigned comparison) BinarySortedState - How does it work with RocksDB?! ● RocksDB offers Prefix Seek and SeekForPrev ● RocksDBMapState.RocksDBMapIterator provides efficient iteration via: ○ Fetching up to 128 RocksDB entries at once ○ RocksDBMapEntry with lazy key/value deserialization
  • 18. BinarySortedState - LexicographicalTypeSerializer ● “Just” need to provide serializers that are compatible with RocksDB’s sort order ● Based on lexicographical binary order as defined by byte-wise unsigned comparison ● Compatible serializers extend LexicographicalTypeSerializer public abstract class LexicographicalTypeSerializer<T> extends TypeSerializer<T> { public Optional<Comparator<T>> findComparator() { return Optional.empty(); } }
  • 19. Stream Sort w/out BinarySortedState (1) private BinarySortedState<Long, Long> events; BinarySortedStateDescriptor<Long, Long> desc = new BinarySortedStateDescriptor<>( "Events", LexicographicLongSerializer.INSTANCE, Types.LONG); events = getRuntimeContext() .getBinarySortedState(desc); private MapState<Long, List<Long>> events; MapStateDescriptor<Long, List<Long>> desc = new MapStateDescriptor<>( "Events", Types.LONG, Types.LIST(Types.LONG)); events = getRuntimeContext() .getMapState(desc); public void onTimer(long ts, /*...*/) { events.valuesAt(ts).forEach(out::collect); events.clearEntryAt(ts); } public void onTimer(long ts, /*...*/) { events.get(ts).forEach(out::collect); events.remove(ts); }
  • 20. Stream Sort w/out BinarySortedState (2) public void processElement(/*...*/) { // ... events.add(ts, event); // ... } public void processElement(/*...*/) { // ... List<Long> listAtTs = events.get(ts); if (listAtTs == null) { listAtTs = new ArrayList<>(); } listAtTs.add(event); events.put(ts, listAtTs); // ... }
  • 21. events.add(ts, event); Stream Sort with BinarySortedState - What’s Happening Underneath State (RocksDB) De-/Serializer add new merge op to memtable Serialized entry as byte[] new list entry
  • 22. Stream Sort with BinarySortedState - What’s Happening Underneath State (RocksDB) De-/Serializer full list as byte[] Lookup deserialized (Java) list search memtable + sst files for all entries events.valuesAt(ts).forEach(out::collect);
  • 23. Mark k/v as deleted (removal during compaction) Stream Sort with BinarySortedState - What’s Happening Underneath State (RocksDB) De-/Serializer events.clearEntryAt(ts); Delete
  • 24. Event-Time Stream Join w/out BinarySortedState (1) private BinarySortedState<Long, Transaction> transactions; private BinarySortedState<Long, Double> rate; private MapState<Long, List<Transaction>> transactions; private MapState<Long, Double> rate; public void processElement1(/*...*/) { // ... if (!isLate(ts, timerSvc)) { // append to BinarySortedState: transactions.add(ts, value); timerSvc.registerEventTimeTimer(ts); } } // similar for processElement2() public void processElement1(/*...*/) { // ... if (!isLate(ts, timerSvc)) { // replace list in MapState: addTransaction(ts, value); timerSvc.registerEventTimeTimer(ts); } } // similar for processElement2()
  • 25. Event-Time Stream Join w/out BinarySortedState (2) public void onTimer(long ts, /*...*/) { Entry<Long, Iterable<Double>> myRate = rate.lastEntry(ts); Double rateVal = Optional .ofNullable(myRate) .map(e -> e.getValue().iterator().next()) .orElse(null); transactions.valuesAt(ts).forEach( tx -> out.collect( new Joined(rateVal, tx))); if (myRate != null) { rate.clearRangeUntil(myRate.getKey(), false); } transactions.clearEntryAt(ts); } public void onTimer(long ts, /*...*/) { TreeSet<Entry<Long, Double>> rates = ratesInRange(ts, rate.entries()); Double myRate = Optional .ofNullable(rates.pollLast()) .map(Entry::getValue) .orElse(null); transactions.get(ts).forEach( tx -> out.collect( new Joined(myRate, tx))); rates.forEach(this::removeRate); transactions.remove(ts); }
  • 26. Stream Sort with BinarySortedState - Optimized ● Idea: ○ Increase efficiency by processing all events between watermarks ● Challenge: ○ Registering a timer for the next watermark will fire too often ➔ Solution: ○ Register timer for the first unprocessed event ○ When the timer fires: ■ Process all events until the current watermark (not the timer timestamp!) ● events.readRangeUntil(currentWatermark, true)
  • 27. Event-Time Stream Join with BinarySortedState - Optimized ● Idea: ○ Increase efficiency by processing all events between watermarks ● Challenge: ○ Registering a timer for the next watermark will fire too often ➔ Solution: ○ Same as Stream Sort: Timer for first unprocessed event, processing until watermark, but: ○ When the timer fires: ■ Iterate both, transactions and rate (in the appropriate time range) in event-time order
  • 32. ● (Custom) stream sorting ● Time-based (custom) joins ● Code with range-reads or bulk-deletes ● Custom window implementations ● Min/Max with retractions ● … ● Basically everything maintaining a MapState<?, List<?>> or requiring range operations BinarySortedState - Who will benefit?
  • 33. What’s left to do? ● Iron out last bits and pieces + tests ○ Start voting thread for FLIP-220 on dev@flink.apache.org ○ Create a PR and get it merged ● Expected to land in Flink 1.17 (as experimental feature) ● Port Table/SQL/DataStream operators to improve efficiency: ○ TemporalRowTimeJoinOperator (PoC already done for validating the API ✓) ○ RowTimeSortOperator ○ IntervalJoinOperator ○ CepOperator ○ … ● Provide more LexicographicalTypeSerializers
  • 34. Get ready to ROK!!! Nico Kruber linkedin.com/in/nico-kruber nico@immerok.com