SlideShare ist ein Scribd-Unternehmen logo
1 von 80
A NETFLIX ORIGINAL SERVICE
SAMZA Edition
@ Samza Meetup, Oct 2015
@monaldax
https://www.linkedin.com/in/monaldaxini
What am I going to learn ?
● Situational Awareness
● Routing Service
● Samza Alterations
● Sprinkles of Best Practices - in our opinion
550 billion events per day
8.5 million events (22 GB per second) peak
Hundreds of event types
Over 1 Petabyte / day
Numbers Galore!
550 billion events per day
8.5 million events (22 GB per second) peak
Hundreds of event types
Over 1 Petabyte / day
Numbers Galore!
550 x 2
Where are we?
Chukwa / Suro
Event
Producer
Druid
Stream
Consumers
EMR
Kafka
Suro Router
Event
Producer
Suro
Where are we going?
At least once
Keystone
Stream
Consumers
Samza
Router
EMR
Fronting
Kafka
Event
Producer
Consumer
Kafka
Control Plane
Events
Injected Event Metadata
● GUID
● Timestamp
● Host
● App
Immutable Event Payload
Keystone Extensible Wire Protocol
● Invisible to source & sinks
● Backwards and forwards compatibility
● Supports JSON, AVRO on the horizon
● Efficient - 10 bytes overhead per message
○ message size - hundreds of bytes to 10MB
Keystone Extensible Wire Protocol
● Packaged as a jar - Chaski
● Why?
○ Evolve Independently
■ event metadata & traceability metadata
■ event payload serialization
Restrictive Access to Fronting Kafka Clusters
Why?
● Better manage 2700 brokers across 4x3 (12) clusters and 3 regions
○ Availability
○ Scalability
○ Decoupling
○ SLA
Access to Fronting Kafka Clusters
● Client Library
○ Wraps Kafka producer
○ Integrates with Netflix ecosystem
● REST proxy
○ Uses the client library
● Wire protocol Non-conformant messages dropped
Side Effect
Routing Service
Keystone
Stream
Consumers
Samza
Router
EMR
Fronting
Kafka
Event
Producer
Consumer
Kafka
Control Plane
Routing Infrastructure
+
Checkpointing
Cluster
+ 0.9.1
Routing Infrastructure
+
Checkpointing
Cluster
+
What?
0.9.1
Router
Job Manager
(Control Plane)
EC2 Instances
Zookeeper
(Instance Id assignment)
Job
Job
Job
ksnode
Checkpointing
Cluster
ASG
Distributed Systems are Hard
Keep it Simple
Minimize Moving Parts
Obey!
Obey the principles without being bound by them.
- Bruce Lee
Mind bender - Sink Isolation
● Multiple Samza jobs for one Kafka source topic
● Each job processes messages for one sink
○ E.g. separate job for each S3 & ElasticSearch cluster sinks
● Tradeoff
○ Sink isolation for extra load on Kafka source topic cluster
● Initial release
○ Each job processes partitions only from one topic
Samza Job Details
● Use window function to implement health check
○ task.window.ms=30000
● Batch requests to sinks
● Explicit offset commits only
○ automatic commits disabled - task.commit.ms=-1
Samza 0.9.1’s hardcoded
checkpoint topic naming scheme
Checkpoint Topic Name
"__samza_checkpoint_ver_%d_for_%s_%s" format
(CHECKPOINT_LOG_VERSION_NUMBER, jobName.
replaceAll("_", "-"), jobId.replaceAll("_", "-"))
Not fully Configurable
Mind bender - Checkpoint Cluster
● __samza_checkpoint_ver_$ver_$job.name_$job.id
● Topic name example
○ __samza_checkpoint_ver_1_for_ksrouter_kafka-elbert-map-audit-s3
○ job.name = ksrouter job.id = $kafkaCluster_$topic_$sink
POWERFULL!
1 checkpoint topic per sink, & source topic
for many Samza Jobs
1 checkpoint topic per kafka cluster, sink, source topic
● Change the number of samza jobs for a topic
● Easily redistribute the partitions across jobs
● Add new partitions seamlessly
● Our naming scheme facilitates migrating topics to other clusters
Job Startup Delays reading Checkpoint
Causing health check failures - timeout 5 min
What to do?
Checkpoint Topic Broker Default Config
default.replication.factor 3
log.cleaner.delete.retention.ms 30000
log.cleaner.enable true
log.cleaner.min.cleanable.ratio 0.25
og.cleaner.threads 5
log.segment.bytes 3145728
●
Checkpoint topic Samza Job Configuration
Replication factor is hard coded to 3
task.checkpoint.system=checkpoint
task.checkpoint.factory=org.apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory
task.checkpoint.segment.bytes=3145728
Add. Checkpoint Information
● About 300 bytes per offset commit
● Change log topic logs into the same checkpoint offset topic
○ Even if not enabled, one time large message with system-stream-partition
inserted into the same checkpoint offset topic
What’s running inside the node?
Custom
Executor
./runJob
Logs
Snapshots
Attach Volumes
./runJob
./runJob
Reconcile Loop
1 min
Health Check
Logs
ZFS Volume
Snapshots
Custom
Executor
.
/runJo
b
.
/runJo
b
.
/runJo
b
Tools
Server
Client Tools
Stream Logs
Browse through
rotated logs by date
Yes! You inferred right!
No Mesos & No Yarn
Samza Alterations
ver 0.9.1
Using ThreadJobFactory in production
job.factory.class=org.apache.samza.job.local.ThreadJobFactory
SAMZA-41 - static partition range assignment
job.systemstreampartition.matcher.class=
org.apache.samza.system.RegexSystemStreamPartitionMatcher
job.systemstreampartition.matcher.config.ranges=[8-10]
^8&|^9$|^10$
you need
SAMZA-41 - static partition range assignment
Simplify...
job.systemstreampartition.matcher.class=
org.apache.samza.system.RangeSystemStreamPartitionMatcher
job.systemstreampartition.matcher.config.ranges=6-10
Prefetch Buffer - When is it going to OOM?
● Default count based per Samza container
○ (50,000 / # partitions) per topic
○ systems.source.samza.fetch.threshold=50000
● Hard to get it right and avoid OOM
○ changing message size
SAMZA-775- size based Prefetch buffer
● How much of heap should I use for prefetching?
○ systems.source.samza.fetch.threshold.bytes=200000000 (200MB)
○ per system / stream / partition
○ if > 0 precedence over systems.source.samza.fetch.threshold
SAMZA-775- size based Prefetch buffer
● systems.source.samza.fetch.threshold.bytes is a Soft limit
○ bytes limit + size of last max message stream
● I don’t get it, where is the example?
SAMZA-775- size based Prefetch buffer
● systems.source.samza.fetch.threshold.bytes=100000 (100K)
● 50 SystemStreamPartitions
● per system-stream-partition threshold is (100000 / 2) / 50 = 1000 bytes.
● Enforced limit would be
○ 1000 bytes + size of last message from the partition
SAMZA-775- size based Prefetch buffer
● Value of systems.source.samza.fetch.threshold.bytes based on
○ Incoming traffic Bps into source Kafka
○ 60 seconds of buffer with region failover traffic
○ Samza in memory data structures (2 x message size)
SAMZA-775- size based Prefetch buffer
● How does it perform?
○ Per message overhead within 0.02% of computed heuristics in the patch
○ Actual footprint of systems.source.samza.fetch.threshold.bytes is 10-15% at
the most in worst case.
■ Example: If set to 200MB, worst case observed 230MB
SAMZA-775- size based Prefetch buffer
● Con
○ Implementation to enforce systems.source.samza.fetch.threshold.bytes is very
dependent on the implementation version of Samza
○ Hence, higher maintenance when code changes. However,
Well Worth It! Ergonmic Config! Adds Stability!
SAMZA-655 & SAMZA-540
● Backported from 0.10
○ environment variable configuration rewriter
■ Pass config from RDS to executor to Docker to Samza Job
○ expose latency related metrics in OffsetManager
■ checkpointed offset guage
Checkpointed offset gauge in action
Immutable Config in Running Job
Integration with NetlixOSS Ecosystem
● Atlas
○ Alert & Monitoring system
● Eureka
○ Service discovery
YMMV
Keystone
Stream
Consumers
Samza
Router
EMR
Fronting
Kafka
Event
Producer
Consumer
Kafka
Control Plane
Router Stats
● S3 sink
○ 620 c3.4xl across 3 regions running 6736 docker containers
○ Avg 21 MBps (168 Mbps) per container
● On its way...
○ Kafka sink - 280 c3.4xl across 3 regions running 3200 docker containers
○ ElasticSearch sink - 70 c3.4xl across 3 regions running 850 containers
AWS c3.4xl
● 16 vCPU
● 30GB Ram
● 320GB SSD - 160 x 2
● High gigabit network
○ support for SR-IOV when we move to VPC
Per Container Reserved
● 2G - 5G memory
○ based on incoming traffic rate, prefetch buffer
● 160 mbps max network bandwidth
● 1 CPU Share
● 20G disk for buffer & logs
● Processes 1-12 partitions
Observed - Per Container
● Avg memory ~1.8G
● Avg memory usage ~ 20G
● Avg CPU utilization 8%
● Avg NetworkIn 256Mbps
● Avg NetworkOut 156Mbps
○ outgoing data compressed
I/O bound, very close to reserved capacity
Metrics
● External process Auditor - continuously monitors & diffs offset checkpoint & source Kafka topic
offset
○ consumerLag
○ missingConsumerOffset
○ stuckConsumer
○ logOffset
○ consumerOffset
page_click
(Topic obfuscated)
page_click
Topic
new_topic
(Topic obfuscated)
End to End metrics
● Producer to Router latency
○ Avg. about 2.5 seconds
○ 90 percentile topics under 2 sec
● Kafka to Router consumer lag (estimated time to catch up)
○ 65 percentile under 500ms
○ 90 percentile under 5 seconds
● Producer event timestamp to Samza job router avg latency - 6 seconds
Additional Metrics
● uploaded-messages
● uploaded-bytes
● upload-ms
● upload
● processed-messages
● processed-bytes
● message-latency
● latest-message-timestamp
● compression-ratio-timer
● compression-ratio-gauge
● window-execution-ms
● window-interval-ms
● window-error
● kssamza-offset-change
● kssamza-offset
● kssamza-messages-behind-high-watermark
● kssamza-high-watermark
● kssamza-checkpointed-offset
● kssamza-buffered-message-size
● kssamza-buffered-message-count
Backpressure
Producer ⇐ Kafka Cluster ⇐ Samza job router ⇐ Sink
Wait there’s more in the pipeline...
● Self service tools
● Multi-tenant Stream Processing as a Service - SPaaS
○ probably add spark streaming to the mix
● Event traceability - on demand and sampled
● As number of jobs increase checkpoint topic may give way to Cassandra
● Optimization & Automation
Fronting Kafka Clusters
Keystone
Stream
Consumers
Samza
Router
EMR
Fronting
Kafka
Event
Producer
Consumer
Kafka
Control Plane
Fronting Kafka Clusters
● Normal-priority (majority)
○ 2 clusters
○ 2 copies, 8 hour retention, 4 hour log roll
● High-priority (streaming activities etc.)
○ 2 clusters
○ 3 copies, 24 hour retention, 12 hour log roll
Fronting Kafka Instances
● 2700 d2.xl AWS instances across 3 regions for regular & failover traffic
● d2.xl
○ Large disk (6TB) - 450-475MB/s of sequential I/O throughput
○ 30GB memory, 700 Mbps medium network capability
○ Replication lag above 18MB/second per broker with thousands of partitions
○ cons: multiple instances on same physical host - increases failures
Kafka Capacity Planning
1. Stay under 20k partitions per cluster (14K)
2. Leave ≅ 40% free disk space on each broker for growth & movement
3. Throughput per partition based on 1, 2, # of brokers, and the retention
period
Partition Assignment
● All assignments Zone / Rack aware
● Strategy 1 - Multiple of brokers
● Stategy 2 - Stateful Round Robin
Kafka Auditor as a Service
● Broker monitoring
● Consumer monitoring
● Heart-beat & Continuous message latency
● On-demand Broker performance testing
● Built as a service deployable on single or multiple instances
Kafka Management UI (Beta)
Open sourcing on the road map
Netflix Keystone Pipeline at Samza Meetup 10-13-2015
Netflix Keystone Pipeline at Samza Meetup 10-13-2015
Netflix Keystone Pipeline at Samza Meetup 10-13-2015
Netflix Keystone Pipeline at Samza Meetup 10-13-2015

Weitere Àhnliche Inhalte

Was ist angesagt?

Data pipeline with kafka
Data pipeline with kafkaData pipeline with kafka
Data pipeline with kafkaMole Wong
 
The Netflix Way to deal with Big Data Problems
The Netflix Way to deal with Big Data ProblemsThe Netflix Way to deal with Big Data Problems
The Netflix Way to deal with Big Data ProblemsMonal Daxini
 
Flink at netflix paypal speaker series
Flink at netflix   paypal speaker seriesFlink at netflix   paypal speaker series
Flink at netflix paypal speaker seriesMonal Daxini
 
(BDT318) How Netflix Handles Up To 8 Million Events Per Second
(BDT318) How Netflix Handles Up To 8 Million Events Per Second(BDT318) How Netflix Handles Up To 8 Million Events Per Second
(BDT318) How Netflix Handles Up To 8 Million Events Per SecondAmazon Web Services
 
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...Amazon Web Services
 
Beaming flink to the cloud @ netflix ff 2016-monal-daxini
Beaming flink to the cloud @ netflix   ff 2016-monal-daxiniBeaming flink to the cloud @ netflix   ff 2016-monal-daxini
Beaming flink to the cloud @ netflix ff 2016-monal-daxiniMonal Daxini
 
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
 
Deploying Kafka at Dropbox, Mark Smith, Sean Fellows
Deploying Kafka at Dropbox, Mark Smith, Sean FellowsDeploying Kafka at Dropbox, Mark Smith, Sean Fellows
Deploying Kafka at Dropbox, Mark Smith, Sean Fellowsconfluent
 
ApacheCon2019 Talk: Kafka, Cassandra and Kubernetes at Scale – Real-time Ano...
ApacheCon2019 Talk: Kafka, Cassandra and Kubernetesat Scale – Real-time Ano...ApacheCon2019 Talk: Kafka, Cassandra and Kubernetesat Scale – Real-time Ano...
ApacheCon2019 Talk: Kafka, Cassandra and Kubernetes at Scale – Real-time Ano...Paul Brebner
 
Kafka At Scale in the Cloud
Kafka At Scale in the CloudKafka At Scale in the Cloud
Kafka At Scale in the Cloudconfluent
 
From Three Nines to Five Nines - A Kafka Journey
From Three Nines to Five Nines - A Kafka JourneyFrom Three Nines to Five Nines - A Kafka Journey
From Three Nines to Five Nines - A Kafka JourneyAllen (Xiaozhong) Wang
 
Architecture of a Kafka camus infrastructure
Architecture of a Kafka camus infrastructureArchitecture of a Kafka camus infrastructure
Architecture of a Kafka camus infrastructuremattlieber
 
Should you read Kafka as a stream or in batch? Should you even care? | Ido Na...
Should you read Kafka as a stream or in batch? Should you even care? | Ido Na...Should you read Kafka as a stream or in batch? Should you even care? | Ido Na...
Should you read Kafka as a stream or in batch? Should you even care? | Ido Na...HostedbyConfluent
 
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARN
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARNApache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARN
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARNblueboxtraveler
 
Apache Kafka at LinkedIn
Apache Kafka at LinkedInApache Kafka at LinkedIn
Apache Kafka at LinkedInDiscover Pinterest
 
Arc305 how netflix leverages multiple regions to increase availability an i...
Arc305 how netflix leverages multiple regions to increase availability   an i...Arc305 how netflix leverages multiple regions to increase availability   an i...
Arc305 how netflix leverages multiple regions to increase availability an i...Ruslan Meshenberg
 
Kafka and Storm - event processing in realtime
Kafka and Storm - event processing in realtimeKafka and Storm - event processing in realtime
Kafka and Storm - event processing in realtimeGuido Schmutz
 
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
 
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...DataWorks Summit/Hadoop Summit
 

Was ist angesagt? (20)

Data pipeline with kafka
Data pipeline with kafkaData pipeline with kafka
Data pipeline with kafka
 
The Netflix Way to deal with Big Data Problems
The Netflix Way to deal with Big Data ProblemsThe Netflix Way to deal with Big Data Problems
The Netflix Way to deal with Big Data Problems
 
Flink at netflix paypal speaker series
Flink at netflix   paypal speaker seriesFlink at netflix   paypal speaker series
Flink at netflix paypal speaker series
 
(BDT318) How Netflix Handles Up To 8 Million Events Per Second
(BDT318) How Netflix Handles Up To 8 Million Events Per Second(BDT318) How Netflix Handles Up To 8 Million Events Per Second
(BDT318) How Netflix Handles Up To 8 Million Events Per Second
 
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...
 
Beaming flink to the cloud @ netflix ff 2016-monal-daxini
Beaming flink to the cloud @ netflix   ff 2016-monal-daxiniBeaming flink to the cloud @ netflix   ff 2016-monal-daxini
Beaming flink to the cloud @ netflix ff 2016-monal-daxini
 
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
 
Deploying Kafka at Dropbox, Mark Smith, Sean Fellows
Deploying Kafka at Dropbox, Mark Smith, Sean FellowsDeploying Kafka at Dropbox, Mark Smith, Sean Fellows
Deploying Kafka at Dropbox, Mark Smith, Sean Fellows
 
ApacheCon2019 Talk: Kafka, Cassandra and Kubernetes at Scale – Real-time Ano...
ApacheCon2019 Talk: Kafka, Cassandra and Kubernetesat Scale – Real-time Ano...ApacheCon2019 Talk: Kafka, Cassandra and Kubernetesat Scale – Real-time Ano...
ApacheCon2019 Talk: Kafka, Cassandra and Kubernetes at Scale – Real-time Ano...
 
Kafka At Scale in the Cloud
Kafka At Scale in the CloudKafka At Scale in the Cloud
Kafka At Scale in the Cloud
 
From Three Nines to Five Nines - A Kafka Journey
From Three Nines to Five Nines - A Kafka JourneyFrom Three Nines to Five Nines - A Kafka Journey
From Three Nines to Five Nines - A Kafka Journey
 
Architecture of a Kafka camus infrastructure
Architecture of a Kafka camus infrastructureArchitecture of a Kafka camus infrastructure
Architecture of a Kafka camus infrastructure
 
Should you read Kafka as a stream or in batch? Should you even care? | Ido Na...
Should you read Kafka as a stream or in batch? Should you even care? | Ido Na...Should you read Kafka as a stream or in batch? Should you even care? | Ido Na...
Should you read Kafka as a stream or in batch? Should you even care? | Ido Na...
 
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARN
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARNApache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARN
Apache Samza: Reliable Stream Processing Atop Apache Kafka and Hadoop YARN
 
Apache Kafka at LinkedIn
Apache Kafka at LinkedInApache Kafka at LinkedIn
Apache Kafka at LinkedIn
 
Kafka - Linkedin's messaging backbone
Kafka - Linkedin's messaging backboneKafka - Linkedin's messaging backbone
Kafka - Linkedin's messaging backbone
 
Arc305 how netflix leverages multiple regions to increase availability an i...
Arc305 how netflix leverages multiple regions to increase availability   an i...Arc305 how netflix leverages multiple regions to increase availability   an i...
Arc305 how netflix leverages multiple regions to increase availability an i...
 
Kafka and Storm - event processing in realtime
Kafka and Storm - event processing in realtimeKafka and Storm - event processing in realtime
Kafka and Storm - event processing in realtime
 
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
 
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
Building Large-Scale Stream Infrastructures Across Multiple Data Centers with...
 

Ähnlich wie Netflix Keystone Pipeline at Samza Meetup 10-13-2015

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...DataWorks Summit/Hadoop Summit
 
Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2aspyker
 
AWS Lambdas are cool - Cheminfo Stories Day 1
AWS Lambdas are cool - Cheminfo Stories Day 1AWS Lambdas are cool - Cheminfo Stories Day 1
AWS Lambdas are cool - Cheminfo Stories Day 1ChemAxon
 
Apache samza past, present and future
Apache samza  past, present and futureApache samza  past, present and future
Apache samza past, present and futureEd Yakabosky
 
CPN302 your-linux-ami-optimization-and-performance
CPN302 your-linux-ami-optimization-and-performanceCPN302 your-linux-ami-optimization-and-performance
CPN302 your-linux-ami-optimization-and-performanceCoburn Watson
 
Your Linux AMI: Optimization and Performance (CPN302) | AWS re:Invent 2013
Your Linux AMI: Optimization and Performance (CPN302) | AWS re:Invent 2013Your Linux AMI: Optimization and Performance (CPN302) | AWS re:Invent 2013
Your Linux AMI: Optimization and Performance (CPN302) | AWS re:Invent 2013Amazon Web Services
 
Apache Samza Past, Present and Future
Apache Samza  Past, Present and FutureApache Samza  Past, Present and Future
Apache Samza Past, Present and FutureKartik Paramasivam
 
Netflix Data Pipeline With Kafka
Netflix Data Pipeline With KafkaNetflix Data Pipeline With Kafka
Netflix Data Pipeline With KafkaSteven Wu
 
Microservices with Micronaut
Microservices with MicronautMicroservices with Micronaut
Microservices with MicronautQAware GmbH
 
Big data Argentina meetup 2020-09: Intro to presto on docker
Big data Argentina meetup 2020-09: Intro to presto on dockerBig data Argentina meetup 2020-09: Intro to presto on docker
Big data Argentina meetup 2020-09: Intro to presto on dockerFederico Palladoro
 
SamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationSamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationYi Pan
 
Introduction to apache kafka
Introduction to apache kafkaIntroduction to apache kafka
Introduction to apache kafkaSamuel Kerrien
 
Beam me up, Samza!
Beam me up, Samza!Beam me up, Samza!
Beam me up, Samza!Xinyu Liu
 
EVCache: Lowering Costs for a Low Latency Cache with RocksDB
EVCache: Lowering Costs for a Low Latency Cache with RocksDBEVCache: Lowering Costs for a Low Latency Cache with RocksDB
EVCache: Lowering Costs for a Low Latency Cache with RocksDBScott Mansfield
 
Salesforce enabling real time scenarios at scale using kafka
Salesforce enabling real time scenarios at scale using kafkaSalesforce enabling real time scenarios at scale using kafka
Salesforce enabling real time scenarios at scale using kafkaThomas Alex
 
Pulsar summit asia 2021 apache pulsar with mqtt for edge computing
Pulsar summit asia 2021   apache pulsar with mqtt for edge computingPulsar summit asia 2021   apache pulsar with mqtt for edge computing
Pulsar summit asia 2021 apache pulsar with mqtt for edge computingTimothy Spann
 
Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)DataWorks Summit
 
