SlideShare a Scribd company logo
1 of 27
Unified processing with the
Samza High-level API
Yi Pan
Streams Team @LinkedIn
Committer and PMC Chair, Apache Samza
1
Agenda
• High-level API
• Flexible Deployment Model
• Convergence between Batch and Stream Processing
2
Application Example
Application logic: Count PageViewEvent for each member in a 5 minute window
and send the counts to PageViewEventPerMemberStream
Re-partition
by memberId
window map sendTo
PageViewEvent
PageViewEventPer
MemberStream
3
Application Example
Re-partition window map sendTo
PageViewEvent
PageViewEvent
ByMemberId
PageViewEventPer
MemberStream
Job-1:
PageViewRepartitionTask
Job-2: PageViewByMemberIdCounterTask
Application in low-level API
4
Application in Low-level API
• Job-1: Repartition job
public class PageViewRepartitionTask implements StreamTask {
private final SystemStream pageViewByMIDStream = new SystemStream("kafka",
"PaveViewEventByMemberId");
@Override
public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator
coordinator) throws Exception {
PageViewEvent pve = (PageViewEvent) envelope.getMessage();
collector.send(new OutgoingMessageEnvelope(pageViewByMIDStream, pve.memberId, pve));
}
}
5
Application in Low-level API
• Job-2: Window-based counter
public class PageViewByMemberIdCounterTask implements InitableTask, StreamTask, WindowableTask {
private final SystemStream pageViewCounterStream = new SystemStream("kafka", "PageViewEventPerMemberStream");
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);
}
}
6
Application in Low-level API
• Job-2: Window-based counter
public class PageViewByMemberIdCounterTask implements InitableTask, StreamTask, WindowableTask {
...
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;
}
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);
}
}
7
High Level API
• Samza High Level API (NEW)
– Ability to express a multi-stage processing
pipeline in a single user program
– Built-in library to provide high-level stream
transformation functions
8
Application in High Level API
(NEW)
public class RepartitionAndCounterExample implements StreamApplication {
@Override public void init(StreamGraph graph, Config config) {
Supplier<Integer> initialValue = () -> 0;
MessageStream<PageViewEvent> pageViewEvents =
graph.getInputStream("pageViewEventStream", (k, m) -> (PageViewEvent) m);
OutputStream<String, MyStreamOutput, MyStreamOutput> pageViewEventPerMemberStream = graph
.getOutputStream("pageViewEventPerMemberStream", 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 transform functions
9
Application in High Level API
(NEW)
• Visualized execution plan
Visualization:
10
High Level API
• Built-in transformation functions in high-level
API
filter select a subset of messages from the stream
map map one input message to an output message
flatMap map one input message to 0 or more output messages
merge union all inputs into a single output stream
partitionBy re-partition the input messages based on a specific field
sendTo send the result to an output stream
sink send the result to an external system (e.g. external DB)
window window aggregation on the input stream
join join messages from two input streams
statelessfunctionsI/Ofunctions
stateful
functions
11
Agenda
• High-level API
• Flexible Deployment Model
• Convergence between Batch and Stream Processing
12
Limitations with current Samza
Deployment
• Tight dependency on YARN
• Can’t easily port over to non-YARN clusters
(e.g. Mesos, Kubernetes, AWS)
• Can’t directly embed stream processing in
other application (eg. a web frontend)
13
Flexible Deployment Model
• Flexible deployment of Samza applications
– Samza-as-a-library (NEW)
• Run embedded stream processing in a user program
• Zookeeper based coordination between multiple
instances of user program
– Samza in a cluster
• Run stream processing as a managed program in a
cluster (e.g. SamzaContainer in YARN)
• Use the cluster manager (e.g. YARN) to provide
deployment, coordination, and resource management
14
Samza-as-a-library
Samza Job is composed of a collection of standalone processes
● Full control on
● Application’s life cycle
● Physical resource allocated to Samza processors
● Configuration and initialization
StreamProcessor
Samza
Container
Job
Coordinator
StreamProcessor
Samza
Container
Job
Coordinator
StreamProcessor
Samza
Container
Job
Coordinator...
Leader
15
● ZooKeeper-based JobCoordinator (stateful use
case)
● JobCoordinator uses ZooKeeper for leader election
● Leader will perform partition assignments among all
active StreamProcessors
Samza-as-a-library
ZooKeeper
StreamProcessor
Samza
Container
Job
Coordinator
StreamProcessor
Samza
Container
Job
Coordinator
StreamProcessor
Samza
Container
Job
Coordinator...
16
Samza-as-a-library
● Embedded application code example
public class WikipediaZkLocalApplication {
/**
* Executes the application using the local application runner.
* It takes two required command line arguments
* config-factory: a fully {@link org.apache.samza.config.factories.PropertiesConfigFactory} class name
* config-path: path to application properties
*
* @param args command line arguments
*/
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);
WikipediaApplication app = new WikipediaApplication();
runner.run(app);
runner.waitForFinish();
}
}
17
Samza-as-a-library
● Embedded application code example
public class WikipediaZkLocalApplication {
/**
* Executes the application using the local application runner.
* It takes two required command line arguments
* config-factory: a fully {@link org.apache.samza.config.factories.PropertiesConfigFactory} class name
* config-path: path to application properties
*
* @param args command line arguments
*/
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);
WikipediaApplication app = new WikipediaApplication();
runner.run(app);
runner.waitForFinish();
}
}
18
job.coordinator.factory=org.apache.samza.zk.ZkJobCoordinatorFactory
job.coordinator.zk.connect=my-zk.server:2191
• Embedded application launch sequence
Samza-as-a-library
myApp.main()
Stream
Application
Local
Application
Runner
Stream
Processor
runner.run() streamProcessor.
start()
n
19
•Cluster-based application launch sequence
Samza in a Cluster
run-app.sh
Remote
Application
Runner
JobRunnerjobRunner.run()
n
main()
app.class=my.app.MyStreamApplication
Yarn
RM
run-jc.sh
task.execute=run-local-app.sh
run-local-app.sh
Stream
Application
myApp.main()
Local
Application
Runner
Stream
Processor
runner.run() streamProcessor.
start()
n
Job
Coordinator
20
Unified StreamProcessor Design
21
Overview
• High-level API
• Flexible Deployment Model
• Convergence between Batch and Stream Processing
22
Stream Application in Batch
Application logic: Count PageViewEvent for each member in a 5 minute window
and send the counts to PageViewEventPerMemberStream
Re-partition
by memberId
window map sendTo
PageViewEvent
PageViewEventPer
MemberStream
HDFS
PageViewEvent: hdfs://mydbsnapshot/PageViewEvent/
PageViewEventPerMemberStream: hdfs://myoutputdb/PageViewEventPerMemberFiles
23
Stream Application in Batch
• No code change in application
streams.pageViewEventStream.system=kafka
streams.pageViewEventPerMemberStream.system=kafka
streams.pageViewEventStream.system=hdfs
streams.pageViewEventStream.physical.name=hdfs://mydbsnapshot/PageViewEvent/
streams.pageViewEventPerMemberStream.system=hdfs
streams.pageViewEventPerMemberStream.physical.name=hdfs://myoutputdb/PageViewEventPerMemberFiles
old config
new config
24
Samza 0.13 Architecture
25
High-level API
Unified Stream & Batch Processing
Remote Runner
Run in Remote Cluster
Cluster-based
Yarn, (Mesos)
Local Runner
Run Locally
Embedded
ZooKeeper, Standalone
APIRUNNERDEPLOY
MENT
PROCESSOR
StreamProcessor
Streams
Kafka, Kinesis, HDFS
...
Local State
RocksDb, In-Memory
Remote Data
Multithreading
25
Future Works
• Samza runner for Apache Beam
• Event-time processing
• Support for Exactly-once processing
• Support partition expansion for stateful
application
• Easy access to Adjunct datasets
• SQL over Streams
26
Thank You!
Q&A
27

