SlideShare a Scribd company logo
1 of 24
KAPPA+
FLINK FORWARD 2019 – SAN FRANCISCO
Moving from Lambda and Kappa Architectures to Kappa+ at
UBER
ROSHAN NAIK
PROBLEM
Realtime jobs often need an offline counterpart:
• Backfill – Retroactively fix, or recompute values once all data has arrived.
• Offline Experimentation & Testing: Before taking the job online.
• Online and offline feature generation: for ML.
• Bootstrapping State: for realtime jobs.
CURRENT SOLUTIONS
1. Lambda Architecture [2011]
• Nathan Marz (Creator of Apache Storm)
• “How to beat the CAP theorem”
• Evidence of prior art [1983]:
• Butler Lampson (Turing Award Laureate)
• “Hints for Computer System Design” – Xerox PARC
• Core Idea: Streaming job for realtime processing. Batch job for offline processing.
2. Kappa Architecture [2014]
• Jay Krepps (Creator of Kafka, CoFounder/CEO Confluent)
• "Questioning the Lambda Architecture”
• Core Idea: Long data retention in Kafka. Replay using realtime code from an older point.
LIMITATIONS : LAMBDA ARCHITECTURE
• Maintain dual code for Batch and Streaming
• Batch APIs often lack required constructs (e.g. sliding windows)
• Variation: Unified API : SQL / Beam. – Offline job run in batch
mode
• Limitations of Batch mode (e.g. Spark):
• Divide large jobs into smaller ones to limit resource consumption
• Manual/automated sub job co-ordination
• Windows that span batch boundaries are problematic
LIMITATIONS : KAPPA ARCHITECTURE
• Longer retention in Kafka: Expensive, Infeasible
• Kafka not really a data warehouse. More expensive than HDFS.
• Retention beyond a few days not feasible. Single node storage limits partition size.
• Workaround 1: Tiered Storage (Pulsar)
• Data duplication: Usually need a separate queryable copy in Hive/warehouse.
• Low utilization: old data accessed only by Backfill jobs.
• Workaround 2: Mini batches
• Load small batches into Kafka and process one batch at a time.
• Sort before loading in Kafka. Try to recreate original arrival order.
• Expensive: Copying to Kafka and sorting are both costly.
• Issues when using multiple sources
• Low volume topic drains faster  messes up windowing  dropped data or OOM
DESIRED CHARACTERISTICS
• Reuse code for Online and Offline Processing.
• Windowing should work well in offline mode as well.
• No splitting jobs. Single job should processes any amount of data.
• Hardware requirements should not balloon with size of input data.
• Not have to rewrite jobs using new APIs.
• Efficient.
KAPPA+
Introducing the Architecture
KEY CHANGE IN PERSPECTIVE
• Decoupling Concepts:
• Bounded vs Unbounded (Nature of Data)
• Batch vs Streaming (Nature of Compute)
• Offline vs Realtime (Mode of Use)
• Instead of thinking: How to enable any job in Streaming and Batch
mode.
• Lambda / SQL / Beam / Unified APIs
• Think: Limits to job types that can run in Realtime (and Offline) mode.
• Kappa+
Impact:
• No need to support every type of batch job (departure from Unified API
approach).
• Identify the types of jobs to support: Kappa+ job classification system.
ARCHITECTURE
Central Idea - counter intuitive
• Use Streaming compute to process data directly from warehouse. (i.e. not tied to
Kafka)
Architectural Components:
1. Job classification system
• 4 categories
2. Processing model
• Same processing basic model with tweaks based on job category
Assumes: Data in warehouse (Hive/Hdfs/etc) is partitioned by time
(hourly/daily/etc).
JOB CLASSIFICATION SYSTEM
• Category 1 : Stateless Jobs
• No windowing. Memory not a concern.
• Data order usually not concern.
• Category 2 : Windowing with
aggregation (Low - Medium Memory)
• Eg: Aggregated Windows: sum / avg / count
/ min / max / reduce
• Retains only aggregate value in each window.
• Order of data is important. But solvable
without need for strict ordering.
• Category 3 : Windowing with retention
(High Memory)
• Holds on to all records till window expiration.
• Eg. Joins, pattern analysis within window.
• Memory requirements much higher than cat 2.
• Cat 4 : Global Windows with retention
• E.g. Sorting entire input data. Joins without
windowing.
• Not found in realtime jobs.
PROCESSING MODEL
1. Partially ordered reads
• Strict Ordering across partitions: Only one partition at a time, older partitions first.
• Constrains memory/container requirements to what is needed to process 1 partition.
• Single job can process any number of partitions with finite resources.
• Helps windowing correctness.
• Un-Ordered reads within partition
• Read records/files within a partition in any order. Opens up concurrency and high throughputs.
• Order could be exploited if necessary.
2. Emit watermark when switching to next partition
• Allows Out-Of-Order reads within partition and windowing correctness.
3. Lockstep progression, in case of multiple sources
• All sources move to next partition at the same time.
• Prevents low volume sources from racing ahead.
HANDLING EACH CATEGORY
• Cat 1 : Stateless
• Nothing special. Set parallelisms based on desired throughput.
• Cat 2 : Windowing with aggregation (Low – Med mem)
• Employ memory state backend.
• Windowing parallelism based on amount of data hosted in
memory for one partition. Other parallelisms, based on
throughput.
• Cat 3 : Windowing with retention (High Mem)
• Either: Use RocksDB state backend.
• Or: reduce partition size, and use Mem state backend.
• Or: Look into exploiting order within partition.
BENEFITS OVER BATCH
BATCH (SQL/ BEAM/ UNIFIED API)
1. Resource requirements grows with total
data volume.
• Tricky to estimate and allocate
2. Split into smaller jobs and coordinate
them.
• Windows that cross batch boundaries are
problematic.
3. Results visible after all data is
processed.
KAPPA+
1. Resources bounded by amount needed
to process 1 partition.
• Easier to estimate and allocate
2. Single job can process any number of
partitions.
3. Results visible after each partition.
Note: Kappa+ processing model could be adopted in Unified APIs to address these
limitations.
IMPLEMENTATION
Architecture is not tied any Streaming Engine.
ADOPTING KAPPA+ ON STREAMING ENGINES
• No new APIs.
• Hdfs/Hive/etc. Sources need behavioral change:
1. One partition at a time, older partitions first.
2. Concurrent reads within partition.
3. Lock step progression in case of multiple sources.
• Kafka source needs to only support #3 since data is already in order.
• Watermarking:
• Emit watermarks at the end of partition to flush windows.
A JOB SUPPORTING REALTIME & OFFLINE
dataSource = offlineMode ? hiveSource : kafkaSource;
watermaker = offlineMode ? new OfflineWM() : new RealtimeWM();
dataSource.assignWatermarkGenerator(watermarker);
// Same logic. Adjust parallelisms for offline & online modes.
job = dataSource.transform(..)
.filter(..)
.keyBy(..)
.window(..)
...
KAPPA+ ON FLINK
UBER internal Hive (/HDFS) source with Kappa+ support.
ONE PARTITION AT A TIME & CONCURRENT READS
File Selector
Source = 1
File Reader
Operator = N
Rest of JOB
ZK Updater
Operator = 1
ZK
- List files from next partition
- Emit names of files in partition
- Create zNode for partition
- Bulk create child zNodes for each
file
- Deserialize file and emit recs
- Inform ZK Updater on
completing file
- Delete zNode for completed file
- Delete zNode for partition if
empty
- Notify File Selector when
zNode for partition is deleted
File
Names
Records
MULTI SOURCE LOCK STEP PROGRESSION
File Selector A
Source = 1
File Reader
Operator = M
Join/Union
ZK Updater
Operator = 1
File Selector B
Source = 1
File Reader
Operator = N
ZK Updater
Operator = 1
Join
ZK
- Enter Barrier and wait for others
- Process next partition
- Exit Barrier and repeat
DETAILS
1. Time Skew: If arrival time is used for partitioning data in the warehouse, instead of
event creation time (used by job). There can be two types of data skews:
• Forward skew: Some events move into a future partition. For example due to late arrival.
• Could lead to appearance of missing data.
• Consider processing an additional partition after the last one, if this is an issue.
• Backward skew: Some events moving into an older partition.
• Can lead to appearance of data loss. As the events are not in the partition that you processed.
• Improper watermarking can close Windows prematurely and cause data loss.
2. Differing partition strategies: Job has two sources. First source reads Hive table
with daily partitions, second source reads table with hourly partition.
• Solution: Watermark progression dictated by daily (i.e. larger) partition
3. May need to throttle throughput of offline job if writing to production critical
destination.
DISTRIBUTED COMPUTING
Realtime
Batch Compute
Spark
(Micro Batching)
Streaming
Compute
Flink, Storm
Offline
Realtime
Batch Compute
Spark
(Micro Batching)
Streaming
Compute
Flink, Storm
Offline
Batch Compute
In Memory Batch
Systems
Full Fledged
Batch
DISTRIBUTED COMPUTING
Realtime
Batch Compute
Spark
(Micro Batching)
Streaming
Compute
Flink, Storm/Trident
Offline
Batch Compute
In Memory
Batch Systems
Full Fledged
Batch
Streaming
Compute
Kappa+
DISTRIBUTED COMPUTING
QUESTIONS
Email: roshan@uber.com Twitter: @naikrosh ,
@UberEng
UBER Engineering Blog: eng.uber.com
UBER is hiring!! Realtime Platform needs your expertise!