Strata Singapore: Gearpump Real time DAG-Processing with Akka at Scale
Strata Singapore: GearpumpReal time DAG-Processing with Akka at ScaleStrata Singapore: GearpumpReal time DAG-Processing with Akka at Scale
Strata Singapore: Gearpump Real time DAG-Processing with Akka at ScaleSean Zhong
 
DevoxxUK: Optimizating Application Performance on Kubernetes
DevoxxUK: Optimizating Application Performance on KubernetesDevoxxUK: Optimizating Application Performance on Kubernetes
DevoxxUK: Optimizating Application Performance on KubernetesDinakar Guniguntala
 
EVCache & Moneta (GoSF)
EVCache & Moneta (GoSF)EVCache & Moneta (GoSF)
EVCache & Moneta (GoSF)Scott Mansfield
 

Ähnlich wie Netflix Keystone Pipeline at Samza Meetup 10-13-2015 (20)

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...
 
Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2
 
AWS Lambdas are cool - Cheminfo Stories Day 1
AWS Lambdas are cool - Cheminfo Stories Day 1AWS Lambdas are cool - Cheminfo Stories Day 1
AWS Lambdas are cool - Cheminfo Stories Day 1
 
Apache samza past, present and future
Apache samza  past, present and futureApache samza  past, present and future
Apache samza past, present and future
 