More Related Content

What's hot

Apache kafka meet_up_zurich_at_swissre_from_zero_to_hero_with_kafka_connect_2...
Apache kafka meet_up_zurich_at_swissre_from_zero_to_hero_with_kafka_connect_2...Apache kafka meet_up_zurich_at_swissre_from_zero_to_hero_with_kafka_connect_2...
Apache kafka meet_up_zurich_at_swissre_from_zero_to_hero_with_kafka_connect_2...confluent
 
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...Flink Forward
 
QCON 2015: Gearpump, Realtime Streaming on Akka
QCON 2015: Gearpump, Realtime Streaming on AkkaQCON 2015: Gearpump, Realtime Streaming on Akka
QCON 2015: Gearpump, Realtime Streaming on AkkaSean Zhong
 
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...HostedbyConfluent
 
Robust Operations of Kafka Streams
Robust Operations of Kafka StreamsRobust Operations of Kafka Streams
Robust Operations of Kafka Streamsconfluent
 
Apache Gearpump next-gen streaming engine
Apache Gearpump next-gen streaming engineApache Gearpump next-gen streaming engine
Apache Gearpump next-gen streaming engineTianlun Zhang
 
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...Flink Forward
 
Introduction to the Processor API
Introduction to the Processor APIIntroduction to the Processor API
Introduction to the Processor APIconfluent
 
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" Flink Forward
 
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...Flink Forward
 
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...Flink Forward
 
Event Sourcing - what could possibly go wrong?
Event Sourcing - what could possibly go wrong?Event Sourcing - what could possibly go wrong?
Event Sourcing - what could possibly go wrong?Andrzej Ludwikowski
 
Andrzej Ludwikowski - Event Sourcing - what could possibly go wrong? - Codemo...
Andrzej Ludwikowski - Event Sourcing - what could possibly go wrong? - Codemo...Andrzej Ludwikowski - Event Sourcing - what could possibly go wrong? - Codemo...
Andrzej Ludwikowski - Event Sourcing - what could possibly go wrong? - Codemo...Codemotion
 
Kafka Summit NYC 2017 - The Best Thing Since Partitioned Bread
Kafka Summit NYC 2017 - The Best Thing Since Partitioned Bread Kafka Summit NYC 2017 - The Best Thing Since Partitioned Bread
Kafka Summit NYC 2017 - The Best Thing Since Partitioned Bread confluent
 
Kafka Streams: the easiest way to start with stream processing
Kafka Streams: the easiest way to start with stream processingKafka Streams: the easiest way to start with stream processing
Kafka Streams: the easiest way to start with stream processingYaroslav Tkachenko
 