More Related Content

What's hot

Iceberg: a fast table format for S3
Iceberg: a fast table format for S3Iceberg: a fast table format for S3
Iceberg: a fast table format for S3DataWorks Summit
 
Kafka replication apachecon_2013
Kafka replication apachecon_2013Kafka replication apachecon_2013
Kafka replication apachecon_2013Jun Rao
 
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Databricks
 
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsFine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsDatabricks
 
Pinot: Near Realtime Analytics @ Uber
Pinot: Near Realtime Analytics @ UberPinot: Near Realtime Analytics @ Uber
Pinot: Near Realtime Analytics @ UberXiang Fu
 
How Apache Kafka® Works
How Apache Kafka® WorksHow Apache Kafka® Works
How Apache Kafka® Worksconfluent
 
Thrift vs Protocol Buffers vs Avro - Biased Comparison
Thrift vs Protocol Buffers vs Avro - Biased ComparisonThrift vs Protocol Buffers vs Avro - Biased Comparison
Thrift vs Protocol Buffers vs Avro - Biased ComparisonIgor Anishchenko
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeDatabricks
 
Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsAlluxio, Inc.
 
Hudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilitiesHudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilitiesNishith Agarwal
 
From my sql to postgresql using kafka+debezium
From my sql to postgresql using kafka+debeziumFrom my sql to postgresql using kafka+debezium
From my sql to postgresql using kafka+debeziumClement Demonchy
 