CPN302 your-linux-ami-optimization-and-performance
CPN302 your-linux-ami-optimization-and-performanceCPN302 your-linux-ami-optimization-and-performance
CPN302 your-linux-ami-optimization-and-performance
 
Your Linux AMI: Optimization and Performance (CPN302) | AWS re:Invent 2013
Your Linux AMI: Optimization and Performance (CPN302) | AWS re:Invent 2013Your Linux AMI: Optimization and Performance (CPN302) | AWS re:Invent 2013
Your Linux AMI: Optimization and Performance (CPN302) | AWS re:Invent 2013
 
Apache Samza Past, Present and Future
Apache Samza  Past, Present and FutureApache Samza  Past, Present and Future
Apache Samza Past, Present and Future
 
Netflix Data Pipeline With Kafka
Netflix Data Pipeline With KafkaNetflix Data Pipeline With Kafka
Netflix Data Pipeline With Kafka
 
Microservices with Micronaut
Microservices with MicronautMicroservices with Micronaut
Microservices with Micronaut
 
Big data Argentina meetup 2020-09: Intro to presto on docker
Big data Argentina meetup 2020-09: Intro to presto on dockerBig data Argentina meetup 2020-09: Intro to presto on docker
Big data Argentina meetup 2020-09: Intro to presto on docker
 
SamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationSamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentation
 
Introduction to apache kafka
Introduction to apache kafkaIntroduction to apache kafka
Introduction to apache kafka
 
Beam me up, Samza!
Beam me up, Samza!Beam me up, Samza!
Beam me up, Samza!
 
EVCache: Lowering Costs for a Low Latency Cache with RocksDB
EVCache: Lowering Costs for a Low Latency Cache with RocksDBEVCache: Lowering Costs for a Low Latency Cache with RocksDB
EVCache: Lowering Costs for a Low Latency Cache with RocksDB
 
Salesforce enabling real time scenarios at scale using kafka
Salesforce enabling real time scenarios at scale using kafkaSalesforce enabling real time scenarios at scale using kafka
Salesforce enabling real time scenarios at scale using kafka
 
Pulsar summit asia 2021 apache pulsar with mqtt for edge computing
Pulsar summit asia 2021   apache pulsar with mqtt for edge computingPulsar summit asia 2021   apache pulsar with mqtt for edge computing
Pulsar summit asia 2021 apache pulsar with mqtt for edge computing
 
Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)Kafka to the Maxka - (Kafka Performance Tuning)
Kafka to the Maxka - (Kafka Performance Tuning)
 
Strata Singapore: Gearpump Real time DAG-Processing with Akka at Scale
Strata Singapore: GearpumpReal time DAG-Processing with Akka at ScaleStrata Singapore: GearpumpReal time DAG-Processing with Akka at Scale
Strata Singapore: Gearpump Real time DAG-Processing with Akka at Scale
 