Webinar: Deep Dive on Apache Flink State - Seth Wiesman
Webinar: Deep Dive on Apache Flink State - Seth WiesmanWebinar: Deep Dive on Apache Flink State - Seth Wiesman
Webinar: Deep Dive on Apache Flink State - Seth WiesmanVerverica
 
High Available Task Scheduling Design using Kafka and Kafka Streams | Naveen ...
High Available Task Scheduling Design using Kafka and Kafka Streams | Naveen ...High Available Task Scheduling Design using Kafka and Kafka Streams | Naveen ...
High Available Task Scheduling Design using Kafka and Kafka Streams | Naveen ...HostedbyConfluent
 
What you need to know for postgresql operation
What you need to know for postgresql operationWhat you need to know for postgresql operation
What you need to know for postgresql operationAnton Bushmelev
 
Flink Forward Berlin 2017: Jörg Schad, Till Rohrmann - Apache Flink meets Apa...
Flink Forward Berlin 2017: Jörg Schad, Till Rohrmann - Apache Flink meets Apa...Flink Forward Berlin 2017: Jörg Schad, Till Rohrmann - Apache Flink meets Apa...
Flink Forward Berlin 2017: Jörg Schad, Till Rohrmann - Apache Flink meets Apa...Flink Forward
 

What's hot (20)

Apache kafka meet_up_zurich_at_swissre_from_zero_to_hero_with_kafka_connect_2...
Apache kafka meet_up_zurich_at_swissre_from_zero_to_hero_with_kafka_connect_2...Apache kafka meet_up_zurich_at_swissre_from_zero_to_hero_with_kafka_connect_2...
Apache kafka meet_up_zurich_at_swissre_from_zero_to_hero_with_kafka_connect_2...
 
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
 
QCON 2015: Gearpump, Realtime Streaming on Akka
QCON 2015: Gearpump, Realtime Streaming on AkkaQCON 2015: Gearpump, Realtime Streaming on Akka
QCON 2015: Gearpump, Realtime Streaming on Akka
 
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
 
Robust Operations of Kafka Streams
Robust Operations of Kafka StreamsRobust Operations of Kafka Streams
Robust Operations of Kafka Streams
 
Apache Gearpump next-gen streaming engine
Apache Gearpump next-gen streaming engineApache Gearpump next-gen streaming engine
Apache Gearpump next-gen streaming engine
 
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
Flink Forward SF 2017: Feng Wang & Zhijiang Wang - Runtime Improvements in Bl...
 
Introduction to the Processor API
Introduction to the Processor APIIntroduction to the Processor API
Introduction to the Processor API
 
ApacheCon BigData Europe 2015
ApacheCon BigData Europe 2015 ApacheCon BigData Europe 2015
ApacheCon BigData Europe 2015
 
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
 
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...
 
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch -  Dynami...
Flink Forward SF 2017: David Hardwick, Sean Hester & David Brelloch - Dynami...
 
Event Sourcing - what could possibly go wrong?
Event Sourcing - what could possibly go wrong?Event Sourcing - what could possibly go wrong?
Event Sourcing - what could possibly go wrong?
 
Andrzej Ludwikowski - Event Sourcing - what could possibly go wrong? - Codemo...
Andrzej Ludwikowski - Event Sourcing - what could possibly go wrong? - Codemo...Andrzej Ludwikowski - Event Sourcing - what could possibly go wrong? - Codemo...
Andrzej Ludwikowski - Event Sourcing - what could possibly go wrong? - Codemo...
 
Kafka Summit NYC 2017 - The Best Thing Since Partitioned Bread
Kafka Summit NYC 2017 - The Best Thing Since Partitioned Bread Kafka Summit NYC 2017 - The Best Thing Since Partitioned Bread
Kafka Summit NYC 2017 - The Best Thing Since Partitioned Bread
 
Kafka Streams: the easiest way to start with stream processing
Kafka Streams: the easiest way to start with stream processingKafka Streams: the easiest way to start with stream processing
Kafka Streams: the easiest way to start with stream processing
 
Webinar: Deep Dive on Apache Flink State - Seth Wiesman
Webinar: Deep Dive on Apache Flink State - Seth WiesmanWebinar: Deep Dive on Apache Flink State - Seth Wiesman
Webinar: Deep Dive on Apache Flink State - Seth Wiesman
 
High Available Task Scheduling Design using Kafka and Kafka Streams | Naveen ...
High Available Task Scheduling Design using Kafka and Kafka Streams | Naveen ...High Available Task Scheduling Design using Kafka and Kafka Streams | Naveen ...
High Available Task Scheduling Design using Kafka and Kafka Streams | Naveen ...
 
What you need to know for postgresql operation
What you need to know for postgresql operationWhat you need to know for postgresql operation
What you need to know for postgresql operation
 
Flink Forward Berlin 2017: Jörg Schad, Till Rohrmann - Apache Flink meets Apa...
Flink Forward Berlin 2017: Jörg Schad, Till Rohrmann - Apache Flink meets Apa...Flink Forward Berlin 2017: Jörg Schad, Till Rohrmann - Apache Flink meets Apa...
Flink Forward Berlin 2017: Jörg Schad, Till Rohrmann - Apache Flink meets Apa...
 