Apache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the CoversApache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the CoversScyllaDB
 
Data Pipline Observability meetup
Data Pipline Observability meetup Data Pipline Observability meetup
Data Pipline Observability meetup Omid Vahdaty
 
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)Ryan Blue
 
Intro to databricks delta lake
 Intro to databricks delta lake Intro to databricks delta lake
Intro to databricks delta lakeMykola Zerniuk
 
From Zero to Hero with Kafka Connect
From Zero to Hero with Kafka ConnectFrom Zero to Hero with Kafka Connect
From Zero to Hero with Kafka Connectconfluent
 
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...Databricks
 
Apache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEAApache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEAAdam Doyle
 

What's hot (20)

Rds data lake @ Robinhood
Rds data lake @ Robinhood Rds data lake @ Robinhood
Rds data lake @ Robinhood
 
Iceberg: a fast table format for S3
Iceberg: a fast table format for S3Iceberg: a fast table format for S3
Iceberg: a fast table format for S3
 
Kafka replication apachecon_2013
Kafka replication apachecon_2013Kafka replication apachecon_2013
Kafka replication apachecon_2013
 
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
Improving SparkSQL Performance by 30%: How We Optimize Parquet Pushdown and P...
 
Fine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark JobsFine Tuning and Enhancing Performance of Apache Spark Jobs
Fine Tuning and Enhancing Performance of Apache Spark Jobs
 
Pinot: Near Realtime Analytics @ Uber
Pinot: Near Realtime Analytics @ UberPinot: Near Realtime Analytics @ Uber
Pinot: Near Realtime Analytics @ Uber
 
How Apache Kafka® Works
How Apache Kafka® WorksHow Apache Kafka® Works
How Apache Kafka® Works
 
Thrift vs Protocol Buffers vs Avro - Biased Comparison
Thrift vs Protocol Buffers vs Avro - Biased ComparisonThrift vs Protocol Buffers vs Avro - Biased Comparison
Thrift vs Protocol Buffers vs Avro - Biased Comparison
 
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta LakeSimplify CDC Pipeline with Spark Streaming SQL and Delta Lake
Simplify CDC Pipeline with Spark Streaming SQL and Delta Lake
 
Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic Datasets
 
Hudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilitiesHudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilities
 
From my sql to postgresql using kafka+debezium
From my sql to postgresql using kafka+debeziumFrom my sql to postgresql using kafka+debezium
From my sql to postgresql using kafka+debezium
 
Apache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the CoversApache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the Covers
 
Data Pipline Observability meetup
Data Pipline Observability meetup Data Pipline Observability meetup
Data Pipline Observability meetup
 
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)
 
Intro to databricks delta lake
 Intro to databricks delta lake Intro to databricks delta lake
Intro to databricks delta lake
 
From Zero to Hero with Kafka Connect
From Zero to Hero with Kafka ConnectFrom Zero to Hero with Kafka Connect
From Zero to Hero with Kafka Connect
 
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...
The Future of Data Science and Machine Learning at Scale: A Look at MLflow, D...
 
