SlideShare a Scribd company logo
1 of 69
Unified Batch & Stream Processing
with Apache Samza
Navina Ramesh
Sr. Software Engineer, LinkedIn
Committer and PMC member, Apache Samza
@navina_r navina@apache.org
Agenda
● Data Processing at LinkedIn
● Data Pipelines in Batch & Stream
● Overview of Apache Samza
● Convergence of Pipelines with Apache Samza
○ Support for Batch Data
○ Unified Data Processing API
○ Flexible Deployment Model
Data Processing at LinkedIn
Azure
EventHub
Oracle
DB
Espresso DB
(NoSQL Store
for all user data)
Brooklin
(DB Change Capture)
HDFS
Hadoop
(Batch Processing)
Import / Export
Services Tier
Ingestion
Processing
Voldemort / Venice
(K-V Store for
Derived Data)
Samza
(Stream Processing)
Amazon
Kinesis
Scale of Processing at LinkedIn
KAFKA
2.3 Trillion
Msgs per Day
0.6 PB in, 2.3 PB out per
Day (compressed)
16 million Msgs per
Second at peaks!
HADOOP
125 TB Ingested per Day
120 PB Hdfs Size
200K Jobs per Day
SAMZA
200+ Applications
Most Applications
require Stateful
Processing ~ several
TBs (overall)
Data Processing Scenarios at LinkedIn
Site Speed
Real-time site-
speed profiling by
facets
Call-graph
Computation
Analysis of
Service calls
Dashboards
Real-time Analytics
Ad CTR
Computation
Tracking Ads Views
and Ads Clicks
Operate primarily using real-time input data
Data Processing Scenarios at LinkedIn
News
Classification
Real-time topic
tagging of articles
Profile
Standardization
Standardizing
titles, gender,
education
Security
Real-time DDoS
protection for
members
● Operate on real-time data & rely on models computed
offline
● Offline computed model must be accessible during
real-time processing
Agenda
● Data Processing at LinkedIn
● Data Pipelines in Batch & Stream
● Overview of Apache Samza
● Convergence of Pipelines with Apache Samza
○ Support for Batch Data
○ Unified Data Processing API
○ Flexible Deployment Model
Ingestion
Service
HDFS
Mappers Reducers
HDFS/
HBase
Processors Processors
KV Store
Partition 0
Partition 1
Partition N
... Query
Data Pipelines in Batch & Stream
Azure
EventHub
Batch / Offline
Stream / Realtime
Streams to Batch
& Batch to Stream
Batch
● Processing on bounded data
● Processing at regular intervals
● Latency ~ order of hours
● Processing on unbounded data
● Processing is continuous
● Latency ~ order of sub-seconds
● Time matters!
Stream
● Overhead of developing and managing multiple source codes
○ Same application logic written using 2 different APIs - one using offline processing APIs and
another using near-realtime processing API
● Same application deployed in potentially 2 different managed platforms
○ Restrictions due to firewalls, acl to environments etc.
● Expensive $$
○ When near-realtime application needs processed data from offline, the data snapshot has to
be made available as a stream. This is expensive!
Data Pipelines in Batch & Stream - Drawbacks
Ingestion
Service
HDFS
Mappers Reducers
HDFS/
HBase
Processors Processors
HDFS
KV
Store
Partition 0
Partition 1
Partition N
... Query
Query
Data Pipelines in Batch & Stream
Azure
EventHub
Data Sources Data Processing Sink / Serving
Batch / Offline
Stream / Realtime
Ingestion
Service
HDFS
Mappers Reducers
HDFS/
HBase
Processors Processors
HDFS
KV
Store
Partition 0
Partition 1
Partition N
... Query
Query
Azure
EventHub
Data Sources
Data Processing
Sink / Serving
Batch / Offline
Stream / Realtime
Converge Pipelines with Apache Samza
Agenda
● Data Processing at LinkedIn
● Data Pipelines in Batch & Stream
● Overview of Apache Samza
● Convergence of Pipelines with Apache Samza
○ Support for Batch Data
○ Unified Data Processing API
○ Flexible Deployment Model
Apache Samza
• Production at LinkedIn since 2013
• Apache TLP since 2014
• Streams as first-class citizen
– Batch as a special case of streaming
Apache Samza
● Provides distributed and scalable data processing platform
with
○ Configurable and heterogeneous data sources and
sinks (Eg. Kafka, HDFS, Kinesis, EventHub etc)
○ Efficient state management - local state and
incremental checkpoints
○ Unified Processing API for Batch & Streaming
○ Flexible deployment models
Apache Samza
Azure
EventHub
Amazon
Kinesis
HDFS
Remote Runner
Standalone
Local Runner
AzureYARN Mesos
System
(Producer&
Consumer)
Local State
(Rocks DB,
In-Memory)
Checkpoint
Manager
Remote Data
(Multithreading)
High-level API
Low-level
API
PROCESSOR
DEPLOYM
ENT
API
SQL
DB
Streams
Batch
Change Data
Capture
Data Processing Model
• Natively supports partitioned data
• Re-partitioning may be required for an un-partitioned source
• Pluggable System and CheckpointManager implementations
Partitions
Partitioned Input
Tasks
1
2
3
Processing
Kafka/Eventhub
Client
Send with PartitionKey
Samza Application
- is a made up of Tasks
- every Task processes a unique
collection of input partitions
1
2
3
4
5
Processing Partitioned Data
Single JVM
(container)
Partitions
Partitioned Input
Tasks
1
2
3
Processing
Kafka/Eventhub
Client
Send with PartitionKey
- Samza master distributes tasks
across JVMs
- Scale up & Distribute –
increasing container count
1
2
3
4
5
Processing Partitioned Data Distributed
across 3 JVMs
Ad View Stream
Samza Application
1
2
3
Ad Click Stream
Ad Click Through
Rate Stream
Tasks
Processing
Joining Co-partitioned Data
1
2
3
1
2
3
Co-partitioned by Ad-ID
Ad View Stream
Samza Application
1
2
3
Ad Click Stream
Ad Click Through
Rate Stream
Tasks
Processing
Joining Co-partitioned Data
Local State Store
(RocksDB)
1
2
3
1
2
3
Co-partitioned by Ad-ID
Ad View Stream
Samza Application
1
2
3
Ad Click Stream
Ad Click Through
Rate Stream
Tasks
Processing
Joining Co-partitioned Data
1
2
3
1
2
3
Co-partitioned by Ad-ID
Changelog Stream
for Replication
(partitioned)
Used for Recovery
upon Task Failure
Agenda
● Data Processing at LinkedIn
● Data Pipelines in Batch & Stream
● Overview of Apache Samza
● Convergence of Pipelines with Apache Samza
○ Support for Batch Data
○ Unified Data Processing API
○ Flexible Deployment Model
❏Support for Bounded Data
❏ Define a boundary over the stream
❏ Batched Processing
❏Unified Data Processing API
❏Flexible Deployment Models – Write once, Run anywhere!
How to converge?
Agenda
● Data Processing at LinkedIn
● Data Pipelines in Batch & Stream
● Overview of Apache Samza
● Convergence of Pipelines with Apache Samza
○ Support for Batch Data
○ Unified Data Processing API
○ Flexible Deployment Model
Support for Batch Data
• Batch as a special Case of Stream:
 Define boundary on stream
 Batched processing – end of batch basically ends the job
Defining a Boundary on the Stream
• Introduced a notion of End-of-Stream (EoS) in the input
• Consumer in the System detects the EoS for a source
– Upon EoS, Samza may invoke EndOfStreamListenerTask handler
implemented by the application (optional)
File Partitions
Partitioned Input
Tasks
1
2
3
Processing
HDFS
Client
Store Partitioned Data
1
2
3
4
5
Processing Bounded Data
- Single File as a Partition
- Directory of Files as a Stream
File Partitions
Partitioned Input
Tasks
1
2
3
Processing
HDFS
Client
Store Partitioned Data
1
2
3
4
5
Processing Bounded Data
- Group of Files as a Partition
- Groups defined using GroupingPattern
Regex
Support for Batch Data
• Batch as a special Case of Stream:
 Define boundary on stream
 Batched processing – end of batch basically ends the job
