SlideShare a Scribd company logo
1 of 82
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:
s3://bucket/checkpoints
<deploy timestamp>/
<job id>
Introduce entropy in checkpoint path
state.checkpoints.dir:
s3://bucket/checkpoints
<4-char random hex>/
<deploy timestamp>/
<job id>
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:
True
state.backend.fs.checkpointdir.injectEntropy.key:
__ENTROPY_KEY__
state.checkpoints.dir:
s3://bucket/__ENTROPY_KEY__/path
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): 950 GB
● 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

Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Databricks
 

What's hot (20)

Stephan Ewen - Experiences running Flink at Very Large Scale
Stephan Ewen -  Experiences running Flink at Very Large ScaleStephan Ewen -  Experiences running Flink at Very Large Scale
Stephan Ewen - Experiences running Flink at Very Large Scale
 
Iceberg: A modern table format for big data (Strata NY 2018)
Iceberg: A modern table format for big data (Strata NY 2018)Iceberg: A modern table format for big data (Strata NY 2018)
Iceberg: A modern table format for big data (Strata NY 2018)
 
Scylla Summit 2022: Scylla 5.0 New Features, Part 1
Scylla Summit 2022: Scylla 5.0 New Features, Part 1Scylla Summit 2022: Scylla 5.0 New Features, Part 1
Scylla Summit 2022: Scylla 5.0 New Features, Part 1
 
Building large scale transactional data lake using apache hudi
Building large scale transactional data lake using apache hudiBuilding large scale transactional data lake using apache hudi
Building large scale transactional data lake using apache hudi
 
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotExactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
 
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
 
“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...
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive Mode
 
Real-time Analytics with Upsert Using Apache Kafka and Apache Pinot | Yupeng ...
Real-time Analytics with Upsert Using Apache Kafka and Apache Pinot | Yupeng ...Real-time Analytics with Upsert Using Apache Kafka and Apache Pinot | Yupeng ...
Real-time Analytics with Upsert Using Apache Kafka and Apache Pinot | Yupeng ...
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guide
 
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
 
The Apache Spark File Format Ecosystem
The Apache Spark File Format EcosystemThe Apache Spark File Format Ecosystem
The Apache Spark File Format Ecosystem
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
 
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
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
 
Performance Troubleshooting Using Apache Spark Metrics
Performance Troubleshooting Using Apache Spark MetricsPerformance Troubleshooting Using Apache Spark Metrics
Performance Troubleshooting Using Apache Spark Metrics
 
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
 
Introduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingIntroduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processing
 
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
 
Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...
Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...
Deep Dive into Project Tungsten: Bringing Spark Closer to Bare Metal-(Josh Ro...
 

Similar to Scaling Flink in Cloud

Running Presto and Spark on the Netflix Big Data Platform
Running Presto and Spark on the Netflix Big Data PlatformRunning Presto and Spark on the Netflix Big Data Platform
Running Presto and Spark on the Netflix Big Data Platform
Eva Tse
 

Similar to Scaling Flink in Cloud (20)

Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
 
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
 
(BDT303) Running Spark and Presto on the Netflix Big Data Platform
(BDT303) Running Spark and Presto on the Netflix Big Data Platform(BDT303) Running Spark and Presto on the Netflix Big Data Platform
(BDT303) Running Spark and Presto on the Netflix Big Data Platform
 
Running Presto and Spark on the Netflix Big Data Platform
Running Presto and Spark on the Netflix Big Data PlatformRunning Presto and Spark on the Netflix Big Data Platform
Running Presto and Spark on the Netflix Big Data Platform
 
Amazed by AWS Series #4
Amazed by AWS Series #4Amazed by AWS Series #4
Amazed by AWS Series #4
 
Introduction to apache kafka
Introduction to apache kafkaIntroduction to apache kafka
Introduction to apache kafka
 
Flink at netflix paypal speaker series
Flink at netflix   paypal speaker seriesFlink at netflix   paypal speaker series
Flink at netflix paypal speaker series
 
spark stream - kafka - the right way
spark stream - kafka - the right way spark stream - kafka - the right way
spark stream - kafka - the right way
 
Using amazon web services with cold fusion 11
Using amazon web services with cold fusion 11Using amazon web services with cold fusion 11
Using amazon web services with cold fusion 11
 
Dok Talks #124 - Intro to Druid on Kubernetes
Dok Talks #124 - Intro to Druid on KubernetesDok Talks #124 - Intro to Druid on Kubernetes
Dok Talks #124 - Intro to Druid on Kubernetes
 
Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...
Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...
Serverless Machine Learning on Modern Hardware Using Apache Spark with Patric...
 
Elasticsearch on Kubernetes
Elasticsearch on KubernetesElasticsearch on Kubernetes
Elasticsearch on Kubernetes
 
Taking advantage of the Amazon Web Services (AWS) Family
Taking advantage of the Amazon Web Services (AWS) FamilyTaking advantage of the Amazon Web Services (AWS) Family
Taking advantage of the Amazon Web Services (AWS) Family
 
AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly
AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly
AWS re:Invent presentation: Unmeltable Infrastructure at Scale by Loggly
 
Docker & ECS: Secure Nearline Execution
Docker & ECS: Secure Nearline ExecutionDocker & ECS: Secure Nearline Execution
Docker & ECS: Secure Nearline Execution
 
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
 
Advanced Task Scheduling with Amazon ECS (June 2017)
Advanced Task Scheduling with Amazon ECS (June 2017)Advanced Task Scheduling with Amazon ECS (June 2017)
Advanced Task Scheduling with Amazon ECS (June 2017)
 
Logging for Production Systems in The Container Era
Logging for Production Systems in The Container EraLogging for Production Systems in The Container Era
Logging for Production Systems in The Container Era
 
ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetup
 
AWS Lambda
AWS LambdaAWS Lambda
AWS Lambda
 

Recently uploaded

+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
Health
 
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
masabamasaba
 
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
masabamasaba
 
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
masabamasaba
 

Recently uploaded (20)

%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
 
WSO2CON 2024 - How to Run a Security Program
WSO2CON 2024 - How to Run a Security ProgramWSO2CON 2024 - How to Run a Security Program
WSO2CON 2024 - How to Run a Security Program
 
+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
 
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
%+27788225528 love spells in Boston Psychic Readings, Attraction spells,Bring...
 
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
Crypto Cloud Review - How To Earn Up To $500 Per DAY Of Bitcoin 100% On AutoP...
 
%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrand%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrand
 
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
 
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
 
%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain
%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain
%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain
 
%in ivory park+277-882-255-28 abortion pills for sale in ivory park
%in ivory park+277-882-255-28 abortion pills for sale in ivory park %in ivory park+277-882-255-28 abortion pills for sale in ivory park
%in ivory park+277-882-255-28 abortion pills for sale in ivory park
 
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
 
WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...
WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...
WSO2CON 2024 - Building the API First Enterprise – Running an API Program, fr...
 
Artyushina_Guest lecture_YorkU CS May 2024.pptx
Artyushina_Guest lecture_YorkU CS May 2024.pptxArtyushina_Guest lecture_YorkU CS May 2024.pptx
Artyushina_Guest lecture_YorkU CS May 2024.pptx
 
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
 
%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisa%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisa
 
WSO2CON 2024 - Freedom First—Unleashing Developer Potential with Open Source
WSO2CON 2024 - Freedom First—Unleashing Developer Potential with Open SourceWSO2CON 2024 - Freedom First—Unleashing Developer Potential with Open Source
WSO2CON 2024 - Freedom First—Unleashing Developer Potential with Open Source
 
WSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go PlatformlessWSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go Platformless
 
WSO2CON 2024 Slides - Open Source to SaaS
WSO2CON 2024 Slides - Open Source to SaaSWSO2CON 2024 Slides - Open Source to SaaS
WSO2CON 2024 Slides - Open Source to SaaS
 
WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...
WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...
WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...
 
WSO2CON 2024 - WSO2's Digital Transformation Journey with Choreo: A Platforml...
WSO2CON 2024 - WSO2's Digital Transformation Journey with Choreo: A Platforml...WSO2CON 2024 - WSO2's Digital Transformation Journey with Choreo: A Platforml...
WSO2CON 2024 - WSO2's Digital Transformation Journey with Choreo: A Platforml...
 

Scaling Flink in Cloud

Editor's Notes

  1. Today, I am going share our experiences on running Flink at scale in cloud environment. What are the challenges and what are the solutions?
  2. We run Flink on our Titus container platform. Titus is similar to Kubernetes. It is developed in house and not open sourced yet.
  3. Flink state backend defines the data structure that holds the state. It also implement the logic to take a snapshot of the job state and store that snapshot to some distributed file system like S3. Checkpoints is how Flink achieve fault tolerance.
  4. Flink support S3 as the distributed storage system for checkpoint state out of the box. Hadoop or presto has S3 adapter that implements HDFS interface on top of Amazon S3. S3 is very cost effective. It is scalable although sometimes you may need to jump through some hoops. It is highly durable with 11 9’s durability.
  5. S3 is designed as a massive storage system (like infinitely large) with very high durability. Netflix uses S3 for our data warehouse stored with over a hundred pera bytes of compressed data.
  6. S3 shard data by range partition. Object keys are stored in order across multiple partitions.
  7. With range partition, S3 can support prefix query efficiently. In this example, when you are querying objects with this date prefix, S3 know it only needs to look into partition 1 and 2
  8. If you have a big rollout and sudden traffic jump, you would want to work with AWS to pre-partition your bucket for higher throughput.
  9. Using a sequential prefix, such as time stamp, increases the likelihood that Amazon S3 will target one specific partition for a large number of your keys, overwhelming the I/O capacity of the partition.
  10. If your workload consistently exceed 100 requests per second for a bucket, Amazon recommend avoiding sequential key names and introduce some random prefix in key names. therefore, the key names and the I/O load will be distributed across more than one partition. Note that with random prefix, you can’t really do prefix query anymore, because there is no more common prefix.
  11. S3 is optimized for high I/O throughput, but not small files. That’s why our Hive data warehouse compacts small files into larger files (like a few hundred MBs large) to improve read performance. If you want to checkpoint at high frequency (e.g. every second), S3 is probably not the best choice. You probably want to consider some state backend that can deliver consistent low latency (e.g. DynamoDB)
  12. At 10,000 feet level, Keystone data pipeline is responsible for moving data from producer to sinks for data consumption. We will get into more details of Keystone pipeline when we are talking about Keystone router later.
  13. Pretty much every applications publishes some data to our data pipeline.
  14. No data shuffling in the job graph
  15. 2,000 jobs. They come in different sizes. Some small jobs only need one container with 1-CPU. Some large jobs have over 100 containers each with 8-CPU.
  16. Let’s zoom in a little bit on how Flink performs checkpoint. As checkpoint barrier, each operator snapshot its state and upload the snapshot to S3. In another word, each operator writes to S3 during each checkpoint cycle.
  17. Actually write is probably 2-3 times smaller than 6,000, because only Kafka source operator has state and needs to write to S3. even 2,000 writes is still a lot. While it is straightforward to do a back-of-envelope calculation for the write volume, it is difficult to estimate request rates for other S3 operations (like get or list) There are also other s3 requests.
  18. At beginning, we set checkpoint path like this. Using a timestamp cause sequential key names. As we said earlier, sequential keys don’t scale well.
  19. We said earlier that we need to avoid sequential key names if we want to scale more than 100 reqs/second without throttling. We introduced this 4-char hex random prefix in S3 path for checkpoint location.Such random hex chars will distribute S3 writes from many different routing jobs to different S3 partitions. This is just a trick from our deployment tooling. There is no change needed from Flink.
  20. Each operator writes a checkpoint file to S3 for its own state. For stateless job, this creates many small files. After writing the snapshot to S3, operators send acknowledgement back to jobmanager.
  21. After jobmanager got the acknowledgements from all operators, it writes a uber checkpoint file with all metadata received from acknowledgements
  22. Flink has this awesome feature of memory-threshold. We set this threshold to 1 MB for Keystone router.
  23. If operator state size is smaller than this threshold (default is 1024 bytes), task manager will ship the state to jobmanager without writing anything to S3.
  24. After jobmanager got the acknowledgements from all operators, it writes the uber checkpoint file with state embedded along with other metadata
  25. Flink has this awesome feature of memory-threshold. We set this threshold to 1 MB for Keystone router.
  26. If you are not familiar with S3, HEAD requests are for querying object metadata and PUT requests are writes. What really caught us by surprise is the fact that HEAD requests are ~150 times of PUT requests. We enabled S3 access log
  27. First request for dir without trailing slash char, which always resulted in 404 NoSuchKey failure. Then second request with trailing slash char, which always succeeds. This is an unfortunate behavior of hadoop s3 file system implementation. But it is actually a minor issue in the whole thing, as it only explains for 2x. What is the other 75x difference. That is the bigger fish that we should target. I believe this minor issue still exists as of today.
  28. I manually spot checked client IP addresses in the access log. Those HEAD request all come from task managers. Task managers do not write any checkpoint file to S3 anymore. Why making so many HEAD requests?
  29. To find out why we are making so many HEAD requests. I started to run BTrace on task manager process.
  30. I don’t expect you to read the stack trace here. Here is the take away. Even though task manager doesn’t actually write to S3, it still goes through the checkpoint code path where a FsCheckpointStreamFactory object is created for each operator for each checkpoint cycle. FsCheckpointStreamFactory constructor calls mkdirs() method which results in S3 metadata requests.
  31. Even though HEAD requests are pretty cheap metadata query. It is still counted when S3 enforcing throttling on request rate. And again, S3 is not optimized on high request rate.
  32. The key problem is CheckpointStreamFactory is created in each checkpoint cycle. After we shared the finding of this issue in 1.2.0, Stephan Ewen quickly fixed it in 1.2.1 release.
  33. For stateless jobs, I strongly encourage you to consider fine grained recovery that Flink implemented since 1.3
  34. Here is an simple embarrassingly parallel job DAG. no data shuffling. three operators running with parallelism of 3. A is source operator and C is sink operator
  35. Here is an simple embarrassingly parallel job DAG. no data shuffling. three operators running with parallelism of 3. A is source operator and C is sink operator
  36. Flink only needs to restart the portion of DAG marked as gray color. Other parallel chains are unaffected and untouched.
  37. This graph shows the impact of full job restart. X axis is time. Y axis the message rate per second. Red line is the incoming message rate to Kafka topic. Blue line is the record consume rate by the Flin job. In this graph, message rate is peaked at 800K messages per sec and it is coming off peak hours. We enabled Chaos Monkey to kill one container every 10 minutes. You can see each kill caused a full job restart and subsequent recovery spike of over 2 times of incoming msg rate. That means significant duplicates, which can be problematic for some jobs. You may wonder why would you run Chaos Monkey killing so frequently. This is to simulate a real-world scenario. As I mentioned earlier, our Flink jobs run on Titus container platform. When our Titus team update code on agent host, Titus team kills one container per ASG every 10 minutes to evacuate containers off old agents.
  38. Those small bumps are fine grained recovery working. Those big spikes are full restart. This flink job is actually not very bad. Only small number of recovery reverted to full restart. In another job, we have seen over 80% of time it reverted to full restart
  39. That is how we reduce or avoid the reversion to full restart.
  40. Same Flink job with fine grained recovery enabled. This is a 20-node cluster. If we kill one task manager, that is about 5% of the job graph. Recovery bump is proportional to that at ~5%.
  41. Now let’s shift gear from stateless computation to stateful computation. Let’s look at the challenges and some of the solutions for scaling large state jobs. By large state, I mean as large as TBs.
  42. Stateful job often has data shuffling to bring events for the same key to the same operator. This is connected graph now. Not embarrassingly parallel anymore.
  43. Here the challenge is hundreds or thousands parallel operators from the same job are writing large state to S3.
  44. We introduced new config to dynamically substitute the “_entroy_key” substring in the checkpoint path with a 4-char random hex string for each S3 write. In another word, each operator got checkpoint path with its own random prefix. This way, we can spread the S3 writes from different operators of the same Flink job to different S3 partitions.
  45. We like to contribute this improvement back. We are discussing it with the community in FLINK-9061
  46. For large-state job, we have to do the following tunings so that Flink job can keep up with the state churning and checkpointing For very large state (like TBs), you probably only want to use RocksDB state backend in Flink. Memory and filesystem statebackends just can’t scale to very large state. Since our container comes with SSD ephemeral disk. Flink has predefined tuning for RocksDB on SSD drive that works well out of the box . Since this job has a large cluster size and high parallelism, we found it helpful to increase the network buffer size from default 1 GB to 4 GB
  47. I want to share some performance test number. By no means we are claiming this is the best you can do with Flink. Just want to give you some ideas what is possible with Flink today. There are plenty of room for improvement both in Flink platform and in our application.
  48. For those who is not familiar with savepoint. Savepoint is like checkpoint but allows you to rescale the job with a different parallelism. We use savepoint to get an ideal of the total state size
  49. We are pretty happy with these numbers. At least, it shows that we can build a large-state application on Flink dealing with TBs of state.
  50. Assume A1, B1, C1 runs on TM #1. Similarly for TM #2 and #3. When TM #3 got terminated, full job got restarted.
  51. Currently, all operators on all task managers download data from S3 and recover operators from downloaded data. Is that really necessary. Obviously for task manager #3, it has no choice since ephemeral disk is lost when container got terminated. Data is gone. But what about task manager #1 and #2? They are still running and their local disk still have the data. If we can reschedule the same operators on the same task managers, potentially they don’t need to download data from S3.
  52. That is exactly what the upcoming new feature, called task local recovery will do. Flink implements schedule affinity that schedule the same operators back to the same task managers. This way, task manager #1 and #2 can recover job from local data. This may not be a big deal with a cluster with 3 task manager nodes. Thinking about a large-state job I shown earlier for performance number. Instead of all 200 task managers go to S3 download 21 TBs of state, with task local recovery only 1 task manager needs to download 100 GBs of state from S3. That makes a huge difference.
  53. Once task local recovery is available, we also want to explore EBS with it. For those who is not familiar with EBS, Elastic Block Store. You can think of EBS volume like a network attached hard drive that can be mounted to an instance and only one instance. Even for task manager #3, after the replacement container come up, it can attach the proper EBS volume from previously terminated container, data is still there in the persistent EBS volume. Task manager #3 can also recover from local data. Nobody needs to download anything from S3. that will make recovery much faster
  54. Before I opening up for questions, I want to mention that I will be at the O'Reilly Booth between 3 and 4 pm this afternoon. If you have more questions or just like to chat, please drop by.