Kafka 101
Kafka 101Kafka 101
Kafka 101
 
Apache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEAApache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEA
 

Similar to Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures to Kappa+ at Uber - Roshan Naik

Spark Overview and Performance Issues
Spark Overview and Performance IssuesSpark Overview and Performance Issues
Spark Overview and Performance IssuesAntonios Katsarakis
 
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life ExampleKafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Exampleconfluent
 
Lessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesLessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesAlexis Seigneurin
 
Building High-Throughput, Low-Latency Pipelines in Kafka
Building High-Throughput, Low-Latency Pipelines in KafkaBuilding High-Throughput, Low-Latency Pipelines in Kafka
Building High-Throughput, Low-Latency Pipelines in Kafkaconfluent
 
[262] netflix 빅데이터 플랫폼
[262] netflix 빅데이터 플랫폼[262] netflix 빅데이터 플랫폼
[262] netflix 빅데이터 플랫폼NAVER D2
 
Diagnosing Problems in Production - Cassandra
Diagnosing Problems in Production - CassandraDiagnosing Problems in Production - Cassandra
Diagnosing Problems in Production - CassandraJon Haddad
 
John adams talk cloudy
John adams   talk cloudyJohn adams   talk cloudy
John adams talk cloudyJohn Adams
 
High performace network of Cloud Native Taiwan User Group
High performace network of Cloud Native Taiwan User GroupHigh performace network of Cloud Native Taiwan User Group
High performace network of Cloud Native Taiwan User GroupHungWei Chiu
 
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the HoodRadical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the HoodDatabricks
 
Composable Futures with Akka 2.0
Composable Futures with Akka 2.0Composable Futures with Akka 2.0
Composable Futures with Akka 2.0Mike Slinn
 
Streaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in ProductionStreaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in Productionconfluent
 
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 ScaleVerverica
 
Fixing twitter
Fixing twitterFixing twitter
Fixing twitterRoger Xia
 
Fixing Twitter Improving The Performance And Scalability Of The Worlds Most ...
Fixing Twitter  Improving The Performance And Scalability Of The Worlds Most ...Fixing Twitter  Improving The Performance And Scalability Of The Worlds Most ...
Fixing Twitter Improving The Performance And Scalability Of The Worlds Most ...smallerror
 
Fixing Twitter Improving The Performance And Scalability Of The Worlds Most ...
Fixing Twitter  Improving The Performance And Scalability Of The Worlds Most ...Fixing Twitter  Improving The Performance And Scalability Of The Worlds Most ...
Fixing Twitter Improving The Performance And Scalability Of The Worlds Most ...xlight
 
Stateful streaming and the challenge of state
Stateful streaming and the challenge of stateStateful streaming and the challenge of state
Stateful streaming and the challenge of stateYoni Farin
 
Building Stream Infrastructure across Multiple Data Centers with Apache Kafka
Building Stream Infrastructure across Multiple Data Centers with Apache KafkaBuilding Stream Infrastructure across Multiple Data Centers with Apache Kafka
Building Stream Infrastructure across Multiple Data Centers with Apache KafkaGuozhang Wang
 

Similar to Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures to Kappa+ at Uber - Roshan Naik (20)

Spark Overview and Performance Issues
Spark Overview and Performance IssuesSpark Overview and Performance Issues
Spark Overview and Performance Issues
 
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life ExampleKafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example
 
Lessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and MicroservicesLessons Learned: Using Spark and Microservices
Lessons Learned: Using Spark and Microservices
 
Building High-Throughput, Low-Latency Pipelines in Kafka
Building High-Throughput, Low-Latency Pipelines in KafkaBuilding High-Throughput, Low-Latency Pipelines in Kafka
Building High-Throughput, Low-Latency Pipelines in Kafka
 
[262] netflix 빅데이터 플랫폼
[262] netflix 빅데이터 플랫폼[262] netflix 빅데이터 플랫폼
[262] netflix 빅데이터 플랫폼
 
Diagnosing Problems in Production - Cassandra
Diagnosing Problems in Production - CassandraDiagnosing Problems in Production - Cassandra
Diagnosing Problems in Production - Cassandra
 
John adams talk cloudy
John adams   talk cloudyJohn adams   talk cloudy
John adams talk cloudy
 
High performace network of Cloud Native Taiwan User Group
High performace network of Cloud Native Taiwan User GroupHigh performace network of Cloud Native Taiwan User Group
High performace network of Cloud Native Taiwan User Group
 
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the HoodRadical Speed for SQL Queries on Databricks: Photon Under the Hood
Radical Speed for SQL Queries on Databricks: Photon Under the Hood
 
Composable Futures with Akka 2.0
Composable Futures with Akka 2.0Composable Futures with Akka 2.0
Composable Futures with Akka 2.0
 