Similar to Samza 0.13 meetup slide v1.0.pptx

Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...
Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...
Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...Big Data Spain
 
Samza Demo @scale 2017
Samza Demo @scale 2017Samza Demo @scale 2017
Samza Demo @scale 2017Xinyu Liu
 
13 asp.net session19
13 asp.net session1913 asp.net session19
13 asp.net session19Vivek chan
 
Fabric - Realtime stream processing framework
Fabric - Realtime stream processing frameworkFabric - Realtime stream processing framework
Fabric - Realtime stream processing frameworkShashank Gautam
 
Samza sql stream processing meetup
Samza sql stream processing meetupSamza sql stream processing meetup
Samza sql stream processing meetupSrinivasulu Punuru
 
using Mithril.js + postgREST to build and consume API's
using Mithril.js + postgREST to build and consume API'susing Mithril.js + postgREST to build and consume API's
using Mithril.js + postgREST to build and consume API'sAntônio Roberto Silva
 
Maciej Treder ''Angular Universal - a medicine for the Angular + SEO/CDN issu...
Maciej Treder ''Angular Universal - a medicine for the Angular + SEO/CDN issu...Maciej Treder ''Angular Universal - a medicine for the Angular + SEO/CDN issu...
Maciej Treder ''Angular Universal - a medicine for the Angular + SEO/CDN issu...OdessaJS Conf
 
Data Microservices In The Cloud + 日本語コメント
Data Microservices In The Cloud + 日本語コメントData Microservices In The Cloud + 日本語コメント
Data Microservices In The Cloud + 日本語コメントTakuya Saeki
 
Rest web service_with_spring_hateoas
Rest web service_with_spring_hateoasRest web service_with_spring_hateoas
Rest web service_with_spring_hateoasZeid Hassan
 
Event Streaming with Kafka Streams and Spring Cloud Stream | Soby Chacko, VMware
Event Streaming with Kafka Streams and Spring Cloud Stream | Soby Chacko, VMwareEvent Streaming with Kafka Streams and Spring Cloud Stream | Soby Chacko, VMware
Event Streaming with Kafka Streams and Spring Cloud Stream | Soby Chacko, VMwareHostedbyConfluent
 
Multi Client Development with Spring
Multi Client Development with SpringMulti Client Development with Spring
Multi Client Development with SpringJoshua Long
 
Reactive programming every day
Reactive programming every dayReactive programming every day
Reactive programming every dayVadym Khondar
 
Spring Web Services: SOAP vs. REST
Spring Web Services: SOAP vs. RESTSpring Web Services: SOAP vs. REST
Spring Web Services: SOAP vs. RESTSam Brannen
 
Developing Microservices using Spring - Beginner's Guide
Developing Microservices using Spring - Beginner's GuideDeveloping Microservices using Spring - Beginner's Guide
Developing Microservices using Spring - Beginner's GuideMohanraj Thirumoorthy
 
SynapseIndia dotnet development ajax client library
SynapseIndia dotnet development ajax client librarySynapseIndia dotnet development ajax client library
SynapseIndia dotnet development ajax client librarySynapseindiappsdevelopment
 
Reactive Application Using METEOR
Reactive Application Using METEORReactive Application Using METEOR
Reactive Application Using METEORNodeXperts
 
SamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationSamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationYi Pan
 
Stream and Batch Processing in the Cloud with Data Microservices
Stream and Batch Processing in the Cloud with Data MicroservicesStream and Batch Processing in the Cloud with Data Microservices
Stream and Batch Processing in the Cloud with Data Microservicesmarius_bogoevici
 
Parallelminds.asp.net with sp
Parallelminds.asp.net with spParallelminds.asp.net with sp
Parallelminds.asp.net with spparallelminder
 

Similar to Samza 0.13 meetup slide v1.0.pptx (20)

Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...
Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...
Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...
 
Samza Demo @scale 2017
Samza Demo @scale 2017Samza Demo @scale 2017
Samza Demo @scale 2017
 
13 asp.net session19
13 asp.net session1913 asp.net session19
13 asp.net session19
 
Fabric - Realtime stream processing framework
Fabric - Realtime stream processing frameworkFabric - Realtime stream processing framework
Fabric - Realtime stream processing framework
 
Samza sql stream processing meetup
Samza sql stream processing meetupSamza sql stream processing meetup
Samza sql stream processing meetup
 
using Mithril.js + postgREST to build and consume API's
using Mithril.js + postgREST to build and consume API'susing Mithril.js + postgREST to build and consume API's
using Mithril.js + postgREST to build and consume API's
 
Maciej Treder ''Angular Universal - a medicine for the Angular + SEO/CDN issu...
Maciej Treder ''Angular Universal - a medicine for the Angular + SEO/CDN issu...Maciej Treder ''Angular Universal - a medicine for the Angular + SEO/CDN issu...
Maciej Treder ''Angular Universal - a medicine for the Angular + SEO/CDN issu...
 
Data Microservices In The Cloud + 日本語コメント
Data Microservices In The Cloud + 日本語コメントData Microservices In The Cloud + 日本語コメント
Data Microservices In The Cloud + 日本語コメント
 
