SlideShare une entreprise Scribd logo
1  sur  33
Apache Samza
Past, Present and Future
Kartik Paramasivam
Director of Engineering, Streams Infra@ LinkedIn
Agenda
1. Stream Processing
2. State of the Union
3. Apache Samza : Key Differentiators
4. Apache Samza Futures
Stream Processing: Processing events as soon as they happen..
● Stateless Processing
■ Transformation etc.
■ Lookup adjunct data (lookup databases/call services )
■ Producing results for every event
● Stateful Processing
■ Triggering/Producing results periodically (time-windows)
● Maintain intermediate state
■ E.g. Joining across multiple streams of events.
Stream Processing: State of the Union
MillwheelStorm
Heron
Spark Streaming
S4
Dempsey
Samza
Flink
Beam
Dataflow
Azure Stream Analytics
AWS Kinesis AnalyticsGearPump
Kafka Streams
Orleans
Not meant to be an
accurate timeline..
Yes It is CROWDED !!
Apache Samza
● Top level Apache project since Dec 2014
● 5 big Releases (0.7, 0.8, 0.9, 0.10, 0.11)
● 62 Contributors
● 14 Committers
● Companies using : LinkedIn, Uber, MetaMarkets, Netflix,
Intuit, TripAdvisor, MobileAware, Optimizely ….
https://cwiki.apache.org/confluence/display/SAMZA/Powered+By
● Applications at LinkedIn : from ~20 to ~200 in 2 years.
Key Differentiators for Apache Samza
● Performance !!
● Stability
● Support for a variety of input sources
● Stream processing as a service AND as an embedded library
Performance : Accessing Adjunct Data
Samza Processor
Database
Remote-read
Samza Processor
Capture changes
Databus,
Brooklin
Rocks-
DB
Local read
Database
Local data access
Remote data access
Input stream
Input stream
Performance : Maintaining Temporary State
Samza Processor Remote
Database
Read-Write
Samza Processor
Backup changes
Kafka Change
Log(Log
compacted)Rocks-
DB
Local read/write
Local data access
Remote data access
Input stream
Input stream
In Memory
Store
Performance : Let us talk numbers !
● 100x Difference between using Local State vs Remote No-Sql store
● Local State details:
○ 1.1 Million TPS on a single processing machine (SSD)
○ Used a 3 node Kafka cluster for storing the durable changelog
● Remote State details:
○ 8500 TPS when the Samza job was changed to accessing a remote No-
Sql store
○ No-Sql Store was also on a 3 node (ssd) cluster
Remote State : Asynchronous Event Processing
Event Loop
(Single thread)
ProcessAsync
Remote
DB
/Services
Asynchronous I/O calls,
using Java Nio, Netty...
Responses sent to
main thread via
callback
Event loop is woken up to process next message
Task.max.concurrency >1 to
enable pipelining
Available with Samza 0.11
Remote State: Synchronous Processing on Multiple
Threads
Event Loop
(Single thread)
Schedule
Process()
Remote
DB/
Services
Built-In
Thread pool
Blocking I/O
calls
Event loop is
woken up by
the worker
thread
job.container.thread.pool.size = N
Available with Samza 0.11
Incremental Checkpointing : MVP for stateful apps
Some applications
have ~ 2 TB state in
production
Stateful apps don’t
really work without
incremental
checkpointing
Samza Task
State
changelog
checkpoint
Host 1
Input stream(e.g. Kafka)
Key Differentiators for Apache Samza
● Performance !!
● Stability
● Support for a variety of input sources
● Stream processing as a service AND as an embedded library
Speed Thrills .. but can kill
● Local State Considerations:
○ State should NOT be reseeded under normal operations (e.g.
Upgrades, Application restarts)
○ Minimal State should be reseeded
- If a container dies/removed
- If a container is added
How Samza keeps Local state ‘stable’ ?
P0
P1 P2 P3
Task-0 Task-1 Task-2 Task-3
P0
P1
P2
P3
Host-E Host-B Host-C
Coordinator Stream:
Task-Container-Host Mapping
Container-0 -> Host-E
Container-1 -> Host-B
Container-2 -> Host-C
AM JC
Yarn-RM
Ask: Host-E Allocate: Host-E
Samza
Job
Input
Stream
Change-log
Enable Continuous Scheduling
Stream A
Stream
B
Stream C
Job 1
Job 2
● Kafka or durable intermediate
queues are leveraged to
avoid backpressure issues in
a pipeline.
● Allows each stage to be
independent of the next stage
Backpressure in a Pipeline
Key Differentiators for Apache Samza
● Performance !!
● Stability
● Support for a variety of input sources
● Stream processing as a service AND as an embedded library
Pluggable system consumers
Samza Processor
Mongo
DB
DynamoDB
Streams
Kafka
Databus/
Brooklin
Kinesis
ZeroMQ
… Azure EventHub,
Azure Document DB,
Google Pub-Sub etc.
Oracle,
Espresso
Dynamo-
DB
Batch processing in Samza!! (NEW)
● HDFS system consumer for Samza
● Same Samza processor can be used for processing
events from Kafka and HDFS with no code changes
● Scenarios :
○ Experimentation and Testing
○ Re-processing of large datasets
○ Some datasets are readily available on HDFS
Samza - HDFS support
HDFS input
(Samza)
Processor HDFS
output
(Samza)
Processor HDFS
output
HDFS input
(Samza)
Processor
Kafka
Kafka
New
Available since
Samza 0.10
The batch job
auto-terminates
when the input is
fully processed.
Reprocessing entire Dataset online
Updates
(Samza)
Processor
(Samza)
Processor
Bootstrap
output
Kafka
Brooklin
Brooklin
Database
(espresso)
set offset=0
Nearline
Processing
Reprocessing ‘large’ Datasets “offline”
Updates
(Samza)
Processor
(Samza)
Processor
Backup
output
Kafka
Databus
Database
(espresso)
Database
Backup
(HDFS)
Nearline
Processing
Offline
Processing
Samza batch pipelines
(Samza)
Processor
HDFS
output
HDFS
input
(Samza)
Processor
Kafka
(Samza)
Processor
HDFS
output
HDFS
input
(Samza)
Processor HDFS
Samza- HDFS Early Performance Results !!
Benchmark : Count number
of records grouped by <Field>
DataSize (bytes): 250 GB
Number of files : 487
Samza
Map/Reduce
Spark
Number of Containers
T
i
m
e
-
s
e
c
o
n
d
s
Key Differentiators for Apache Samza
● Performance !!
● Stability
● Support for a variety of input sources (batch and streaming)
● Stream processing as a service AND (coming soon) as an
embedded library
Stream Processing as a Service
● Based on YARN
○ Yarn-RM high availability
○ Work preserving RM
○ Support for Heterogenous hardware with Node Labels (NEW)
● Easy upgrade of Samza framework :
Use the Samza version deployed on the machine instead of packaging it with the application.
● Disk Quotas for local state (e.g. rocksDB state)
● Samza Management Service(SAMZA-REST)-> Next Slide
YARN
Resource
Managers
Nodes
in the
YARN
cluster
RM SRR RM SRR RM SRR
NM SRN
Samza Management Service (Samza REST) (NEW)
NM SRN NM SRN NM SRN
NM SRN NM SRN NM SRN NM SRN
/v1/jobs /v1/jobs /v1/jobs
Samza Containers
1. Exposes /jobs
resource to
start, stop, get
status of jobs
etc.
1. Cleans up
stores from
dead jobs
Samza
REST
YARN
processes(RM/NM)
Agenda
1. Stream processing
2. State of the union
3. Apache Samza : Key differentiators
4. Apache Samza Futures
Coming Soon : Samza as a Library
Stream Processor
Code
Job
Coordinator
Stream Processor
Code
Job
Coordinator
Stream Processor
Code
Job
Coordinator
...
Leader
● No YARN dependency
● Will use ZK for leader
election
● Embed stream processing
into your bigger application
StreamProcessor processor = new StreamProcessor (config, “job-name”,
“job-id”);
processor.start();
processor.awaitStart();
…
processor.stop();
Coming Soon: High Level API and Event Time (SAMZA-
914/915)
Count the number of PageViews by Region, every 30 minutes.
@Override public void init(Collection<SystemMessageStream> sources) {
sources.forEach(source -> {
Function<PageView, String> keyExtractor = view -> view.getRegion();
source.map(msg -> new PageViewMessage(msg))
.window(Windows.<PageViewMessage, String>intoSessionCounter(keyExtractor,
WindowType.Tumbling, 30*60 ))
});
}
Coming Soon: First class support for Pipelines
(Samza- 1041)
public class MyPipeline implements PipelineFactory {
public Pipeline create(Config config) {
Processor myShuffler = getShuffle(config);
Processor myJoiner = getJoin(config);
Stream inStream = getStream(config, “inStream1”);
// … omitted for brevity
PipelineBuilder builder = new PipelineBuilder();
return builder.addInputStreams(myShuffler, inStream1)
.addOutputStreams(myShuffler, intermediateOutStream)
.addInputStreams(myJoiner, intermediateOutStream, inStream2)
.addOutputStreams(myJoiner, finalOutStream)
.build();
}
}
Shuffle
Join
input output
Future: Miscellaneous
● Exactly once processing
● Making it easier to auto-scale even with Local State (on-
demand Standby containers)
● Turnkey Disaster Recovery for stateful applications
○ Easy Restore of changelog and checkpoints from
some other datacenter.
● Improved support for Batch jobs
● SQL over Streams
● A default Dashboard :)
Questions ?