Hadoop
HadoopHadoop
Hadoop
 
Streaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in ProductionStreaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in Production
 
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
 
Fixing twitter
Fixing twitterFixing twitter
Fixing twitter
 
Fixing_Twitter
Fixing_TwitterFixing_Twitter
Fixing_Twitter
 
Fixing Twitter Improving The Performance And Scalability Of The Worlds Most ...
Fixing Twitter  Improving The Performance And Scalability Of The Worlds Most ...Fixing Twitter  Improving The Performance And Scalability Of The Worlds Most ...
Fixing Twitter Improving The Performance And Scalability Of The Worlds Most ...
 
Fixing Twitter Improving The Performance And Scalability Of The Worlds Most ...
Fixing Twitter  Improving The Performance And Scalability Of The Worlds Most ...Fixing Twitter  Improving The Performance And Scalability Of The Worlds Most ...
Fixing Twitter Improving The Performance And Scalability Of The Worlds Most ...
 
Stateful streaming and the challenge of state
Stateful streaming and the challenge of stateStateful streaming and the challenge of state
Stateful streaming and the challenge of state
 
Background processing with hangfire
Background processing with hangfireBackground processing with hangfire
Background processing with hangfire
 
Building Stream Infrastructure across Multiple Data Centers with Apache Kafka
Building Stream Infrastructure across Multiple Data Centers with Apache KafkaBuilding Stream Infrastructure across Multiple Data Centers with Apache Kafka
Building Stream Infrastructure across Multiple Data Centers with Apache Kafka
 

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
 
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...Flink Forward
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Flink Forward
 
Introducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorIntroducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorFlink Forward
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeFlink Forward
 
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...Flink Forward
 
One sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkOne sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkFlink Forward
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxFlink Forward
 
Flink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink Forward
 
Apache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraApache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraFlink Forward
 
Where is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in FlinkWhere is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in FlinkFlink Forward
 
Using the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentUsing the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentFlink Forward
 
The Current State of Table API in 2022
The Current State of Table API in 2022The Current State of Table API in 2022
The Current State of Table API in 2022Flink Forward
 
Flink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink Forward
 
Dynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsDynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsFlink Forward
 
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotExactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotFlink Forward
 
Processing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesProcessing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesFlink Forward
 
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
 

More from Flink Forward (20)

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

Recently uploaded

Pigging Solutions Piggable Sweeping Elbows
Pigging Solutions Piggable Sweeping ElbowsPigging Solutions Piggable Sweeping Elbows
Pigging Solutions Piggable Sweeping ElbowsPigging Solutions
 
08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking MenDelhi Call girls
 
How to Remove Document Management Hurdles with X-Docs?
How to Remove Document Management Hurdles with X-Docs?How to Remove Document Management Hurdles with X-Docs?
How to Remove Document Management Hurdles with X-Docs?XfilesPro
 
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 3652toLead Limited
 
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure serviceWhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure servicePooja Nehwal
 
08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking MenDelhi Call girls
 
Benefits Of Flutter Compared To Other Frameworks
Benefits Of Flutter Compared To Other FrameworksBenefits Of Flutter Compared To Other Frameworks
Benefits Of Flutter Compared To Other FrameworksSoftradix Technologies
 
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...Neo4j
 
Injustice - Developers Among Us (SciFiDevCon 2024)
Injustice - Developers Among Us (SciFiDevCon 2024)Injustice - Developers Among Us (SciFiDevCon 2024)
Injustice - Developers Among Us (SciFiDevCon 2024)Allon Mureinik
 
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...shyamraj55
 
Install Stable Diffusion in windows machine
Install Stable Diffusion in windows machineInstall Stable Diffusion in windows machine
Install Stable Diffusion in windows machinePadma Pradeep
 
FULL ENJOY 🔝 8264348440 🔝 Call Girls in Diplomatic Enclave | Delhi
FULL ENJOY 🔝 8264348440 🔝 Call Girls in Diplomatic Enclave | DelhiFULL ENJOY 🔝 8264348440 🔝 Call Girls in Diplomatic Enclave | Delhi
FULL ENJOY 🔝 8264348440 🔝 Call Girls in Diplomatic Enclave | Delhisoniya singh
 
Maximizing Board Effectiveness 2024 Webinar.pptx
Maximizing Board Effectiveness 2024 Webinar.pptxMaximizing Board Effectiveness 2024 Webinar.pptx
Maximizing Board Effectiveness 2024 Webinar.pptxOnBoard
 
Swan(sea) Song – personal research during my six years at Swansea ... and bey...
Swan(sea) Song – personal research during my six years at Swansea ... and bey...Swan(sea) Song – personal research during my six years at Swansea ... and bey...
Swan(sea) Song – personal research during my six years at Swansea ... and bey...Alan Dix
 