Rest web service_with_spring_hateoas
Rest web service_with_spring_hateoasRest web service_with_spring_hateoas
Rest web service_with_spring_hateoas
 
Event Streaming with Kafka Streams and Spring Cloud Stream | Soby Chacko, VMware
Event Streaming with Kafka Streams and Spring Cloud Stream | Soby Chacko, VMwareEvent Streaming with Kafka Streams and Spring Cloud Stream | Soby Chacko, VMware
Event Streaming with Kafka Streams and Spring Cloud Stream | Soby Chacko, VMware
 
Multi Client Development with Spring
Multi Client Development with SpringMulti Client Development with Spring
Multi Client Development with Spring
 
Reactive programming every day
Reactive programming every dayReactive programming every day
Reactive programming every day
 
Spring Web Services: SOAP vs. REST
Spring Web Services: SOAP vs. RESTSpring Web Services: SOAP vs. REST
Spring Web Services: SOAP vs. REST
 
Developing Microservices using Spring - Beginner's Guide
Developing Microservices using Spring - Beginner's GuideDeveloping Microservices using Spring - Beginner's Guide
Developing Microservices using Spring - Beginner's Guide
 
SynapseIndia dotnet development ajax client library
SynapseIndia dotnet development ajax client librarySynapseIndia dotnet development ajax client library
SynapseIndia dotnet development ajax client library
 
Reactive Application Using METEOR
Reactive Application Using METEORReactive Application Using METEOR
Reactive Application Using METEOR
 
SamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationSamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentation
 
Stream and Batch Processing in the Cloud with Data Microservices
Stream and Batch Processing in the Cloud with Data MicroservicesStream and Batch Processing in the Cloud with Data Microservices
Stream and Batch Processing in the Cloud with Data Microservices
 
Parallelminds.asp.net with sp
Parallelminds.asp.net with spParallelminds.asp.net with sp
Parallelminds.asp.net with sp
 
Asp.net control
Asp.net controlAsp.net control
Asp.net control
 

Recently uploaded

Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur Escorts
Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur EscortsCall Girls in Nagpur Suman Call 7001035870 Meet With Nagpur Escorts
Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur EscortsCall Girls in Nagpur High Profile
 
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur EscortsHigh Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escortsranjana rawat
 
MANUFACTURING PROCESS-II UNIT-2 LATHE MACHINE
MANUFACTURING PROCESS-II UNIT-2 LATHE MACHINEMANUFACTURING PROCESS-II UNIT-2 LATHE MACHINE
MANUFACTURING PROCESS-II UNIT-2 LATHE MACHINESIVASHANKAR N
 
University management System project report..pdf
University management System project report..pdfUniversity management System project report..pdf
University management System project report..pdfKamal Acharya
 
Call Girls Service Nagpur Tanvi Call 7001035870 Meet With Nagpur Escorts
Call Girls Service Nagpur Tanvi Call 7001035870 Meet With Nagpur EscortsCall Girls Service Nagpur Tanvi Call 7001035870 Meet With Nagpur Escorts
Call Girls Service Nagpur Tanvi Call 7001035870 Meet With Nagpur EscortsCall Girls in Nagpur High Profile
 
Porous Ceramics seminar and technical writing
Porous Ceramics seminar and technical writingPorous Ceramics seminar and technical writing
Porous Ceramics seminar and technical writingrakeshbaidya232001
 
College Call Girls Nashik Nehal 7001305949 Independent Escort Service Nashik
College Call Girls Nashik Nehal 7001305949 Independent Escort Service NashikCollege Call Girls Nashik Nehal 7001305949 Independent Escort Service Nashik
College Call Girls Nashik Nehal 7001305949 Independent Escort Service NashikCall Girls in Nagpur High Profile
 
(SHREYA) Chakan Call Girls Just Call 7001035870 [ Cash on Delivery ] Pune Esc...
(SHREYA) Chakan Call Girls Just Call 7001035870 [ Cash on Delivery ] Pune Esc...(SHREYA) Chakan Call Girls Just Call 7001035870 [ Cash on Delivery ] Pune Esc...
(SHREYA) Chakan Call Girls Just Call 7001035870 [ Cash on Delivery ] Pune Esc...ranjana rawat
 
Call Girls Pimpri Chinchwad Call Me 7737669865 Budget Friendly No Advance Boo...
Call Girls Pimpri Chinchwad Call Me 7737669865 Budget Friendly No Advance Boo...Call Girls Pimpri Chinchwad Call Me 7737669865 Budget Friendly No Advance Boo...
Call Girls Pimpri Chinchwad Call Me 7737669865 Budget Friendly No Advance Boo...roncy bisnoi
 
(ANVI) Koregaon Park Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...
(ANVI) Koregaon Park Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...(ANVI) Koregaon Park Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...
(ANVI) Koregaon Park Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...ranjana rawat
 
Online banking management system project.pdf
Online banking management system project.pdfOnline banking management system project.pdf
Online banking management system project.pdfKamal Acharya
 
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...Christo Ananth
 
UNIT-II FMM-Flow Through Circular Conduits
UNIT-II FMM-Flow Through Circular ConduitsUNIT-II FMM-Flow Through Circular Conduits
UNIT-II FMM-Flow Through Circular Conduitsrknatarajan
 
