SlideShare a Scribd company logo
Scaling Flink in Cloud
Steven Wu @stevenzwu
Agenda
● Introduction
● Scaling stateless jobs
● Scaling stateful jobs
Agenda
● Introduction
● Scaling stateless jobs
● Scaling stateful jobs
Running Flink on Titus
(Netflix’s in-house container runtime)
Job isolation: single job
Job
Manager
Task
Manager
Task
Manager
Task
Manager
...
Titus Job #1
Titus Job #2
Flink
standalone
cluster
State backend and checkpoint store
State backend
● Memory
● File system
● RocksDB
Source: http://flink.apache.org/
checkpoint store
● HDFS
● S3
Why S3 as the snapshot store
● Only out-of-the-box support for Amazon cloud
● Cost-effective, scalability, durability
S3 concepts
● Massive storage system
● Bucket: container for objects
● Object: identified by a key (and a version)
● Filesystem like operations
○ GET, PUT, DELETE, LIST, HEAD
examplebucket/2018-04-01/00/data1.avro
examplebucket/2018-04-01/00/data2.avro
examplebucket/2018-04-02/01/data1.avro
examplebucket/2018-04-02/01/data2.avro
examplebucket/2018-04-02/03/data1.avro
examplebucket/2018-04-02/08/data1.avro
examplebucket/2018-04-03/23/data1.avro
S3 sharding: range partition
Partition 1
Partition 2
Partition 3
date / hour / file
examplebucket/2018-04-01/00/data1.avro
examplebucket/2018-04-01/00/data2.avro
examplebucket/2018-04-02/01/data1.avro
examplebucket/2018-04-02/01/data2.avro
examplebucket/2018-04-02/03/data1.avro
examplebucket/2018-04-02/08/data1.avro
examplebucket/2018-04-03/23/data1.avro
LIST (prefix query)
Partition 1
Partition 2
Partition 3
date / hour / file
S3 scaling
● If request rate grows steadily, S3 automatically
partitions buckets as needed to support higher
request rates
examplebucket/2018-04-01/00/data1.avro
examplebucket/2018-04-01/00/data2.avro
examplebucket/2018-04-01/00/data3.avro
examplebucket/2018-04-01/00/data4.avro
examplebucket/2018-04-01/00/data5.avro
examplebucket/2018-04-01/00/data6.avro
examplebucket/2018-04-01/00/data7.avro
Avoid sequential key names
if over 100 reqs/second
examplebucket/232a/2018-04-01/00/data1.avro
examplebucket/7b54/2018-04-01/00/data2.avro
examplebucket/921c/2018-04-01/00/data3.avro
examplebucket/ba65/2018-04-01/00/data4.avro
examplebucket/8761/2018-04-01/00/data5.avro
examplebucket/a390/2018-04-01/00/data6.avro
examplebucket/5d6c/2018-04-01/00/data7.avro
Introduce random prefix in key name
S3 Performance
● Optimized for high I/O throughput
● Not optimized for high request rate without
tweaking key names
● Not optimized for small files
● Not optimized for consistent low latency
Agenda
● Introduction
● Scaling stateless jobs
● Scaling stateful jobs
Event
Producers
Sinks
highly available ingest pipelines - the
backbone of a real-time data infrastructure
Events are published to fronting Kafka
directly or via proxy
KSGateway
Stream
Consumers
Event
Producer
Keystone
Management
Fronting
Kafka
Flink
Router
Consumer
Kafka
HTTP /
gRPC
Events land up in fronting Kafka cluster
KSGateway
Stream
Consumers
Event
Producer
Keystone
Management
Fronting
Kafka
Flink
Router
Consumer
Kafka
HTTP /
gRPC
Events are polled by router, filter and
projection applied
KSGateway
Stream
Consumers
Keystone
Management
Fronting
Kafka
Flink
Router
Consumer
Kafka
Event
Producer
HTTP /
gRPC
Router sends events to destination
KSGateway
Stream
Consumers
Keystone Management
Fronting
Kafka
Flink
Router
Consumer
Kafka
Event
Producer
HTTP /
gRPC
KSGateway
Stream
Consumers
Keystone
Management
Fronting
Kafka
Flink
Router
Consumer
Kafka
Event
Producer
HTTP /
gRPC
Keystone routing jobs
● Stateless
● Embarrassingly parallel
Keystone router scale
● ~3 trillion events/day
● ~2,000 routing jobs
● ~10,000 containers
● ~200,000 parallel operator instances
Math 101: S3 writes
● ~2,000 routing jobs
● checkpoint interval is 30 seconds
● ~67 (= 2,000 / 30) S3 writes per second?
Adapted from http://flink.apache.org/
Each operator writes to S3
S3
State
Snapshot
barriers
Math 201: S3 writes
● ~200,000 operators. Each operator writes
checkpoint to S3
● checkpoint interval is 30 seconds
● ~6,600 writes (= 200,000 / 30) per second
○ Actual writes 2-3x smaller because only Kafka
source operators have state
S3 throttling!
S3 not optimized for high request rate
without tweaking key names
Checkpoint path
state.checkpoints.dir:
Introduce entropy in checkpoint path
state.checkpoints.dir:
S3 not optimized for small files
Checkpoint ack with metadata
Job
Manager
ACK
(metadata)
S3
State
Snapshot
barriers
Uber checkpoint file after all ACKs
Job
Manager
S3
Uber file with
metadata
State.backend.fs.memory-threshold:
1,024
Checkpoint ack with state
Job
Manager
S3
barriers
ACK
(state)
Uber checkpoint file after all ACKs
Job
Manager
S3
Uber file with
metadata + state
State.backend.fs.memory-threshold:
1,024,000
Avoid S3 writes from task managers
● Only job manager writes one uber checkpoint file
● Reduced checkpoint duration by 10x
S3 HEADs are 100x of PUTs
Issue #1 Hadoop S3 file system
● Half of the HEADs failed for non-exist objects
● Always two HEADs for the same object (with and
without trailing slash)
○ checkpoints/<flink job>/fe68ab5591614163c19b55ff4aa66ac
○ checkpoints/<flink job>/fe68ab5591614163c19b55ff4aa66ac/
HEAD requests coming from task
managers
BTrace: dynamic tracing tool for Java
● Dynamically trace a running Java process
● Dynamically instruments the classes of the target
application to inject tracing code ("bytecode
tracing")
import com.sun.btrace.annotations.*;
import static com.sun.btrace.BTraceUtils.*;
import static com.sun.btrace.BTraceUtils.Sys.*;
import java.util.concurrent.atomic.*;
@BTrace
public class S3ReqTracing {
private static AtomicLong putCounter = newAtomicLong(0);
private static AtomicLong headCounter = newAtomicLong(0);
@OnMethod(
clazz = "com.amazonaws.services.s3.AmazonS3Client",
method = "putObject"
)
public static void trackPut() {
addAndGet(putCounter, 1L);
}
@OnMethod(
clazz = "com.amazonaws.services.s3.AmazonS3Client",
method = "getObjectMetadata",
location = @Location(value=Kind.LINE, line=966)
)
public static void trackHead() {
addAndGet(headCounter, 1L);
jstack();
}
@OnTimer(30000)
public static void dumpCounters() {
printNumber("put", getAndSet(putCounter, 0));
printNumber("head", getAndSet(headCounter, 0));
}
}
Run btrace on task manager
● bin/btrace <PID> S3ReqTracing.java
Setup
● 1-CPU container
● 1 subtask with two operators
Findings from task manager
● No S3 writes
● 4 HEAD requests per checkpoint interval
○ 1 (subtask) * 2 (operators) * 2 (with and without
trailing slash)
com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:966)
com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:956)
org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:892)
org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:848)
org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:1877)
org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.mkdirs(HadoopFileSystem.java:433)
org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory.<init>(FsChec
kpointStreamFactory.java:105)
org.apache.flink.runtime.state.filesystem.FsStateBackend.createStreamFactory(FsStateBackend.java:174)
org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.createStreamFactory(Strea
mTask.java:987)
org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.executeCheckpointing(Stre
amTask.java:956)
org.apache.flink.streaming.runtime.tasks.StreamTask.checkpointState(StreamTask.java:583)
org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:551)
org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:511)
Math 301: metadata reqs
● ~200,000 operators
● Each operator creates 2 HEAD requests (with and
without trailing slash)
● checkpoint interval is 30 seconds
● ~13,000 (200,000 * 2 / 30) HEAD reqs/s from
task managers even though they write zero S3
files
Create CheckpointStreamFactory
only once during operator
initialization (FLINK-5800)
Fixed in 1.2.1 (https://github.com/apache/flink/pull/3312)
Fine grained recovery (FLIP-1)
What is fine grained recovery
A1
A2
A3
B1
B2
B3
C1
C2
C3
What is fine grained recovery
A1
A2
A3
B1
B2
B3
C1
C2
C3
What is fine grained recovery
A1
A2
A3
B1
B2
B3
C1
C2
C3
What is fine grained recovery
A1
A2
A3
B1
B2
B3
C1
C2
C3
Life without fine grained recovery
Each kill (every 10 minutes) caused ~2x spikes
Sometimes revert to full restart
Full restartFine grained
recovery
Current implementation issue (FLINK-8042)
● Revert to full restart immediately if replacement
container didn’t come back in time (FLINK-8042)
● Fix expected in FLIP-6
Workaround: +1 standby container
Job
Manager
Task
Manager
Task
Manager
Task
Manager
... Task
Manager
+1
Fine grained recovery in action
Recap of scaling stateless jobs
● Introduce random prefix in checkpoint path to
spread S3 writes from many different jobs
● Avoid S3 writes from task managers
● Enable fine grained recovery (+1 standby)
Agenda
● Introduction
● Scaling stateless jobs
● Scaling stateful jobs
Often come with data shuffling
A1
A2
A3
B1
B2
B3
C1
C2
C3
keyBysource window sink
Challenges of large-state job
● Introduce random hex chars in checkpoint path to
spread S3 writes from different jobs
○ Single job writes large state to S3
● Avoid S3 writes from task managers
○ Each task manager has large state
● Enable fine grained recovery (+1 standby)
○ Connected job graph
Challenges of large-state job
● Single job writes large state to S3
● Each task manager has large state
● Connected job graph
Inject dynamic entropy in S3 path
state.backend.fs.checkpointdir.injectEntropy.enabled:
state.backend.fs.checkpointdir.injectEntropy.key:
state.checkpoints.dir:
2.5x throughput improvement
Contributing back: FLINK-9061
Challenges of large-state job
● Single job writes large state to S3
● Each task manager has large state
● Connected job graph
Tuning Flink to stabilize
● Enable incremental checkpoint with RocksDB
● RocksDB tuning: FLASH_SSD_OPTIMIZED
● Network buffer: taskmanager.network.memory.max=4 GB
Setup for performance test
● Cluster size: 200 nodes
○ 16 CPUs
○ 54 GB memory
○ 108 GB SSD-backed EBS volume
● Parallelism: 3,200
Numbers for savepoint
● Size: 21 TBs
● Take time: 27 minutes
● Recovery time: 6 minutes
Numbers for incremental checkpoint
● Checkpoint interval: 15 mins
● Size (avg):
● Duration (avg): 2.5 minutes
Challenges of large-state job
● Single job writes large state to S3
● Each task manager has large state
● Connected job graph
Full restart with connected graph
A1
A2
A3
B1
B2
B3
C1
C2
C3
Full restart with connected graph
A1
A2
A3
B1
B2
B3
C1
C2
C3
Full restart with connected graph
A1
A2
A3
B1
B2
B3
C1
C2
C3
Recover data from S3
A1
A2
A3
B1
B2
B3
C1
C2
C3
TM #1
TM #2 S3
HDD
HDD
HDDTM #3
Recover data from S3
A1
A2
A3
B1
B2
B3
C1
C2
C3
TM #1
TM #2 S3
HDD
HDD
HDDTM #3
Task local recovery (FLINK-8360)
A1
A2
A3
B1
B2
B3
C1
C2
C3
TM #1
TM #2 S3
HDD
HDD
HDDTM #3
Task local recovery with EBS
A1
A2
A3
B1
B2
B3
C1
C2
C3
TM #1
TM #2 S3
EBS
EBS
EBSTM #3
Recap of scaling stateful jobs
● Inject dynamic random prefix in checkpoint path
to spread S3 writes from operators in the same
job
● Enable incremental checkpoint with RocksDB
● Challenge: connected graph makes recovery
more expensive
Thank you!
Steven Wu @stevenzwu

More Related Content

What's hot

Flink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink powered stream processing platform at Pinterest
Flink powered stream processing platform at Pinterest
Flink Forward
 
Introduction to the Disruptor
Introduction to the DisruptorIntroduction to the Disruptor
Introduction to the Disruptor
Trisha Gee
 
HBase and HDFS: Understanding FileSystem Usage in HBase
HBase and HDFS: Understanding FileSystem Usage in HBaseHBase and HDFS: Understanding FileSystem Usage in HBase
HBase and HDFS: Understanding FileSystem Usage in HBase
enissoz
 
The Current State of Table API in 2022
The Current State of Table API in 2022The Current State of Table API in 2022
The Current State of Table API in 2022
Flink Forward
 
One sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkOne sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async Sink
Flink Forward
 
Pinot: Near Realtime Analytics @ Uber
Pinot: Near Realtime Analytics @ UberPinot: Near Realtime Analytics @ Uber
Pinot: Near Realtime Analytics @ Uber
Xiang Fu
 
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
 
Apache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraApache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native Era
Flink Forward
 
Scaling Flink in Cloud
Scaling Flink in CloudScaling Flink in Cloud
Scaling Flink in Cloud
Steven Wu
 
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
Flink Forward
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptx
Flink Forward
 
Flink Forward Berlin 2017: Matt Zimmer - Custom, Complex Windows at Scale Usi...
Flink Forward Berlin 2017: Matt Zimmer - Custom, Complex Windows at Scale Usi...Flink Forward Berlin 2017: Matt Zimmer - Custom, Complex Windows at Scale Usi...
Flink Forward Berlin 2017: Matt Zimmer - Custom, Complex Windows at Scale Usi...
Flink 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
 
Using the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentUsing the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production Deployment
Flink Forward
 
Linux Performance Analysis: New Tools and Old Secrets
Linux Performance Analysis: New Tools and Old SecretsLinux Performance Analysis: New Tools and Old Secrets
Linux Performance Analysis: New Tools and Old Secrets
Brendan Gregg
 
Using Queryable State for Fun and Profit
Using Queryable State for Fun and ProfitUsing Queryable State for Fun and Profit
Using Queryable State for Fun and Profit
Flink Forward
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache Kafka
Jiangjie Qin
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Flink Forward
 
Introducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorIntroducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes Operator
Flink Forward
 
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache FlinkTzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Ververica
 

What's hot (20)

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
 
Introduction to the Disruptor
Introduction to the DisruptorIntroduction to the Disruptor
Introduction to the Disruptor
 
HBase and HDFS: Understanding FileSystem Usage in HBase
HBase and HDFS: Understanding FileSystem Usage in HBaseHBase and HDFS: Understanding FileSystem Usage in HBase
HBase and HDFS: Understanding FileSystem Usage in HBase
 
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
 
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
 
Pinot: Near Realtime Analytics @ Uber
Pinot: Near Realtime Analytics @ UberPinot: Near Realtime Analytics @ Uber
Pinot: Near Realtime Analytics @ Uber
 
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...
 
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
 
Scaling Flink in Cloud
Scaling Flink in CloudScaling Flink in Cloud
Scaling Flink in Cloud
 
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
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptx
 
Flink Forward Berlin 2017: Matt Zimmer - Custom, Complex Windows at Scale Usi...
Flink Forward Berlin 2017: Matt Zimmer - Custom, Complex Windows at Scale Usi...Flink Forward Berlin 2017: Matt Zimmer - Custom, Complex Windows at Scale Usi...
Flink Forward Berlin 2017: Matt Zimmer - Custom, Complex Windows at Scale Usi...
 
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!
 
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
 
Linux Performance Analysis: New Tools and Old Secrets
Linux Performance Analysis: New Tools and Old SecretsLinux Performance Analysis: New Tools and Old Secrets
Linux Performance Analysis: New Tools and Old Secrets
 
Using Queryable State for Fun and Profit
Using Queryable State for Fun and ProfitUsing Queryable State for Fun and Profit
Using Queryable State for Fun and Profit
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache Kafka
 
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
 
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache FlinkTzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
 

Similar to Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"

Flink at netflix paypal speaker series
Flink at netflix   paypal speaker seriesFlink at netflix   paypal speaker series
Flink at netflix paypal speaker series
Monal Daxini
 
Spark and S3 with Ryan Blue
Spark and S3 with Ryan BlueSpark and S3 with Ryan Blue
Spark and S3 with Ryan Blue
Databricks
 
Mongodb meetup
Mongodb meetupMongodb meetup
Mongodb meetup
Eytan Daniyalzade
 
3 Flink Mistakes We Made So You Won't Have To
3 Flink Mistakes We Made So You Won't Have To3 Flink Mistakes We Made So You Won't Have To
3 Flink Mistakes We Made So You Won't Have To
HostedbyConfluent
 
PGConf.ASIA 2019 Bali - PostgreSQL on K8S at Zalando - Alexander Kukushkin
PGConf.ASIA 2019 Bali - PostgreSQL on K8S at Zalando - Alexander KukushkinPGConf.ASIA 2019 Bali - PostgreSQL on K8S at Zalando - Alexander Kukushkin
PGConf.ASIA 2019 Bali - PostgreSQL on K8S at Zalando - Alexander Kukushkin
Equnix Business Solutions
 
Varnish - PLNOG 4
Varnish - PLNOG 4Varnish - PLNOG 4
Varnish - PLNOG 4
Leszek Urbanski
 
Scale-Out Using Spark in Serverless Herd Mode!
Scale-Out Using Spark in Serverless Herd Mode!Scale-Out Using Spark in Serverless Herd Mode!
Scale-Out Using Spark in Serverless Herd Mode!
Databricks
 
Introduction to apache kafka
Introduction to apache kafkaIntroduction to apache kafka
Introduction to apache kafka
Samuel Kerrien
 
Extreme Replication - RMOUG Presentation
Extreme Replication - RMOUG PresentationExtreme Replication - RMOUG Presentation
Extreme Replication - RMOUG Presentation
Bobby Curtis
 
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward
 
Improving Logging Ingestion Quality At Pinterest: Fighting Data Corruption An...
Improving Logging Ingestion Quality At Pinterest: Fighting Data Corruption An...Improving Logging Ingestion Quality At Pinterest: Fighting Data Corruption An...
Improving Logging Ingestion Quality At Pinterest: Fighting Data Corruption An...
HostedbyConfluent
 
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
 
Scaling 100PB Data Warehouse in Cloud
Scaling 100PB Data Warehouse in CloudScaling 100PB Data Warehouse in Cloud
Scaling 100PB Data Warehouse in Cloud
Changshu Liu
 
spark stream - kafka - the right way
spark stream - kafka - the right way spark stream - kafka - the right way
spark stream - kafka - the right way
Dori Waldman
 
Kafka Tiered Storage | Satish Duggana and Sriharsha Chintalapani, Uber
Kafka Tiered Storage | Satish Duggana and Sriharsha Chintalapani, UberKafka Tiered Storage | Satish Duggana and Sriharsha Chintalapani, Uber
Kafka Tiered Storage | Satish Duggana and Sriharsha Chintalapani, Uber
HostedbyConfluent
 
Mirko Damiani - An Embedded soft real time distributed system in Go
Mirko Damiani - An Embedded soft real time distributed system in GoMirko Damiani - An Embedded soft real time distributed system in Go
Mirko Damiani - An Embedded soft real time distributed system in Go
linuxlab_conf
 
AWS Lambdas are cool - Cheminfo Stories Day 1
AWS Lambdas are cool - Cheminfo Stories Day 1AWS Lambdas are cool - Cheminfo Stories Day 1
AWS Lambdas are cool - Cheminfo Stories Day 1
ChemAxon
 
Haproxy - zastosowania
Haproxy - zastosowaniaHaproxy - zastosowania
Haproxy - zastosowania
Łukasz Jagiełło
 
Migrating to Apache Spark at Netflix
Migrating to Apache Spark at NetflixMigrating to Apache Spark at Netflix
Migrating to Apache Spark at Netflix
Databricks
 
Netflix Keystone Pipeline at Samza Meetup 10-13-2015
Netflix Keystone Pipeline at Samza Meetup 10-13-2015Netflix Keystone Pipeline at Samza Meetup 10-13-2015
Netflix Keystone Pipeline at Samza Meetup 10-13-2015
Monal Daxini
 

Similar to Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" (20)

Flink at netflix paypal speaker series
Flink at netflix   paypal speaker seriesFlink at netflix   paypal speaker series
Flink at netflix paypal speaker series
 
Spark and S3 with Ryan Blue
Spark and S3 with Ryan BlueSpark and S3 with Ryan Blue
Spark and S3 with Ryan Blue
 
Mongodb meetup
Mongodb meetupMongodb meetup
Mongodb meetup
 
3 Flink Mistakes We Made So You Won't Have To
3 Flink Mistakes We Made So You Won't Have To3 Flink Mistakes We Made So You Won't Have To
3 Flink Mistakes We Made So You Won't Have To
 
PGConf.ASIA 2019 Bali - PostgreSQL on K8S at Zalando - Alexander Kukushkin
PGConf.ASIA 2019 Bali - PostgreSQL on K8S at Zalando - Alexander KukushkinPGConf.ASIA 2019 Bali - PostgreSQL on K8S at Zalando - Alexander Kukushkin
PGConf.ASIA 2019 Bali - PostgreSQL on K8S at Zalando - Alexander Kukushkin
 
Varnish - PLNOG 4
Varnish - PLNOG 4Varnish - PLNOG 4
Varnish - PLNOG 4
 
Scale-Out Using Spark in Serverless Herd Mode!
Scale-Out Using Spark in Serverless Herd Mode!Scale-Out Using Spark in Serverless Herd Mode!
Scale-Out Using Spark in Serverless Herd Mode!
 
Introduction to apache kafka
Introduction to apache kafkaIntroduction to apache kafka
Introduction to apache kafka
 
Extreme Replication - RMOUG Presentation
Extreme Replication - RMOUG PresentationExtreme Replication - RMOUG Presentation
Extreme Replication - RMOUG Presentation
 
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
 
Improving Logging Ingestion Quality At Pinterest: Fighting Data Corruption An...
Improving Logging Ingestion Quality At Pinterest: Fighting Data Corruption An...Improving Logging Ingestion Quality At Pinterest: Fighting Data Corruption An...
Improving Logging Ingestion Quality At Pinterest: Fighting Data Corruption An...
 
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...
 
Scaling 100PB Data Warehouse in Cloud
Scaling 100PB Data Warehouse in CloudScaling 100PB Data Warehouse in Cloud
Scaling 100PB Data Warehouse in Cloud
 
spark stream - kafka - the right way
spark stream - kafka - the right way spark stream - kafka - the right way
spark stream - kafka - the right way
 
Kafka Tiered Storage | Satish Duggana and Sriharsha Chintalapani, Uber
Kafka Tiered Storage | Satish Duggana and Sriharsha Chintalapani, UberKafka Tiered Storage | Satish Duggana and Sriharsha Chintalapani, Uber
Kafka Tiered Storage | Satish Duggana and Sriharsha Chintalapani, Uber
 
Mirko Damiani - An Embedded soft real time distributed system in Go
Mirko Damiani - An Embedded soft real time distributed system in GoMirko Damiani - An Embedded soft real time distributed system in Go
Mirko Damiani - An Embedded soft real time distributed system in Go
 
AWS Lambdas are cool - Cheminfo Stories Day 1
AWS Lambdas are cool - Cheminfo Stories Day 1AWS Lambdas are cool - Cheminfo Stories Day 1
AWS Lambdas are cool - Cheminfo Stories Day 1
 
Haproxy - zastosowania
Haproxy - zastosowaniaHaproxy - zastosowania
Haproxy - zastosowania
 
Migrating to Apache Spark at Netflix
Migrating to Apache Spark at NetflixMigrating to Apache Spark at Netflix
Migrating to Apache Spark at Netflix
 
Netflix Keystone Pipeline at Samza Meetup 10-13-2015
Netflix Keystone Pipeline at Samza Meetup 10-13-2015Netflix Keystone Pipeline at Samza Meetup 10-13-2015
Netflix Keystone Pipeline at Samza Meetup 10-13-2015
 

More from Flink Forward

“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
Flink Forward
 
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 SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink SQL on Pulsar made easy
Flink SQL on Pulsar made easy
Flink Forward
 
Dynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsDynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data Alerts
Flink Forward
 
Processing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesProcessing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial Services
Flink Forward
 
Batch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergBatch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & Iceberg
Flink Forward
 
Welcome to the Flink Community!
Welcome to the Flink Community!Welcome to the Flink Community!
Welcome to the Flink Community!
Flink 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 scale
Flink Forward
 
Changelog Stream Processing with Apache Flink
Changelog Stream Processing with Apache FlinkChangelog Stream Processing with Apache Flink
Changelog Stream Processing with Apache Flink
Flink 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 Detection
Flink Forward
 
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
 
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
Flink Forward
 

More from Flink Forward (12)

“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...
 
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 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
 
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
 
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!
 
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
 
Changelog Stream Processing with Apache Flink
Changelog Stream Processing with Apache FlinkChangelog Stream Processing with Apache Flink
Changelog Stream Processing with Apache Flink
 
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
 
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...
 
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
 

Recently uploaded

GraphSummit Singapore | Graphing Success: Revolutionising Organisational Stru...
GraphSummit Singapore | Graphing Success: Revolutionising Organisational Stru...GraphSummit Singapore | Graphing Success: Revolutionising Organisational Stru...
GraphSummit Singapore | Graphing Success: Revolutionising Organisational Stru...
Neo4j
 
Why You Should Replace Windows 11 with Nitrux Linux 3.5.0 for enhanced perfor...
Why You Should Replace Windows 11 with Nitrux Linux 3.5.0 for enhanced perfor...Why You Should Replace Windows 11 with Nitrux Linux 3.5.0 for enhanced perfor...
Why You Should Replace Windows 11 with Nitrux Linux 3.5.0 for enhanced perfor...
SOFTTECHHUB
 
Infrastructure Challenges in Scaling RAG with Custom AI models
Infrastructure Challenges in Scaling RAG with Custom AI modelsInfrastructure Challenges in Scaling RAG with Custom AI models
Infrastructure Challenges in Scaling RAG with Custom AI models
Zilliz
 
Microsoft - Power Platform_G.Aspiotis.pdf
Microsoft - Power Platform_G.Aspiotis.pdfMicrosoft - Power Platform_G.Aspiotis.pdf
Microsoft - Power Platform_G.Aspiotis.pdf
Uni Systems S.M.S.A.
 
Removing Uninteresting Bytes in Software Fuzzing
Removing Uninteresting Bytes in Software FuzzingRemoving Uninteresting Bytes in Software Fuzzing
Removing Uninteresting Bytes in Software Fuzzing
Aftab Hussain
 
RESUME BUILDER APPLICATION Project for students
RESUME BUILDER APPLICATION Project for studentsRESUME BUILDER APPLICATION Project for students
RESUME BUILDER APPLICATION Project for students
KAMESHS29
 
GenAI Pilot Implementation in the organizations
GenAI Pilot Implementation in the organizationsGenAI Pilot Implementation in the organizations
GenAI Pilot Implementation in the organizations
kumardaparthi1024
 
GraphRAG for Life Science to increase LLM accuracy
GraphRAG for Life Science to increase LLM accuracyGraphRAG for Life Science to increase LLM accuracy
GraphRAG for Life Science to increase LLM accuracy
Tomaz Bratanic
 
Best 20 SEO Techniques To Improve Website Visibility In SERP
Best 20 SEO Techniques To Improve Website Visibility In SERPBest 20 SEO Techniques To Improve Website Visibility In SERP
Best 20 SEO Techniques To Improve Website Visibility In SERP
Pixlogix Infotech
 
Serial Arm Control in Real Time Presentation
Serial Arm Control in Real Time PresentationSerial Arm Control in Real Time Presentation
Serial Arm Control in Real Time Presentation
tolgahangng
 
Presentation of the OECD Artificial Intelligence Review of Germany
Presentation of the OECD Artificial Intelligence Review of GermanyPresentation of the OECD Artificial Intelligence Review of Germany
Presentation of the OECD Artificial Intelligence Review of Germany
innovationoecd
 
Introduction to CHERI technology - Cybersecurity
Introduction to CHERI technology - CybersecurityIntroduction to CHERI technology - Cybersecurity
Introduction to CHERI technology - Cybersecurity
mikeeftimakis1
 
“I’m still / I’m still / Chaining from the Block”
“I’m still / I’m still / Chaining from the Block”“I’m still / I’m still / Chaining from the Block”
“I’m still / I’m still / Chaining from the Block”
Claudio Di Ciccio
 
AI 101: An Introduction to the Basics and Impact of Artificial Intelligence
AI 101: An Introduction to the Basics and Impact of Artificial IntelligenceAI 101: An Introduction to the Basics and Impact of Artificial Intelligence
AI 101: An Introduction to the Basics and Impact of Artificial Intelligence
IndexBug
 
Video Streaming: Then, Now, and in the Future
Video Streaming: Then, Now, and in the FutureVideo Streaming: Then, Now, and in the Future
Video Streaming: Then, Now, and in the Future
Alpen-Adria-Universität
 
Building Production Ready Search Pipelines with Spark and Milvus
Building Production Ready Search Pipelines with Spark and MilvusBuilding Production Ready Search Pipelines with Spark and Milvus
Building Production Ready Search Pipelines with Spark and Milvus
Zilliz
 
How to use Firebase Data Connect For Flutter
How to use Firebase Data Connect For FlutterHow to use Firebase Data Connect For Flutter
How to use Firebase Data Connect For Flutter
Daiki Mogmet Ito
 
How to Get CNIC Information System with Paksim Ga.pptx
How to Get CNIC Information System with Paksim Ga.pptxHow to Get CNIC Information System with Paksim Ga.pptx
How to Get CNIC Information System with Paksim Ga.pptx
danishmna97
 
20240605 QFM017 Machine Intelligence Reading List May 2024
20240605 QFM017 Machine Intelligence Reading List May 202420240605 QFM017 Machine Intelligence Reading List May 2024
20240605 QFM017 Machine Intelligence Reading List May 2024
Matthew Sinclair
 
Observability Concepts EVERY Developer Should Know -- DeveloperWeek Europe.pdf
Observability Concepts EVERY Developer Should Know -- DeveloperWeek Europe.pdfObservability Concepts EVERY Developer Should Know -- DeveloperWeek Europe.pdf
Observability Concepts EVERY Developer Should Know -- DeveloperWeek Europe.pdf
Paige Cruz
 

Recently uploaded (20)

GraphSummit Singapore | Graphing Success: Revolutionising Organisational Stru...
GraphSummit Singapore | Graphing Success: Revolutionising Organisational Stru...GraphSummit Singapore | Graphing Success: Revolutionising Organisational Stru...
GraphSummit Singapore | Graphing Success: Revolutionising Organisational Stru...
 
Why You Should Replace Windows 11 with Nitrux Linux 3.5.0 for enhanced perfor...
Why You Should Replace Windows 11 with Nitrux Linux 3.5.0 for enhanced perfor...Why You Should Replace Windows 11 with Nitrux Linux 3.5.0 for enhanced perfor...
Why You Should Replace Windows 11 with Nitrux Linux 3.5.0 for enhanced perfor...
 
Infrastructure Challenges in Scaling RAG with Custom AI models
Infrastructure Challenges in Scaling RAG with Custom AI modelsInfrastructure Challenges in Scaling RAG with Custom AI models
Infrastructure Challenges in Scaling RAG with Custom AI models
 
Microsoft - Power Platform_G.Aspiotis.pdf
Microsoft - Power Platform_G.Aspiotis.pdfMicrosoft - Power Platform_G.Aspiotis.pdf
Microsoft - Power Platform_G.Aspiotis.pdf
 
Removing Uninteresting Bytes in Software Fuzzing
Removing Uninteresting Bytes in Software FuzzingRemoving Uninteresting Bytes in Software Fuzzing
Removing Uninteresting Bytes in Software Fuzzing
 
RESUME BUILDER APPLICATION Project for students
RESUME BUILDER APPLICATION Project for studentsRESUME BUILDER APPLICATION Project for students
RESUME BUILDER APPLICATION Project for students
 
GenAI Pilot Implementation in the organizations
GenAI Pilot Implementation in the organizationsGenAI Pilot Implementation in the organizations
GenAI Pilot Implementation in the organizations
 
GraphRAG for Life Science to increase LLM accuracy
GraphRAG for Life Science to increase LLM accuracyGraphRAG for Life Science to increase LLM accuracy
GraphRAG for Life Science to increase LLM accuracy
 
Best 20 SEO Techniques To Improve Website Visibility In SERP
Best 20 SEO Techniques To Improve Website Visibility In SERPBest 20 SEO Techniques To Improve Website Visibility In SERP
Best 20 SEO Techniques To Improve Website Visibility In SERP
 
Serial Arm Control in Real Time Presentation
Serial Arm Control in Real Time PresentationSerial Arm Control in Real Time Presentation
Serial Arm Control in Real Time Presentation
 
Presentation of the OECD Artificial Intelligence Review of Germany
Presentation of the OECD Artificial Intelligence Review of GermanyPresentation of the OECD Artificial Intelligence Review of Germany
Presentation of the OECD Artificial Intelligence Review of Germany
 
Introduction to CHERI technology - Cybersecurity
Introduction to CHERI technology - CybersecurityIntroduction to CHERI technology - Cybersecurity
Introduction to CHERI technology - Cybersecurity
 
“I’m still / I’m still / Chaining from the Block”
“I’m still / I’m still / Chaining from the Block”“I’m still / I’m still / Chaining from the Block”
“I’m still / I’m still / Chaining from the Block”
 
AI 101: An Introduction to the Basics and Impact of Artificial Intelligence
AI 101: An Introduction to the Basics and Impact of Artificial IntelligenceAI 101: An Introduction to the Basics and Impact of Artificial Intelligence
AI 101: An Introduction to the Basics and Impact of Artificial Intelligence
 
Video Streaming: Then, Now, and in the Future
Video Streaming: Then, Now, and in the FutureVideo Streaming: Then, Now, and in the Future
Video Streaming: Then, Now, and in the Future
 
Building Production Ready Search Pipelines with Spark and Milvus
Building Production Ready Search Pipelines with Spark and MilvusBuilding Production Ready Search Pipelines with Spark and Milvus
Building Production Ready Search Pipelines with Spark and Milvus
 
How to use Firebase Data Connect For Flutter
How to use Firebase Data Connect For FlutterHow to use Firebase Data Connect For Flutter
How to use Firebase Data Connect For Flutter
 
How to Get CNIC Information System with Paksim Ga.pptx
How to Get CNIC Information System with Paksim Ga.pptxHow to Get CNIC Information System with Paksim Ga.pptx
How to Get CNIC Information System with Paksim Ga.pptx
 
20240605 QFM017 Machine Intelligence Reading List May 2024
20240605 QFM017 Machine Intelligence Reading List May 202420240605 QFM017 Machine Intelligence Reading List May 2024
20240605 QFM017 Machine Intelligence Reading List May 2024
 
Observability Concepts EVERY Developer Should Know -- DeveloperWeek Europe.pdf
Observability Concepts EVERY Developer Should Know -- DeveloperWeek Europe.pdfObservability Concepts EVERY Developer Should Know -- DeveloperWeek Europe.pdf
Observability Concepts EVERY Developer Should Know -- DeveloperWeek Europe.pdf
 

Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"