AI as an Interface for Commercial Buildings
AI as an Interface for Commercial BuildingsAI as an Interface for Commercial Buildings
AI as an Interface for Commercial BuildingsMemoori
 
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...Integration and Automation in Practice: CI/CD in Mule Integration and Automat...
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...Patryk Bandurski
 
Unblocking The Main Thread Solving ANRs and Frozen Frames
Unblocking The Main Thread Solving ANRs and Frozen FramesUnblocking The Main Thread Solving ANRs and Frozen Frames
Unblocking The Main Thread Solving ANRs and Frozen FramesSinan KOZAK
 
Key Features Of Token Development (1).pptx
Key  Features Of Token  Development (1).pptxKey  Features Of Token  Development (1).pptx
Key Features Of Token Development (1).pptxLBM Solutions
 
My Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationMy Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationRidwan Fadjar
 

Recently uploaded (20)

Pigging Solutions Piggable Sweeping Elbows
Pigging Solutions Piggable Sweeping ElbowsPigging Solutions Piggable Sweeping Elbows
Pigging Solutions Piggable Sweeping Elbows
 
08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men
 
How to Remove Document Management Hurdles with X-Docs?
How to Remove Document Management Hurdles with X-Docs?How to Remove Document Management Hurdles with X-Docs?
How to Remove Document Management Hurdles with X-Docs?
 
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365
Tech-Forward - Achieving Business Readiness For Copilot in Microsoft 365
 
Vulnerability_Management_GRC_by Sohang Sengupta.pptx
Vulnerability_Management_GRC_by Sohang Sengupta.pptxVulnerability_Management_GRC_by Sohang Sengupta.pptx
Vulnerability_Management_GRC_by Sohang Sengupta.pptx
 
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure serviceWhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
WhatsApp 9892124323 ✓Call Girls In Kalyan ( Mumbai ) secure service
 
08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men
 
Benefits Of Flutter Compared To Other Frameworks
Benefits Of Flutter Compared To Other FrameworksBenefits Of Flutter Compared To Other Frameworks
Benefits Of Flutter Compared To Other Frameworks
 
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...
 
Injustice - Developers Among Us (SciFiDevCon 2024)
Injustice - Developers Among Us (SciFiDevCon 2024)Injustice - Developers Among Us (SciFiDevCon 2024)
Injustice - Developers Among Us (SciFiDevCon 2024)
 
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...
Automating Business Process via MuleSoft Composer | Bangalore MuleSoft Meetup...
 
Install Stable Diffusion in windows machine
Install Stable Diffusion in windows machineInstall Stable Diffusion in windows machine
Install Stable Diffusion in windows machine
 
FULL ENJOY 🔝 8264348440 🔝 Call Girls in Diplomatic Enclave | Delhi
FULL ENJOY 🔝 8264348440 🔝 Call Girls in Diplomatic Enclave | DelhiFULL ENJOY 🔝 8264348440 🔝 Call Girls in Diplomatic Enclave | Delhi
FULL ENJOY 🔝 8264348440 🔝 Call Girls in Diplomatic Enclave | Delhi
 
Maximizing Board Effectiveness 2024 Webinar.pptx
Maximizing Board Effectiveness 2024 Webinar.pptxMaximizing Board Effectiveness 2024 Webinar.pptx
Maximizing Board Effectiveness 2024 Webinar.pptx
 
Swan(sea) Song – personal research during my six years at Swansea ... and bey...
Swan(sea) Song – personal research during my six years at Swansea ... and bey...Swan(sea) Song – personal research during my six years at Swansea ... and bey...
Swan(sea) Song – personal research during my six years at Swansea ... and bey...
 
AI as an Interface for Commercial Buildings
AI as an Interface for Commercial BuildingsAI as an Interface for Commercial Buildings
AI as an Interface for Commercial Buildings
 
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...Integration and Automation in Practice: CI/CD in Mule Integration and Automat...
Integration and Automation in Practice: CI/CD in Mule Integration and Automat...
 
Unblocking The Main Thread Solving ANRs and Frozen Frames
Unblocking The Main Thread Solving ANRs and Frozen FramesUnblocking The Main Thread Solving ANRs and Frozen Frames
Unblocking The Main Thread Solving ANRs and Frozen Frames
 
Key Features Of Token Development (1).pptx
Key  Features Of Token  Development (1).pptxKey  Features Of Token  Development (1).pptx
Key Features Of Token Development (1).pptx
 
My Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 PresentationMy Hashitalk Indonesia April 2024 Presentation
My Hashitalk Indonesia April 2024 Presentation
 

