SlideShare una empresa de Scribd logo
1 de 32
Beam me up, Samza!
How we built a Samza Runner for Apache Beam
Xinyu Liu
Apache Samza PMC
Software Engineer@LinkedIn
Agenda
● Apache Beam
● Samza Runner
● Use Cases
● Future Work
Apache Beam Overview
Apache Beam is an advanced unified programming model designed to provide
efficient and portable data processing pipelines.
● Unified - Single programming model for both batch and streaming
● Advanced - Strong consistency via event-time, i.e. windowing, triggering, late
arrival handling, accumulation, etc
● Portable - Execute pipelines of multiple programming language SDKs,
including Java, Python and Go
● Efficient - Write and share SDKs, IO connectors, and transformation libraries
https://beam.apache.org/
Beam Model
● A Pipeline encapsulates your entire data
processing task, from start to finish
● IO is the end points for data input and
output
● A PCollection represents an
immutable distributed data set that
your Beam pipeline operates on
● A PTransform represents a data
processing operation, or a step, in
your pipeline
IO.read
IO.write
PTransform
IO.read
IO.write
PCollection
Pipeline
Beam Model Example
public static void main(String[] args) {
// Create Pipeline
PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
Pipeline pipeline = Pipeline.create(options);
// Read from KafkaIO
PCollection<KafkaRecord<String, PageViewEvent>> input =
pipeline
.apply(KafkaIO.<String, PageViewEvent>read()
.withTopic("PageViewEvent")
.withTimestampFn(kv ->
new Instant(kv.getValue().timestamp))
.updateConsumerProperties(kafkaConfig)
.withKeyDeserializer(StringDeserializer.class)
.withValueDeserializer(AvroDeserializer.class));
// Apply PTransforms
PCollection<KV<String, Long>> counts = input
.apply(MapElements
.into(TypeDescriptor.of(PageViewEvent.class))
.via(r -> r.getKV().getValue()))
.apply(Window
.<PageViewEvent>into(FixedWindows.of(Duration.standardMinutes(1))))
.apply(WithKeys
.of((PageViewEvent pv) -> pv.pageKey)
.withKeyType(TypeDescriptors.strings()))
.apply(Count.perKey());
}
KafkaIO.Read
(KafkaUnboundedSource)
MapElements
PCollection<KafkaRecord>
Window
WithKeys
Count.perKey
PCollection<PageViewEvent>
PCollection<PageViewEvent>
PCollection<KV<String, PageViewEvent>>
PCollection<KV<String, Long>>
Beam Event Time
12:00 12:01 12:02 12:03
12:02
12:03
12:04
2
3
4
5
6
7
8
1
1-min fixed
window using
processing time
12:01
ProcessingTime
Event time
Beam Event Time
12:00 12:01 12:02 12:03
12:02
12:03
12:04
1
2
1-min fixed
window using
event time
- Watermark: a
timestamp that all
events before that have
arrived.
- Data that arrives with a
timestamp after the
watermark is
considered late data.
- Example using simple
watermark of event
timestamp 12:01
watermark
ProcessingTime
Event time
Beam Event Time
12:00 12:01 12:02 12:03
12:02
12:03
12:04
1
2
3
4
5
6
12:01
1-min fixed
window using
event time
- Watermark: a
timestamp that all
events before that have
arrived.
- Data that arrives with a
timestamp after the
watermark is
considered late data.
- Example using simple
watermark of event
timestamp
watermark
ProcessingTime
Event time
Beam Event Time
12:00 12:01 12:02 12:03
12:02
12:03
12:04
1
2
3
4
5
6
7
8
12:01
1-min fixed
window using
event time
- Watermark: a
timestamp that all
events before that have
arrived.
- Data that arrives with a
timestamp after the
watermark is
considered late data.
- Example using simple
watermark of event
timestamp
watermark
lateProcessingTime
Event time
Beam Windowing
Windowing divides data into event-time-based finite chunks.
Often required when doing aggregations over unbounded data.
Fixed Sliding
1 2 3
54
Sessions
2
431
Key
2
Key
1
Key
3
Time
2 3 4
Beam Stateful Processing
12:00 12:01 12:02 12:03
12:02
12:03
12:04
12:01
(news)
(msg)
(msg)
(jobs)
(msg)
(network)
(news) 12:00-12:01 12:01-12:02 12:02-12:03
news 1 0 1
msg 0 3 0
network 0 0 1
jobs 0 1 0
● Beam provides several state abstractions,
e.g. ValueState, BagState, MapState,
CombineState
● State is on a per-key-and-window basis
State for counting PageKey:
ProcessingTime
Event time
Beam IO
In-progress:
Agenda
● Apache Beam Overview
● Samza Runner
● Use Cases
● Future Work
The Goal of Samza Runner
Bring the easy-to-use, but powerful, model of
Beam to Samza users for state-of-art stream
and batch data processing, with portability
across a variety of programming languages.
Samza Overview
● The runner combines the large-scale stream processing capabilities of
Samza with the the advanced programming model of Beam
● First class support for local state (with RocksDB store)
● Fault-tolerance with support for incremental checkpointing of state
instead of full snapshots
● A fully asynchronous processing engine that makes remote calls
efficient
● Flexible deployment models, e.g. Yarn and standalone with
Zookeeper
Samza Runner: Capability Matrix
How Samza Runner Works?
● A Beam runner translates the Beam API into its native API + runtime
logic, and executed it in a distributed data processing system.
● Samza Runner translates Beam API into Samza high-level API and
execute the logic in a distributed manner, e.g. Yarn, Standalone.
● Samza runner contains the logic to support Beam features
- Beam IO - Event time/Watermark - GroupByKey
- Keyed State - Triggering Timers - Side
Input
BoundedSourceSystem
UnboundedSourceSystem
Unbounded/Bounded IO
● UnboundedSourceSystem adapts any
Unbounded IO.Read into a Samza
SystemConsumer. It will 1) split the sources
according to the parallelism needed; 2) generate
IncomingMessageEnvelopes of either event or
watermark
● BoundedSourceSystem adapts any Bounded
IO.Read into a Samza SystemConsumer
● Direct translation is also supported for Samza
native data connectors, e.g. translating
KafkaIO.Read directly into KafkaSystemConsumer
KafkaIO.Read
(KafkaUnboundedSource)
TextIO.Read
(TextSource)
Samza
StreamProcessor
Events/Watermarks
Events/End-of-stream
Watermark
● Watermark is injected at a fixed interval from unbounded sources
● Watermarks are propagated through each downstream operators and
aggregated using the following logic:
InputWatermak(op) = max (CurrentInputWatermark(op), min(OutputWatermark(op') | op' is upstream of op))
OutputWatermark(op) = max (CurrentOutputWatermark(op), InputWatermark(op))
Watermark Example
KafkaIO.Read KafkaIO.Read
FlatMap
Filter
Map
Merge
PartitionBy
0 1 2 3
FlatMap
Task Watermark
0 8
1 6
2 10
3 3 -> 7
7
7
7
7
9
9
7
7
6
6
6
6
IntermediateMessage => [MessageType
MessageData]
MessageType => byte
MessageData => byte[]
MessageData => [UserMessage/ControlMessage]
ControlMessage =>
Type => int
TaskName => String
1. WatermarkMessage =>
Timestamp => long
2. EndOfStreamMessage
Aggregate
GroupByKey
GroupByKey
● Automatically inserting partitionBy before
reduce
● The intermediate aggregation results are
stored in Samza key-value stores (RocksDb by
default)
● The output is triggered by watermarks by
default
KafkaIO.Read
partitionBy
FlatMap
Run
ReduceFn State
KV<key, value>
State Support
● Beam states are provided by
SamzaStoreStateInternals
● The key for each state cell is
(element key, window id, address)
● Samza also provides an readIterator()
interface for large states that won’t fit
in memory
ValueState
BagState
SetState
MapState
CombingState
WatermarkState
SamzaStoreStateInternals
RocksDb
Timer Support
● Beam timers are provided by
SamzaTimerInternalsFactory
● Support both event-time and
processing-time timers
● Event-time timers are managed using a
sorted set ordered by timestamp
● Processing-time timers are managed by
Samza TimerRegistry via
TimerFunction API
● All timers are keyed by TimerKey
(id, namespace, element key)
SamzaTimerInternals
Event-time Timers
GroupByKey
k1
timer1
k2
timer2 k3
timer3
k4
timer4
Processing-time Timers
Key Timer
k1 timer1
k2 timer2
k3 timer3
setTimer
watermark
register
Samza
SystemTimerScheduler
View/Side Input
● Beam views: SingletonView,
IterableView, ListView, MapView,
MultimapView
● Beam views are materialized into a
physical stream and broadcast to all
tasks using Samza broadcast
operator
● ParDo will consume the broadcasted
view as side input
KafkaIO.Read
0 1 2 3
ParDo
ParDo
ParDo
ParDo
TextIO.Read
ParDo
Combine.
GloballyAsSingletonView
0
broadcast
Broadcast
Stream
Side input
Main input
Deployment
Local (single JVM)
●Default mode: No config
required
●LocalApplicationRunner
●PassthroughJobCoordinator
●All tasks grouped into one
container
Yarn
●RemoteApplicationRunner
●YarnJobFactory
●Configure containers using
job.container.count
N
M
N
M
N
M
N
M
N
M
N
M
RM
Yarn
Cluster
JVM
Process
Standalone (zookeeper)
●LocalApplicationRunner
●ZkJobCoordinator
●Configure zk connection
job.coordinator.zk.connect
StreamProcessor
Samza
Contai
ner
Job
Coordi
nator
StreamProcessor
Samza
Contai
ner
Job
Coordi
nator
StreamProcessor
Samza
Contai
ner
Job
Coordi
nator
StreamProcessor
Samza
Contai
ner
Job
Coordi
nator
Zookeeper
Agenda
● Apache Beam Overview
● Samza Runner
● Use Cases
● Future Work
Use Case 1: Fixed-window Join to Track Location
Onboard
location
transmitt
er
Radar
Monitor
WithKey
(key by ID)
WithKey
(key by ID)
FixedWindow
(10 min)
FixedWindow
(10 min)
CoGroupByKey
(join)
Location
Info
DB
Suppose you own a Star Trek fleet, and you want to track the location of your Starships. The
location data are gathered through Starship on-board transmitters as well as your radar
monitors. Now let’s track their location in event time of a 10-min window.
(T1,
Enterprise,
SF, 1)
(R1,
Enterprise
SV, 9)
Enterprise,
(SF, 1)
Enterprise,
(SV, 9)
Enterprise,
(SF, 1)
Window(0 : 9)
Enterprise,
(SV, 9)
Window(0 : 9)
ParDo
KafkaIO.Write
DbIO.Write
Enterprise,
(SF, 1), (SV, 9)
Window(0 : 9)
Enterprise,
SV
Window(0 : 9)
Enterprise,
SV
Enterprise,
SV
Use Case 2: Session-window Join to Gather Activities
Suppose we are heading out to Disneyland. We would like to know the activity
count for each person. Here we use session window join to gather the activities
done per person.
Ticket Purchase
Event
Membership
Purchase Event
Activity Event
CoGroupByKey
SessionWindow
(4 hour)
SessionWindow
(4 hour)
SessionWindow
(4 hour)
Join by id
Xinyu: G00001 Boris: M00001
G00001: Space Mountain
M00001: Harry Potter
M00001: Small World
Count.perKey
Xinyu: 1
Boris: 2
Use Case 3: Sliding-window Aggr. for Feature Generation
Calculate the features of count, top N and sum for particular key for PageView
events using a 1-day sliding window with 1-min update interval.
PageView Event
SlidingWindow
(1day, every min)
Count.perKey Top.largestPerKey(n)
Filtter.by
Sum.globally
Schema pageViewSchema = RowSqlTypes
.builder()
.withVarcharField("pageKey")
.withTimestampField("timestamp")
.build();
PCollection<Row> pageViewsRows = pageViews
.apply(MapElements
.into(TypeDescriptor.of(Row.class))
.via((PageViewEvent pv) ->
Row.withSchema(pageViewSchema)
.addValues(pv.pageKey.toString(),
new DateTime(pv.time)).build()))
.setCoder(pageViewSchema.getRowCoder());
PCollection<KV<String, Long>> counts = pageViewsRows
.apply(BeamSql.query(
"SELECT COUNT(*) AS `count` FROM pageView "
+ "GROUP BY pageKey, "
+ "HOP(timestamp, INTERVAL '1' MINUTE, INTERVAL '1' DAY)"));
Alternatively, using SQL:
Agenda
● Apache Beam Overview
● Samza Runner
● Use Cases
● Future Work
Future Work
● Python! ● Async Support ● Table API
# A sample word count
p =Pipeline(options=pipeline_options)
# Read the text file[pattern] into a PCollection.
lines = p | 'read' >> ReadFromText(known_args.input)
# Count the occurrences of each word.
counts = (lines
| 'split' >> (ParDo(WordExtractingDoFn())
.with_output_types(unicode))
| 'pair_with_one' >>Map(lambda x: (x, 1))
| 'group' >> GroupByKey()
| 'count' >> Map(lambda (word, ones): (word,
sum(ones))))
# Format the counts into a PCollection of strings.
output = counts
| 'format' >>Map(lambda (word, c): '%s: %s' %
(word, c))
# Write the output using a "Write" transform that has side
effects.
# pylint: disable=expression-not-assigned
output | 'write' >> WriteToText(known_args.output)
result = p.run()
result.wait_until_finish()
// Use CompletionStage for asynchronous processing
input.apply(ParDo.of(
new DoFn<InputT, OutputT>() {
@ProcessElement
public void process
(@Element CompletionStage<InputT> element, ...) {
element.thenApply(...)
}
}
));
// PTable is the Table abstraction
PTable<KV<String, User>> userTable =
pipeline.apply(
EspressoTable.readWrite()
.withDb("dbname")
.withTable("user"));
pageView
.apply(TableParDo.of(
new DoFn<KV<String, PageViewEvent>, String>() {
@ProcessElement
public void processElement(ProcessContext c,
@TableContext.Inject TableContext tc) {
String id = c.element().getKey();
//table lookup
Table<String, User> users = tc.getTable(userTable);
User user = settings.get(id);
c.output(id + “:” + user.getName().toString());
}
})
.withTables(userTable));
// Convenient helper class to do the same thing
PCollection<String> result = PCollectionTableJoin
.of(pageView, userTable)
.into(TypeDescriptors.strings())
.via((pv, user) ->
pv.getKey() + “:” + user.getName().toString());
Thank you!
And
Special Thanks to Our Early Adopters:
Yingkai Hu, Froila Dsouza, Zhongen Tao,
Nithin Reddy, Bruce Su
https://beam.apache.org/documentation/runners/samza/

Más contenido relacionado

La actualidad más candente

Stream processing in python with Apache Samza and Beam
Stream processing in python with Apache Samza and BeamStream processing in python with Apache Samza and Beam
Stream processing in python with Apache Samza and BeamHai Lu
 
Apache samza past, present and future
Apache samza  past, present and futureApache samza  past, present and future
Apache samza past, present and futureEd Yakabosky
 
Unified Stream Processing at Scale with Apache Samza - BDS2017
Unified Stream Processing at Scale with Apache Samza - BDS2017Unified Stream Processing at Scale with Apache Samza - BDS2017
Unified Stream Processing at Scale with Apache Samza - BDS2017Jacob Maes
 
Function Mesh for Apache Pulsar, the Way for Simple Streaming Solutions
Function Mesh for Apache Pulsar, the Way for Simple Streaming SolutionsFunction Mesh for Apache Pulsar, the Way for Simple Streaming Solutions
Function Mesh for Apache Pulsar, the Way for Simple Streaming SolutionsStreamNative
 
Strata Singapore: Gearpump Real time DAG-Processing with Akka at Scale
Strata Singapore: GearpumpReal time DAG-Processing with Akka at ScaleStrata Singapore: GearpumpReal time DAG-Processing with Akka at Scale
Strata Singapore: Gearpump Real time DAG-Processing with Akka at ScaleSean Zhong
 
Why @Loggly Loves Apache Kafka, and How We Use Its Unbreakable Messaging for ...
Why @Loggly Loves Apache Kafka, and How We Use Its Unbreakable Messaging for ...Why @Loggly Loves Apache Kafka, and How We Use Its Unbreakable Messaging for ...
Why @Loggly Loves Apache Kafka, and How We Use Its Unbreakable Messaging for ...SolarWinds Loggly
 
Building Stream Processing as a Service
Building Stream Processing as a ServiceBuilding Stream Processing as a Service
Building Stream Processing as a ServiceSteven Wu
 
Samza portable runner for beam
Samza portable runner for beamSamza portable runner for beam
Samza portable runner for beamHai Lu
 
M|18 Choosing the Right High Availability Strategy for You
M|18 Choosing the Right High Availability Strategy for YouM|18 Choosing the Right High Availability Strategy for You
M|18 Choosing the Right High Availability Strategy for YouMariaDB plc
 
Administrative techniques to reduce Kafka costs | Anna Kepler, Viasat
Administrative techniques to reduce Kafka costs | Anna Kepler, ViasatAdministrative techniques to reduce Kafka costs | Anna Kepler, Viasat
Administrative techniques to reduce Kafka costs | Anna Kepler, ViasatHostedbyConfluent
 
Reactive mistakes - ScalaDays Chicago 2017
Reactive mistakes -  ScalaDays Chicago 2017Reactive mistakes -  ScalaDays Chicago 2017
Reactive mistakes - ScalaDays Chicago 2017Petr Zapletal
 
Understanding time in structured streaming
Understanding time in structured streamingUnderstanding time in structured streaming
Understanding time in structured streamingdatamantra
 
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
 
Consumer offset management in Kafka
Consumer offset management in KafkaConsumer offset management in Kafka
Consumer offset management in KafkaJoel Koshy
 
Flink Forward Berlin 2017: Robert Metzger - Keep it going - How to reliably a...
Flink Forward Berlin 2017: Robert Metzger - Keep it going - How to reliably a...Flink Forward Berlin 2017: Robert Metzger - Keep it going - How to reliably a...
Flink Forward Berlin 2017: Robert Metzger - Keep it going - How to reliably a...Flink Forward
 
Connecting kafka message systems with scylla
Connecting kafka message systems with scylla   Connecting kafka message systems with scylla
Connecting kafka message systems with scylla Maheedhar Gunturu
 
Pulsar connector on flink 1.14
Pulsar connector on flink 1.14Pulsar connector on flink 1.14
Pulsar connector on flink 1.14宇帆 盛
 
Using Apache Pulsar as a Modern, Scalable, High Performing JMS Platform - Pus...
Using Apache Pulsar as a Modern, Scalable, High Performing JMS Platform - Pus...Using Apache Pulsar as a Modern, Scalable, High Performing JMS Platform - Pus...
Using Apache Pulsar as a Modern, Scalable, High Performing JMS Platform - Pus...StreamNative
 
Streaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in ProductionStreaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in Productionconfluent
 

La actualidad más candente (20)

Stream processing in python with Apache Samza and Beam
Stream processing in python with Apache Samza and BeamStream processing in python with Apache Samza and Beam
Stream processing in python with Apache Samza and Beam
 
Apache samza past, present and future
Apache samza  past, present and futureApache samza  past, present and future
Apache samza past, present and future
 
Unified Stream Processing at Scale with Apache Samza - BDS2017
Unified Stream Processing at Scale with Apache Samza - BDS2017Unified Stream Processing at Scale with Apache Samza - BDS2017
Unified Stream Processing at Scale with Apache Samza - BDS2017
 
Function Mesh for Apache Pulsar, the Way for Simple Streaming Solutions
Function Mesh for Apache Pulsar, the Way for Simple Streaming SolutionsFunction Mesh for Apache Pulsar, the Way for Simple Streaming Solutions
Function Mesh for Apache Pulsar, the Way for Simple Streaming Solutions
 
Strata Singapore: Gearpump Real time DAG-Processing with Akka at Scale
Strata Singapore: GearpumpReal time DAG-Processing with Akka at ScaleStrata Singapore: GearpumpReal time DAG-Processing with Akka at Scale
Strata Singapore: Gearpump Real time DAG-Processing with Akka at Scale
 
Apache samza
Apache samzaApache samza
Apache samza
 
Why @Loggly Loves Apache Kafka, and How We Use Its Unbreakable Messaging for ...
Why @Loggly Loves Apache Kafka, and How We Use Its Unbreakable Messaging for ...Why @Loggly Loves Apache Kafka, and How We Use Its Unbreakable Messaging for ...
Why @Loggly Loves Apache Kafka, and How We Use Its Unbreakable Messaging for ...
 
Building Stream Processing as a Service
Building Stream Processing as a ServiceBuilding Stream Processing as a Service
Building Stream Processing as a Service
 
Samza portable runner for beam
Samza portable runner for beamSamza portable runner for beam
Samza portable runner for beam
 
M|18 Choosing the Right High Availability Strategy for You
M|18 Choosing the Right High Availability Strategy for YouM|18 Choosing the Right High Availability Strategy for You
M|18 Choosing the Right High Availability Strategy for You
 
Administrative techniques to reduce Kafka costs | Anna Kepler, Viasat
Administrative techniques to reduce Kafka costs | Anna Kepler, ViasatAdministrative techniques to reduce Kafka costs | Anna Kepler, Viasat
Administrative techniques to reduce Kafka costs | Anna Kepler, Viasat
 
Reactive mistakes - ScalaDays Chicago 2017
Reactive mistakes -  ScalaDays Chicago 2017Reactive mistakes -  ScalaDays Chicago 2017
Reactive mistakes - ScalaDays Chicago 2017
 
Understanding time in structured streaming
Understanding time in structured streamingUnderstanding time in structured streaming
Understanding time in structured streaming
 
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"
 
Consumer offset management in Kafka
Consumer offset management in KafkaConsumer offset management in Kafka
Consumer offset management in Kafka
 
Flink Forward Berlin 2017: Robert Metzger - Keep it going - How to reliably a...
Flink Forward Berlin 2017: Robert Metzger - Keep it going - How to reliably a...Flink Forward Berlin 2017: Robert Metzger - Keep it going - How to reliably a...
Flink Forward Berlin 2017: Robert Metzger - Keep it going - How to reliably a...
 
Connecting kafka message systems with scylla
Connecting kafka message systems with scylla   Connecting kafka message systems with scylla
Connecting kafka message systems with scylla
 
Pulsar connector on flink 1.14
Pulsar connector on flink 1.14Pulsar connector on flink 1.14
Pulsar connector on flink 1.14
 
Using Apache Pulsar as a Modern, Scalable, High Performing JMS Platform - Pus...
Using Apache Pulsar as a Modern, Scalable, High Performing JMS Platform - Pus...Using Apache Pulsar as a Modern, Scalable, High Performing JMS Platform - Pus...
Using Apache Pulsar as a Modern, Scalable, High Performing JMS Platform - Pus...
 
Streaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in ProductionStreaming in Practice - Putting Apache Kafka in Production
Streaming in Practice - Putting Apache Kafka in Production
 

Similar a Beam me up, Samza!

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
 
Apache Samza Past, Present and Future
Apache Samza  Past, Present and FutureApache Samza  Past, Present and Future
Apache Samza Past, Present and FutureKartik Paramasivam
 
Data Streaming in Kafka
Data Streaming in KafkaData Streaming in Kafka
Data Streaming in KafkaSilviuMarcu1
 
Samza tech talk_2015 - huawei
Samza tech talk_2015 - huaweiSamza tech talk_2015 - huawei
Samza tech talk_2015 - huaweiYi Pan
 
Spark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedSpark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedMichael Spector
 
SamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationSamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationYi Pan
 
Porting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to RustPorting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to RustEvan Chan
 
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry confluent
 
Flink forward-2017-netflix keystones-paas
Flink forward-2017-netflix keystones-paasFlink forward-2017-netflix keystones-paas
Flink forward-2017-netflix keystones-paasMonal Daxini
 
TenMax Data Pipeline Experience Sharing
TenMax Data Pipeline Experience SharingTenMax Data Pipeline Experience Sharing
TenMax Data Pipeline Experience SharingChen-en Lu
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Guido Schmutz
 
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...Amazon Web Services
 
Flink at netflix paypal speaker series
Flink at netflix   paypal speaker seriesFlink at netflix   paypal speaker series
Flink at netflix paypal speaker seriesMonal Daxini
 
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...DataWorks Summit/Hadoop Summit
 
Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2aspyker
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Guido Schmutz
 
What no one tells you about writing a streaming app
What no one tells you about writing a streaming appWhat no one tells you about writing a streaming app
What no one tells you about writing a streaming apphadooparchbook
 
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...Spark Summit
 
Python Streaming Pipelines with Beam on Flink
Python Streaming Pipelines with Beam on FlinkPython Streaming Pipelines with Beam on Flink
Python Streaming Pipelines with Beam on FlinkAljoscha Krettek
 

Similar a Beam me up, Samza! (20)

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...
 
Apache Samza Past, Present and Future
Apache Samza  Past, Present and FutureApache Samza  Past, Present and Future
Apache Samza Past, Present and Future
 
Data Streaming in Kafka
Data Streaming in KafkaData Streaming in Kafka
Data Streaming in Kafka
 
Samza tech talk_2015 - huawei
Samza tech talk_2015 - huaweiSamza tech talk_2015 - huawei
Samza tech talk_2015 - huawei
 
Spark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics RevisedSpark Streaming Recipes and "Exactly Once" Semantics Revised
Spark Streaming Recipes and "Exactly Once" Semantics Revised
 
SamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationSamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentation
 
Porting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to RustPorting a Streaming Pipeline from Scala to Rust
Porting a Streaming Pipeline from Scala to Rust
 
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry
 
Flink forward-2017-netflix keystones-paas
Flink forward-2017-netflix keystones-paasFlink forward-2017-netflix keystones-paas
Flink forward-2017-netflix keystones-paas
 
TenMax Data Pipeline Experience Sharing
TenMax Data Pipeline Experience SharingTenMax Data Pipeline Experience Sharing
TenMax Data Pipeline Experience Sharing
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
About time
About timeAbout time
About time
 
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
Infrastructure at Scale: Apache Kafka, Twitter Storm & Elastic Search (ARC303...
 
Flink at netflix paypal speaker series
Flink at netflix   paypal speaker seriesFlink at netflix   paypal speaker series
Flink at netflix paypal speaker series
 
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
End to End Processing of 3.7 Million Telemetry Events per Second using Lambda...
 
Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2Netflix Open Source Meetup Season 4 Episode 2
Netflix Open Source Meetup Season 4 Episode 2
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
What no one tells you about writing a streaming app
What no one tells you about writing a streaming appWhat no one tells you about writing a streaming app
What no one tells you about writing a streaming app
 
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
What No One Tells You About Writing a Streaming App: Spark Summit East talk b...
 
Python Streaming Pipelines with Beam on Flink
Python Streaming Pipelines with Beam on FlinkPython Streaming Pipelines with Beam on Flink
Python Streaming Pipelines with Beam on Flink
 

Último

University management System project report..pdf
University management System project report..pdfUniversity management System project report..pdf
University management System project report..pdfKamal Acharya
 
Coefficient of Thermal Expansion and their Importance.pptx
Coefficient of Thermal Expansion and their Importance.pptxCoefficient of Thermal Expansion and their Importance.pptx
Coefficient of Thermal Expansion and their Importance.pptxAsutosh Ranjan
 
Introduction and different types of Ethernet.pptx
Introduction and different types of Ethernet.pptxIntroduction and different types of Ethernet.pptx
Introduction and different types of Ethernet.pptxupamatechverse
 
Processing & Properties of Floor and Wall Tiles.pptx
Processing & Properties of Floor and Wall Tiles.pptxProcessing & Properties of Floor and Wall Tiles.pptx
Processing & Properties of Floor and Wall Tiles.pptxpranjaldaimarysona
 
(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
 
Structural Analysis and Design of Foundations: A Comprehensive Handbook for S...
Structural Analysis and Design of Foundations: A Comprehensive Handbook for S...Structural Analysis and Design of Foundations: A Comprehensive Handbook for S...
Structural Analysis and Design of Foundations: A Comprehensive Handbook for S...Dr.Costas Sachpazis
 
CCS335 _ Neural Networks and Deep Learning Laboratory_Lab Complete Record
CCS335 _ Neural Networks and Deep Learning Laboratory_Lab Complete RecordCCS335 _ Neural Networks and Deep Learning Laboratory_Lab Complete Record
CCS335 _ Neural Networks and Deep Learning Laboratory_Lab Complete RecordAsst.prof M.Gokilavani
 
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
 
Call for Papers - Educational Administration: Theory and Practice, E-ISSN: 21...
Call for Papers - Educational Administration: Theory and Practice, E-ISSN: 21...Call for Papers - Educational Administration: Theory and Practice, E-ISSN: 21...
Call for Papers - Educational Administration: Theory and Practice, E-ISSN: 21...Christo Ananth
 
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
 
Online banking management system project.pdf
Online banking management system project.pdfOnline banking management system project.pdf
Online banking management system project.pdfKamal Acharya
 
UNIT - IV - Air Compressors and its Performance
UNIT - IV - Air Compressors and its PerformanceUNIT - IV - Air Compressors and its Performance
UNIT - IV - Air Compressors and its Performancesivaprakash250
 
UNIT-V FMM.HYDRAULIC TURBINE - Construction and working
UNIT-V FMM.HYDRAULIC TURBINE - Construction and workingUNIT-V FMM.HYDRAULIC TURBINE - Construction and working
UNIT-V FMM.HYDRAULIC TURBINE - Construction and workingrknatarajan
 
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTING
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTINGMANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTING
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTINGSIVASHANKAR N
 
Extrusion Processes and Their Limitations
Extrusion Processes and Their LimitationsExtrusion Processes and Their Limitations
Extrusion Processes and Their Limitations120cr0395
 
Introduction to Multiple Access Protocol.pptx
Introduction to Multiple Access Protocol.pptxIntroduction to Multiple Access Protocol.pptx
Introduction to Multiple Access Protocol.pptxupamatechverse
 
UNIT-III FMM. DIMENSIONAL ANALYSIS
UNIT-III FMM.        DIMENSIONAL ANALYSISUNIT-III FMM.        DIMENSIONAL ANALYSIS
UNIT-III FMM. DIMENSIONAL ANALYSISrknatarajan
 
Russian Call Girls in Nagpur Grishma Call 7001035870 Meet With Nagpur Escorts
Russian Call Girls in Nagpur Grishma Call 7001035870 Meet With Nagpur EscortsRussian Call Girls in Nagpur Grishma Call 7001035870 Meet With Nagpur Escorts
Russian Call Girls in Nagpur Grishma Call 7001035870 Meet With Nagpur EscortsCall Girls in Nagpur High Profile
 
KubeKraft presentation @CloudNativeHooghly
KubeKraft presentation @CloudNativeHooghlyKubeKraft presentation @CloudNativeHooghly
KubeKraft presentation @CloudNativeHooghlysanyuktamishra911
 

Último (20)

University management System project report..pdf
University management System project report..pdfUniversity management System project report..pdf
University management System project report..pdf
 
Water Industry Process Automation & Control Monthly - April 2024
Water Industry Process Automation & Control Monthly - April 2024Water Industry Process Automation & Control Monthly - April 2024
Water Industry Process Automation & Control Monthly - April 2024
 
Coefficient of Thermal Expansion and their Importance.pptx
Coefficient of Thermal Expansion and their Importance.pptxCoefficient of Thermal Expansion and their Importance.pptx
Coefficient of Thermal Expansion and their Importance.pptx
 
Introduction and different types of Ethernet.pptx
Introduction and different types of Ethernet.pptxIntroduction and different types of Ethernet.pptx
Introduction and different types of Ethernet.pptx
 
Processing & Properties of Floor and Wall Tiles.pptx
Processing & Properties of Floor and Wall Tiles.pptxProcessing & Properties of Floor and Wall Tiles.pptx
Processing & Properties of Floor and Wall Tiles.pptx
 
(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...
 
Structural Analysis and Design of Foundations: A Comprehensive Handbook for S...
Structural Analysis and Design of Foundations: A Comprehensive Handbook for S...Structural Analysis and Design of Foundations: A Comprehensive Handbook for S...
Structural Analysis and Design of Foundations: A Comprehensive Handbook for S...
 
CCS335 _ Neural Networks and Deep Learning Laboratory_Lab Complete Record
CCS335 _ Neural Networks and Deep Learning Laboratory_Lab Complete RecordCCS335 _ Neural Networks and Deep Learning Laboratory_Lab Complete Record
CCS335 _ Neural Networks and Deep Learning Laboratory_Lab Complete Record
 
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 for Papers - Educational Administration: Theory and Practice, E-ISSN: 21...
Call for Papers - Educational Administration: Theory and Practice, E-ISSN: 21...Call for Papers - Educational Administration: Theory and Practice, E-ISSN: 21...
Call for Papers - Educational Administration: Theory and Practice, E-ISSN: 21...
 
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
 
Online banking management system project.pdf
Online banking management system project.pdfOnline banking management system project.pdf
Online banking management system project.pdf
 
UNIT - IV - Air Compressors and its Performance
UNIT - IV - Air Compressors and its PerformanceUNIT - IV - Air Compressors and its Performance
UNIT - IV - Air Compressors and its Performance
 
UNIT-V FMM.HYDRAULIC TURBINE - Construction and working
UNIT-V FMM.HYDRAULIC TURBINE - Construction and workingUNIT-V FMM.HYDRAULIC TURBINE - Construction and working
UNIT-V FMM.HYDRAULIC TURBINE - Construction and working
 
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTING
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTINGMANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTING
MANUFACTURING PROCESS-II UNIT-1 THEORY OF METAL CUTTING
 
Extrusion Processes and Their Limitations
Extrusion Processes and Their LimitationsExtrusion Processes and Their Limitations
Extrusion Processes and Their Limitations
 
Introduction to Multiple Access Protocol.pptx
Introduction to Multiple Access Protocol.pptxIntroduction to Multiple Access Protocol.pptx
Introduction to Multiple Access Protocol.pptx
 
UNIT-III FMM. DIMENSIONAL ANALYSIS
UNIT-III FMM.        DIMENSIONAL ANALYSISUNIT-III FMM.        DIMENSIONAL ANALYSIS
UNIT-III FMM. DIMENSIONAL ANALYSIS
 
Russian Call Girls in Nagpur Grishma Call 7001035870 Meet With Nagpur Escorts
Russian Call Girls in Nagpur Grishma Call 7001035870 Meet With Nagpur EscortsRussian Call Girls in Nagpur Grishma Call 7001035870 Meet With Nagpur Escorts
Russian Call Girls in Nagpur Grishma Call 7001035870 Meet With Nagpur Escorts
 
KubeKraft presentation @CloudNativeHooghly
KubeKraft presentation @CloudNativeHooghlyKubeKraft presentation @CloudNativeHooghly
KubeKraft presentation @CloudNativeHooghly
 

Beam me up, Samza!

  • 1. Beam me up, Samza! How we built a Samza Runner for Apache Beam Xinyu Liu Apache Samza PMC Software Engineer@LinkedIn
  • 2. Agenda ● Apache Beam ● Samza Runner ● Use Cases ● Future Work
  • 3. Apache Beam Overview Apache Beam is an advanced unified programming model designed to provide efficient and portable data processing pipelines. ● Unified - Single programming model for both batch and streaming ● Advanced - Strong consistency via event-time, i.e. windowing, triggering, late arrival handling, accumulation, etc ● Portable - Execute pipelines of multiple programming language SDKs, including Java, Python and Go ● Efficient - Write and share SDKs, IO connectors, and transformation libraries https://beam.apache.org/
  • 4. Beam Model ● A Pipeline encapsulates your entire data processing task, from start to finish ● IO is the end points for data input and output ● A PCollection represents an immutable distributed data set that your Beam pipeline operates on ● A PTransform represents a data processing operation, or a step, in your pipeline IO.read IO.write PTransform IO.read IO.write PCollection Pipeline
  • 5. Beam Model Example public static void main(String[] args) { // Create Pipeline PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create(); Pipeline pipeline = Pipeline.create(options); // Read from KafkaIO PCollection<KafkaRecord<String, PageViewEvent>> input = pipeline .apply(KafkaIO.<String, PageViewEvent>read() .withTopic("PageViewEvent") .withTimestampFn(kv -> new Instant(kv.getValue().timestamp)) .updateConsumerProperties(kafkaConfig) .withKeyDeserializer(StringDeserializer.class) .withValueDeserializer(AvroDeserializer.class)); // Apply PTransforms PCollection<KV<String, Long>> counts = input .apply(MapElements .into(TypeDescriptor.of(PageViewEvent.class)) .via(r -> r.getKV().getValue())) .apply(Window .<PageViewEvent>into(FixedWindows.of(Duration.standardMinutes(1)))) .apply(WithKeys .of((PageViewEvent pv) -> pv.pageKey) .withKeyType(TypeDescriptors.strings())) .apply(Count.perKey()); } KafkaIO.Read (KafkaUnboundedSource) MapElements PCollection<KafkaRecord> Window WithKeys Count.perKey PCollection<PageViewEvent> PCollection<PageViewEvent> PCollection<KV<String, PageViewEvent>> PCollection<KV<String, Long>>
  • 6. Beam Event Time 12:00 12:01 12:02 12:03 12:02 12:03 12:04 2 3 4 5 6 7 8 1 1-min fixed window using processing time 12:01 ProcessingTime Event time
  • 7. Beam Event Time 12:00 12:01 12:02 12:03 12:02 12:03 12:04 1 2 1-min fixed window using event time - Watermark: a timestamp that all events before that have arrived. - Data that arrives with a timestamp after the watermark is considered late data. - Example using simple watermark of event timestamp 12:01 watermark ProcessingTime Event time
  • 8. Beam Event Time 12:00 12:01 12:02 12:03 12:02 12:03 12:04 1 2 3 4 5 6 12:01 1-min fixed window using event time - Watermark: a timestamp that all events before that have arrived. - Data that arrives with a timestamp after the watermark is considered late data. - Example using simple watermark of event timestamp watermark ProcessingTime Event time
  • 9. Beam Event Time 12:00 12:01 12:02 12:03 12:02 12:03 12:04 1 2 3 4 5 6 7 8 12:01 1-min fixed window using event time - Watermark: a timestamp that all events before that have arrived. - Data that arrives with a timestamp after the watermark is considered late data. - Example using simple watermark of event timestamp watermark lateProcessingTime Event time
  • 10. Beam Windowing Windowing divides data into event-time-based finite chunks. Often required when doing aggregations over unbounded data. Fixed Sliding 1 2 3 54 Sessions 2 431 Key 2 Key 1 Key 3 Time 2 3 4
  • 11. Beam Stateful Processing 12:00 12:01 12:02 12:03 12:02 12:03 12:04 12:01 (news) (msg) (msg) (jobs) (msg) (network) (news) 12:00-12:01 12:01-12:02 12:02-12:03 news 1 0 1 msg 0 3 0 network 0 0 1 jobs 0 1 0 ● Beam provides several state abstractions, e.g. ValueState, BagState, MapState, CombineState ● State is on a per-key-and-window basis State for counting PageKey: ProcessingTime Event time
  • 13. Agenda ● Apache Beam Overview ● Samza Runner ● Use Cases ● Future Work
  • 14. The Goal of Samza Runner Bring the easy-to-use, but powerful, model of Beam to Samza users for state-of-art stream and batch data processing, with portability across a variety of programming languages.
  • 15. Samza Overview ● The runner combines the large-scale stream processing capabilities of Samza with the the advanced programming model of Beam ● First class support for local state (with RocksDB store) ● Fault-tolerance with support for incremental checkpointing of state instead of full snapshots ● A fully asynchronous processing engine that makes remote calls efficient ● Flexible deployment models, e.g. Yarn and standalone with Zookeeper
  • 17. How Samza Runner Works? ● A Beam runner translates the Beam API into its native API + runtime logic, and executed it in a distributed data processing system. ● Samza Runner translates Beam API into Samza high-level API and execute the logic in a distributed manner, e.g. Yarn, Standalone. ● Samza runner contains the logic to support Beam features - Beam IO - Event time/Watermark - GroupByKey - Keyed State - Triggering Timers - Side Input
  • 18. BoundedSourceSystem UnboundedSourceSystem Unbounded/Bounded IO ● UnboundedSourceSystem adapts any Unbounded IO.Read into a Samza SystemConsumer. It will 1) split the sources according to the parallelism needed; 2) generate IncomingMessageEnvelopes of either event or watermark ● BoundedSourceSystem adapts any Bounded IO.Read into a Samza SystemConsumer ● Direct translation is also supported for Samza native data connectors, e.g. translating KafkaIO.Read directly into KafkaSystemConsumer KafkaIO.Read (KafkaUnboundedSource) TextIO.Read (TextSource) Samza StreamProcessor Events/Watermarks Events/End-of-stream
  • 19. Watermark ● Watermark is injected at a fixed interval from unbounded sources ● Watermarks are propagated through each downstream operators and aggregated using the following logic: InputWatermak(op) = max (CurrentInputWatermark(op), min(OutputWatermark(op') | op' is upstream of op)) OutputWatermark(op) = max (CurrentOutputWatermark(op), InputWatermark(op))
  • 20. Watermark Example KafkaIO.Read KafkaIO.Read FlatMap Filter Map Merge PartitionBy 0 1 2 3 FlatMap Task Watermark 0 8 1 6 2 10 3 3 -> 7 7 7 7 7 9 9 7 7 6 6 6 6 IntermediateMessage => [MessageType MessageData] MessageType => byte MessageData => byte[] MessageData => [UserMessage/ControlMessage] ControlMessage => Type => int TaskName => String 1. WatermarkMessage => Timestamp => long 2. EndOfStreamMessage Aggregate
  • 21. GroupByKey GroupByKey ● Automatically inserting partitionBy before reduce ● The intermediate aggregation results are stored in Samza key-value stores (RocksDb by default) ● The output is triggered by watermarks by default KafkaIO.Read partitionBy FlatMap Run ReduceFn State KV<key, value>
  • 22. State Support ● Beam states are provided by SamzaStoreStateInternals ● The key for each state cell is (element key, window id, address) ● Samza also provides an readIterator() interface for large states that won’t fit in memory ValueState BagState SetState MapState CombingState WatermarkState SamzaStoreStateInternals RocksDb
  • 23. Timer Support ● Beam timers are provided by SamzaTimerInternalsFactory ● Support both event-time and processing-time timers ● Event-time timers are managed using a sorted set ordered by timestamp ● Processing-time timers are managed by Samza TimerRegistry via TimerFunction API ● All timers are keyed by TimerKey (id, namespace, element key) SamzaTimerInternals Event-time Timers GroupByKey k1 timer1 k2 timer2 k3 timer3 k4 timer4 Processing-time Timers Key Timer k1 timer1 k2 timer2 k3 timer3 setTimer watermark register Samza SystemTimerScheduler
  • 24. View/Side Input ● Beam views: SingletonView, IterableView, ListView, MapView, MultimapView ● Beam views are materialized into a physical stream and broadcast to all tasks using Samza broadcast operator ● ParDo will consume the broadcasted view as side input KafkaIO.Read 0 1 2 3 ParDo ParDo ParDo ParDo TextIO.Read ParDo Combine. GloballyAsSingletonView 0 broadcast Broadcast Stream Side input Main input
  • 25. Deployment Local (single JVM) ●Default mode: No config required ●LocalApplicationRunner ●PassthroughJobCoordinator ●All tasks grouped into one container Yarn ●RemoteApplicationRunner ●YarnJobFactory ●Configure containers using job.container.count N M N M N M N M N M N M RM Yarn Cluster JVM Process Standalone (zookeeper) ●LocalApplicationRunner ●ZkJobCoordinator ●Configure zk connection job.coordinator.zk.connect StreamProcessor Samza Contai ner Job Coordi nator StreamProcessor Samza Contai ner Job Coordi nator StreamProcessor Samza Contai ner Job Coordi nator StreamProcessor Samza Contai ner Job Coordi nator Zookeeper
  • 26. Agenda ● Apache Beam Overview ● Samza Runner ● Use Cases ● Future Work
  • 27. Use Case 1: Fixed-window Join to Track Location Onboard location transmitt er Radar Monitor WithKey (key by ID) WithKey (key by ID) FixedWindow (10 min) FixedWindow (10 min) CoGroupByKey (join) Location Info DB Suppose you own a Star Trek fleet, and you want to track the location of your Starships. The location data are gathered through Starship on-board transmitters as well as your radar monitors. Now let’s track their location in event time of a 10-min window. (T1, Enterprise, SF, 1) (R1, Enterprise SV, 9) Enterprise, (SF, 1) Enterprise, (SV, 9) Enterprise, (SF, 1) Window(0 : 9) Enterprise, (SV, 9) Window(0 : 9) ParDo KafkaIO.Write DbIO.Write Enterprise, (SF, 1), (SV, 9) Window(0 : 9) Enterprise, SV Window(0 : 9) Enterprise, SV Enterprise, SV
  • 28. Use Case 2: Session-window Join to Gather Activities Suppose we are heading out to Disneyland. We would like to know the activity count for each person. Here we use session window join to gather the activities done per person. Ticket Purchase Event Membership Purchase Event Activity Event CoGroupByKey SessionWindow (4 hour) SessionWindow (4 hour) SessionWindow (4 hour) Join by id Xinyu: G00001 Boris: M00001 G00001: Space Mountain M00001: Harry Potter M00001: Small World Count.perKey Xinyu: 1 Boris: 2
  • 29. Use Case 3: Sliding-window Aggr. for Feature Generation Calculate the features of count, top N and sum for particular key for PageView events using a 1-day sliding window with 1-min update interval. PageView Event SlidingWindow (1day, every min) Count.perKey Top.largestPerKey(n) Filtter.by Sum.globally Schema pageViewSchema = RowSqlTypes .builder() .withVarcharField("pageKey") .withTimestampField("timestamp") .build(); PCollection<Row> pageViewsRows = pageViews .apply(MapElements .into(TypeDescriptor.of(Row.class)) .via((PageViewEvent pv) -> Row.withSchema(pageViewSchema) .addValues(pv.pageKey.toString(), new DateTime(pv.time)).build())) .setCoder(pageViewSchema.getRowCoder()); PCollection<KV<String, Long>> counts = pageViewsRows .apply(BeamSql.query( "SELECT COUNT(*) AS `count` FROM pageView " + "GROUP BY pageKey, " + "HOP(timestamp, INTERVAL '1' MINUTE, INTERVAL '1' DAY)")); Alternatively, using SQL:
  • 30. Agenda ● Apache Beam Overview ● Samza Runner ● Use Cases ● Future Work
  • 31. Future Work ● Python! ● Async Support ● Table API # A sample word count p =Pipeline(options=pipeline_options) # Read the text file[pattern] into a PCollection. lines = p | 'read' >> ReadFromText(known_args.input) # Count the occurrences of each word. counts = (lines | 'split' >> (ParDo(WordExtractingDoFn()) .with_output_types(unicode)) | 'pair_with_one' >>Map(lambda x: (x, 1)) | 'group' >> GroupByKey() | 'count' >> Map(lambda (word, ones): (word, sum(ones)))) # Format the counts into a PCollection of strings. output = counts | 'format' >>Map(lambda (word, c): '%s: %s' % (word, c)) # Write the output using a "Write" transform that has side effects. # pylint: disable=expression-not-assigned output | 'write' >> WriteToText(known_args.output) result = p.run() result.wait_until_finish() // Use CompletionStage for asynchronous processing input.apply(ParDo.of( new DoFn<InputT, OutputT>() { @ProcessElement public void process (@Element CompletionStage<InputT> element, ...) { element.thenApply(...) } } )); // PTable is the Table abstraction PTable<KV<String, User>> userTable = pipeline.apply( EspressoTable.readWrite() .withDb("dbname") .withTable("user")); pageView .apply(TableParDo.of( new DoFn<KV<String, PageViewEvent>, String>() { @ProcessElement public void processElement(ProcessContext c, @TableContext.Inject TableContext tc) { String id = c.element().getKey(); //table lookup Table<String, User> users = tc.getTable(userTable); User user = settings.get(id); c.output(id + “:” + user.getName().toString()); } }) .withTables(userTable)); // Convenient helper class to do the same thing PCollection<String> result = PCollectionTableJoin .of(pageView, userTable) .into(TypeDescriptors.strings()) .via((pv, user) -> pv.getKey() + “:” + user.getName().toString());
  • 32. Thank you! And Special Thanks to Our Early Adopters: Yingkai Hu, Froila Dsouza, Zhongen Tao, Nithin Reddy, Bruce Su https://beam.apache.org/documentation/runners/samza/

Notas del editor

  1. Talk about the key features that are not available in current samza.
  2. How samza runner works? What runner is. What we need to support Beam features.
  3. When propagating watermarks across stages (connected by intermediate streams), partitionBy operator will send the watermarks to a single downstream task to aggregate the watermarks and then broadcast aggregated watermark to all the peer tasks.
  4. As far as we know, most of the existing Beam runners don’t support bigger-than-memory state.
  5. Needs to explain more details, which is a timer.
  6. Reduce the time on this slides