SlideShare uma empresa Scribd logo
1 de 50
Baixar para ler offline
Scaling up Near real-time
Analytics
@ Uber and LinkedIn
Chinmay Soman @ChinmaySoman
● Tech lead Streaming Platform team at Uber
● Worked on distributed storage and distributed filesystems in the past
● Apache Samza Committer, PMC
Yi Pan @nickpan47
● Tech lead Samza team at LinkedIn
● Worked on NoSQL databases and messaging systems in the past
● 8 years of experience in building distributed systems
● Apache Samza Committer and PMC.
Who we are
Agenda
Part I
● Use cases for near real-time analytics
● Operational / Scalability challenges
● New Streaming Analytics platform
Part II
● SamzaSQL: Apache Calcite - Apache Samza Integration
● Operators
● Multi-stage DAG
Why Streaming Analytics
Raw Data
(Input)
Real-time
Decision
(Output)
Big data processing &
query within secs
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Use Cases
Stream
Processing
Real-time Price Surging
SURGE
MULTIPLIERS
Rider eyeballs
Open car information
KAFKA
Ad Ranking at LinkedIn
Ads
Ranked by
Quality
LinkedIn Ad View
LinkedIn Ad Click
Stream
Processing
KAFKA
Real-time Machine Learning - UberEats
Online Prediction
Service
Stream
Processing
Real-time Machine Learning - UberEats
Kafka
Average ETD in the last
1/5/10/15/30 mins
Cassandra
Hadoop/Hive
Trained Model
Real-time data
Batch data
Experimentation Platform
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Introduction to Apache Samza
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Basic structure of a task
class PageKeyViewsCounterTask implements StreamTask {
public void process(IncomingMessageEnvelope envelope,
MessageCollector collector,
TaskCoordinator coordinator) {
GenericRecord record = ((GenericRecord) envelope.getMsg());
String pageKey = record.get("page-key").toString();
int newCount = pageKeyViews.get(pageKey).incrementAndGet();
collector.send(countStream, pageKey, newCount);
}
}
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Samza Deployment
RocksDB
(local store)
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Why Samza ?
● Stability
● Predictable scalability
● Built in Local state - with changelog support
● High Throughput: 1.1 Million msgs/second on 1 SSD box (with stateful
computation)
● Ease of debuggability
● Matured operationality
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Athena
Stream Processing platform @ Uber
Athena Platform - Technology stack
Kafka
Alerts
Cassandra
YARN
Challenges
● Manually track an end-end data flow
● Write code
● Manual provisioning
○ Schema inference
○ Kafka topics
○ Pinot tables
● Do your own Capacity Planning
● Create your own Metrics and Alerts
● Long time to production: 1-2 weeks
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Proposed Solution
SQL semantics
SURGE
MULTIPLIERS
Ads
Ranked by
popularity
JOIN FILTERING /
PROJECTION
Machine Learning
AGGREGATION
Job Definition
New Workflow: AthenaX
Job Evaluation
1 32
Managed
deployment
Job Definition
New Workflow: AthenaX
Job Evaluation
1 32
Managed
deployment
1) Select Inputs
2) Define SQL query
3) Select Outputs
Job definition in AthenaX
DEMO
SQL Expression: Example join job
Parameterized Queries
Config DB
select count(*) from hp_api_created_trips
where driver_uuid = f956e-ad11c-ff451-d34c2
AND city_id = 34
AND fare > 10
select count(*) from hp_api_created_trips
where driver_uuid = 80ac4-11ac5-efd63-a7de9
AND city_id = 2
AND fare > 100
Job Definition
New Workflow: AthenaX
Job Evaluation
1 32
Managed
deployment
1) Schema inference
2) Validation
3) Capacity Estimation
Job Evaluation: Schema Inference
Schema
Service
Job Evaluation: Capacity Estimator
Analyze
Input(s)
msg/s
bytes/s
Analyze
Query
Lookup
Table
Test
Deployment
● Yarn Containers
● Heap Memory
● Yarn memory
● CPU
● ...
Job Definition
New Workflow: AthenaX
Job Evaluation
1 32
Managed
deployment
1) Sandbox, Staging,
Production envs
2) Automated alerts
3) Job profiling
Job Profiling
Kafka Offset lag
CPU idle
Centralized Monitoring System
Managed Deployments
Sandbox
● Functional Correctness
● Play around with SQL
Staging
● System generated estimates
● Production like load
Production
● Well guarded
● Continuous profiling
AthenaX
Promote
AthenaX: Wins
● Flexible SQL* abstraction
● 1 click deployment to staging and promotion to production (within mins)
● Centralized place to track the data flow.
● Minimal manual intervention
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Athena X
Streaming Processing
Streaming Query
Samza Operator
Samza Core
SamzaSQL Planner
SQL on Streams
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Part II: Apache Calcite and Apache Samza
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Samza Operator
Samza Core
SamzaSQL Planner
SQL on Samza
Calcite: A data management framework w/ SQL parser, a query
optimizer, and adapters to different data sources. It allows
customized logical and physical algebras as well.
SamzaSQL Planner: Implementing Samza’s extension of
customized logical and physical algebras to Calcite.
Samza Operator: Samza’s physical operator APIs, used to
generate physical plan of a query
Samza Core: Samza’s execution engine that process the query as
a Samza job
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Samza Operator
Samza Core
SamzaSQL Planner
SQL on Samza:
Example
Logical plan from Calcite
LogicalStreamScan LogicalStreamScan
LogicalJoin
LogicalWindow
LogicalAggregate
Samza Physical plan
MessageStream.input() MessageStream.input()
join
WindowedCounter
join windowed
counter
StreamOperatorTask
SQL query
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Samza Operator APIs
• Used to describe Samza operators in the physical plan in SamzaSQL
• Support general transformation methods on a stream of messages:
‐ map <--> project in SQL
‐ filter <--> filter in SQL
‐ window <--> window/aggregation in SQL
‐ join <--> join in SQL
‐ flatMap
Job configuration
task.inputs=hp.driver_log,hp.rider_log
MessageStream.input(“hp.driver_log”).
join(MessageStream.input(“hp.rider_log”), ...).
window(Windows.intoSessionCounter(
m -> new Key(m.get(“trip_uuid”), m.get(“event_time”)),
WindowType.TUMBLE, 3600))
@Override void initOperators(Collection<SystemMessageStream> sources) {
Iterator iter = sources.iterator();
SystemMessageStream t1 = iter.next();
SystemMessageStream t2 = iter.next();
MessageStream.input(t1).join(MessageStream.input(t2).
window(Windows.intoSessionCounter(
m -> new Key(m.get(“trip_uuid”), m.get(“event_time”)),
WindowType.TUMBLE, 3600));
}
Java code for task initialization
Example of Operator API
Samza Physical plan
MessageStream.input() MessageStream.input()
join
WindowedCounter
Physical plan via operator APIs
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
SQL on Samza - Query Planner
SamzaSQL: Scalable Fast Data Management with Streaming SQL presented at IEEE International Parallel and Distributed Processing Symposium Workshops (IPDPSW) in May 2016
Samza parser/planner extension to Calcite
● How do we run the same SQL query on event time window?
SELECT STREAM t1.trip_uuid, TUMBLE_END(event_time, INTERVAL '1' HOUR) AS
event_time, count(*)
FROM hp.driver_log as t1
JOIN hp.rider_log as t2
ON t1.driver_uuid = t2.driver_uuid
GROUP BY TUMBLE(event_time, INTERVAL '1' HOUR), t1.trip_uuid;
Event Time Window in Samza SQL
● Accurate event-time window output in realtime stream processing is hard
○ Uncertain latency in message arrival
○ Possible out-of-order due to re-partitioning
Samza Operator for Event Time Window
● Solution
○ Use early trigger to calculate the window output in realtime
○ Keep the window state
○ Handle late arrivals using late trigger to re-compute the corrected window output
Concept from Google MillWheel and Stream Processing 101/102
Samza Operator for Event Time Window
● Key to implement the solution:
○ Need to keep past window states
○ Need high read/write rates to
update window states
● Samza’s local KV-store is the
perfect choice for the event time
window!
Operator API: Triggers for Event-Time Window
● Samza Operator APIs allow setting early and late triggers for window
inputStream.window(Windows.<JsonMessage, String>intoSessionCounter(
keyExtractor, WindowType.TUMBLE, 3600).
setTriggers(TriggerBuilder.
<JsonMessage, Integer>earlyTriggerOnEventTime(m -> getEventTime(m), 3600).
addLateTrigger((m, s) -> true). //always re-compute output for late arrivals
addTimeoutSinceLastMessage(30)))
Samza SQL: Scaling out to Multiple Stages
● Supporting embedded SQL statements
○ LinkedIn standardizing pipelines
Title Updates
Company Updates
Title standardizer
Company standardizer
Standard titleupdates
Standard company
updates
LinkedIn Member
LinkedIn Member
Join by memberId
Combined member
profile update
Samza SQL: Scaling out to Multiple Stages
● Supporting embedded SQL statements
○ LinkedIn standardizing pipelines in SQL statement
○ Motivations to move the above embedded query statements in different Samza jobs
■ Update machine learning models w/o changing join logic
■ Scaling differently for title_standardizer and company_standardizer due to
● Different traffic volumes
● Different resource utilization to run ML models
SELECT STREAM mid, title, company_info
FROM (
SELECT STREAM mid, title_standardizer(*)
FROM isb.member_title_updates) AS t1
OUTER_JOIN (
SELECT STREAM mid, company_standardizer(*)
FROM isb.member_company_updates) AS t2
ON t1.mid = t2.mid;
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Samza SQL: Samza Pipeline for SQL (WIP)
Samza
Operator
Samza Core
Samza
Pipeline
SamzaSQL
Parser/Planner
Samza Pipeline: allows a single SQL statement to be grouped
into sub-queries and to be instantiated and deployed separately
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
SQL on Samza - Query Planner for Pipelines
SamzaSQL: Scalable Fast Data Management with Streaming SQL presented at IEEE International Parallel and Distributed Processing Symposium Workshops (IPDPSW) in May 2016
Samza parser/planner extension to Calcite
Samza Pipeline
Configuration
Pipelines for SamzaSQL (WIP)
public class StandardizerJoinPipeline implements PipelineFactory {
public Pipeline create(Config config) {
Processor title = getTitleStandardizer(config);
Processor comp = getTitleStandardizer(config);
Processor join = getJoin(config);
Stream inStream1 = getStream(config, “inStream1”);
Stream inStream2 = getStream(config, “inStream2”);
// … omitted for brevity
PipelineBuilder builder = new PipelineBuilder();
return builder.addInputStreams(title, inStream1)
.addInputStreams(comp, inStream2)
.addIntermediateStreams(title, join, midStream1)
.addIntermediateStreams(comp, join, midStream2)
.addOutputStreams(join, outStream)
.build();
}
}
output
Title
standard
izer
Join
Compan
y
standard
izer
Future work
● Apache Beam integration
● Samza support for batch jobs
● Exactly once processing
● Automated scale out
● Disaster Recovery for stateful applications
References
● http://samza.apache.org/
● Milinda Pathirage, Julian Hyde, Yi Pan, Beth Plale. "SamzaSQL: Scalable Fast Data
Management with Streaming SQL"
● https://calcite.apache.org/
● Samza operator API design and implementation (SAMZA-914, SAMZA-915)
● Tyler Akidau The world beyond batch: Streaming 101
● Tyler Akidau The world beyond batch: Streaming 102
● Samza window operator design and implementation (SAMZA-552)
Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed
quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent.
Questions ?
csoman@uber.com
yipan@linkedin.com

Mais conteúdo relacionado

Mais procurados

Essential ingredients for real time stream processing @Scale by Kartik pParam...
Essential ingredients for real time stream processing @Scale by Kartik pParam...Essential ingredients for real time stream processing @Scale by Kartik pParam...
Essential ingredients for real time stream processing @Scale by Kartik pParam...Big Data Spain
 
Performance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State StoresPerformance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State Storesconfluent
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database Systemconfluent
 
Use ksqlDB to migrate core-banking processing from batch to streaming | Mark ...
Use ksqlDB to migrate core-banking processing from batch to streaming | Mark ...Use ksqlDB to migrate core-banking processing from batch to streaming | Mark ...
Use ksqlDB to migrate core-banking processing from batch to streaming | Mark ...HostedbyConfluent
 
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
 
Bravo Six, Going Realtime. Transitioning Activision Data Pipeline to Streamin...
Bravo Six, Going Realtime. Transitioning Activision Data Pipeline to Streamin...Bravo Six, Going Realtime. Transitioning Activision Data Pipeline to Streamin...
Bravo Six, Going Realtime. Transitioning Activision Data Pipeline to Streamin...HostedbyConfluent
 
Scalable complex event processing on samza @UBER
Scalable complex event processing on samza @UBERScalable complex event processing on samza @UBER
Scalable complex event processing on samza @UBERShuyi Chen
 
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
 
Developing streaming applications with apache apex (strata + hadoop world)
Developing streaming applications with apache apex (strata + hadoop world)Developing streaming applications with apache apex (strata + hadoop world)
Developing streaming applications with apache apex (strata + hadoop world)Apache Apex
 
Timeline Service v.2 (Hadoop Summit 2016)
Timeline Service v.2 (Hadoop Summit 2016)Timeline Service v.2 (Hadoop Summit 2016)
Timeline Service v.2 (Hadoop Summit 2016)Sangjin Lee
 
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life ExampleKafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Exampleconfluent
 
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ UberKafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uberconfluent
 
KSQL: Streaming SQL for Kafka
KSQL: Streaming SQL for KafkaKSQL: Streaming SQL for Kafka
KSQL: Streaming SQL for Kafkaconfluent
 
Scaling up uber's real time data analytics
Scaling up uber's real time data analyticsScaling up uber's real time data analytics
Scaling up uber's real time data analyticsXiang Fu
 
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
 
Introduction to apache kafka, confluent and why they matter
Introduction to apache kafka, confluent and why they matterIntroduction to apache kafka, confluent and why they matter
Introduction to apache kafka, confluent and why they matterPaolo Castagna
 
Low Latency Polyglot Model Scoring using Apache Apex
Low Latency Polyglot Model Scoring using Apache ApexLow Latency Polyglot Model Scoring using Apache Apex
Low Latency Polyglot Model Scoring using Apache ApexApache Apex
 
Streaming all over the world Real life use cases with Kafka Streams
Streaming all over the world  Real life use cases with Kafka StreamsStreaming all over the world  Real life use cases with Kafka Streams
Streaming all over the world Real life use cases with Kafka Streamsconfluent
 
HBaseCon2017 Highly-Available HBase
HBaseCon2017 Highly-Available HBaseHBaseCon2017 Highly-Available HBase
HBaseCon2017 Highly-Available HBaseHBaseCon
 

Mais procurados (20)

Essential ingredients for real time stream processing @Scale by Kartik pParam...
Essential ingredients for real time stream processing @Scale by Kartik pParam...Essential ingredients for real time stream processing @Scale by Kartik pParam...
Essential ingredients for real time stream processing @Scale by Kartik pParam...
 
Performance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State StoresPerformance Tuning RocksDB for Kafka Streams’ State Stores
Performance Tuning RocksDB for Kafka Streams’ State Stores
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database System
 
Use ksqlDB to migrate core-banking processing from batch to streaming | Mark ...
Use ksqlDB to migrate core-banking processing from batch to streaming | Mark ...Use ksqlDB to migrate core-banking processing from batch to streaming | Mark ...
Use ksqlDB to migrate core-banking processing from batch to streaming | Mark ...
 
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 ...
 
Bravo Six, Going Realtime. Transitioning Activision Data Pipeline to Streamin...
Bravo Six, Going Realtime. Transitioning Activision Data Pipeline to Streamin...Bravo Six, Going Realtime. Transitioning Activision Data Pipeline to Streamin...
Bravo Six, Going Realtime. Transitioning Activision Data Pipeline to Streamin...
 
Scalable complex event processing on samza @UBER
Scalable complex event processing on samza @UBERScalable complex event processing on samza @UBER
Scalable complex event processing on samza @UBER
 
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
 
Developing streaming applications with apache apex (strata + hadoop world)
Developing streaming applications with apache apex (strata + hadoop world)Developing streaming applications with apache apex (strata + hadoop world)
Developing streaming applications with apache apex (strata + hadoop world)
 
Timeline Service v.2 (Hadoop Summit 2016)
Timeline Service v.2 (Hadoop Summit 2016)Timeline Service v.2 (Hadoop Summit 2016)
Timeline Service v.2 (Hadoop Summit 2016)
 
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life ExampleKafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example
Kafka Summit NYC 2017 Introduction to Kafka Streams with a Real-life Example
 
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ UberKafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
Kafka Summit NYC 2017 - Scalable Real-Time Complex Event Processing @ Uber
 
KSQL: Streaming SQL for Kafka
KSQL: Streaming SQL for KafkaKSQL: Streaming SQL for Kafka
KSQL: Streaming SQL for Kafka
 
Scaling up uber's real time data analytics
Scaling up uber's real time data analyticsScaling up uber's real time data analytics
Scaling up uber's real time data analytics
 
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...
 
Introduction to apache kafka, confluent and why they matter
Introduction to apache kafka, confluent and why they matterIntroduction to apache kafka, confluent and why they matter
Introduction to apache kafka, confluent and why they matter
 
Data Pipeline at Tapad
Data Pipeline at TapadData Pipeline at Tapad
Data Pipeline at Tapad
 
Low Latency Polyglot Model Scoring using Apache Apex
Low Latency Polyglot Model Scoring using Apache ApexLow Latency Polyglot Model Scoring using Apache Apex
Low Latency Polyglot Model Scoring using Apache Apex
 
Streaming all over the world Real life use cases with Kafka Streams
Streaming all over the world  Real life use cases with Kafka StreamsStreaming all over the world  Real life use cases with Kafka Streams
Streaming all over the world Real life use cases with Kafka Streams
 
HBaseCon2017 Highly-Available HBase
HBaseCon2017 Highly-Available HBaseHBaseCon2017 Highly-Available HBase
HBaseCon2017 Highly-Available HBase
 

Semelhante a SamzaSQL QCon'16 presentation

Scaling up Near Real-time Analytics @Uber &LinkedIn
Scaling up Near Real-time Analytics @Uber &LinkedInScaling up Near Real-time Analytics @Uber &LinkedIn
Scaling up Near Real-time Analytics @Uber &LinkedInC4Media
 
Apache Samza 1.0 - What's New, What's Next
Apache Samza 1.0 - What's New, What's NextApache Samza 1.0 - What's New, What's Next
Apache Samza 1.0 - What's New, What's NextPrateek Maheshwari
 
Apache Spark 2.0: Faster, Easier, and Smarter
Apache Spark 2.0: Faster, Easier, and SmarterApache Spark 2.0: Faster, Easier, and Smarter
Apache Spark 2.0: Faster, Easier, and SmarterDatabricks
 
SnappyData overview NikeTechTalk 11/19/15
SnappyData overview NikeTechTalk 11/19/15SnappyData overview NikeTechTalk 11/19/15
SnappyData overview NikeTechTalk 11/19/15SnappyData
 
SCALABLE MONITORING USING PROMETHEUS WITH APACHE SPARK
SCALABLE MONITORING USING PROMETHEUS WITH APACHE SPARKSCALABLE MONITORING USING PROMETHEUS WITH APACHE SPARK
SCALABLE MONITORING USING PROMETHEUS WITH APACHE SPARKzmhassan
 
Building a serverless company on AWS lambda and Serverless framework
Building a serverless company on AWS lambda and Serverless frameworkBuilding a serverless company on AWS lambda and Serverless framework
Building a serverless company on AWS lambda and Serverless frameworkLuciano Mammino
 
Extending Spark Streaming to Support Complex Event Processing
Extending Spark Streaming to Support Complex Event ProcessingExtending Spark Streaming to Support Complex Event Processing
Extending Spark Streaming to Support Complex Event ProcessingOh Chan Kwon
 
Chicago Kafka Meetup
Chicago Kafka MeetupChicago Kafka Meetup
Chicago Kafka MeetupCliff Gilmore
 
XStream: stream processing platform at facebook
XStream:  stream processing platform at facebookXStream:  stream processing platform at facebook
XStream: stream processing platform at facebookAniket Mokashi
 
Scalable Stream Processing with Apache Samza
Scalable Stream Processing with Apache SamzaScalable Stream Processing with Apache Samza
Scalable Stream Processing with Apache SamzaPrateek Maheshwari
 
Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...
Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...
Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...Databricks
 
Flink Forward SF 2017: Malo Deniélou - No shard left behind: Dynamic work re...
Flink Forward SF 2017: Malo Deniélou -  No shard left behind: Dynamic work re...Flink Forward SF 2017: Malo Deniélou -  No shard left behind: Dynamic work re...
Flink Forward SF 2017: Malo Deniélou - No shard left behind: Dynamic work re...Flink Forward
 
Event Driven Microservices
Event Driven MicroservicesEvent Driven Microservices
Event Driven MicroservicesFabrizio Fortino
 
Data analytics master class: predict hotel revenue
Data analytics master class: predict hotel revenueData analytics master class: predict hotel revenue
Data analytics master class: predict hotel revenueKris Peeters
 
Nextcon samza preso july - final
Nextcon samza preso   july - finalNextcon samza preso   july - final
Nextcon samza preso july - finalYi Pan
 
SnappyData Ad Analytics Use Case -- BDAM Meetup Sept 14th
SnappyData Ad Analytics Use Case -- BDAM Meetup Sept 14thSnappyData Ad Analytics Use Case -- BDAM Meetup Sept 14th
SnappyData Ad Analytics Use Case -- BDAM Meetup Sept 14thSnappyData
 
How Opera Syncs Tens of Millions of Browsers and Sleeps Well at Night
How Opera Syncs Tens of Millions of Browsers and Sleeps Well at NightHow Opera Syncs Tens of Millions of Browsers and Sleeps Well at Night
How Opera Syncs Tens of Millions of Browsers and Sleeps Well at NightScyllaDB
 
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
 

Semelhante a SamzaSQL QCon'16 presentation (20)

Scaling up Near Real-time Analytics @Uber &LinkedIn
Scaling up Near Real-time Analytics @Uber &LinkedInScaling up Near Real-time Analytics @Uber &LinkedIn
Scaling up Near Real-time Analytics @Uber &LinkedIn
 
Apache Samza 1.0 - What's New, What's Next
Apache Samza 1.0 - What's New, What's NextApache Samza 1.0 - What's New, What's Next
Apache Samza 1.0 - What's New, What's Next
 
Nike tech talk.2
Nike tech talk.2Nike tech talk.2
Nike tech talk.2
 
Apache Spark 2.0: Faster, Easier, and Smarter
Apache Spark 2.0: Faster, Easier, and SmarterApache Spark 2.0: Faster, Easier, and Smarter
Apache Spark 2.0: Faster, Easier, and Smarter
 
SnappyData overview NikeTechTalk 11/19/15
SnappyData overview NikeTechTalk 11/19/15SnappyData overview NikeTechTalk 11/19/15
SnappyData overview NikeTechTalk 11/19/15
 
SCALABLE MONITORING USING PROMETHEUS WITH APACHE SPARK
SCALABLE MONITORING USING PROMETHEUS WITH APACHE SPARKSCALABLE MONITORING USING PROMETHEUS WITH APACHE SPARK
SCALABLE MONITORING USING PROMETHEUS WITH APACHE SPARK
 
Building a serverless company on AWS lambda and Serverless framework
Building a serverless company on AWS lambda and Serverless frameworkBuilding a serverless company on AWS lambda and Serverless framework
Building a serverless company on AWS lambda and Serverless framework
 
Spark cep
Spark cepSpark cep
Spark cep
 
Extending Spark Streaming to Support Complex Event Processing
Extending Spark Streaming to Support Complex Event ProcessingExtending Spark Streaming to Support Complex Event Processing
Extending Spark Streaming to Support Complex Event Processing
 
Chicago Kafka Meetup
Chicago Kafka MeetupChicago Kafka Meetup
Chicago Kafka Meetup
 
XStream: stream processing platform at facebook
XStream:  stream processing platform at facebookXStream:  stream processing platform at facebook
XStream: stream processing platform at facebook
 
Scalable Stream Processing with Apache Samza
Scalable Stream Processing with Apache SamzaScalable Stream Processing with Apache Samza
Scalable Stream Processing with Apache Samza
 
Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...
Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...
Spark SQL Catalyst Code Optimization using Function Outlining with Kavana Bha...
 
Flink Forward SF 2017: Malo Deniélou - No shard left behind: Dynamic work re...
Flink Forward SF 2017: Malo Deniélou -  No shard left behind: Dynamic work re...Flink Forward SF 2017: Malo Deniélou -  No shard left behind: Dynamic work re...
Flink Forward SF 2017: Malo Deniélou - No shard left behind: Dynamic work re...
 
Event Driven Microservices
Event Driven MicroservicesEvent Driven Microservices
Event Driven Microservices
 
Data analytics master class: predict hotel revenue
Data analytics master class: predict hotel revenueData analytics master class: predict hotel revenue
Data analytics master class: predict hotel revenue
 
Nextcon samza preso july - final
Nextcon samza preso   july - finalNextcon samza preso   july - final
Nextcon samza preso july - final
 
SnappyData Ad Analytics Use Case -- BDAM Meetup Sept 14th
SnappyData Ad Analytics Use Case -- BDAM Meetup Sept 14thSnappyData Ad Analytics Use Case -- BDAM Meetup Sept 14th
SnappyData Ad Analytics Use Case -- BDAM Meetup Sept 14th
 
How Opera Syncs Tens of Millions of Browsers and Sleeps Well at Night
How Opera Syncs Tens of Millions of Browsers and Sleeps Well at NightHow Opera Syncs Tens of Millions of Browsers and Sleeps Well at Night
How Opera Syncs Tens of Millions of Browsers and Sleeps Well at Night
 
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...
 

Último

Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Boston Institute of Analytics
 
Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...Seán Kennedy
 
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Boston Institute of Analytics
 
What To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptxWhat To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptxSimranPal17
 
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxmodul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxaleedritatuxx
 
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...Unveiling the Role of Social Media Suspect Investigators in Preventing Online...
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...Milind Agarwal
 
Principles and Practices of Data Visualization
Principles and Practices of Data VisualizationPrinciples and Practices of Data Visualization
Principles and Practices of Data VisualizationKianJazayeri1
 
Cyber awareness ppt on the recorded data
Cyber awareness ppt on the recorded dataCyber awareness ppt on the recorded data
Cyber awareness ppt on the recorded dataTecnoIncentive
 
Networking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptxNetworking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptxHimangsuNath
 
Decoding Patterns: Customer Churn Prediction Data Analysis Project
Decoding Patterns: Customer Churn Prediction Data Analysis ProjectDecoding Patterns: Customer Churn Prediction Data Analysis Project
Decoding Patterns: Customer Churn Prediction Data Analysis ProjectBoston Institute of Analytics
 
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Boston Institute of Analytics
 
Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Cathrine Wilhelmsen
 
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesConf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesTimothy Spann
 
Real-Time AI Streaming - AI Max Princeton
Real-Time AI  Streaming - AI Max PrincetonReal-Time AI  Streaming - AI Max Princeton
Real-Time AI Streaming - AI Max PrincetonTimothy Spann
 
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...Amil Baba Dawood bangali
 
INTRODUCTION TO Natural language processing
INTRODUCTION TO Natural language processingINTRODUCTION TO Natural language processing
INTRODUCTION TO Natural language processingsocarem879
 
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...Dr Arash Najmaei ( Phd., MBA, BSc)
 
Semantic Shed - Squashing and Squeezing.pptx
Semantic Shed - Squashing and Squeezing.pptxSemantic Shed - Squashing and Squeezing.pptx
Semantic Shed - Squashing and Squeezing.pptxMike Bennett
 

Último (20)

Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
 
Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...
 
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
 
What To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptxWhat To Do For World Nature Conservation Day by Slidesgo.pptx
What To Do For World Nature Conservation Day by Slidesgo.pptx
 
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxmodul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
 
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...Unveiling the Role of Social Media Suspect Investigators in Preventing Online...
Unveiling the Role of Social Media Suspect Investigators in Preventing Online...
 
Principles and Practices of Data Visualization
Principles and Practices of Data VisualizationPrinciples and Practices of Data Visualization
Principles and Practices of Data Visualization
 
Cyber awareness ppt on the recorded data
Cyber awareness ppt on the recorded dataCyber awareness ppt on the recorded data
Cyber awareness ppt on the recorded data
 
Networking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptxNetworking Case Study prepared by teacher.pptx
Networking Case Study prepared by teacher.pptx
 
Decoding Patterns: Customer Churn Prediction Data Analysis Project
Decoding Patterns: Customer Churn Prediction Data Analysis ProjectDecoding Patterns: Customer Churn Prediction Data Analysis Project
Decoding Patterns: Customer Churn Prediction Data Analysis Project
 
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
 
Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)
 
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming PipelinesConf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
Conf42-LLM_Adding Generative AI to Real-Time Streaming Pipelines
 