Contenu connexe

Tendances

Exploring Reactive Integrations With Akka Streams, Alpakka And Apache Kafka
Exploring Reactive Integrations With Akka Streams, Alpakka And Apache KafkaExploring Reactive Integrations With Akka Streams, Alpakka And Apache Kafka
Exploring Reactive Integrations With Akka Streams, Alpakka And Apache Kafka
Lightbend
 

Tendances (20)

Netflix keystone streaming data pipeline @scale in the cloud-dbtb-2016
Netflix keystone   streaming data pipeline @scale in the cloud-dbtb-2016Netflix keystone   streaming data pipeline @scale in the cloud-dbtb-2016
Netflix keystone streaming data pipeline @scale in the cloud-dbtb-2016
 
Netflix Keystone Pipeline at Samza Meetup 10-13-2015
Netflix Keystone Pipeline at Samza Meetup 10-13-2015Netflix Keystone Pipeline at Samza Meetup 10-13-2015
Netflix Keystone Pipeline at Samza Meetup 10-13-2015
 
Netflix at-disney-09-26-2014
Netflix at-disney-09-26-2014Netflix at-disney-09-26-2014
Netflix at-disney-09-26-2014
 
Real Time Data Streaming using Kafka & Storm
Real Time Data Streaming using Kafka & StormReal Time Data Streaming using Kafka & Storm
Real Time Data Streaming using Kafka & Storm
 