Processing Bounded Data
1 2 3 4 5 6
1 2 3 4 5
1 2 3
1 2 3 4 5
Partition 0
Partition 1
Partition 2
Partition 3
Bounded Input
Task-0
Task-1
1 2 3 4 5 6
1 2 3 4 5
1 2 3
1 2 3 4 5
Partition 0
Partition 1
Partition 2
Partition 3
Bounded Input
Task-0
Task-1
Processing Bounded Data
Samza’s SystemConsumer
detects EoS for Partition 1 -
doesn’t shut-down the task
yet.
1 2 3 4 5 6
1 2 3 4 5
1 2 3
1 2 3 4 5
Partition 0
Partition 1
Partition 2
Partition 3
Bounded Input
Task-0
Task-1
Processing Bounded Data
Continues Processing
Partition-0
1 2 3 4 5 6
1 2 3 4 5
1 2 3
1 2 3 4 5
Partition 0
Partition 1
Partition 2
Partition 3
Bounded Input
Task-0
Task-1
Samza detects EoS for the
Partition 1 - shuts-down the
task.
Processing Bounded Data
1 2 3 4 5 6
1 2 3 4 5
1 2 3
1 2 3 4 5
Partition 0
Partition 1
Partition 2
Partition 3
Bounded Input
Task-0
Task-1
Task has stopped
processing
Processing Bounded Data
1 2 3 4 5 6
1 2 3 4 5
1 2 3
1 2 3 4 5
Partition 0
Partition 1
Partition 2
Partition 3
Bounded Input
Task-0
Task-1
When all Tasks in JVM
finish processing, Samza
job itself shuts-down.
Processing Bounded Data
Batch as a Special Case of Stream
 Support Bounded nature of data
 Define a boundary over the stream
 Processing at regular intervals
 Tasks exit upon complete consumption of the batch