DevoxxUK: Optimizating Application Performance on Kubernetes
DevoxxUK: Optimizating Application Performance on KubernetesDevoxxUK: Optimizating Application Performance on Kubernetes
DevoxxUK: Optimizating Application Performance on Kubernetes
 
EVCache & Moneta (GoSF)
EVCache & Moneta (GoSF)EVCache & Moneta (GoSF)
EVCache & Moneta (GoSF)
 

KĂŒrzlich hochgeladen

Understanding the FAA Part 107 License ..
Understanding the FAA Part 107 License ..Understanding the FAA Part 107 License ..
Understanding the FAA Part 107 License ..Christopher Logan Kennedy
 
MINDCTI Revenue Release Quarter One 2024
MINDCTI Revenue Release Quarter One 2024MINDCTI Revenue Release Quarter One 2024
MINDCTI Revenue Release Quarter One 2024MIND CTI
 
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...apidays
 
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProduct Anonymous
 
ICT role in 21st century education and its challenges
ICT role in 21st century education and its challengesICT role in 21st century education and its challenges
ICT role in 21st century education and its challengesrafiqahmad00786416
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century educationjfdjdjcjdnsjd
 
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherRemote DBA Services
 
Mcleodganj Call Girls đŸ„° 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls đŸ„° 8617370543 Service Offer VIP Hot ModelMcleodganj Call Girls đŸ„° 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls đŸ„° 8617370543 Service Offer VIP Hot ModelDeepika Singh
 