Unbounded bounded-data-strangeloop-2016-monal-daxini
Unbounded bounded-data-strangeloop-2016-monal-daxiniUnbounded bounded-data-strangeloop-2016-monal-daxini
Unbounded bounded-data-strangeloop-2016-monal-daxini
 
Kafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 People
Kafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 PeopleKafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 People
Kafka Summit NYC 2017 - Running Hundreds of Kafka Clusters with 5 People
 
Bootstrapping Microservices with Kafka, Akka and Spark
Bootstrapping Microservices with Kafka, Akka and SparkBootstrapping Microservices with Kafka, Akka and Spark
Bootstrapping Microservices with Kafka, Akka and Spark
 
Netflix Keystone Pipeline at Big Data Bootcamp, Santa Clara, Nov 2015
Netflix Keystone Pipeline at Big Data Bootcamp, Santa Clara, Nov 2015Netflix Keystone Pipeline at Big Data Bootcamp, Santa Clara, Nov 2015
Netflix Keystone Pipeline at Big Data Bootcamp, Santa Clara, Nov 2015
 
Apache Samza Past, Present and Future
Apache Samza  Past, Present and FutureApache Samza  Past, Present and Future
Apache Samza Past, Present and Future
 
Exactly-once Stream Processing with Kafka Streams
Exactly-once Stream Processing with Kafka StreamsExactly-once Stream Processing with Kafka Streams
Exactly-once Stream Processing with Kafka Streams
 
Kafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache Kafka
Kafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache KafkaKafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache Kafka
Kafka Summit NYC 2017 - Introducing Exactly Once Semantics in Apache Kafka
 
Event Stream Processing with Kafka and Samza
Event Stream Processing with Kafka and SamzaEvent Stream Processing with Kafka and Samza
Event Stream Processing with Kafka and Samza
 
Exploring Reactive Integrations With Akka Streams, Alpakka And Apache Kafka
Exploring Reactive Integrations With Akka Streams, Alpakka And Apache KafkaExploring Reactive Integrations With Akka Streams, Alpakka And Apache Kafka
Exploring Reactive Integrations With Akka Streams, Alpakka And Apache Kafka
 
Temporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, Confluent
Temporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, ConfluentTemporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, Confluent
Temporal-Joins in Kafka Streams and ksqlDB | Matthias Sax, Confluent
 
Netflix Keystone - How Netflix Handles Data Streams up to 11M Events/Sec
Netflix Keystone - How Netflix Handles Data Streams up to 11M Events/SecNetflix Keystone - How Netflix Handles Data Streams up to 11M Events/Sec
Netflix Keystone - How Netflix Handles Data Streams up to 11M Events/Sec
 
Building Stream Processing Applications with Apache Kafka Using KSQL (Robin M...
Building Stream Processing Applications with Apache Kafka Using KSQL (Robin M...Building Stream Processing Applications with Apache Kafka Using KSQL (Robin M...
Building Stream Processing Applications with Apache Kafka Using KSQL (Robin M...
 
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
 
How to manage large amounts of data with akka streams
How to manage large amounts of data with akka streamsHow to manage large amounts of data with akka streams
How to manage large amounts of data with akka streams
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database System
 
Netflix Data Pipeline With Kafka
Netflix Data Pipeline With KafkaNetflix Data Pipeline With Kafka
Netflix Data Pipeline With Kafka
 

En vedette

En vedette (10)

Apache Samza - New features in the upcoming Samza release 0.10.0
Apache Samza - New features in the upcoming Samza release 0.10.0Apache Samza - New features in the upcoming Samza release 0.10.0
Apache Samza - New features in the upcoming Samza release 0.10.0
 
Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop Ecosystem Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop Ecosystem
 
Samza: Real-time Stream Processing at LinkedIn
Samza: Real-time Stream Processing at LinkedInSamza: Real-time Stream Processing at LinkedIn
Samza: Real-time Stream Processing at LinkedIn
 
IOT, Streaming Analytics and Machine Learning
IOT, Streaming Analytics and Machine Learning IOT, Streaming Analytics and Machine Learning
IOT, Streaming Analytics and Machine Learning
 
The art of infrastructure elasticity
The art of infrastructure elasticityThe art of infrastructure elasticity
The art of infrastructure elasticity
 
Performance Comparison of Streaming Big Data Platforms
Performance Comparison of Streaming Big Data PlatformsPerformance Comparison of Streaming Big Data Platforms
Performance Comparison of Streaming Big Data Platforms
 
Comparison of Open Source Frameworks for Integrating the Internet of Things
Comparison of Open Source Frameworks for Integrating the Internet of ThingsComparison of Open Source Frameworks for Integrating the Internet of Things
Comparison of Open Source Frameworks for Integrating the Internet of Things
 
Apache Flink: Real-World Use Cases for Streaming Analytics
Apache Flink: Real-World Use Cases for Streaming AnalyticsApache Flink: Real-World Use Cases for Streaming Analytics
Apache Flink: Real-World Use Cases for Streaming Analytics
 
Click-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer CheckpointingClick-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer Checkpointing
 
Flink vs. Spark
Flink vs. SparkFlink vs. Spark
Flink vs. Spark
 

Similaire à Apache samza past, present and future

Scala like distributed collections - dumping time-series data with apache spark
Scala like distributed collections - dumping time-series data with apache sparkScala like distributed collections - dumping time-series data with apache spark
Scala like distributed collections - dumping time-series data with apache spark
Demi Ben-Ari
 

Similaire à Apache samza past, present and future (20)

Samza portable runner for beam
Samza portable runner for beamSamza portable runner for beam
Samza portable runner for beam
 
Stream processing in python with Apache Samza and Beam
Stream processing in python with Apache Samza and BeamStream processing in python with Apache Samza and Beam
Stream processing in python with Apache Samza and Beam
 
Beam me up, Samza!
Beam me up, Samza!Beam me up, Samza!
Beam me up, Samza!
 
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
 
EVCache & Moneta (GoSF)
EVCache & Moneta (GoSF)EVCache & Moneta (GoSF)
EVCache & Moneta (GoSF)
 
SamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationSamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentation
 
Spark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedSpark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics Revised
 
Stream processing using Kafka
Stream processing using KafkaStream processing using Kafka
Stream processing using Kafka
 
Leverage Mesos for running Spark Streaming production jobs by Iulian Dragos a...
Leverage Mesos for running Spark Streaming production jobs by Iulian Dragos a...Leverage Mesos for running Spark Streaming production jobs by Iulian Dragos a...
Leverage Mesos for running Spark Streaming production jobs by Iulian Dragos a...
 
What no one tells you about writing a streaming app
What no one tells you about writing a streaming appWhat no one tells you about writing a streaming app
What no one tells you about writing a streaming app
 
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
 
Real time big data stream processing
Real time big data stream processing Real time big data stream processing
Real time big data stream processing
 
Real-time Data Processing Using AWS Lambda
Real-time Data Processing Using AWS LambdaReal-time Data Processing Using AWS Lambda
Real-time Data Processing Using AWS Lambda
 
Scalable complex event processing on samza @UBER
Scalable complex event processing on samza @UBERScalable complex event processing on samza @UBER
Scalable complex event processing on samza @UBER
 
Nike tech talk.2
Nike tech talk.2Nike tech talk.2
Nike tech talk.2
 
Netflix Keystone—Cloud scale event processing pipeline
Netflix Keystone—Cloud scale event processing pipelineNetflix Keystone—Cloud scale event processing pipeline
Netflix Keystone—Cloud scale event processing pipeline
 
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
 
Migrating Data Pipeline from MongoDB to Cassandra
Migrating Data Pipeline from MongoDB to CassandraMigrating Data Pipeline from MongoDB to Cassandra
Migrating Data Pipeline from MongoDB to Cassandra
 
Scala like distributed collections - dumping time-series data with apache spark
Scala like distributed collections - dumping time-series data with apache sparkScala like distributed collections - dumping time-series data with apache spark
Scala like distributed collections - dumping time-series data with apache spark
 
Scalable Stream Processing with Apache Samza
Scalable Stream Processing with Apache SamzaScalable Stream Processing with Apache Samza
Scalable Stream Processing with Apache Samza
 

Dernier

IAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI SolutionsIAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI Solutions
Enterprise Knowledge
 

Dernier (20)

Exploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone ProcessorsExploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone Processors
 
Data Cloud, More than a CDP by Matt Robison
Data Cloud, More than a CDP by Matt RobisonData Cloud, More than a CDP by Matt Robison
Data Cloud, More than a CDP by Matt Robison
 
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
 
Presentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreterPresentation on how to chat with PDF using ChatGPT code interpreter
Presentation on how to chat with PDF using ChatGPT code interpreter
 
TrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
TrustArc Webinar - Stay Ahead of US State Data Privacy Law DevelopmentsTrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
TrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
 
Factors to Consider When Choosing Accounts Payable Services Providers.pptx
Factors to Consider When Choosing Accounts Payable Services Providers.pptxFactors to Consider When Choosing Accounts Payable Services Providers.pptx
Factors to Consider When Choosing Accounts Payable Services Providers.pptx
 
IAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI SolutionsIAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI Solutions
 
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
 
🐬 The future of MySQL is Postgres 🐘
🐬  The future of MySQL is Postgres   🐘🐬  The future of MySQL is Postgres   🐘
🐬 The future of MySQL is Postgres 🐘
 
Breaking the Kubernetes Kill Chain: Host Path Mount
Breaking the Kubernetes Kill Chain: Host Path MountBreaking the Kubernetes Kill Chain: Host Path Mount
Breaking the Kubernetes Kill Chain: Host Path Mount
 
Boost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityBoost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivity
 
A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected Worker
 
The Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptxThe Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptx
 
Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024
 
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
 
GenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationGenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day Presentation
 
What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...
 

Apache samza past, present and future

  • 1. Apache Samza Past, Present and Future Kartik Paramasivam Director of Engineering, Streams Infra@ LinkedIn
  • 2. Agenda 1. Stream Processing 2. State of the Union 3. Apache Samza : Key Differentiators 4. Apache Samza Futures
  • 3. Stream Processing: Processing events as soon as they happen.. ● Stateless Processing ■ Transformation etc. ■ Lookup adjunct data (lookup databases/call services ) ■ Producing results for every event ● Stateful Processing ■ Triggering/Producing results periodically (time-windows) ● Maintain intermediate state ■ E.g. Joining across multiple streams of events.
  • 4. Stream Processing: State of the Union MillwheelStorm Heron Spark Streaming S4 Dempsey Samza Flink Beam Dataflow Azure Stream Analytics AWS Kinesis AnalyticsGearPump Kafka Streams Orleans Not meant to be an accurate timeline.. Yes It is CROWDED !!
  • 5. Apache Samza ● Top level Apache project since Dec 2014 ● 5 big Releases (0.7, 0.8, 0.9, 0.10, 0.11) ● 62 Contributors ● 14 Committers ● Companies using : LinkedIn, Uber, MetaMarkets, Netflix, Intuit, TripAdvisor, MobileAware, Optimizely …. https://cwiki.apache.org/confluence/display/SAMZA/Powered+By ● Applications at LinkedIn : from ~20 to ~200 in 2 years.
  • 6. Key Differentiators for Apache Samza ● Performance !! ● Stability ● Support for a variety of input sources ● Stream processing as a service AND as an embedded library
  • 7. Performance : Accessing Adjunct Data Samza Processor Database Remote-read Samza Processor Capture changes Databus, Brooklin Rocks- DB Local read Database Local data access Remote data access Input stream Input stream
  • 8. Performance : Maintaining Temporary State Samza Processor Remote Database Read-Write Samza Processor Backup changes Kafka Change Log(Log compacted)Rocks- DB Local read/write Local data access Remote data access Input stream Input stream In Memory Store
  • 9. Performance : Let us talk numbers ! ● 100x Difference between using Local State vs Remote No-Sql store ● Local State details: ○ 1.1 Million TPS on a single processing machine (SSD) ○ Used a 3 node Kafka cluster for storing the durable changelog ● Remote State details: ○ 8500 TPS when the Samza job was changed to accessing a remote No- Sql store ○ No-Sql Store was also on a 3 node (ssd) cluster
  • 10. Remote State : Asynchronous Event Processing Event Loop (Single thread) ProcessAsync Remote DB /Services Asynchronous I/O calls, using Java Nio, Netty... Responses sent to main thread via callback Event loop is woken up to process next message Task.max.concurrency >1 to enable pipelining Available with Samza 0.11
  • 11. Remote State: Synchronous Processing on Multiple Threads Event Loop (Single thread) Schedule Process() Remote DB/ Services Built-In Thread pool Blocking I/O calls Event loop is woken up by the worker thread job.container.thread.pool.size = N Available with Samza 0.11
  • 12. Incremental Checkpointing : MVP for stateful apps Some applications have ~ 2 TB state in production Stateful apps don’t really work without incremental checkpointing Samza Task State changelog checkpoint Host 1 Input stream(e.g. Kafka)
  • 13. Key Differentiators for Apache Samza ● Performance !! ● Stability ● Support for a variety of input sources ● Stream processing as a service AND as an embedded library
  • 14. Speed Thrills .. but can kill ● Local State Considerations: ○ State should NOT be reseeded under normal operations (e.g. Upgrades, Application restarts) ○ Minimal State should be reseeded - If a container dies/removed - If a container is added
  • 15. How Samza keeps Local state ‘stable’ ? P0 P1 P2 P3 Task-0 Task-1 Task-2 Task-3 P0 P1 P2 P3 Host-E Host-B Host-C Coordinator Stream: Task-Container-Host Mapping Container-0 -> Host-E Container-1 -> Host-B Container-2 -> Host-C AM JC Yarn-RM Ask: Host-E Allocate: Host-E Samza Job Input Stream Change-log Enable Continuous Scheduling
  • 16. Stream A Stream B Stream C Job 1 Job 2 ● Kafka or durable intermediate queues are leveraged to avoid backpressure issues in a pipeline. ● Allows each stage to be independent of the next stage Backpressure in a Pipeline
  • 17. Key Differentiators for Apache Samza ● Performance !! ● Stability ● Support for a variety of input sources ● Stream processing as a service AND as an embedded library
  • 18. Pluggable system consumers Samza Processor Mongo DB DynamoDB Streams Kafka Databus/ Brooklin Kinesis ZeroMQ … Azure EventHub, Azure Document DB, Google Pub-Sub etc. Oracle, Espresso Dynamo- DB
  • 19. Batch processing in Samza!! (NEW) ● HDFS system consumer for Samza ● Same Samza processor can be used for processing events from Kafka and HDFS with no code changes ● Scenarios : ○ Experimentation and Testing ○ Re-processing of large datasets ○ Some datasets are readily available on HDFS
  • 20. Samza - HDFS support HDFS input (Samza) Processor HDFS output (Samza) Processor HDFS output HDFS input (Samza) Processor Kafka Kafka New Available since Samza 0.10 The batch job auto-terminates when the input is fully processed.
  • 21. Reprocessing entire Dataset online Updates (Samza) Processor (Samza) Processor Bootstrap output Kafka Brooklin Brooklin Database (espresso) set offset=0 Nearline Processing
  • 22. Reprocessing ‘large’ Datasets “offline” Updates (Samza) Processor (Samza) Processor Backup output Kafka Databus Database (espresso) Database Backup (HDFS) Nearline Processing Offline Processing
  • 24. Samza- HDFS Early Performance Results !! Benchmark : Count number of records grouped by <Field> DataSize (bytes): 250 GB Number of files : 487 Samza Map/Reduce Spark Number of Containers T i m e - s e c o n d s
  • 25. Key Differentiators for Apache Samza ● Performance !! ● Stability ● Support for a variety of input sources (batch and streaming) ● Stream processing as a service AND (coming soon) as an embedded library
  • 26. Stream Processing as a Service ● Based on YARN ○ Yarn-RM high availability ○ Work preserving RM ○ Support for Heterogenous hardware with Node Labels (NEW) ● Easy upgrade of Samza framework : Use the Samza version deployed on the machine instead of packaging it with the application. ● Disk Quotas for local state (e.g. rocksDB state) ● Samza Management Service(SAMZA-REST)-> Next Slide
  • 27. YARN Resource Managers Nodes in the YARN cluster RM SRR RM SRR RM SRR NM SRN Samza Management Service (Samza REST) (NEW) NM SRN NM SRN NM SRN NM SRN NM SRN NM SRN NM SRN /v1/jobs /v1/jobs /v1/jobs Samza Containers 1. Exposes /jobs resource to start, stop, get status of jobs etc. 1. Cleans up stores from dead jobs Samza REST YARN processes(RM/NM)
  • 28. Agenda 1. Stream processing 2. State of the union 3. Apache Samza : Key differentiators 4. Apache Samza Futures
  • 29. Coming Soon : Samza as a Library Stream Processor Code Job Coordinator Stream Processor Code Job Coordinator Stream Processor Code Job Coordinator ... Leader ● No YARN dependency ● Will use ZK for leader election ● Embed stream processing into your bigger application StreamProcessor processor = new StreamProcessor (config, “job-name”, “job-id”); processor.start(); processor.awaitStart(); … processor.stop();
  • 30. Coming Soon: High Level API and Event Time (SAMZA- 914/915) Count the number of PageViews by Region, every 30 minutes. @Override public void init(Collection<SystemMessageStream> sources) { sources.forEach(source -> { Function<PageView, String> keyExtractor = view -> view.getRegion(); source.map(msg -> new PageViewMessage(msg)) .window(Windows.<PageViewMessage, String>intoSessionCounter(keyExtractor, WindowType.Tumbling, 30*60 )) }); }
  • 31. Coming Soon: First class support for Pipelines (Samza- 1041) public class MyPipeline implements PipelineFactory { public Pipeline create(Config config) { Processor myShuffler = getShuffle(config); Processor myJoiner = getJoin(config); Stream inStream = getStream(config, “inStream1”); // … omitted for brevity PipelineBuilder builder = new PipelineBuilder(); return builder.addInputStreams(myShuffler, inStream1) .addOutputStreams(myShuffler, intermediateOutStream) .addInputStreams(myJoiner, intermediateOutStream, inStream2) .addOutputStreams(myJoiner, finalOutStream) .build(); } } Shuffle Join input output
  • 32. Future: Miscellaneous ● Exactly once processing ● Making it easier to auto-scale even with Local State (on- demand Standby containers) ● Turnkey Disaster Recovery for stateful applications ○ Easy Restore of changelog and checkpoints from some other datacenter. ● Improved support for Batch jobs ● SQL over Streams ● A default Dashboard :)

Notes de l'éditeur

  1. Looking up data.. Sometimes people call it a Stream Table join.
  2. Talk about bootstrapping feature of Samza.
  3. Talk about bootstrapping feature of Samza.
  4. Sure .. We can add caches.. But then how do the caches get populated and kept in sync.
  5. Take-away: Samza REST is designed to be a “man on the ground” for every node in your cluster. It can play different roles on different hosts. Legend: Blue boxes are YARN processes Green boxes are Samza REST processes SRR - Samza REST with a Resource Manager config Exposes /jobs resource to start, stop, get status of jobs SRN - Samza REST with a Node Manager config Monitors node manager process Cleans host affinity stores Red boxes are Samza containers (jobs)
  6. Talk about partition coordination