Profile count,
group-by country
500 files
250GB input
Samza HDFS Benchmark
* Multiple threads per container
*
Agenda
● Data Processing at LinkedIn
● Data Pipelines in Batch & Stream
● Overview of Apache Samza
● Convergence of Pipelines with Apache Samza
○ Support for Batch Data
○ Unified Data Processing API
○ Flexible Deployment Model
Example Application
Count PageViewEvent for each mobile Device OS in a 5 minute
window and send the counts to PageViewEventPerDeviceOS
PageViewEvent PageViewCountPerDeviceOS
Filter & Re-
partition
Window Map SendTo
Samza Low-level API
public interface StreamTask {
void process(IncomingMessageEnvelope envelope,
MessageCollector collector,
TaskCoordinator coordinator) {
// process message
}
}
PageViewEvent PageViewCountPerDeviceOS
Filter & Re-
partition
Window Map SendTo
Job 1: PageViewRepartitionTask Job 2: PageViewByDeviceOSCounterTask
PageViewEventByDeviceOS
Application using Low-level API
public class PageViewRepartitionTask implements StreamTask {
private final SystemStream pageViewEventByDeviceOSStream = new SystemStream("kafka",
"PaveViewEventByDeviceOS");
@Override
public void process(IncomingMessageEnvelope envelope,
MessageCollector collector,
TaskCoordinator coordinator) throws Exception {
PageViewEvent pve = (PageViewEvent) envelope.getMessage();
collector.send(new OutgoingMessageEnvelope(pageViewEventByDeviceOSStream, pve.memberId, pve));
}
}
Job-1: Filter & Repartition Job
PageViewEvent
PageViewCountPerDeviceOS
Filter & Re-
partition
Window Map SendTo
PageViewEventByDeviceOS
Application using Low-level API
public class PageViewByDeviceOSCounterTask implements InitableTask, StreamTask, WindowableTask {
private final SystemStream pageViewCounterStream = new SystemStream("kafka", "PageViewCountPerDeviceOS");
private KeyValueStore<String, PageViewPerMemberIdCounterEvent> windowedCounters;
private Long windowSize;
@Override
public void init(Config config, TaskContext context) throws Exception {
this.windowedCounters = (KeyValueStore<String, PageViewPerMemberIdCounterEvent>)
context.getStore("windowed-counter-store");
this.windowSize = config.getLong("task.window.ms");
}
@Override
public void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
getWindowCounterEvent().forEach(counter ->
collector.send(new OutgoingMessageEnvelope(pageViewCounterStream, counter.memberId, counter)));
}
@Override
public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception {
PageViewEvent pve = (PageViewEvent) envelope.getMessage();
countPageViewEvent(pve);
}
}
Job-2: Window-based Counter
PageViewEvent
PageViewCountPerDeviceOS
Filter & Re-
partition
Window Map SendTo
PageViewEventByDeviceOS
Application using Low-level API
public class PageViewByDeviceOSCounterTask implements InitableTask, StreamTask, WindowableTask {
private final SystemStream pageViewCounterStream = new SystemStream("kafka", "PageViewCountPerDeviceOS");
private KeyValueStore<String, PageViewPerMemberIdCounterEvent> windowedCounters;
private Long windowSize;
@Override
public void init(Config config, TaskContext context) throws Exception {
this.windowedCounters = (KeyValueStore<String, PageViewPerMemberIdCounterEvent>)
context.getStore("windowed-counter-store");
this.windowSize = config.getLong("task.window.ms");
}
@Override
public void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
getWindowCounterEvent().forEach(counter ->
collector.send(new OutgoingMessageEnvelope(pageViewCounterStream, counter.memberId, counter)));
}
@Override
public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception {
PageViewEvent pve = (PageViewEvent) envelope.getMessage();
countPageViewEvent(pve);
}
}
...
void countPageViewEvent(PageViewEvent pve) {
String key = String.format("%08d-%s", (pve.timestamp - pve.timestamp % this.windowSize),
pve.memberId);
PageViewPerMemberIdCounterEvent counter = this.windowedCounters.get(key);
if (counter == null) {
counter = new PageViewPerMemberIdCounterEvent(pve.memberId, (pve.timestamp -
pve.timestamp % this.windowSize), 0);
}
counter.count ++;
this.windowedCounters.put(key, counter);
}
...
Job-2: Window-based Counter
PageViewEvent
PageViewCountPerDeviceOS
Filter & Re-
partition
Window Map SendTo
PageViewEventByDeviceOS
Application using Low-level API
public class PageViewByDeviceOSCounterTask implements InitableTask, StreamTask, WindowableTask {
private final SystemStream pageViewCounterStream = new SystemStream("kafka", "PageViewCountPerDeviceOS");
private KeyValueStore<String, PageViewPerMemberIdCounterEvent> windowedCounters;
private Long windowSize;
@Override
public void init(Config config, TaskContext context) throws Exception {
this.windowedCounters = (KeyValueStore<String, PageViewPerMemberIdCounterEvent>)
context.getStore("windowed-counter-store");
this.windowSize = config.getLong("task.window.ms");
}
@Override
public void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
getWindowCounterEvent().forEach(counter ->
collector.send(new OutgoingMessageEnvelope(pageViewCounterStream, counter.memberId, counter)));
}
@Override
public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception {
PageViewEvent pve = (PageViewEvent) envelope.getMessage();
countPageViewEvent(pve);
}
}
...
List<PageViewPerMemberIdCounterEvent> getWindowCounterEvent() {
List<PageViewPerMemberIdCounterEvent> retList = new ArrayList<>();
Long currentTimestamp = System.currentTimeMillis();
Long cutoffTimestamp = currentTimestamp - this.windowSize;
String lowerBound = String.format("%08d-", cutoffTimestamp);
String upperBound = String.format("%08d-", currentTimestamp + 1);
this.windowedCounters.range(lowerBound, upperBound).forEachRemaining(entry
->
retList.add(entry.getValue()));
return retList;
}
...
Job-2: Window-based Counter
PageViewEvent
PageViewCountPerDeviceOS
Filter & Re-
partition
Window Map SendTo
PageViewEventByDeviceOS
Samza High-level API
public interface StreamApplication {
void init(StreamGraph streamGraph,
Config config) {
// Process message using DSL-
// like declarations
}
}
- Ability to express a multi-stage
processing pipeline in a single user
program
- Built-in library to provide high-level
stream transformation functions -> Map,
Filter, Window, Partition, Join etc.
- Automatically generates the DAG for
the application
public class CountByDeviceOSApplication implements StreamApplication {
@Override
public void init(StreamGraph graph, Config config) {
Supplier<Integer> initialValue = () -> 0;
MessageStream<PageViewEvent> pageViewEvents =
graph.getInputStream("pageViewEvent", (k, m) -> (PageViewEvent) m);
OutputStream<String, MyStreamOutput, MyStreamOutput> pageViewEventPerMemberStream = graph
.getOutputStream("pageViewCountPerDevice", m -> m.memberId, m -> m);
pageViewEvents
.partitionBy(m -> m.memberId)
.window(Windows.keyedTumblingWindow(
m -> m.memberId, Duration.ofMinutes(5),initialValue,(m, c) -> c + 1))
.map(MyStreamOutput::new)
.sendTo(pageViewEventPerMemberStream);
}
}
Built-in
Transforms
Application using High-level API
PageViewEvent
PageViewCountPerDeviceOS
Filter & Re-
partition
Window Map SendTo
PageViewEventByDeviceOS
public class CountByDeviceOSApplication implements StreamApplication {
@Override
public void init(StreamGraph graph, Config config) {
Supplier<Integer> initialValue = () -> 0;
MessageStream<PageViewEvent> pageViewEvents =
graph.getInputStream("pageViewEvent", (k, m) -> (PageViewEvent) m);
OutputStream<String, MyStreamOutput, MyStreamOutput> pageViewEventPerMemberStream = graph
.getOutputStream("pageViewCountPerDevice", m -> m.memberId, m -> m);
pageViewEvents
.partitionBy(m -> m.memberId)
.window(Windows.keyedTumblingWindow(
m -> m.memberId, Duration.ofMinutes(5), initialValue, (m, c) -> c + 1))
.map(MyStreamOutput::new)
.sendTo(pageViewEventPerMemberStream);
}
}
Unified for Batch & Stream
Configuration for Stream Input (Kafka):
systems.kafka.samza.factory =
org.apache.samza.system.KafkaSystemFactory
streams.PageViewEvent.samza.system = kafka
streams.PageViewEvent.samza.physical.name = PageViewEvent
public class CountByDeviceOSApplication implements StreamApplication {
@Override
public void init(StreamGraph graph, Config config) {
Supplier<Integer> initialValue = () -> 0;
MessageStream<PageViewEvent> pageViewEvents =
graph.getInputStream("pageViewEvent", (k, m) -> (PageViewEvent) m);
OutputStream<String, MyStreamOutput, MyStreamOutput> pageViewEventPerMemberStream = graph
.getOutputStream("pageViewCountPerDevice", m -> m.memberId, m -> m);
pageViewEvents
.partitionBy(m -> m.memberId)
.window(Windows.keyedTumblingWindow(
m -> m.memberId, Duration.ofMinutes(5), initialValue, (m, c) -> c + 1))
.map(MyStreamOutput::new)
.sendTo(pageViewEventPerMemberStream);
}
}
Unified for Batch & Stream
Configuration for Stream Input (Kafka):
systems.kafka.samza.factory =
org.apache.samza.system.KafkaSystemFactory
streams.PageViewEvent.samza.system = kafka
streams.PageViewEvent.samza.physical.name = PageViewEvent
Configuration for Batch Input (HDFS):
systems.hdfs.samza.factory =
org.apache.samza.system.HdfsSystemFactory
streams.PageViewEvent.samza.system = hdfs
streams.PageViewEvent.samza.physical.name =
hdfs:/user/nramesh/PageViewEvent
public class CountByDeviceOSApplication implements StreamApplication {
@Override
public void init(StreamGraph graph, Config config) {
Supplier<Integer> initialValue = () -> 0;
MessageStream<PageViewEvent> pageViewEvents =
graph.getInputStream("pageViewEvent", (k, m) -> (PageViewEvent) m);
OutputStream<String, MyStreamOutput, MyStreamOutput> pageViewEventPerMemberStream = graph
.getOutputStream("pageViewCountPerDevice", m -> m.memberId, m -> m);
pageViewEvents
.partitionBy(m -> m.memberId)
.window(Windows.keyedTumblingWindow(
m -> m.memberId, Duration.ofMinutes(5), initialValue, (m, c) -> c + 1))
.map(MyStreamOutput::new)
.sendTo(pageViewEventPerMemberStream);
}
}
Unified for Batch & Stream
Configuration for Stream Input (Kafka):
systems.kafka.samza.factory =
org.apache.samza.system.KafkaSystemFactory
streams.PageViewEvent.samza.system = kafka
streams.PageViewEvent.samza.physical.name = PageViewEvent
Configuration for Batch Input (HDFS):
systems.hdfs.samza.factory =
org.apache.samza.system.HdfsSystemFactory
streams.PageViewEvent.samza.system = hdfs
streams.PageViewEvent.samza.physical.name =
hdfs:/user/nramesh/PageViewEvent
Only Config Change!
High-level API - Visualization for DAG
SAMZA Visualizer
A visualization of application samza-count-by-device-i001, which consists of 1 job(s), 1 input
stream(s), and 1 output stream(s).
High-level API Transforms
Agenda
● Data Processing at LinkedIn
● Data Pipelines in Batch & Stream
● Overview of Apache Samza
● Convergence of Pipelines with Apache Samza
○ Support for Batch Data
○ Unified Data Processing API
○ Flexible Deployment Model
Coordination Model
• Coordination layer is pluggable in Samza
• Samza master / leader
– Distributes tasks to processor JVMs
– On processor failure, it re-distributes
• Available Coordination Mechanisms
– Apache Yarn
• ApplicationMaster is the leader
– Apache Zookeeper
• One of the processors is the leader and co-ordinates via Zookeeper
– Microsoft Azure
• One of the processors is the leader and co-ordinates via Azure’s
Blob/Tables Storage
Embedding Processor within Application
- An instance of the processor is
embedded within user’s application
- LocalApplicationRunner helps launch
the processor within the application
public static void main(String[] args) {
CommandLine cmdLine = new CommandLine();
OptionSet options = cmdLine.parser().parse(args);
Config config = cmdLine.loadConfig(options);
LocalApplicationRunner runner = new
LocalApplicationRunner(config);
CountByDeviceOSApplication app = new
CountByDeviceOSApplication();
runner.run(app);
runner.waitForFinish();
}
Pluggable Coordination Config
public static void main(String[] args) {
CommandLine cmdLine = new CommandLine();
OptionSet options = cmdLine.parser().parse(args);
Config config = cmdLine.loadConfig(options);
LocalApplicationRunner runner = new
LocalApplicationRunner(config);
CountByDeviceOSApplication app = new
CountByDeviceOSApplication();
runner.run(app);
runner.waitForFinish();
}
Configs with Zk-based coordination
job.coordinator.factory =
org.apache.samza.zk.ZkJobCoordinatorFactory
job.coordinator.zk.connect = foobar:2181/samza
Pluggable Coordination Config
public static void main(String[] args) {
CommandLine cmdLine = new CommandLine();
OptionSet options = cmdLine.parser().parse(args);
Config config = cmdLine.loadConfig(options);
LocalApplicationRunner runner = new
LocalApplicationRunner(config);
CountByDeviceOSApplication app = new
CountByDeviceOSApplication();
runner.run(app);
runner.waitForFinish();
}
Configs with Azure-based coordination:
job.coordinator.factory =
org.apache.samza.azure.AzureJobCoordinatorFactory
job.coordinator.azure.connect = http://foobar:29892/storage/
Configs with Zk-based coordination
job.coordinator.factory =
org.apache.samza.zk.ZkJobCoordinatorFactory
job.coordinator.zk.connect = foobar:2181/samza
Pluggable Coordination Config
public static void main(String[] args) {
CommandLine cmdLine = new CommandLine();
OptionSet options = cmdLine.parser().parse(args);
Config config = cmdLine.loadConfig(options);
LocalApplicationRunner runner = new
LocalApplicationRunner(config);
CountByDeviceOSApplication app = new
CountByDeviceOSApplication();
runner.run(app);
runner.waitForFinish();
}
Only Config Change!
Configs with Azure-based coordination:
job.coordinator.factory =
org.apache.samza.azure.AzureJobCoordinatorFactory
job.coordinator.azure.connect = http://foobar:29892/storage/
Configs with Zk-based coordination
job.coordinator.factory =
org.apache.samza.zk.ZkJobCoordinatorFactory
job.coordinator.zk.connect = foobar:2181/samza
Deploying Samza in a Managed Cluster (Yarn)
app.class = MyStreamApplication
RemoteAppplicationRunner: main()
RM
NM
LocalApplicationRunner
StreamProcessor
JobCoordinator
NM
NM
LocalApplicationRunner
StreamProcessor
Client
Submits JAR
run-jc.sh
run-app.sh
run-local-app.sh run-local-app.sh
Flexible Deployment Models
Samza as a Library
- Run embedded stream processing in
user program
- Use Zookeeper for partition distribution
among tasks and liveness of processors
- Seamlessly scale by spinning a new
processor instance
Samza as a Service
- Run stream processing as a
managed program in a cluster (eg.
Yarn)
- Works with the cluster manager (Eg.
AM/RM) for partition distribution
among tasks and liveness of
processors
- Better for resource sharing in a multi-
tenant environment
Conclusion
● Easily Composable Architecture allows varied data source consumption
● Write Once, Run Anywhere paradigm
○ Unified API - application logic to be written only once
○ Pluggable Coordination Model - allows application deployment across different execution
environment
Future Work
● Support SQL on Streams with Samza
● Table Abstraction in Samza
● Event-time processing
● Samza runner for Apache Beam
Contributions are welcome!
● Contributor’s Corner - http://samza.apache.org/contribute/contributors-corner.html
● Ask any question - dev@samza.apache.org
● Follow or tweet us @apachesamza
Questions?
Extra Slides
Lambda-less Architecture with Samza
Profile Updates
Kafka stream
Standardization
Normalized Profile
Updates
Kafka stream
Member
Profiles
Lambda-less Architecture with Samza
Profile Updates
Kafka stream
Standardization
Normalized Profile
Updates
Kafka stream
Member
Profiles
update the standardization job
Lambda-less Architecture with Samza
Profile Updates
Kafka stream
Standardization
Normalized Profile
Updates
Kafka stream
Member
Profiles
update the standardization job
DB Snapshot
Standardization
Merge
& Store Results
Lambda-less Architecture with Samza
Profile Updates
Kafka stream
Standardization
Normalized Profile
Updates
Kafka stream
Member
Profiles
DB Snapshot
Standardization
Merge
& Store Results
Stream
Processing
Batch
Processing
Lambda Architecture with Samza