Flink Forward San Francisco 2019: Moving from Lambda and Kappa Architectures to Kappa+ at Uber - Roshan Naik

  • 1. KAPPA+ FLINK FORWARD 2019 – SAN FRANCISCO Moving from Lambda and Kappa Architectures to Kappa+ at UBER ROSHAN NAIK
  • 2. PROBLEM Realtime jobs often need an offline counterpart: • Backfill – Retroactively fix, or recompute values once all data has arrived. • Offline Experimentation & Testing: Before taking the job online. • Online and offline feature generation: for ML. • Bootstrapping State: for realtime jobs.
  • 3. CURRENT SOLUTIONS 1. Lambda Architecture [2011] • Nathan Marz (Creator of Apache Storm) • “How to beat the CAP theorem” • Evidence of prior art [1983]: • Butler Lampson (Turing Award Laureate) • “Hints for Computer System Design” – Xerox PARC • Core Idea: Streaming job for realtime processing. Batch job for offline processing. 2. Kappa Architecture [2014] • Jay Krepps (Creator of Kafka, CoFounder/CEO Confluent) • "Questioning the Lambda Architecture” • Core Idea: Long data retention in Kafka. Replay using realtime code from an older point.
  • 4. LIMITATIONS : LAMBDA ARCHITECTURE • Maintain dual code for Batch and Streaming • Batch APIs often lack required constructs (e.g. sliding windows) • Variation: Unified API : SQL / Beam. – Offline job run in batch mode • Limitations of Batch mode (e.g. Spark): • Divide large jobs into smaller ones to limit resource consumption • Manual/automated sub job co-ordination • Windows that span batch boundaries are problematic
  • 5. LIMITATIONS : KAPPA ARCHITECTURE • Longer retention in Kafka: Expensive, Infeasible • Kafka not really a data warehouse. More expensive than HDFS. • Retention beyond a few days not feasible. Single node storage limits partition size. • Workaround 1: Tiered Storage (Pulsar) • Data duplication: Usually need a separate queryable copy in Hive/warehouse. • Low utilization: old data accessed only by Backfill jobs. • Workaround 2: Mini batches • Load small batches into Kafka and process one batch at a time. • Sort before loading in Kafka. Try to recreate original arrival order. • Expensive: Copying to Kafka and sorting are both costly. • Issues when using multiple sources • Low volume topic drains faster  messes up windowing  dropped data or OOM
  • 6. DESIRED CHARACTERISTICS • Reuse code for Online and Offline Processing. • Windowing should work well in offline mode as well. • No splitting jobs. Single job should processes any amount of data. • Hardware requirements should not balloon with size of input data. • Not have to rewrite jobs using new APIs. • Efficient.
  • 8. KEY CHANGE IN PERSPECTIVE • Decoupling Concepts: • Bounded vs Unbounded (Nature of Data) • Batch vs Streaming (Nature of Compute) • Offline vs Realtime (Mode of Use) • Instead of thinking: How to enable any job in Streaming and Batch mode. • Lambda / SQL / Beam / Unified APIs • Think: Limits to job types that can run in Realtime (and Offline) mode. • Kappa+ Impact: • No need to support every type of batch job (departure from Unified API approach). • Identify the types of jobs to support: Kappa+ job classification system.
  • 9. ARCHITECTURE Central Idea - counter intuitive • Use Streaming compute to process data directly from warehouse. (i.e. not tied to Kafka) Architectural Components: 1. Job classification system • 4 categories 2. Processing model • Same processing basic model with tweaks based on job category Assumes: Data in warehouse (Hive/Hdfs/etc) is partitioned by time (hourly/daily/etc).
  • 10. JOB CLASSIFICATION SYSTEM • Category 1 : Stateless Jobs • No windowing. Memory not a concern. • Data order usually not concern. • Category 2 : Windowing with aggregation (Low - Medium Memory) • Eg: Aggregated Windows: sum / avg / count / min / max / reduce • Retains only aggregate value in each window. • Order of data is important. But solvable without need for strict ordering. • Category 3 : Windowing with retention (High Memory) • Holds on to all records till window expiration. • Eg. Joins, pattern analysis within window. • Memory requirements much higher than cat 2. • Cat 4 : Global Windows with retention • E.g. Sorting entire input data. Joins without windowing. • Not found in realtime jobs.
  • 11. PROCESSING MODEL 1. Partially ordered reads • Strict Ordering across partitions: Only one partition at a time, older partitions first. • Constrains memory/container requirements to what is needed to process 1 partition. • Single job can process any number of partitions with finite resources. • Helps windowing correctness. • Un-Ordered reads within partition • Read records/files within a partition in any order. Opens up concurrency and high throughputs. • Order could be exploited if necessary. 2. Emit watermark when switching to next partition • Allows Out-Of-Order reads within partition and windowing correctness. 3. Lockstep progression, in case of multiple sources • All sources move to next partition at the same time. • Prevents low volume sources from racing ahead.
  • 12. HANDLING EACH CATEGORY • Cat 1 : Stateless • Nothing special. Set parallelisms based on desired throughput. • Cat 2 : Windowing with aggregation (Low – Med mem) • Employ memory state backend. • Windowing parallelism based on amount of data hosted in memory for one partition. Other parallelisms, based on throughput. • Cat 3 : Windowing with retention (High Mem) • Either: Use RocksDB state backend. • Or: reduce partition size, and use Mem state backend. • Or: Look into exploiting order within partition.
  • 13. BENEFITS OVER BATCH BATCH (SQL/ BEAM/ UNIFIED API) 1. Resource requirements grows with total data volume. • Tricky to estimate and allocate 2. Split into smaller jobs and coordinate them. • Windows that cross batch boundaries are problematic. 3. Results visible after all data is processed. KAPPA+ 1. Resources bounded by amount needed to process 1 partition. • Easier to estimate and allocate 2. Single job can process any number of partitions. 3. Results visible after each partition. Note: Kappa+ processing model could be adopted in Unified APIs to address these limitations.
  • 14. IMPLEMENTATION Architecture is not tied any Streaming Engine.
  • 15. ADOPTING KAPPA+ ON STREAMING ENGINES • No new APIs. • Hdfs/Hive/etc. Sources need behavioral change: 1. One partition at a time, older partitions first. 2. Concurrent reads within partition. 3. Lock step progression in case of multiple sources. • Kafka source needs to only support #3 since data is already in order. • Watermarking: • Emit watermarks at the end of partition to flush windows.
  • 16. A JOB SUPPORTING REALTIME & OFFLINE dataSource = offlineMode ? hiveSource : kafkaSource; watermaker = offlineMode ? new OfflineWM() : new RealtimeWM(); dataSource.assignWatermarkGenerator(watermarker); // Same logic. Adjust parallelisms for offline & online modes. job = dataSource.transform(..) .filter(..) .keyBy(..) .window(..) ...
  • 17. KAPPA+ ON FLINK UBER internal Hive (/HDFS) source with Kappa+ support.
  • 18. ONE PARTITION AT A TIME & CONCURRENT READS File Selector Source = 1 File Reader Operator = N Rest of JOB ZK Updater Operator = 1 ZK - List files from next partition - Emit names of files in partition - Create zNode for partition - Bulk create child zNodes for each file - Deserialize file and emit recs - Inform ZK Updater on completing file - Delete zNode for completed file - Delete zNode for partition if empty - Notify File Selector when zNode for partition is deleted File Names Records
  • 19. MULTI SOURCE LOCK STEP PROGRESSION File Selector A Source = 1 File Reader Operator = M Join/Union ZK Updater Operator = 1 File Selector B Source = 1 File Reader Operator = N ZK Updater Operator = 1 Join ZK - Enter Barrier and wait for others - Process next partition - Exit Barrier and repeat
  • 20. DETAILS 1. Time Skew: If arrival time is used for partitioning data in the warehouse, instead of event creation time (used by job). There can be two types of data skews: • Forward skew: Some events move into a future partition. For example due to late arrival. • Could lead to appearance of missing data. • Consider processing an additional partition after the last one, if this is an issue. • Backward skew: Some events moving into an older partition. • Can lead to appearance of data loss. As the events are not in the partition that you processed. • Improper watermarking can close Windows prematurely and cause data loss. 2. Differing partition strategies: Job has two sources. First source reads Hive table with daily partitions, second source reads table with hourly partition. • Solution: Watermark progression dictated by daily (i.e. larger) partition 3. May need to throttle throughput of offline job if writing to production critical destination.
  • 21. DISTRIBUTED COMPUTING Realtime Batch Compute Spark (Micro Batching) Streaming Compute Flink, Storm Offline
  • 22. Realtime Batch Compute Spark (Micro Batching) Streaming Compute Flink, Storm Offline Batch Compute In Memory Batch Systems Full Fledged Batch DISTRIBUTED COMPUTING
  • 23. Realtime Batch Compute Spark (Micro Batching) Streaming Compute Flink, Storm/Trident Offline Batch Compute In Memory Batch Systems Full Fledged Batch Streaming Compute Kappa+ DISTRIBUTED COMPUTING
  • 24. QUESTIONS Email: roshan@uber.com Twitter: @naikrosh , @UberEng UBER Engineering Blog: eng.uber.com UBER is hiring!! Realtime Platform needs your expertise!