Polkadot JAM Slides - Token2049 - By Dr. Gavin Wood
Polkadot JAM Slides - Token2049 - By Dr. Gavin WoodPolkadot JAM Slides - Token2049 - By Dr. Gavin Wood
Polkadot JAM Slides - Token2049 - By Dr. Gavin WoodJuan lago vĂĄzquez
 
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ..."I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...Zilliz
 
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobe
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, AdobeApidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobe
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobeapidays
 
Modular Monolith - a Practical Alternative to Microservices @ Devoxx UK 2024
Modular Monolith - a Practical Alternative to Microservices @ Devoxx UK 2024Modular Monolith - a Practical Alternative to Microservices @ Devoxx UK 2024
Modular Monolith - a Practical Alternative to Microservices @ Devoxx UK 2024Victor Rentea
 
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWER
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWEREMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWER
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWERMadyBayot
 
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Angeliki Cooney
 
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...apidays
 
MS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsMS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsNanddeep Nachan
 
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers:  A Deep Dive into Serverless Spatial Data and FMECloud Frontiers:  A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FMESafe Software
 
Boost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdfBoost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdfsudhanshuwaghmare1
 
Artificial Intelligence Chap.5 : Uncertainty
Artificial Intelligence Chap.5 : UncertaintyArtificial Intelligence Chap.5 : Uncertainty
Artificial Intelligence Chap.5 : UncertaintyKhushali Kathiriya
 

KĂŒrzlich hochgeladen (20)

Understanding the FAA Part 107 License ..
Understanding the FAA Part 107 License ..Understanding the FAA Part 107 License ..
Understanding the FAA Part 107 License ..
 
MINDCTI Revenue Release Quarter One 2024
MINDCTI Revenue Release Quarter One 2024MINDCTI Revenue Release Quarter One 2024
MINDCTI Revenue Release Quarter One 2024
 
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
 
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
 
ICT role in 21st century education and its challenges
ICT role in 21st century education and its challengesICT role in 21st century education and its challenges
ICT role in 21st century education and its challenges
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century education
 
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a Fresher
 
Mcleodganj Call Girls đŸ„° 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls đŸ„° 8617370543 Service Offer VIP Hot ModelMcleodganj Call Girls đŸ„° 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls đŸ„° 8617370543 Service Offer VIP Hot Model
 
Polkadot JAM Slides - Token2049 - By Dr. Gavin Wood
Polkadot JAM Slides - Token2049 - By Dr. Gavin WoodPolkadot JAM Slides - Token2049 - By Dr. Gavin Wood
Polkadot JAM Slides - Token2049 - By Dr. Gavin Wood
 
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ..."I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
 
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobe
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, AdobeApidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobe
Apidays New York 2024 - Scaling API-first by Ian Reasor and Radu Cotescu, Adobe
 
Modular Monolith - a Practical Alternative to Microservices @ Devoxx UK 2024
Modular Monolith - a Practical Alternative to Microservices @ Devoxx UK 2024Modular Monolith - a Practical Alternative to Microservices @ Devoxx UK 2024
Modular Monolith - a Practical Alternative to Microservices @ Devoxx UK 2024
 
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWER
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWEREMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWER
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWER
 
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
 
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
 
MS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsMS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectors
 
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers:  A Deep Dive into Serverless Spatial Data and FMECloud Frontiers:  A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
 
Boost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdfBoost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdf
 
Artificial Intelligence Chap.5 : Uncertainty
Artificial Intelligence Chap.5 : UncertaintyArtificial Intelligence Chap.5 : Uncertainty
Artificial Intelligence Chap.5 : Uncertainty
 