More Related Content

What's hot

How Uber scaled its Real Time Infrastructure to Trillion events per day
How Uber scaled its Real Time Infrastructure to Trillion events per dayHow Uber scaled its Real Time Infrastructure to Trillion events per day
How Uber scaled its Real Time Infrastructure to Trillion events per dayDataWorks Summit
 
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
 
Overview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics FrameworkOverview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics FrameworkSlim Baltagi
 
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 Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationDatabricks
 
Integrating NiFi and Flink
Integrating NiFi and FlinkIntegrating NiFi and Flink
Integrating NiFi and FlinkBryan Bende
 
Spark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka StreamsSpark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka StreamsGuido Schmutz
 
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...Flink Forward
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internalsKostas Tzoumas
 
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...Databricks
 
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
 
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013Introduction and Overview of Apache Kafka, TriHUG July 23, 2013
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013mumrah
 
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
 
0-60: Tesla's Streaming Data Platform ( Jesse Yates, Tesla) Kafka Summit SF 2019
0-60: Tesla's Streaming Data Platform ( Jesse Yates, Tesla) Kafka Summit SF 20190-60: Tesla's Streaming Data Platform ( Jesse Yates, Tesla) Kafka Summit SF 2019
0-60: Tesla's Streaming Data Platform ( Jesse Yates, Tesla) Kafka Summit SF 2019confluent
 