Booking open Available Pune Call Girls Koregaon Park 6297143586 Call Hot Ind...
Booking open Available Pune Call Girls Koregaon Park  6297143586 Call Hot Ind...Booking open Available Pune Call Girls Koregaon Park  6297143586 Call Hot Ind...
Booking open Available Pune Call Girls Koregaon Park 6297143586 Call Hot Ind...Call Girls in Nagpur High Profile
 
ONLINE FOOD ORDER SYSTEM PROJECT REPORT.pdf
ONLINE FOOD ORDER SYSTEM PROJECT REPORT.pdfONLINE FOOD ORDER SYSTEM PROJECT REPORT.pdf
ONLINE FOOD ORDER SYSTEM PROJECT REPORT.pdfKamal Acharya
 
Call Girls Service Nashik Vaishnavi 7001305949 Independent Escort Service Nashik
Call Girls Service Nashik Vaishnavi 7001305949 Independent Escort Service NashikCall Girls Service Nashik Vaishnavi 7001305949 Independent Escort Service Nashik
Call Girls Service Nashik Vaishnavi 7001305949 Independent Escort Service NashikCall Girls in Nagpur High Profile
 
AKTU Computer Networks notes --- Unit 3.pdf
AKTU Computer Networks notes ---  Unit 3.pdfAKTU Computer Networks notes ---  Unit 3.pdf
AKTU Computer Networks notes --- Unit 3.pdfankushspencer015
 
Sheet Pile Wall Design and Construction: A Practical Guide for Civil Engineer...
Sheet Pile Wall Design and Construction: A Practical Guide for Civil Engineer...Sheet Pile Wall Design and Construction: A Practical Guide for Civil Engineer...
Sheet Pile Wall Design and Construction: A Practical Guide for Civil Engineer...Dr.Costas Sachpazis
 
The Most Attractive Pune Call Girls Budhwar Peth 8250192130 Will You Miss Thi...
The Most Attractive Pune Call Girls Budhwar Peth 8250192130 Will You Miss Thi...The Most Attractive Pune Call Girls Budhwar Peth 8250192130 Will You Miss Thi...
The Most Attractive Pune Call Girls Budhwar Peth 8250192130 Will You Miss Thi...ranjana rawat
 
High Profile Call Girls Nagpur Meera Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Meera Call 7001035870 Meet With Nagpur EscortsHigh Profile Call Girls Nagpur Meera Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Meera Call 7001035870 Meet With Nagpur EscortsCall Girls in Nagpur High Profile
 

Recently uploaded (20)

Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur Escorts
Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur EscortsCall Girls in Nagpur Suman Call 7001035870 Meet With Nagpur Escorts
Call Girls in Nagpur Suman Call 7001035870 Meet With Nagpur Escorts
 
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur EscortsHigh Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Isha Call 7001035870 Meet With Nagpur Escorts
 
MANUFACTURING PROCESS-II UNIT-2 LATHE MACHINE
MANUFACTURING PROCESS-II UNIT-2 LATHE MACHINEMANUFACTURING PROCESS-II UNIT-2 LATHE MACHINE
MANUFACTURING PROCESS-II UNIT-2 LATHE MACHINE
 
University management System project report..pdf
University management System project report..pdfUniversity management System project report..pdf
University management System project report..pdf
 
Call Girls Service Nagpur Tanvi Call 7001035870 Meet With Nagpur Escorts
Call Girls Service Nagpur Tanvi Call 7001035870 Meet With Nagpur EscortsCall Girls Service Nagpur Tanvi Call 7001035870 Meet With Nagpur Escorts
Call Girls Service Nagpur Tanvi Call 7001035870 Meet With Nagpur Escorts
 
Porous Ceramics seminar and technical writing
Porous Ceramics seminar and technical writingPorous Ceramics seminar and technical writing
Porous Ceramics seminar and technical writing
 
College Call Girls Nashik Nehal 7001305949 Independent Escort Service Nashik
College Call Girls Nashik Nehal 7001305949 Independent Escort Service NashikCollege Call Girls Nashik Nehal 7001305949 Independent Escort Service Nashik
College Call Girls Nashik Nehal 7001305949 Independent Escort Service Nashik
 
(SHREYA) Chakan Call Girls Just Call 7001035870 [ Cash on Delivery ] Pune Esc...
(SHREYA) Chakan Call Girls Just Call 7001035870 [ Cash on Delivery ] Pune Esc...(SHREYA) Chakan Call Girls Just Call 7001035870 [ Cash on Delivery ] Pune Esc...
(SHREYA) Chakan Call Girls Just Call 7001035870 [ Cash on Delivery ] Pune Esc...
 
Call Girls Pimpri Chinchwad Call Me 7737669865 Budget Friendly No Advance Boo...
Call Girls Pimpri Chinchwad Call Me 7737669865 Budget Friendly No Advance Boo...Call Girls Pimpri Chinchwad Call Me 7737669865 Budget Friendly No Advance Boo...
Call Girls Pimpri Chinchwad Call Me 7737669865 Budget Friendly No Advance Boo...
 
(ANVI) Koregaon Park Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...
(ANVI) Koregaon Park Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...(ANVI) Koregaon Park Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...
(ANVI) Koregaon Park Call Girls Just Call 7001035870 [ Cash on Delivery ] Pun...
 