Real-Time AI Streaming - AI Max Princeton
Real-Time AI  Streaming - AI Max PrincetonReal-Time AI  Streaming - AI Max Princeton
Real-Time AI Streaming - AI Max Princeton
 
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
 
Data Analysis Project: Stroke Prediction
Data Analysis Project: Stroke PredictionData Analysis Project: Stroke Prediction
Data Analysis Project: Stroke Prediction
 
INTRODUCTION TO Natural language processing
INTRODUCTION TO Natural language processingINTRODUCTION TO Natural language processing
INTRODUCTION TO Natural language processing
 
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
6 Tips for Interpretable Topic Models _ by Nicha Ruchirawat _ Towards Data Sc...
 
Semantic Shed - Squashing and Squeezing.pptx
Semantic Shed - Squashing and Squeezing.pptxSemantic Shed - Squashing and Squeezing.pptx
Semantic Shed - Squashing and Squeezing.pptx
 
Insurance Churn Prediction Data Analysis Project
Insurance Churn Prediction Data Analysis ProjectInsurance Churn Prediction Data Analysis Project
Insurance Churn Prediction Data Analysis Project
 

SamzaSQL QCon'16 presentation

  • 1. Scaling up Near real-time Analytics @ Uber and LinkedIn
  • 2. Chinmay Soman @ChinmaySoman ● Tech lead Streaming Platform team at Uber ● Worked on distributed storage and distributed filesystems in the past ● Apache Samza Committer, PMC Yi Pan @nickpan47 ● Tech lead Samza team at LinkedIn ● Worked on NoSQL databases and messaging systems in the past ● 8 years of experience in building distributed systems ● Apache Samza Committer and PMC. Who we are
  • 3. Agenda Part I ● Use cases for near real-time analytics ● Operational / Scalability challenges ● New Streaming Analytics platform Part II ● SamzaSQL: Apache Calcite - Apache Samza Integration ● Operators ● Multi-stage DAG
  • 4. Why Streaming Analytics Raw Data (Input) Real-time Decision (Output) Big data processing & query within secs
  • 5. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Use Cases
  • 7. Ad Ranking at LinkedIn Ads Ranked by Quality LinkedIn Ad View LinkedIn Ad Click Stream Processing KAFKA
  • 9. Online Prediction Service Stream Processing Real-time Machine Learning - UberEats Kafka Average ETD in the last 1/5/10/15/30 mins Cassandra Hadoop/Hive Trained Model Real-time data Batch data
  • 11. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Introduction to Apache Samza
  • 12. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Basic structure of a task class PageKeyViewsCounterTask implements StreamTask { public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) { GenericRecord record = ((GenericRecord) envelope.getMsg()); String pageKey = record.get("page-key").toString(); int newCount = pageKeyViews.get(pageKey).incrementAndGet(); collector.send(countStream, pageKey, newCount); } }
  • 13. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Samza Deployment RocksDB (local store)
  • 14. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Why Samza ? ● Stability ● Predictable scalability ● Built in Local state - with changelog support ● High Throughput: 1.1 Million msgs/second on 1 SSD box (with stateful computation) ● Ease of debuggability ● Matured operationality
  • 15. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Athena Stream Processing platform @ Uber
  • 16. Athena Platform - Technology stack Kafka Alerts Cassandra YARN
  • 17. Challenges ● Manually track an end-end data flow ● Write code ● Manual provisioning ○ Schema inference ○ Kafka topics ○ Pinot tables ● Do your own Capacity Planning ● Create your own Metrics and Alerts ● Long time to production: 1-2 weeks
  • 18. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Proposed Solution
  • 19. SQL semantics SURGE MULTIPLIERS Ads Ranked by popularity JOIN FILTERING / PROJECTION Machine Learning AGGREGATION
  • 20. Job Definition New Workflow: AthenaX Job Evaluation 1 32 Managed deployment
  • 21. Job Definition New Workflow: AthenaX Job Evaluation 1 32 Managed deployment 1) Select Inputs 2) Define SQL query 3) Select Outputs
  • 22. Job definition in AthenaX DEMO
  • 24. Parameterized Queries Config DB select count(*) from hp_api_created_trips where driver_uuid = f956e-ad11c-ff451-d34c2 AND city_id = 34 AND fare > 10 select count(*) from hp_api_created_trips where driver_uuid = 80ac4-11ac5-efd63-a7de9 AND city_id = 2 AND fare > 100
  • 25. Job Definition New Workflow: AthenaX Job Evaluation 1 32 Managed deployment 1) Schema inference 2) Validation 3) Capacity Estimation
  • 26. Job Evaluation: Schema Inference Schema Service
  • 27. Job Evaluation: Capacity Estimator Analyze Input(s) msg/s bytes/s Analyze Query Lookup Table Test Deployment ● Yarn Containers ● Heap Memory ● Yarn memory ● CPU ● ...
  • 28. Job Definition New Workflow: AthenaX Job Evaluation 1 32 Managed deployment 1) Sandbox, Staging, Production envs 2) Automated alerts 3) Job profiling
  • 29. Job Profiling Kafka Offset lag CPU idle Centralized Monitoring System
  • 30. Managed Deployments Sandbox ● Functional Correctness ● Play around with SQL Staging ● System generated estimates ● Production like load Production ● Well guarded ● Continuous profiling AthenaX Promote
  • 31. AthenaX: Wins ● Flexible SQL* abstraction ● 1 click deployment to staging and promotion to production (within mins) ● Centralized place to track the data flow. ● Minimal manual intervention
  • 32. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Athena X Streaming Processing Streaming Query Samza Operator Samza Core SamzaSQL Planner SQL on Streams
  • 33. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Part II: Apache Calcite and Apache Samza
  • 34. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Samza Operator Samza Core SamzaSQL Planner SQL on Samza Calcite: A data management framework w/ SQL parser, a query optimizer, and adapters to different data sources. It allows customized logical and physical algebras as well. SamzaSQL Planner: Implementing Samza’s extension of customized logical and physical algebras to Calcite. Samza Operator: Samza’s physical operator APIs, used to generate physical plan of a query Samza Core: Samza’s execution engine that process the query as a Samza job
  • 35. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Samza Operator Samza Core SamzaSQL Planner SQL on Samza: Example Logical plan from Calcite LogicalStreamScan LogicalStreamScan LogicalJoin LogicalWindow LogicalAggregate Samza Physical plan MessageStream.input() MessageStream.input() join WindowedCounter join windowed counter StreamOperatorTask SQL query
  • 36. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Samza Operator APIs • Used to describe Samza operators in the physical plan in SamzaSQL • Support general transformation methods on a stream of messages: ‐ map <--> project in SQL ‐ filter <--> filter in SQL ‐ window <--> window/aggregation in SQL ‐ join <--> join in SQL ‐ flatMap
  • 37. Job configuration task.inputs=hp.driver_log,hp.rider_log MessageStream.input(“hp.driver_log”). join(MessageStream.input(“hp.rider_log”), ...). window(Windows.intoSessionCounter( m -> new Key(m.get(“trip_uuid”), m.get(“event_time”)), WindowType.TUMBLE, 3600)) @Override void initOperators(Collection<SystemMessageStream> sources) { Iterator iter = sources.iterator(); SystemMessageStream t1 = iter.next(); SystemMessageStream t2 = iter.next(); MessageStream.input(t1).join(MessageStream.input(t2). window(Windows.intoSessionCounter( m -> new Key(m.get(“trip_uuid”), m.get(“event_time”)), WindowType.TUMBLE, 3600)); } Java code for task initialization Example of Operator API Samza Physical plan MessageStream.input() MessageStream.input() join WindowedCounter Physical plan via operator APIs
  • 38. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. SQL on Samza - Query Planner SamzaSQL: Scalable Fast Data Management with Streaming SQL presented at IEEE International Parallel and Distributed Processing Symposium Workshops (IPDPSW) in May 2016 Samza parser/planner extension to Calcite
  • 39. ● How do we run the same SQL query on event time window? SELECT STREAM t1.trip_uuid, TUMBLE_END(event_time, INTERVAL '1' HOUR) AS event_time, count(*) FROM hp.driver_log as t1 JOIN hp.rider_log as t2 ON t1.driver_uuid = t2.driver_uuid GROUP BY TUMBLE(event_time, INTERVAL '1' HOUR), t1.trip_uuid; Event Time Window in Samza SQL ● Accurate event-time window output in realtime stream processing is hard ○ Uncertain latency in message arrival ○ Possible out-of-order due to re-partitioning
  • 40. Samza Operator for Event Time Window ● Solution ○ Use early trigger to calculate the window output in realtime ○ Keep the window state ○ Handle late arrivals using late trigger to re-compute the corrected window output Concept from Google MillWheel and Stream Processing 101/102
  • 41. Samza Operator for Event Time Window ● Key to implement the solution: ○ Need to keep past window states ○ Need high read/write rates to update window states ● Samza’s local KV-store is the perfect choice for the event time window!
  • 42. Operator API: Triggers for Event-Time Window ● Samza Operator APIs allow setting early and late triggers for window inputStream.window(Windows.<JsonMessage, String>intoSessionCounter( keyExtractor, WindowType.TUMBLE, 3600). setTriggers(TriggerBuilder. <JsonMessage, Integer>earlyTriggerOnEventTime(m -> getEventTime(m), 3600). addLateTrigger((m, s) -> true). //always re-compute output for late arrivals addTimeoutSinceLastMessage(30)))
  • 43. Samza SQL: Scaling out to Multiple Stages ● Supporting embedded SQL statements ○ LinkedIn standardizing pipelines Title Updates Company Updates Title standardizer Company standardizer Standard titleupdates Standard company updates LinkedIn Member LinkedIn Member Join by memberId Combined member profile update
  • 44. Samza SQL: Scaling out to Multiple Stages ● Supporting embedded SQL statements ○ LinkedIn standardizing pipelines in SQL statement ○ Motivations to move the above embedded query statements in different Samza jobs ■ Update machine learning models w/o changing join logic ■ Scaling differently for title_standardizer and company_standardizer due to ● Different traffic volumes ● Different resource utilization to run ML models SELECT STREAM mid, title, company_info FROM ( SELECT STREAM mid, title_standardizer(*) FROM isb.member_title_updates) AS t1 OUTER_JOIN ( SELECT STREAM mid, company_standardizer(*) FROM isb.member_company_updates) AS t2 ON t1.mid = t2.mid;
  • 45. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Samza SQL: Samza Pipeline for SQL (WIP) Samza Operator Samza Core Samza Pipeline SamzaSQL Parser/Planner Samza Pipeline: allows a single SQL statement to be grouped into sub-queries and to be instantiated and deployed separately
  • 46. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. SQL on Samza - Query Planner for Pipelines SamzaSQL: Scalable Fast Data Management with Streaming SQL presented at IEEE International Parallel and Distributed Processing Symposium Workshops (IPDPSW) in May 2016 Samza parser/planner extension to Calcite Samza Pipeline Configuration
  • 47. Pipelines for SamzaSQL (WIP) public class StandardizerJoinPipeline implements PipelineFactory { public Pipeline create(Config config) { Processor title = getTitleStandardizer(config); Processor comp = getTitleStandardizer(config); Processor join = getJoin(config); Stream inStream1 = getStream(config, “inStream1”); Stream inStream2 = getStream(config, “inStream2”); // … omitted for brevity PipelineBuilder builder = new PipelineBuilder(); return builder.addInputStreams(title, inStream1) .addInputStreams(comp, inStream2) .addIntermediateStreams(title, join, midStream1) .addIntermediateStreams(comp, join, midStream2) .addOutputStreams(join, outStream) .build(); } } output Title standard izer Join Compan y standard izer
  • 48. Future work ● Apache Beam integration ● Samza support for batch jobs ● Exactly once processing ● Automated scale out ● Disaster Recovery for stateful applications
  • 49. References ● http://samza.apache.org/ ● Milinda Pathirage, Julian Hyde, Yi Pan, Beth Plale. "SamzaSQL: Scalable Fast Data Management with Streaming SQL" ● https://calcite.apache.org/ ● Samza operator API design and implementation (SAMZA-914, SAMZA-915) ● Tyler Akidau The world beyond batch: Streaming 101 ● Tyler Akidau The world beyond batch: Streaming 102 ● Samza window operator design and implementation (SAMZA-552)
  • 50. Edit or delete footer text in Master ipsandella doloreium dem isciame ndaestia nessed quibus aut hiligenet ut ea debisci eturiate poresti vid min core, vercidigent. Questions ? csoman@uber.com yipan@linkedin.com