(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
 
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
 
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 AnalyticsSlim Baltagi
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeDatabricks
 

What's hot (20)

How Uber scaled its Real Time Infrastructure to Trillion events per day
How Uber scaled its Real Time Infrastructure to Trillion events per dayHow Uber scaled its Real Time Infrastructure to Trillion events per day
How Uber scaled its Real Time Infrastructure to Trillion events per day
 
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
 
Overview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics FrameworkOverview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics Framework
 
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 Spark Core – Practical Optimization
Apache Spark Core – Practical OptimizationApache Spark Core – Practical Optimization
Apache Spark Core – Practical Optimization
 
Integrating NiFi and Flink
Integrating NiFi and FlinkIntegrating NiFi and Flink
Integrating NiFi and Flink
 
Spark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka StreamsSpark (Structured) Streaming vs. Kafka Streams
Spark (Structured) Streaming vs. Kafka Streams
 
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
 
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...
The Top Five Mistakes Made When Writing Streaming Applications with Mark Grov...
 
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
 
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013Introduction and Overview of Apache Kafka, TriHUG July 23, 2013
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013
 
Unified Stream and Batch Processing with Apache Flink
Unified Stream and Batch Processing with Apache FlinkUnified Stream and Batch Processing with Apache Flink
Unified Stream and Batch Processing with Apache Flink
 
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 Streaming
Flink StreamingFlink Streaming
Flink Streaming
 
0-60: Tesla's Streaming Data Platform ( Jesse Yates, Tesla) Kafka Summit SF 2019
0-60: Tesla's Streaming Data Platform ( Jesse Yates, Tesla) Kafka Summit SF 20190-60: Tesla's Streaming Data Platform ( Jesse Yates, Tesla) Kafka Summit SF 2019
0-60: Tesla's Streaming Data Platform ( Jesse Yates, Tesla) Kafka Summit SF 2019
 
(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
 
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
 
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
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta Lake
 

Similar to Unified Batch & Stream Processing with Apache Samza

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
 
Stream processing on mobile networks
Stream processing on mobile networksStream processing on mobile networks
Stream processing on mobile networkspbelko82
 
Hadoop 3.0 - Revolution or evolution?
Hadoop 3.0 - Revolution or evolution?Hadoop 3.0 - Revolution or evolution?
Hadoop 3.0 - Revolution or evolution?Uwe Printz
 
ApacheCon BigData - What it takes to process a trillion events a day?
ApacheCon BigData - What it takes to process a trillion events a day?ApacheCon BigData - What it takes to process a trillion events a day?
ApacheCon BigData - What it takes to process a trillion events a day?Jagadish Venkatraman
 
TenMax Data Pipeline Experience Sharing
TenMax Data Pipeline Experience SharingTenMax Data Pipeline Experience Sharing
TenMax Data Pipeline Experience SharingChen-en Lu
 
Cloud Lambda Architecture Patterns
Cloud Lambda Architecture PatternsCloud Lambda Architecture Patterns
Cloud Lambda Architecture PatternsAsis Mohanty
 
Fluentd Overview, Now and Then
Fluentd Overview, Now and ThenFluentd Overview, Now and Then
Fluentd Overview, Now and ThenSATOSHI TAGOMORI
 
Will it Scale? The Secrets behind Scaling Stream Processing Applications
Will it Scale? The Secrets behind Scaling Stream Processing ApplicationsWill it Scale? The Secrets behind Scaling Stream Processing Applications
Will it Scale? The Secrets behind Scaling Stream Processing ApplicationsNavina Ramesh
 
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
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformApache Apex
 
Tez big datacamp-la-bikas_saha
Tez big datacamp-la-bikas_sahaTez big datacamp-la-bikas_saha
Tez big datacamp-la-bikas_sahaData Con LA
 
Data Analysis on AWS
Data Analysis on AWSData Analysis on AWS
Data Analysis on AWSPaolo latella
 
AWS re:Invent 2016: Streaming ETL for RDS and DynamoDB (DAT315)
AWS re:Invent 2016: Streaming ETL for RDS and DynamoDB (DAT315)AWS re:Invent 2016: Streaming ETL for RDS and DynamoDB (DAT315)
AWS re:Invent 2016: Streaming ETL for RDS and DynamoDB (DAT315)Amazon Web Services
 
Hadoop - Just the Basics for Big Data Rookies (SpringOne2GX 2013)
Hadoop - Just the Basics for Big Data Rookies (SpringOne2GX 2013)Hadoop - Just the Basics for Big Data Rookies (SpringOne2GX 2013)
Hadoop - Just the Basics for Big Data Rookies (SpringOne2GX 2013)VMware Tanzu
 
Building a high-performance data lake analytics engine at Alibaba Cloud with ...
Building a high-performance data lake analytics engine at Alibaba Cloud with ...Building a high-performance data lake analytics engine at Alibaba Cloud with ...
Building a high-performance data lake analytics engine at Alibaba Cloud with ...Alluxio, Inc.
 
Highlights of AWS ReInvent 2023 (Announcements and Best Practices)
Highlights of AWS ReInvent 2023 (Announcements and Best Practices)Highlights of AWS ReInvent 2023 (Announcements and Best Practices)
Highlights of AWS ReInvent 2023 (Announcements and Best Practices)Emprovise
 

Similar to Unified Batch & Stream Processing with Apache Samza (20)

Amazon Kinesis
Amazon KinesisAmazon Kinesis
Amazon Kinesis
 
Apache Spark Components
Apache Spark ComponentsApache Spark Components
Apache Spark Components
 
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
 
Stream processing on mobile networks
Stream processing on mobile networksStream processing on mobile networks
Stream processing on mobile networks
 
Hadoop 3.0 - Revolution or evolution?
Hadoop 3.0 - Revolution or evolution?Hadoop 3.0 - Revolution or evolution?
Hadoop 3.0 - Revolution or evolution?
 
MYSQL
MYSQLMYSQL
MYSQL
 
ApacheCon BigData - What it takes to process a trillion events a day?
ApacheCon BigData - What it takes to process a trillion events a day?ApacheCon BigData - What it takes to process a trillion events a day?
ApacheCon BigData - What it takes to process a trillion events a day?
 
Kafka & Hadoop in Rakuten
Kafka & Hadoop in RakutenKafka & Hadoop in Rakuten
Kafka & Hadoop in Rakuten
 
TenMax Data Pipeline Experience Sharing
TenMax Data Pipeline Experience SharingTenMax Data Pipeline Experience Sharing
TenMax Data Pipeline Experience Sharing
 
Cloud Lambda Architecture Patterns
Cloud Lambda Architecture PatternsCloud Lambda Architecture Patterns
Cloud Lambda Architecture Patterns
 
Fluentd Overview, Now and Then
Fluentd Overview, Now and ThenFluentd Overview, Now and Then
Fluentd Overview, Now and Then
 
Will it Scale? The Secrets behind Scaling Stream Processing Applications
Will it Scale? The Secrets behind Scaling Stream Processing ApplicationsWill it Scale? The Secrets behind Scaling Stream Processing Applications
Will it Scale? The Secrets behind Scaling Stream Processing Applications
 
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
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
 
Tez big datacamp-la-bikas_saha
Tez big datacamp-la-bikas_sahaTez big datacamp-la-bikas_saha
Tez big datacamp-la-bikas_saha
 
Data Analysis on AWS
Data Analysis on AWSData Analysis on AWS
Data Analysis on AWS
 
AWS re:Invent 2016: Streaming ETL for RDS and DynamoDB (DAT315)
AWS re:Invent 2016: Streaming ETL for RDS and DynamoDB (DAT315)AWS re:Invent 2016: Streaming ETL for RDS and DynamoDB (DAT315)
AWS re:Invent 2016: Streaming ETL for RDS and DynamoDB (DAT315)
 
Hadoop - Just the Basics for Big Data Rookies (SpringOne2GX 2013)
Hadoop - Just the Basics for Big Data Rookies (SpringOne2GX 2013)Hadoop - Just the Basics for Big Data Rookies (SpringOne2GX 2013)
Hadoop - Just the Basics for Big Data Rookies (SpringOne2GX 2013)
 
Building a high-performance data lake analytics engine at Alibaba Cloud with ...
Building a high-performance data lake analytics engine at Alibaba Cloud with ...Building a high-performance data lake analytics engine at Alibaba Cloud with ...
Building a high-performance data lake analytics engine at Alibaba Cloud with ...
 
Highlights of AWS ReInvent 2023 (Announcements and Best Practices)
Highlights of AWS ReInvent 2023 (Announcements and Best Practices)Highlights of AWS ReInvent 2023 (Announcements and Best Practices)
Highlights of AWS ReInvent 2023 (Announcements and Best Practices)
 

More from DataWorks Summit

Floating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisFloating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisDataWorks Summit
 
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiTracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiDataWorks Summit
 
HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...DataWorks Summit
 
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...DataWorks Summit
 
Managing the Dewey Decimal System
Managing the Dewey Decimal SystemManaging the Dewey Decimal System
Managing the Dewey Decimal SystemDataWorks Summit
 
Practical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExamplePractical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExampleDataWorks Summit
 
HBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberHBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberDataWorks Summit
 
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixScaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixDataWorks Summit
 
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiBuilding the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiDataWorks Summit
 
Supporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsSupporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsDataWorks Summit
 
Security Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureSecurity Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureDataWorks Summit
 
Presto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EnginePresto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EngineDataWorks Summit
 
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...DataWorks Summit
 
Extending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudExtending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudDataWorks Summit
 
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiEvent-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiDataWorks Summit
 
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerSecuring Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerDataWorks Summit
 
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...DataWorks Summit
 
Computer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouComputer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouDataWorks Summit
 
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkBig Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkDataWorks Summit
 

More from DataWorks Summit (20)

Data Science Crash Course
Data Science Crash CourseData Science Crash Course
Data Science Crash Course
 
Floating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisFloating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache Ratis
 
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiTracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
 
HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...
 
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
 
Managing the Dewey Decimal System
Managing the Dewey Decimal SystemManaging the Dewey Decimal System
Managing the Dewey Decimal System
 
Practical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExamplePractical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist Example
 
HBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberHBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at Uber
 
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixScaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
 
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiBuilding the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
 
Supporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsSupporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability Improvements
 
Security Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureSecurity Framework for Multitenant Architecture
Security Framework for Multitenant Architecture
 
Presto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EnginePresto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything Engine
 
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
 
Extending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudExtending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google Cloud
 
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiEvent-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
 
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerSecuring Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
 
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
 
Computer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouComputer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near You
 
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkBig Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
 

Recently uploaded

From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationSafe Software
 
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 Processorsdebabhi2
 
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
 
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...apidays
 
A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?Igalia
 
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
 
Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...
Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...
Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...Principled Technologies
 
Why Teams call analytics are critical to your entire business
Why Teams call analytics are critical to your entire businessWhy Teams call analytics are critical to your entire business
Why Teams call analytics are critical to your entire businesspanagenda
 
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...Martijn de Jong
 
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 WorkerThousandEyes
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationRadu Cotescu
 
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
 
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
 
Top 10 Most Downloaded Games on Play Store in 2024
Top 10 Most Downloaded Games on Play Store in 2024Top 10 Most Downloaded Games on Play Store in 2024
Top 10 Most Downloaded Games on Play Store in 2024SynarionITSolutions
 
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProduct Anonymous
 
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot TakeoffStrategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoffsammart93
 
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 DevelopmentsTrustArc
 
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024The Digital Insurer
 
Top 5 Benefits OF Using Muvi Live Paywall For Live Streams
Top 5 Benefits OF Using Muvi Live Paywall For Live StreamsTop 5 Benefits OF Using Muvi Live Paywall For Live Streams
Top 5 Benefits OF Using Muvi Live Paywall For Live StreamsRoshan Dwivedi
 

Recently uploaded (20)

From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
 
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
 
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
 
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...
Apidays New York 2024 - The Good, the Bad and the Governed by David O'Neill, ...
 
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
 
A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?
 
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
 
Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...
Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...
Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...
 
Why Teams call analytics are critical to your entire business
Why Teams call analytics are critical to your entire businessWhy Teams call analytics are critical to your entire business
Why Teams call analytics are critical to your entire business
 
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...
 
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
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organization
 
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
 
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
 
Top 10 Most Downloaded Games on Play Store in 2024
Top 10 Most Downloaded Games on Play Store in 2024Top 10 Most Downloaded Games on Play Store in 2024
Top 10 Most Downloaded Games on Play Store in 2024
 
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
 
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot TakeoffStrategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
 
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
 
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
 
Top 5 Benefits OF Using Muvi Live Paywall For Live Streams
Top 5 Benefits OF Using Muvi Live Paywall For Live StreamsTop 5 Benefits OF Using Muvi Live Paywall For Live Streams
Top 5 Benefits OF Using Muvi Live Paywall For Live Streams
 

Unified Batch & Stream Processing with Apache Samza

  • 1. Unified Batch & Stream Processing with Apache Samza Navina Ramesh Sr. Software Engineer, LinkedIn Committer and PMC member, Apache Samza @navina_r navina@apache.org
  • 2. Agenda ● Data Processing at LinkedIn ● Data Pipelines in Batch & Stream ● Overview of Apache Samza ● Convergence of Pipelines with Apache Samza ○ Support for Batch Data ○ Unified Data Processing API ○ Flexible Deployment Model
  • 3. Data Processing at LinkedIn Azure EventHub Oracle DB Espresso DB (NoSQL Store for all user data) Brooklin (DB Change Capture) HDFS Hadoop (Batch Processing) Import / Export Services Tier Ingestion Processing Voldemort / Venice (K-V Store for Derived Data) Samza (Stream Processing) Amazon Kinesis
  • 4. Scale of Processing at LinkedIn KAFKA 2.3 Trillion Msgs per Day 0.6 PB in, 2.3 PB out per Day (compressed) 16 million Msgs per Second at peaks! HADOOP 125 TB Ingested per Day 120 PB Hdfs Size 200K Jobs per Day SAMZA 200+ Applications Most Applications require Stateful Processing ~ several TBs (overall)
  • 5. Data Processing Scenarios at LinkedIn Site Speed Real-time site- speed profiling by facets Call-graph Computation Analysis of Service calls Dashboards Real-time Analytics Ad CTR Computation Tracking Ads Views and Ads Clicks Operate primarily using real-time input data
  • 6. Data Processing Scenarios at LinkedIn News Classification Real-time topic tagging of articles Profile Standardization Standardizing titles, gender, education Security Real-time DDoS protection for members ● Operate on real-time data & rely on models computed offline ● Offline computed model must be accessible during real-time processing
  • 7. Agenda ● Data Processing at LinkedIn ● Data Pipelines in Batch & Stream ● Overview of Apache Samza ● Convergence of Pipelines with Apache Samza ○ Support for Batch Data ○ Unified Data Processing API ○ Flexible Deployment Model
  • 8. Ingestion Service HDFS Mappers Reducers HDFS/ HBase Processors Processors KV Store Partition 0 Partition 1 Partition N ... Query Data Pipelines in Batch & Stream Azure EventHub Batch / Offline Stream / Realtime Streams to Batch & Batch to Stream
  • 9. Batch ● Processing on bounded data ● Processing at regular intervals ● Latency ~ order of hours ● Processing on unbounded data ● Processing is continuous ● Latency ~ order of sub-seconds ● Time matters! Stream
  • 10. ● Overhead of developing and managing multiple source codes ○ Same application logic written using 2 different APIs - one using offline processing APIs and another using near-realtime processing API ● Same application deployed in potentially 2 different managed platforms ○ Restrictions due to firewalls, acl to environments etc. ● Expensive $$ ○ When near-realtime application needs processed data from offline, the data snapshot has to be made available as a stream. This is expensive! Data Pipelines in Batch & Stream - Drawbacks
  • 11. Ingestion Service HDFS Mappers Reducers HDFS/ HBase Processors Processors HDFS KV Store Partition 0 Partition 1 Partition N ... Query Query Data Pipelines in Batch & Stream Azure EventHub Data Sources Data Processing Sink / Serving Batch / Offline Stream / Realtime
  • 12. Ingestion Service HDFS Mappers Reducers HDFS/ HBase Processors Processors HDFS KV Store Partition 0 Partition 1 Partition N ... Query Query Azure EventHub Data Sources Data Processing Sink / Serving Batch / Offline Stream / Realtime Converge Pipelines with Apache Samza
  • 13. Agenda ● Data Processing at LinkedIn ● Data Pipelines in Batch & Stream ● Overview of Apache Samza ● Convergence of Pipelines with Apache Samza ○ Support for Batch Data ○ Unified Data Processing API ○ Flexible Deployment Model
  • 14. Apache Samza • Production at LinkedIn since 2013 • Apache TLP since 2014 • Streams as first-class citizen – Batch as a special case of streaming
  • 15. Apache Samza ● Provides distributed and scalable data processing platform with ○ Configurable and heterogeneous data sources and sinks (Eg. Kafka, HDFS, Kinesis, EventHub etc) ○ Efficient state management - local state and incremental checkpoints ○ Unified Processing API for Batch & Streaming ○ Flexible deployment models
  • 16. Apache Samza Azure EventHub Amazon Kinesis HDFS Remote Runner Standalone Local Runner AzureYARN Mesos System (Producer& Consumer) Local State (Rocks DB, In-Memory) Checkpoint Manager Remote Data (Multithreading) High-level API Low-level API PROCESSOR DEPLOYM ENT API SQL DB Streams Batch Change Data Capture
  • 17. Data Processing Model • Natively supports partitioned data • Re-partitioning may be required for an un-partitioned source • Pluggable System and CheckpointManager implementations
  • 18. Partitions Partitioned Input Tasks 1 2 3 Processing Kafka/Eventhub Client Send with PartitionKey Samza Application - is a made up of Tasks - every Task processes a unique collection of input partitions 1 2 3 4 5 Processing Partitioned Data Single JVM (container)
  • 19. Partitions Partitioned Input Tasks 1 2 3 Processing Kafka/Eventhub Client Send with PartitionKey - Samza master distributes tasks across JVMs - Scale up & Distribute – increasing container count 1 2 3 4 5 Processing Partitioned Data Distributed across 3 JVMs
  • 20. Ad View Stream Samza Application 1 2 3 Ad Click Stream Ad Click Through Rate Stream Tasks Processing Joining Co-partitioned Data 1 2 3 1 2 3 Co-partitioned by Ad-ID
  • 21. Ad View Stream Samza Application 1 2 3 Ad Click Stream Ad Click Through Rate Stream Tasks Processing Joining Co-partitioned Data Local State Store (RocksDB) 1 2 3 1 2 3 Co-partitioned by Ad-ID
  • 22. Ad View Stream Samza Application 1 2 3 Ad Click Stream Ad Click Through Rate Stream Tasks Processing Joining Co-partitioned Data 1 2 3 1 2 3 Co-partitioned by Ad-ID Changelog Stream for Replication (partitioned) Used for Recovery upon Task Failure
  • 23. Agenda ● Data Processing at LinkedIn ● Data Pipelines in Batch & Stream ● Overview of Apache Samza ● Convergence of Pipelines with Apache Samza ○ Support for Batch Data ○ Unified Data Processing API ○ Flexible Deployment Model
  • 24. ❏Support for Bounded Data ❏ Define a boundary over the stream ❏ Batched Processing ❏Unified Data Processing API ❏Flexible Deployment Models – Write once, Run anywhere! How to converge?
  • 25. Agenda ● Data Processing at LinkedIn ● Data Pipelines in Batch & Stream ● Overview of Apache Samza ● Convergence of Pipelines with Apache Samza ○ Support for Batch Data ○ Unified Data Processing API ○ Flexible Deployment Model
  • 26. Support for Batch Data • Batch as a special Case of Stream:  Define boundary on stream  Batched processing – end of batch basically ends the job
  • 27. Defining a Boundary on the Stream • Introduced a notion of End-of-Stream (EoS) in the input • Consumer in the System detects the EoS for a source – Upon EoS, Samza may invoke EndOfStreamListenerTask handler implemented by the application (optional)
  • 28. File Partitions Partitioned Input Tasks 1 2 3 Processing HDFS Client Store Partitioned Data 1 2 3 4 5 Processing Bounded Data - Single File as a Partition - Directory of Files as a Stream
  • 29. File Partitions Partitioned Input Tasks 1 2 3 Processing HDFS Client Store Partitioned Data 1 2 3 4 5 Processing Bounded Data - Group of Files as a Partition - Groups defined using GroupingPattern Regex
  • 30. Support for Batch Data • Batch as a special Case of Stream:  Define boundary on stream  Batched processing – end of batch basically ends the job
  • 31. Processing Bounded Data 1 2 3 4 5 6 1 2 3 4 5 1 2 3 1 2 3 4 5 Partition 0 Partition 1 Partition 2 Partition 3 Bounded Input Task-0 Task-1
  • 32. 1 2 3 4 5 6 1 2 3 4 5 1 2 3 1 2 3 4 5 Partition 0 Partition 1 Partition 2 Partition 3 Bounded Input Task-0 Task-1 Processing Bounded Data Samza’s SystemConsumer detects EoS for Partition 1 - doesn’t shut-down the task yet.
  • 33. 1 2 3 4 5 6 1 2 3 4 5 1 2 3 1 2 3 4 5 Partition 0 Partition 1 Partition 2 Partition 3 Bounded Input Task-0 Task-1 Processing Bounded Data Continues Processing Partition-0
  • 34. 1 2 3 4 5 6 1 2 3 4 5 1 2 3 1 2 3 4 5 Partition 0 Partition 1 Partition 2 Partition 3 Bounded Input Task-0 Task-1 Samza detects EoS for the Partition 1 - shuts-down the task. Processing Bounded Data
  • 35. 1 2 3 4 5 6 1 2 3 4 5 1 2 3 1 2 3 4 5 Partition 0 Partition 1 Partition 2 Partition 3 Bounded Input Task-0 Task-1 Task has stopped processing Processing Bounded Data
  • 36. 1 2 3 4 5 6 1 2 3 4 5 1 2 3 1 2 3 4 5 Partition 0 Partition 1 Partition 2 Partition 3 Bounded Input Task-0 Task-1 When all Tasks in JVM finish processing, Samza job itself shuts-down. Processing Bounded Data
  • 37. Batch as a Special Case of Stream  Support Bounded nature of data  Define a boundary over the stream  Processing at regular intervals  Tasks exit upon complete consumption of the batch
  • 38. Profile count, group-by country 500 files 250GB input Samza HDFS Benchmark * Multiple threads per container *
  • 39. Agenda ● Data Processing at LinkedIn ● Data Pipelines in Batch & Stream ● Overview of Apache Samza ● Convergence of Pipelines with Apache Samza ○ Support for Batch Data ○ Unified Data Processing API ○ Flexible Deployment Model
  • 40. Example Application Count PageViewEvent for each mobile Device OS in a 5 minute window and send the counts to PageViewEventPerDeviceOS PageViewEvent PageViewCountPerDeviceOS Filter & Re- partition Window Map SendTo
  • 41. Samza Low-level API public interface StreamTask { void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) { // process message } } PageViewEvent PageViewCountPerDeviceOS Filter & Re- partition Window Map SendTo Job 1: PageViewRepartitionTask Job 2: PageViewByDeviceOSCounterTask PageViewEventByDeviceOS
  • 42. Application using Low-level API public class PageViewRepartitionTask implements StreamTask { private final SystemStream pageViewEventByDeviceOSStream = new SystemStream("kafka", "PaveViewEventByDeviceOS"); @Override public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception { PageViewEvent pve = (PageViewEvent) envelope.getMessage(); collector.send(new OutgoingMessageEnvelope(pageViewEventByDeviceOSStream, pve.memberId, pve)); } } Job-1: Filter & Repartition Job PageViewEvent PageViewCountPerDeviceOS Filter & Re- partition Window Map SendTo PageViewEventByDeviceOS
  • 43. Application using Low-level API public class PageViewByDeviceOSCounterTask implements InitableTask, StreamTask, WindowableTask { private final SystemStream pageViewCounterStream = new SystemStream("kafka", "PageViewCountPerDeviceOS"); private KeyValueStore<String, PageViewPerMemberIdCounterEvent> windowedCounters; private Long windowSize; @Override public void init(Config config, TaskContext context) throws Exception { this.windowedCounters = (KeyValueStore<String, PageViewPerMemberIdCounterEvent>) context.getStore("windowed-counter-store"); this.windowSize = config.getLong("task.window.ms"); } @Override public void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception { getWindowCounterEvent().forEach(counter -> collector.send(new OutgoingMessageEnvelope(pageViewCounterStream, counter.memberId, counter))); } @Override public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception { PageViewEvent pve = (PageViewEvent) envelope.getMessage(); countPageViewEvent(pve); } } Job-2: Window-based Counter PageViewEvent PageViewCountPerDeviceOS Filter & Re- partition Window Map SendTo PageViewEventByDeviceOS
  • 44. Application using Low-level API public class PageViewByDeviceOSCounterTask implements InitableTask, StreamTask, WindowableTask { private final SystemStream pageViewCounterStream = new SystemStream("kafka", "PageViewCountPerDeviceOS"); private KeyValueStore<String, PageViewPerMemberIdCounterEvent> windowedCounters; private Long windowSize; @Override public void init(Config config, TaskContext context) throws Exception { this.windowedCounters = (KeyValueStore<String, PageViewPerMemberIdCounterEvent>) context.getStore("windowed-counter-store"); this.windowSize = config.getLong("task.window.ms"); } @Override public void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception { getWindowCounterEvent().forEach(counter -> collector.send(new OutgoingMessageEnvelope(pageViewCounterStream, counter.memberId, counter))); } @Override public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception { PageViewEvent pve = (PageViewEvent) envelope.getMessage(); countPageViewEvent(pve); } } ... void countPageViewEvent(PageViewEvent pve) { String key = String.format("%08d-%s", (pve.timestamp - pve.timestamp % this.windowSize), pve.memberId); PageViewPerMemberIdCounterEvent counter = this.windowedCounters.get(key); if (counter == null) { counter = new PageViewPerMemberIdCounterEvent(pve.memberId, (pve.timestamp - pve.timestamp % this.windowSize), 0); } counter.count ++; this.windowedCounters.put(key, counter); } ... Job-2: Window-based Counter PageViewEvent PageViewCountPerDeviceOS Filter & Re- partition Window Map SendTo PageViewEventByDeviceOS
  • 45. Application using Low-level API public class PageViewByDeviceOSCounterTask implements InitableTask, StreamTask, WindowableTask { private final SystemStream pageViewCounterStream = new SystemStream("kafka", "PageViewCountPerDeviceOS"); private KeyValueStore<String, PageViewPerMemberIdCounterEvent> windowedCounters; private Long windowSize; @Override public void init(Config config, TaskContext context) throws Exception { this.windowedCounters = (KeyValueStore<String, PageViewPerMemberIdCounterEvent>) context.getStore("windowed-counter-store"); this.windowSize = config.getLong("task.window.ms"); } @Override public void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception { getWindowCounterEvent().forEach(counter -> collector.send(new OutgoingMessageEnvelope(pageViewCounterStream, counter.memberId, counter))); } @Override public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception { PageViewEvent pve = (PageViewEvent) envelope.getMessage(); countPageViewEvent(pve); } } ... List<PageViewPerMemberIdCounterEvent> getWindowCounterEvent() { List<PageViewPerMemberIdCounterEvent> retList = new ArrayList<>(); Long currentTimestamp = System.currentTimeMillis(); Long cutoffTimestamp = currentTimestamp - this.windowSize; String lowerBound = String.format("%08d-", cutoffTimestamp); String upperBound = String.format("%08d-", currentTimestamp + 1); this.windowedCounters.range(lowerBound, upperBound).forEachRemaining(entry -> retList.add(entry.getValue())); return retList; } ... Job-2: Window-based Counter PageViewEvent PageViewCountPerDeviceOS Filter & Re- partition Window Map SendTo PageViewEventByDeviceOS
  • 46. Samza High-level API public interface StreamApplication { void init(StreamGraph streamGraph, Config config) { // Process message using DSL- // like declarations } } - Ability to express a multi-stage processing pipeline in a single user program - Built-in library to provide high-level stream transformation functions -> Map, Filter, Window, Partition, Join etc. - Automatically generates the DAG for the application
  • 47. public class CountByDeviceOSApplication implements StreamApplication { @Override public void init(StreamGraph graph, Config config) { Supplier<Integer> initialValue = () -> 0; MessageStream<PageViewEvent> pageViewEvents = graph.getInputStream("pageViewEvent", (k, m) -> (PageViewEvent) m); OutputStream<String, MyStreamOutput, MyStreamOutput> pageViewEventPerMemberStream = graph .getOutputStream("pageViewCountPerDevice", m -> m.memberId, m -> m); pageViewEvents .partitionBy(m -> m.memberId) .window(Windows.keyedTumblingWindow( m -> m.memberId, Duration.ofMinutes(5),initialValue,(m, c) -> c + 1)) .map(MyStreamOutput::new) .sendTo(pageViewEventPerMemberStream); } } Built-in Transforms Application using High-level API PageViewEvent PageViewCountPerDeviceOS Filter & Re- partition Window Map SendTo PageViewEventByDeviceOS
  • 48. public class CountByDeviceOSApplication implements StreamApplication { @Override public void init(StreamGraph graph, Config config) { Supplier<Integer> initialValue = () -> 0; MessageStream<PageViewEvent> pageViewEvents = graph.getInputStream("pageViewEvent", (k, m) -> (PageViewEvent) m); OutputStream<String, MyStreamOutput, MyStreamOutput> pageViewEventPerMemberStream = graph .getOutputStream("pageViewCountPerDevice", m -> m.memberId, m -> m); pageViewEvents .partitionBy(m -> m.memberId) .window(Windows.keyedTumblingWindow( m -> m.memberId, Duration.ofMinutes(5), initialValue, (m, c) -> c + 1)) .map(MyStreamOutput::new) .sendTo(pageViewEventPerMemberStream); } } Unified for Batch & Stream Configuration for Stream Input (Kafka): systems.kafka.samza.factory = org.apache.samza.system.KafkaSystemFactory streams.PageViewEvent.samza.system = kafka streams.PageViewEvent.samza.physical.name = PageViewEvent
  • 49. public class CountByDeviceOSApplication implements StreamApplication { @Override public void init(StreamGraph graph, Config config) { Supplier<Integer> initialValue = () -> 0; MessageStream<PageViewEvent> pageViewEvents = graph.getInputStream("pageViewEvent", (k, m) -> (PageViewEvent) m); OutputStream<String, MyStreamOutput, MyStreamOutput> pageViewEventPerMemberStream = graph .getOutputStream("pageViewCountPerDevice", m -> m.memberId, m -> m); pageViewEvents .partitionBy(m -> m.memberId) .window(Windows.keyedTumblingWindow( m -> m.memberId, Duration.ofMinutes(5), initialValue, (m, c) -> c + 1)) .map(MyStreamOutput::new) .sendTo(pageViewEventPerMemberStream); } } Unified for Batch & Stream Configuration for Stream Input (Kafka): systems.kafka.samza.factory = org.apache.samza.system.KafkaSystemFactory streams.PageViewEvent.samza.system = kafka streams.PageViewEvent.samza.physical.name = PageViewEvent Configuration for Batch Input (HDFS): systems.hdfs.samza.factory = org.apache.samza.system.HdfsSystemFactory streams.PageViewEvent.samza.system = hdfs streams.PageViewEvent.samza.physical.name = hdfs:/user/nramesh/PageViewEvent
  • 50. public class CountByDeviceOSApplication implements StreamApplication { @Override public void init(StreamGraph graph, Config config) { Supplier<Integer> initialValue = () -> 0; MessageStream<PageViewEvent> pageViewEvents = graph.getInputStream("pageViewEvent", (k, m) -> (PageViewEvent) m); OutputStream<String, MyStreamOutput, MyStreamOutput> pageViewEventPerMemberStream = graph .getOutputStream("pageViewCountPerDevice", m -> m.memberId, m -> m); pageViewEvents .partitionBy(m -> m.memberId) .window(Windows.keyedTumblingWindow( m -> m.memberId, Duration.ofMinutes(5), initialValue, (m, c) -> c + 1)) .map(MyStreamOutput::new) .sendTo(pageViewEventPerMemberStream); } } Unified for Batch & Stream Configuration for Stream Input (Kafka): systems.kafka.samza.factory = org.apache.samza.system.KafkaSystemFactory streams.PageViewEvent.samza.system = kafka streams.PageViewEvent.samza.physical.name = PageViewEvent Configuration for Batch Input (HDFS): systems.hdfs.samza.factory = org.apache.samza.system.HdfsSystemFactory streams.PageViewEvent.samza.system = hdfs streams.PageViewEvent.samza.physical.name = hdfs:/user/nramesh/PageViewEvent Only Config Change!
  • 51. High-level API - Visualization for DAG SAMZA Visualizer A visualization of application samza-count-by-device-i001, which consists of 1 job(s), 1 input stream(s), and 1 output stream(s).
  • 53. Agenda ● Data Processing at LinkedIn ● Data Pipelines in Batch & Stream ● Overview of Apache Samza ● Convergence of Pipelines with Apache Samza ○ Support for Batch Data ○ Unified Data Processing API ○ Flexible Deployment Model
  • 54. Coordination Model • Coordination layer is pluggable in Samza • Samza master / leader – Distributes tasks to processor JVMs – On processor failure, it re-distributes • Available Coordination Mechanisms – Apache Yarn • ApplicationMaster is the leader – Apache Zookeeper • One of the processors is the leader and co-ordinates via Zookeeper – Microsoft Azure • One of the processors is the leader and co-ordinates via Azure’s Blob/Tables Storage
  • 55. Embedding Processor within Application - An instance of the processor is embedded within user’s application - LocalApplicationRunner helps launch the processor within the application public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); OptionSet options = cmdLine.parser().parse(args); Config config = cmdLine.loadConfig(options); LocalApplicationRunner runner = new LocalApplicationRunner(config); CountByDeviceOSApplication app = new CountByDeviceOSApplication(); runner.run(app); runner.waitForFinish(); }
  • 56. Pluggable Coordination Config public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); OptionSet options = cmdLine.parser().parse(args); Config config = cmdLine.loadConfig(options); LocalApplicationRunner runner = new LocalApplicationRunner(config); CountByDeviceOSApplication app = new CountByDeviceOSApplication(); runner.run(app); runner.waitForFinish(); } Configs with Zk-based coordination job.coordinator.factory = org.apache.samza.zk.ZkJobCoordinatorFactory job.coordinator.zk.connect = foobar:2181/samza
  • 57. Pluggable Coordination Config public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); OptionSet options = cmdLine.parser().parse(args); Config config = cmdLine.loadConfig(options); LocalApplicationRunner runner = new LocalApplicationRunner(config); CountByDeviceOSApplication app = new CountByDeviceOSApplication(); runner.run(app); runner.waitForFinish(); } Configs with Azure-based coordination: job.coordinator.factory = org.apache.samza.azure.AzureJobCoordinatorFactory job.coordinator.azure.connect = http://foobar:29892/storage/ Configs with Zk-based coordination job.coordinator.factory = org.apache.samza.zk.ZkJobCoordinatorFactory job.coordinator.zk.connect = foobar:2181/samza
  • 58. Pluggable Coordination Config public static void main(String[] args) { CommandLine cmdLine = new CommandLine(); OptionSet options = cmdLine.parser().parse(args); Config config = cmdLine.loadConfig(options); LocalApplicationRunner runner = new LocalApplicationRunner(config); CountByDeviceOSApplication app = new CountByDeviceOSApplication(); runner.run(app); runner.waitForFinish(); } Only Config Change! Configs with Azure-based coordination: job.coordinator.factory = org.apache.samza.azure.AzureJobCoordinatorFactory job.coordinator.azure.connect = http://foobar:29892/storage/ Configs with Zk-based coordination job.coordinator.factory = org.apache.samza.zk.ZkJobCoordinatorFactory job.coordinator.zk.connect = foobar:2181/samza
  • 59. Deploying Samza in a Managed Cluster (Yarn) app.class = MyStreamApplication RemoteAppplicationRunner: main() RM NM LocalApplicationRunner StreamProcessor JobCoordinator NM NM LocalApplicationRunner StreamProcessor Client Submits JAR run-jc.sh run-app.sh run-local-app.sh run-local-app.sh
  • 60. Flexible Deployment Models Samza as a Library - Run embedded stream processing in user program - Use Zookeeper for partition distribution among tasks and liveness of processors - Seamlessly scale by spinning a new processor instance Samza as a Service - Run stream processing as a managed program in a cluster (eg. Yarn) - Works with the cluster manager (Eg. AM/RM) for partition distribution among tasks and liveness of processors - Better for resource sharing in a multi- tenant environment
  • 61. Conclusion ● Easily Composable Architecture allows varied data source consumption ● Write Once, Run Anywhere paradigm ○ Unified API - application logic to be written only once ○ Pluggable Coordination Model - allows application deployment across different execution environment
  • 62. Future Work ● Support SQL on Streams with Samza ● Table Abstraction in Samza ● Event-time processing ● Samza runner for Apache Beam Contributions are welcome! ● Contributor’s Corner - http://samza.apache.org/contribute/contributors-corner.html ● Ask any question - dev@samza.apache.org ● Follow or tweet us @apachesamza
  • 65. Lambda-less Architecture with Samza Profile Updates Kafka stream Standardization Normalized Profile Updates Kafka stream Member Profiles
  • 66. Lambda-less Architecture with Samza Profile Updates Kafka stream Standardization Normalized Profile Updates Kafka stream Member Profiles update the standardization job
  • 67. Lambda-less Architecture with Samza Profile Updates Kafka stream Standardization Normalized Profile Updates Kafka stream Member Profiles update the standardization job DB Snapshot Standardization Merge & Store Results
  • 68. Lambda-less Architecture with Samza Profile Updates Kafka stream Standardization Normalized Profile Updates Kafka stream Member Profiles DB Snapshot Standardization Merge & Store Results Stream Processing Batch Processing