Online banking management system project.pdf
Online banking management system project.pdfOnline banking management system project.pdf
Online banking management system project.pdf
 
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...
Call for Papers - African Journal of Biological Sciences, E-ISSN: 2663-2187, ...
 
UNIT-II FMM-Flow Through Circular Conduits
UNIT-II FMM-Flow Through Circular ConduitsUNIT-II FMM-Flow Through Circular Conduits
UNIT-II FMM-Flow Through Circular Conduits
 
Booking open Available Pune Call Girls Koregaon Park 6297143586 Call Hot Ind...
Booking open Available Pune Call Girls Koregaon Park  6297143586 Call Hot Ind...Booking open Available Pune Call Girls Koregaon Park  6297143586 Call Hot Ind...
Booking open Available Pune Call Girls Koregaon Park 6297143586 Call Hot Ind...
 
ONLINE FOOD ORDER SYSTEM PROJECT REPORT.pdf
ONLINE FOOD ORDER SYSTEM PROJECT REPORT.pdfONLINE FOOD ORDER SYSTEM PROJECT REPORT.pdf
ONLINE FOOD ORDER SYSTEM PROJECT REPORT.pdf
 
Call Girls Service Nashik Vaishnavi 7001305949 Independent Escort Service Nashik
Call Girls Service Nashik Vaishnavi 7001305949 Independent Escort Service NashikCall Girls Service Nashik Vaishnavi 7001305949 Independent Escort Service Nashik
Call Girls Service Nashik Vaishnavi 7001305949 Independent Escort Service Nashik
 
AKTU Computer Networks notes --- Unit 3.pdf
AKTU Computer Networks notes ---  Unit 3.pdfAKTU Computer Networks notes ---  Unit 3.pdf
AKTU Computer Networks notes --- Unit 3.pdf
 
Sheet Pile Wall Design and Construction: A Practical Guide for Civil Engineer...
Sheet Pile Wall Design and Construction: A Practical Guide for Civil Engineer...Sheet Pile Wall Design and Construction: A Practical Guide for Civil Engineer...
Sheet Pile Wall Design and Construction: A Practical Guide for Civil Engineer...
 
The Most Attractive Pune Call Girls Budhwar Peth 8250192130 Will You Miss Thi...
The Most Attractive Pune Call Girls Budhwar Peth 8250192130 Will You Miss Thi...The Most Attractive Pune Call Girls Budhwar Peth 8250192130 Will You Miss Thi...
The Most Attractive Pune Call Girls Budhwar Peth 8250192130 Will You Miss Thi...
 
High Profile Call Girls Nagpur Meera Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Meera Call 7001035870 Meet With Nagpur EscortsHigh Profile Call Girls Nagpur Meera Call 7001035870 Meet With Nagpur Escorts
High Profile Call Girls Nagpur Meera Call 7001035870 Meet With Nagpur Escorts
 