Netflix Keystone Pipeline at Samza Meetup 10-13-2015

  • 1. A NETFLIX ORIGINAL SERVICE SAMZA Edition @ Samza Meetup, Oct 2015 @monaldax https://www.linkedin.com/in/monaldaxini
  • 2. What am I going to learn ? ● Situational Awareness ● Routing Service ● Samza Alterations ● Sprinkles of Best Practices - in our opinion
  • 3. 550 billion events per day 8.5 million events (22 GB per second) peak Hundreds of event types Over 1 Petabyte / day Numbers Galore!
  • 4. 550 billion events per day 8.5 million events (22 GB per second) peak Hundreds of event types Over 1 Petabyte / day Numbers Galore! 550 x 2
  • 7. Where are we going?
  • 11. Injected Event Metadata ● GUID ● Timestamp ● Host ● App
  • 13. Keystone Extensible Wire Protocol ● Invisible to source & sinks ● Backwards and forwards compatibility ● Supports JSON, AVRO on the horizon ● Efficient - 10 bytes overhead per message ○ message size - hundreds of bytes to 10MB
  • 14. Keystone Extensible Wire Protocol ● Packaged as a jar - Chaski ● Why? ○ Evolve Independently ■ event metadata & traceability metadata ■ event payload serialization
  • 15. Restrictive Access to Fronting Kafka Clusters
  • 16. Why? ● Better manage 2700 brokers across 4x3 (12) clusters and 3 regions ○ Availability ○ Scalability ○ Decoupling ○ SLA
  • 17. Access to Fronting Kafka Clusters ● Client Library ○ Wraps Kafka producer ○ Integrates with Netflix ecosystem ● REST proxy ○ Uses the client library ● Wire protocol Non-conformant messages dropped Side Effect
  • 22. Router Job Manager (Control Plane) EC2 Instances Zookeeper (Instance Id assignment) Job Job Job ksnode Checkpointing Cluster ASG
  • 23. Distributed Systems are Hard Keep it Simple Minimize Moving Parts
  • 24. Obey! Obey the principles without being bound by them. - Bruce Lee
  • 25. Mind bender - Sink Isolation ● Multiple Samza jobs for one Kafka source topic ● Each job processes messages for one sink ○ E.g. separate job for each S3 & ElasticSearch cluster sinks ● Tradeoff ○ Sink isolation for extra load on Kafka source topic cluster ● Initial release ○ Each job processes partitions only from one topic
  • 26. Samza Job Details ● Use window function to implement health check ○ task.window.ms=30000 ● Batch requests to sinks ● Explicit offset commits only ○ automatic commits disabled - task.commit.ms=-1
  • 28. Checkpoint Topic Name "__samza_checkpoint_ver_%d_for_%s_%s" format (CHECKPOINT_LOG_VERSION_NUMBER, jobName. replaceAll("_", "-"), jobId.replaceAll("_", "-")) Not fully Configurable
  • 29. Mind bender - Checkpoint Cluster ● __samza_checkpoint_ver_$ver_$job.name_$job.id ● Topic name example ○ __samza_checkpoint_ver_1_for_ksrouter_kafka-elbert-map-audit-s3 ○ job.name = ksrouter job.id = $kafkaCluster_$topic_$sink
  • 30. POWERFULL! 1 checkpoint topic per sink, & source topic for many Samza Jobs
  • 31. 1 checkpoint topic per kafka cluster, sink, source topic ● Change the number of samza jobs for a topic ● Easily redistribute the partitions across jobs ● Add new partitions seamlessly ● Our naming scheme facilitates migrating topics to other clusters
  • 32. Job Startup Delays reading Checkpoint Causing health check failures - timeout 5 min What to do?
  • 33. Checkpoint Topic Broker Default Config default.replication.factor 3 log.cleaner.delete.retention.ms 30000 log.cleaner.enable true log.cleaner.min.cleanable.ratio 0.25 og.cleaner.threads 5 log.segment.bytes 3145728 ●
  • 34. Checkpoint topic Samza Job Configuration Replication factor is hard coded to 3 task.checkpoint.system=checkpoint task.checkpoint.factory=org.apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory task.checkpoint.segment.bytes=3145728
  • 35. Add. Checkpoint Information ● About 300 bytes per offset commit ● Change log topic logs into the same checkpoint offset topic ○ Even if not enabled, one time large message with system-stream-partition inserted into the same checkpoint offset topic
  • 39. Yes! You inferred right! No Mesos & No Yarn
  • 41. Using ThreadJobFactory in production job.factory.class=org.apache.samza.job.local.ThreadJobFactory
  • 42. SAMZA-41 - static partition range assignment job.systemstreampartition.matcher.class= org.apache.samza.system.RegexSystemStreamPartitionMatcher job.systemstreampartition.matcher.config.ranges=[8-10] ^8&|^9$|^10$ you need
  • 43. SAMZA-41 - static partition range assignment Simplify... job.systemstreampartition.matcher.class= org.apache.samza.system.RangeSystemStreamPartitionMatcher job.systemstreampartition.matcher.config.ranges=6-10
  • 44. Prefetch Buffer - When is it going to OOM? ● Default count based per Samza container ○ (50,000 / # partitions) per topic ○ systems.source.samza.fetch.threshold=50000 ● Hard to get it right and avoid OOM ○ changing message size
  • 45. SAMZA-775- size based Prefetch buffer ● How much of heap should I use for prefetching? ○ systems.source.samza.fetch.threshold.bytes=200000000 (200MB) ○ per system / stream / partition ○ if > 0 precedence over systems.source.samza.fetch.threshold
  • 46. SAMZA-775- size based Prefetch buffer ● systems.source.samza.fetch.threshold.bytes is a Soft limit ○ bytes limit + size of last max message stream ● I don’t get it, where is the example?
  • 47. SAMZA-775- size based Prefetch buffer ● systems.source.samza.fetch.threshold.bytes=100000 (100K) ● 50 SystemStreamPartitions ● per system-stream-partition threshold is (100000 / 2) / 50 = 1000 bytes. ● Enforced limit would be ○ 1000 bytes + size of last message from the partition
  • 48. SAMZA-775- size based Prefetch buffer ● Value of systems.source.samza.fetch.threshold.bytes based on ○ Incoming traffic Bps into source Kafka ○ 60 seconds of buffer with region failover traffic ○ Samza in memory data structures (2 x message size)
  • 49. SAMZA-775- size based Prefetch buffer ● How does it perform? ○ Per message overhead within 0.02% of computed heuristics in the patch ○ Actual footprint of systems.source.samza.fetch.threshold.bytes is 10-15% at the most in worst case. ■ Example: If set to 200MB, worst case observed 230MB
  • 50. SAMZA-775- size based Prefetch buffer ● Con ○ Implementation to enforce systems.source.samza.fetch.threshold.bytes is very dependent on the implementation version of Samza ○ Hence, higher maintenance when code changes. However, Well Worth It! Ergonmic Config! Adds Stability!
  • 51. SAMZA-655 & SAMZA-540 ● Backported from 0.10 ○ environment variable configuration rewriter ■ Pass config from RDS to executor to Docker to Samza Job ○ expose latency related metrics in OffsetManager ■ checkpointed offset guage
  • 53. Immutable Config in Running Job
  • 54. Integration with NetlixOSS Ecosystem ● Atlas ○ Alert & Monitoring system ● Eureka ○ Service discovery
  • 55. YMMV
  • 57. Router Stats ● S3 sink ○ 620 c3.4xl across 3 regions running 6736 docker containers ○ Avg 21 MBps (168 Mbps) per container ● On its way... ○ Kafka sink - 280 c3.4xl across 3 regions running 3200 docker containers ○ ElasticSearch sink - 70 c3.4xl across 3 regions running 850 containers
  • 58. AWS c3.4xl ● 16 vCPU ● 30GB Ram ● 320GB SSD - 160 x 2 ● High gigabit network ○ support for SR-IOV when we move to VPC
  • 59. Per Container Reserved ● 2G - 5G memory ○ based on incoming traffic rate, prefetch buffer ● 160 mbps max network bandwidth ● 1 CPU Share ● 20G disk for buffer & logs ● Processes 1-12 partitions
  • 60. Observed - Per Container ● Avg memory ~1.8G ● Avg memory usage ~ 20G ● Avg CPU utilization 8% ● Avg NetworkIn 256Mbps ● Avg NetworkOut 156Mbps ○ outgoing data compressed I/O bound, very close to reserved capacity
  • 61. Metrics ● External process Auditor - continuously monitors & diffs offset checkpoint & source Kafka topic offset ○ consumerLag ○ missingConsumerOffset ○ stuckConsumer ○ logOffset ○ consumerOffset
  • 65. End to End metrics ● Producer to Router latency ○ Avg. about 2.5 seconds ○ 90 percentile topics under 2 sec ● Kafka to Router consumer lag (estimated time to catch up) ○ 65 percentile under 500ms ○ 90 percentile under 5 seconds ● Producer event timestamp to Samza job router avg latency - 6 seconds
  • 66. Additional Metrics ● uploaded-messages ● uploaded-bytes ● upload-ms ● upload ● processed-messages ● processed-bytes ● message-latency ● latest-message-timestamp ● compression-ratio-timer ● compression-ratio-gauge ● window-execution-ms ● window-interval-ms ● window-error ● kssamza-offset-change ● kssamza-offset ● kssamza-messages-behind-high-watermark ● kssamza-high-watermark ● kssamza-checkpointed-offset ● kssamza-buffered-message-size ● kssamza-buffered-message-count
  • 67. Backpressure Producer ⇐ Kafka Cluster ⇐ Samza job router ⇐ Sink
  • 68. Wait there’s more in the pipeline... ● Self service tools ● Multi-tenant Stream Processing as a Service - SPaaS ○ probably add spark streaming to the mix ● Event traceability - on demand and sampled ● As number of jobs increase checkpoint topic may give way to Cassandra ● Optimization & Automation
  • 71. Fronting Kafka Clusters ● Normal-priority (majority) ○ 2 clusters ○ 2 copies, 8 hour retention, 4 hour log roll ● High-priority (streaming activities etc.) ○ 2 clusters ○ 3 copies, 24 hour retention, 12 hour log roll
  • 72. Fronting Kafka Instances ● 2700 d2.xl AWS instances across 3 regions for regular & failover traffic ● d2.xl ○ Large disk (6TB) - 450-475MB/s of sequential I/O throughput ○ 30GB memory, 700 Mbps medium network capability ○ Replication lag above 18MB/second per broker with thousands of partitions ○ cons: multiple instances on same physical host - increases failures
  • 73. Kafka Capacity Planning 1. Stay under 20k partitions per cluster (14K) 2. Leave ≅ 40% free disk space on each broker for growth & movement 3. Throughput per partition based on 1, 2, # of brokers, and the retention period
  • 74. Partition Assignment ● All assignments Zone / Rack aware ● Strategy 1 - Multiple of brokers ● Stategy 2 - Stateful Round Robin
  • 75. Kafka Auditor as a Service ● Broker monitoring ● Consumer monitoring ● Heart-beat & Continuous message latency ● On-demand Broker performance testing ● Built as a service deployable on single or multiple instances
  • 76. Kafka Management UI (Beta) Open sourcing on the road map