Samza 0.13 meetup slide v1.0.pptx

  • 1. Unified processing with the Samza High-level API Yi Pan Streams Team @LinkedIn Committer and PMC Chair, Apache Samza 1
  • 2. Agenda • High-level API • Flexible Deployment Model • Convergence between Batch and Stream Processing 2
  • 3. Application Example Application logic: Count PageViewEvent for each member in a 5 minute window and send the counts to PageViewEventPerMemberStream Re-partition by memberId window map sendTo PageViewEvent PageViewEventPer MemberStream 3
  • 4. Application Example Re-partition window map sendTo PageViewEvent PageViewEvent ByMemberId PageViewEventPer MemberStream Job-1: PageViewRepartitionTask Job-2: PageViewByMemberIdCounterTask Application in low-level API 4
  • 5. Application in Low-level API • Job-1: Repartition job public class PageViewRepartitionTask implements StreamTask { private final SystemStream pageViewByMIDStream = new SystemStream("kafka", "PaveViewEventByMemberId"); @Override public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception { PageViewEvent pve = (PageViewEvent) envelope.getMessage(); collector.send(new OutgoingMessageEnvelope(pageViewByMIDStream, pve.memberId, pve)); } } 5
  • 6. Application in Low-level API • Job-2: Window-based counter public class PageViewByMemberIdCounterTask implements InitableTask, StreamTask, WindowableTask { private final SystemStream pageViewCounterStream = new SystemStream("kafka", "PageViewEventPerMemberStream"); 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); } } 6
  • 7. Application in Low-level API • Job-2: Window-based counter public class PageViewByMemberIdCounterTask implements InitableTask, StreamTask, WindowableTask { ... 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; } 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); } } 7
  • 8. High Level API • Samza High Level API (NEW) – Ability to express a multi-stage processing pipeline in a single user program – Built-in library to provide high-level stream transformation functions 8
  • 9. Application in High Level API (NEW) public class RepartitionAndCounterExample implements StreamApplication { @Override public void init(StreamGraph graph, Config config) { Supplier<Integer> initialValue = () -> 0; MessageStream<PageViewEvent> pageViewEvents = graph.getInputStream("pageViewEventStream", (k, m) -> (PageViewEvent) m); OutputStream<String, MyStreamOutput, MyStreamOutput> pageViewEventPerMemberStream = graph .getOutputStream("pageViewEventPerMemberStream", 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 transform functions 9
  • 10. Application in High Level API (NEW) • Visualized execution plan Visualization: 10
  • 11. High Level API • Built-in transformation functions in high-level API filter select a subset of messages from the stream map map one input message to an output message flatMap map one input message to 0 or more output messages merge union all inputs into a single output stream partitionBy re-partition the input messages based on a specific field sendTo send the result to an output stream sink send the result to an external system (e.g. external DB) window window aggregation on the input stream join join messages from two input streams statelessfunctionsI/Ofunctions stateful functions 11
  • 12. Agenda • High-level API • Flexible Deployment Model • Convergence between Batch and Stream Processing 12
  • 13. Limitations with current Samza Deployment • Tight dependency on YARN • Can’t easily port over to non-YARN clusters (e.g. Mesos, Kubernetes, AWS) • Can’t directly embed stream processing in other application (eg. a web frontend) 13
  • 14. Flexible Deployment Model • Flexible deployment of Samza applications – Samza-as-a-library (NEW) • Run embedded stream processing in a user program • Zookeeper based coordination between multiple instances of user program – Samza in a cluster • Run stream processing as a managed program in a cluster (e.g. SamzaContainer in YARN) • Use the cluster manager (e.g. YARN) to provide deployment, coordination, and resource management 14
  • 15. Samza-as-a-library Samza Job is composed of a collection of standalone processes ● Full control on ● Application’s life cycle ● Physical resource allocated to Samza processors ● Configuration and initialization StreamProcessor Samza Container Job Coordinator StreamProcessor Samza Container Job Coordinator StreamProcessor Samza Container Job Coordinator... Leader 15
  • 16. ● ZooKeeper-based JobCoordinator (stateful use case) ● JobCoordinator uses ZooKeeper for leader election ● Leader will perform partition assignments among all active StreamProcessors Samza-as-a-library ZooKeeper StreamProcessor Samza Container Job Coordinator StreamProcessor Samza Container Job Coordinator StreamProcessor Samza Container Job Coordinator... 16
  • 17. Samza-as-a-library ● Embedded application code example public class WikipediaZkLocalApplication { /** * Executes the application using the local application runner. * It takes two required command line arguments * config-factory: a fully {@link org.apache.samza.config.factories.PropertiesConfigFactory} class name * config-path: path to application properties * * @param args command line arguments */ 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); WikipediaApplication app = new WikipediaApplication(); runner.run(app); runner.waitForFinish(); } } 17
  • 18. Samza-as-a-library ● Embedded application code example public class WikipediaZkLocalApplication { /** * Executes the application using the local application runner. * It takes two required command line arguments * config-factory: a fully {@link org.apache.samza.config.factories.PropertiesConfigFactory} class name * config-path: path to application properties * * @param args command line arguments */ 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); WikipediaApplication app = new WikipediaApplication(); runner.run(app); runner.waitForFinish(); } } 18 job.coordinator.factory=org.apache.samza.zk.ZkJobCoordinatorFactory job.coordinator.zk.connect=my-zk.server:2191
  • 19. • Embedded application launch sequence Samza-as-a-library myApp.main() Stream Application Local Application Runner Stream Processor runner.run() streamProcessor. start() n 19
  • 20. •Cluster-based application launch sequence Samza in a Cluster run-app.sh Remote Application Runner JobRunnerjobRunner.run() n main() app.class=my.app.MyStreamApplication Yarn RM run-jc.sh task.execute=run-local-app.sh run-local-app.sh Stream Application myApp.main() Local Application Runner Stream Processor runner.run() streamProcessor. start() n Job Coordinator 20
  • 22. Overview • High-level API • Flexible Deployment Model • Convergence between Batch and Stream Processing 22
  • 23. Stream Application in Batch Application logic: Count PageViewEvent for each member in a 5 minute window and send the counts to PageViewEventPerMemberStream Re-partition by memberId window map sendTo PageViewEvent PageViewEventPer MemberStream HDFS PageViewEvent: hdfs://mydbsnapshot/PageViewEvent/ PageViewEventPerMemberStream: hdfs://myoutputdb/PageViewEventPerMemberFiles 23
  • 24. Stream Application in Batch • No code change in application streams.pageViewEventStream.system=kafka streams.pageViewEventPerMemberStream.system=kafka streams.pageViewEventStream.system=hdfs streams.pageViewEventStream.physical.name=hdfs://mydbsnapshot/PageViewEvent/ streams.pageViewEventPerMemberStream.system=hdfs streams.pageViewEventPerMemberStream.physical.name=hdfs://myoutputdb/PageViewEventPerMemberFiles old config new config 24
  • 25. Samza 0.13 Architecture 25 High-level API Unified Stream & Batch Processing Remote Runner Run in Remote Cluster Cluster-based Yarn, (Mesos) Local Runner Run Locally Embedded ZooKeeper, Standalone APIRUNNERDEPLOY MENT PROCESSOR StreamProcessor Streams Kafka, Kinesis, HDFS ... Local State RocksDb, In-Memory Remote Data Multithreading 25
  • 26. Future Works • Samza runner for Apache Beam • Event-time processing • Support for Exactly-once processing • Support partition expansion for stateful application • Easy access to Adjunct datasets • SQL over Streams 26