SlideShare a Scribd company logo
1 of 32
Download to read offline
Stateful Streaming Data
Pipelines with Apache Apex
Chandni Singh
PMC and Committer, Apache Apex
Founder, Simplifi.it
Timothy Farkas
Committer, Apache Apex
Founder, Simplifi.it
Agenda
● Introduction to Apache Apex
● Managed State
● Spillable Data-structures
● Questions
What is Apache Apex
● Distributed data processing engine
● Runs on Hadoop
● Real-time streaming
● Fault-tolerant
Anatomy of An Apex Application
● Tuple: Discrete unit of information sent from one operator to another.
● Operator: Java code that performs an operation on tuples. The code runs in
a yarn container on a yarn cluster.
● DAG: Operators can be connected to form an application. Tuple transfer
between operators is 1-way, so the application forms a Directed Acyclic
Graph.
● Window Marker: An Id that is associated with tuples and operators, and is
used for fault-tolerance.
Anatomy of An Apex Operator
public class MyOperator implements Operator {
private Map<String, String> inMemState = new HashMap<>(); // checkpointed in memory state
private int myProperty;
public final transient DefaultInputPort<String> inputPort = new DefaultInputPort<String>() {
public void process(String event) {
// Custom event processing logic
}
}
public void setup(Context context) { // One time setup tasks to be performed when the operator first starts
}
public void beginWindow(long windowId) { // Next window has started
}
public void endWindow() {
}
public void teardown() { // Operator is shutting down. Any cleanup needs to be done here.
}
public void setMyProperty(int myProperty) {
this.myProperty = myProperty
}
public int getMyProperty() { return myProperty}
}
Fault tolerance in Apex
● Apex inserts window markers with IDs in the data stream, which operators are
notified of.
● It provides fault-tolerance by checkpointing the state of every operator in the
pipeline every N windows.
● If an operator crashes, it restores the operator with the state corresponding to
a checkpointed window.
● Committed window: In the simple case, when all operators are checkpointed
at the same frequency, committed window is the latest window which has
been checkpointed by all the operators in the DAG.
What is the problem?
● Time to checkpoint ∝ size of operator state
● With increasing state, the operator will eventually crash.
● Even before the operator crashes, the platform may assume that the operator
is unresponsive and instruct Yarn to kill it.
Managed State - Introduction
A reusable component that can be added to any operator to manage its
key/value state.
● Checkpoints key/value state incrementally.
● Allows to set a threshold on the size of data in memory. Data that has been
persisted, is off-loaded from memory when the threshold is reached.
● Keys can be partitioned in user-defined buckets which helps with operator
partitioning and efficient off-loading from memory.
● Key/values are persisted on hdfs in a state that is optimized for querying.
● Purges stale data from disk.
Managed State API
● Write to managed state
managedState.put(1L, key, value)
● Read from managed state
managedState.getSync(1L, key)
managedState.getAsync(1L, key)
Architecture
For simplicity, in the following examples we will use window Ids for time buckets
because window Ids roughly correspond to processing time.
Read from Managed State
Writes to Managed State
● Key/Values are put in the bucket cache.
● At checkpoints, data from the bucket cache is moved to checkpoint cache
which is written to WAL.
● When a window is committed, data in the WAL till the current committed
window is transferred to key/value store which is the Bucket File System.
Writes to Managed State - Continued
Purging of Data
Delete time-buckets older than 2 days. 2 days are approximately equivalent to 5760 windows.
Fault-tolerance in Managed State
Scenario 1: Operator failure
Fault-tolerance in Managed State
Scenario 2: Transferring data from WAL to Bucket File System
Implementations of Managed State
ManagedStateImpl ManagedTimeStateImpl ManagedTimeUnifiedStateImpl
Buckets Users specify buckets Users specify buckets Users specify time properties which are used
to create buckets.
Example:
bucketSpan = 30 minutes
expireBefore = 60 minutes
referenceInstant = now, then
Number of buckets = 60/30 = 2
Data on Disk A bucket data is partitioned
into time-buckets.
Time-buckets are derived
using processing time.
A bucket data is partitioned
into time-buckets.
Time-buckets are derived
using event time.
In this implementation a bucket is already a
time-bucket so it is not partitioned further on
disk.
Operator
Partitioning
A bucket belongs to a single
partition. Multiple partitions
cannot write to the same
bucket.
Same as ManagedStateImpl Multiple partitions can write to the same
time-bucket. On the disk each partition’s data
is segregated by the operator id.
Spillable Data Structures
Why Spillable Data Structures?
store.put(0L, new Slice(keyBytes), new Slice(valueBytes));
valueSlice = store.getSync(0L, new Slice(keyBytes));
● More cognitive load to worry about the details of storing data.
● We are used to working with Maps, Lists, and Sets.
● But we can’t work with simple in memory data structures.
● We need to decouple data from how we serialize and deserialize it.
Spillable Data Structures Architecture
● Spillable Data Structures
are created by a factory
● Backend store is
pluggable
● The factory has an Id
Generator, which
generates a unique Id
(key prefix) for each
Spillable Data Structure
● Serializer and deserializer
a configured for each data
structure individually
public class MyOperator implements Operator {
private SpillableStateStore store;
private SpillableComplexComponent spillableComplexComponent;
private Spillable.SpillableByteMap<String, String> mapString = null;
public final transient DefaultInputPort<String> inputPort = new DefaultInputPort<String>() {
public void process(String event) { /* Custom event processing logic */ }
}
public void setup(Context context) {
if (spillableComplexComponent == null) {
spillableComplexComponent = new SpillableComplexComponentImpl(store);
mapString = spillableComplexComponent.newSpillableByteMap(0, new StringSerde(), new StringSerde());
}
spillableComplexComponent.setup(context);
}
public void beginWindow(long windowId) { spillableComplexComponent.beginWindow(windowId); }
public void endWindow() { spillableComplexComponent.endWindow(); }
public void teardown() { spillableComplexComponent.teardown(); }
// Some other checkpointed callbacks need to be overridden and called on spillableComplexComponent, but are omitted
for shortness.
public void setStore(SpillableStateStore store) { this.store = Preconditions.checkNotNull(store); }
public SpillableStateStore getStore() { return store; }}
Spillable Data Structures Usage
Building a Map on top Of Managed State
// Psuedo code
public static class SpillableMap<K, V> implements Map<K, V> {
private ManagedState store;
private Serde<K> serdeKey;
private Serde<V> serdeValue;
public SpillableMap(ManagedState store, Serde<K> serdeKey, Serde<V> serdeValue) {
this.store = store;
this.serdeKey = serdeKey;
this.serdeValue = serdeValue;
}
public V get(K key) {
byte[] keyBytes = serdeKey.serialize(key)
byte[] valueBytes = store.getSync(0L, new Slice(keyBytes)).toByteArray()
return serdeValue.deserialize(valueBytes);
}
public void put(K key, V value) { /* code similar to above */ }
}
What If I Wanted To Store Multiple Maps?
Key collisions for multiple maps
Handling Multiple Maps (And Data-structures)
Keys have a fixed bit-width prefix
Implementing ArrayLists
Index keys are 4 bytes wide
Implementing an ArrayListMultimap
Implementing a Linked List
Implementing An Iterable Set
Caching Strategy
Simple write and read through cache is kept in memory.
Implementations For Apache Apex
● SpillableMap:https://github.com/apache/apex-malhar/blob/master/library/src/main/java/org/apache/
apex/malhar/lib/state/spillable/SpillableMapImpl.java
● SpillablArrayList:https://github.com/apache/apex-malhar/blob/master/library/src/main/java/org/apac
he/apex/malhar/lib/state/spillable/SpillableArrayListImpl.java
● SpillableArrayListMultimap:https://github.com/apache/apex-malhar/blob/master/library/src/main/ja
va/org/apache/apex/malhar/lib/state/spillable/SpillableArrayListMultimapImpl.java
● SpillableSetImpl:https://github.com/apache/apex-malhar/blob/master/library/src/main/java/org/apac
he/apex/malhar/lib/state/spillable/SpillableSetImpl.java
● SpillableFactory:https://github.com/apache/apex-malhar/blob/master/library/src/main/java/org/apac
he/apex/malhar/lib/state/spillable/SpillableComplexComponentImpl.java
Spillable Data Structures In Action
We use them at Simplifi.it to run a Data Aggregation Pipeline built on Apache Apex.
Questions?

More Related Content

What's hot

Hadoop institutes in Bangalore
Hadoop institutes in BangaloreHadoop institutes in Bangalore
Hadoop institutes in Bangaloresrikanthhadoop
 
Skytools: PgQ Queues and applications
Skytools: PgQ Queues and applicationsSkytools: PgQ Queues and applications
Skytools: PgQ Queues and applicationselliando dias
 
Parallel programming patterns - Олександр Павлишак
Parallel programming patterns - Олександр ПавлишакParallel programming patterns - Олександр Павлишак
Parallel programming patterns - Олександр ПавлишакIgor Bronovskyy
 
PgQ Generic high-performance queue for PostgreSQL
PgQ Generic high-performance queue for PostgreSQLPgQ Generic high-performance queue for PostgreSQL
PgQ Generic high-performance queue for PostgreSQLelliando dias
 
MySQL Time Machine by replicating into HBase - Slides from Percona Live Amste...
MySQL Time Machine by replicating into HBase - Slides from Percona Live Amste...MySQL Time Machine by replicating into HBase - Slides from Percona Live Amste...
MySQL Time Machine by replicating into HBase - Slides from Percona Live Amste...Boško Devetak
 
Operating Systems - Process Synchronization and Deadlocks
Operating Systems - Process Synchronization and DeadlocksOperating Systems - Process Synchronization and Deadlocks
Operating Systems - Process Synchronization and DeadlocksMukesh Chinta
 
Cluster Dynamics in Solr Autoscaling - Andrzej Białecki, Lucidworks
Cluster Dynamics in Solr Autoscaling - Andrzej Białecki, LucidworksCluster Dynamics in Solr Autoscaling - Andrzej Białecki, Lucidworks
Cluster Dynamics in Solr Autoscaling - Andrzej Białecki, LucidworksLucidworks
 
Effective testing for spark programs Strata NY 2015
Effective testing for spark programs   Strata NY 2015Effective testing for spark programs   Strata NY 2015
Effective testing for spark programs Strata NY 2015Holden Karau
 
Process Synchronization And Deadlocks
Process Synchronization And DeadlocksProcess Synchronization And Deadlocks
Process Synchronization And Deadlockstech2click
 
Flink Forward Berlin 2017: Stefan Richter - A look at Flink's internal data s...
Flink Forward Berlin 2017: Stefan Richter - A look at Flink's internal data s...Flink Forward Berlin 2017: Stefan Richter - A look at Flink's internal data s...
Flink Forward Berlin 2017: Stefan Richter - A look at Flink's internal data s...Flink Forward
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured StreamingKnoldus Inc.
 
Think of oracle and mysql bind value
Think of oracle and mysql bind value Think of oracle and mysql bind value
Think of oracle and mysql bind value Louis liu
 
Flink Batch Processing and Iterations
Flink Batch Processing and IterationsFlink Batch Processing and Iterations
Flink Batch Processing and IterationsSameer Wadkar
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...Databricks
 
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...Flink Forward
 
The Newest in Session Types
The Newest in Session TypesThe Newest in Session Types
The Newest in Session TypesRoland Kuhn
 

What's hot (20)

Hadoop institutes in Bangalore
Hadoop institutes in BangaloreHadoop institutes in Bangalore
Hadoop institutes in Bangalore
 
Skytools: PgQ Queues and applications
Skytools: PgQ Queues and applicationsSkytools: PgQ Queues and applications
Skytools: PgQ Queues and applications
 
Parallel programming patterns - Олександр Павлишак
Parallel programming patterns - Олександр ПавлишакParallel programming patterns - Олександр Павлишак
Parallel programming patterns - Олександр Павлишак
 
PgQ Generic high-performance queue for PostgreSQL
PgQ Generic high-performance queue for PostgreSQLPgQ Generic high-performance queue for PostgreSQL
PgQ Generic high-performance queue for PostgreSQL
 
MySQL Time Machine by replicating into HBase - Slides from Percona Live Amste...
MySQL Time Machine by replicating into HBase - Slides from Percona Live Amste...MySQL Time Machine by replicating into HBase - Slides from Percona Live Amste...
MySQL Time Machine by replicating into HBase - Slides from Percona Live Amste...
 
JavaCro'15 - Spring @Async - Dragan Juričić
JavaCro'15 - Spring @Async - Dragan JuričićJavaCro'15 - Spring @Async - Dragan Juričić
JavaCro'15 - Spring @Async - Dragan Juričić
 
Operating Systems - Process Synchronization and Deadlocks
Operating Systems - Process Synchronization and DeadlocksOperating Systems - Process Synchronization and Deadlocks
Operating Systems - Process Synchronization and Deadlocks
 
Spark streaming: Best Practices
Spark streaming: Best PracticesSpark streaming: Best Practices
Spark streaming: Best Practices
 
Cluster Dynamics in Solr Autoscaling - Andrzej Białecki, Lucidworks
Cluster Dynamics in Solr Autoscaling - Andrzej Białecki, LucidworksCluster Dynamics in Solr Autoscaling - Andrzej Białecki, Lucidworks
Cluster Dynamics in Solr Autoscaling - Andrzej Białecki, Lucidworks
 
Effective testing for spark programs Strata NY 2015
Effective testing for spark programs   Strata NY 2015Effective testing for spark programs   Strata NY 2015
Effective testing for spark programs Strata NY 2015
 
Process Synchronization And Deadlocks
Process Synchronization And DeadlocksProcess Synchronization And Deadlocks
Process Synchronization And Deadlocks
 
Flink Forward Berlin 2017: Stefan Richter - A look at Flink's internal data s...
Flink Forward Berlin 2017: Stefan Richter - A look at Flink's internal data s...Flink Forward Berlin 2017: Stefan Richter - A look at Flink's internal data s...
Flink Forward Berlin 2017: Stefan Richter - A look at Flink's internal data s...
 
Introduction to Structured Streaming
Introduction to Structured StreamingIntroduction to Structured Streaming
Introduction to Structured Streaming
 
Final_Presentation_Docker_KP
Final_Presentation_Docker_KPFinal_Presentation_Docker_KP
Final_Presentation_Docker_KP
 
Think of oracle and mysql bind value
Think of oracle and mysql bind value Think of oracle and mysql bind value
Think of oracle and mysql bind value
 
Flink Batch Processing and Iterations
Flink Batch Processing and IterationsFlink Batch Processing and Iterations
Flink Batch Processing and Iterations
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...
 
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
 
Deadlocks
DeadlocksDeadlocks
Deadlocks
 
The Newest in Session Types
The Newest in Session TypesThe Newest in Session Types
The Newest in Session Types
 

Similar to Stateful streaming data pipelines

Stream processing - Apache flink
Stream processing - Apache flinkStream processing - Apache flink
Stream processing - Apache flinkRenato Guimaraes
 
Introduction to Apache Apex - CoDS 2016
Introduction to Apache Apex - CoDS 2016Introduction to Apache Apex - CoDS 2016
Introduction to Apache Apex - CoDS 2016Bhupesh Chawda
 
Real-time Stream Processing using Apache Apex
Real-time Stream Processing using Apache ApexReal-time Stream Processing using Apache Apex
Real-time Stream Processing using Apache ApexApache Apex
 
Container Orchestration from Theory to Practice
Container Orchestration from Theory to PracticeContainer Orchestration from Theory to Practice
Container Orchestration from Theory to PracticeDocker, Inc.
 
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
 
Container orchestration from theory to practice
Container orchestration from theory to practiceContainer orchestration from theory to practice
Container orchestration from theory to practiceDocker, Inc.
 
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)Apache Apex
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformApache Apex
 
Actionable Insights with Apache Apex at Apache Big Data 2017 by Devendra Tagare
Actionable Insights with Apache Apex at Apache Big Data 2017 by Devendra TagareActionable Insights with Apache Apex at Apache Big Data 2017 by Devendra Tagare
Actionable Insights with Apache Apex at Apache Big Data 2017 by Devendra TagareApache Apex
 
Unified stateful big data processing in Apache Beam (incubating)
Unified stateful big data processing in Apache Beam (incubating)Unified stateful big data processing in Apache Beam (incubating)
Unified stateful big data processing in Apache Beam (incubating)Aljoscha Krettek
 
Aljoscha Krettek - Portable stateful big data processing in Apache Beam
Aljoscha Krettek - Portable stateful big data processing in Apache BeamAljoscha Krettek - Portable stateful big data processing in Apache Beam
Aljoscha Krettek - Portable stateful big data processing in Apache BeamVerverica
 
Apache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data 2016: Next Gen Big Data Analytics with Apache ApexApache Apex
 
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Apex
 
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...Flink Forward
 
Introduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas WeiseIntroduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas WeiseBig Data Spain
 
App Grid Dev With Coherence
App Grid Dev With CoherenceApp Grid Dev With Coherence
App Grid Dev With CoherenceJames Bayer
 
Application Grid Dev with Coherence
Application Grid Dev with CoherenceApplication Grid Dev with Coherence
Application Grid Dev with CoherenceJames Bayer
 
App Grid Dev With Coherence
App Grid Dev With CoherenceApp Grid Dev With Coherence
App Grid Dev With CoherenceJames Bayer
 

Similar to Stateful streaming data pipelines (20)

Stream processing - Apache flink
Stream processing - Apache flinkStream processing - Apache flink
Stream processing - Apache flink
 
Introduction to Apache Apex - CoDS 2016
Introduction to Apache Apex - CoDS 2016Introduction to Apache Apex - CoDS 2016
Introduction to Apache Apex - CoDS 2016
 
Real-time Stream Processing using Apache Apex
Real-time Stream Processing using Apache ApexReal-time Stream Processing using Apache Apex
Real-time Stream Processing using Apache Apex
 
Container Orchestration from Theory to Practice
Container Orchestration from Theory to PracticeContainer Orchestration from Theory to Practice
Container Orchestration from Theory to Practice
 
Apache flink
Apache flinkApache flink
Apache flink
 
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
 
G pars
G parsG pars
G pars
 
Container orchestration from theory to practice
Container orchestration from theory to practiceContainer orchestration from theory to practice
Container orchestration from theory to practice
 
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)
GE IOT Predix Time Series & Data Ingestion Service using Apache Apex (Hadoop)
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
 
Actionable Insights with Apache Apex at Apache Big Data 2017 by Devendra Tagare
Actionable Insights with Apache Apex at Apache Big Data 2017 by Devendra TagareActionable Insights with Apache Apex at Apache Big Data 2017 by Devendra Tagare
Actionable Insights with Apache Apex at Apache Big Data 2017 by Devendra Tagare
 
Unified stateful big data processing in Apache Beam (incubating)
Unified stateful big data processing in Apache Beam (incubating)Unified stateful big data processing in Apache Beam (incubating)
Unified stateful big data processing in Apache Beam (incubating)
 
Aljoscha Krettek - Portable stateful big data processing in Apache Beam
Aljoscha Krettek - Portable stateful big data processing in Apache BeamAljoscha Krettek - Portable stateful big data processing in Apache Beam
Aljoscha Krettek - Portable stateful big data processing in Apache Beam
 
Apache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
 
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
 
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
Flink Forward SF 2017: Joe Olson - Using Flink and Queryable State to Buffer ...
 
Introduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas WeiseIntroduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas Weise
 
App Grid Dev With Coherence
App Grid Dev With CoherenceApp Grid Dev With Coherence
App Grid Dev With Coherence
 
Application Grid Dev with Coherence
Application Grid Dev with CoherenceApplication Grid Dev with Coherence
Application Grid Dev with Coherence
 
App Grid Dev With Coherence
App Grid Dev With CoherenceApp Grid Dev With Coherence
App Grid Dev With Coherence
 

Recently uploaded

Full night 🥵 Call Girls Delhi New Friends Colony {9711199171} Sanya Reddy ✌️o...
Full night 🥵 Call Girls Delhi New Friends Colony {9711199171} Sanya Reddy ✌️o...Full night 🥵 Call Girls Delhi New Friends Colony {9711199171} Sanya Reddy ✌️o...
Full night 🥵 Call Girls Delhi New Friends Colony {9711199171} Sanya Reddy ✌️o...shivangimorya083
 
RadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfRadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfgstagge
 
20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdfHuman37
 
Call Girls In Mahipalpur O9654467111 Escorts Service
Call Girls In Mahipalpur O9654467111  Escorts ServiceCall Girls In Mahipalpur O9654467111  Escorts Service
Call Girls In Mahipalpur O9654467111 Escorts ServiceSapana Sha
 
Spark3's new memory model/management
Spark3's new memory model/managementSpark3's new memory model/management
Spark3's new memory model/managementakshesh doshi
 
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...dajasot375
 
Unveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystUnveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystSamantha Rae Coolbeth
 
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.pptdokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.pptSonatrach
 
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Callshivangimorya083
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfLars Albertsson
 
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130Suhani Kapoor
 
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...Suhani Kapoor
 
Industrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfIndustrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfLars Albertsson
 
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfSocial Samosa
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptxAnupama Kate
 
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Callshivangimorya083
 
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...Suhani Kapoor
 

Recently uploaded (20)

Full night 🥵 Call Girls Delhi New Friends Colony {9711199171} Sanya Reddy ✌️o...
Full night 🥵 Call Girls Delhi New Friends Colony {9711199171} Sanya Reddy ✌️o...Full night 🥵 Call Girls Delhi New Friends Colony {9711199171} Sanya Reddy ✌️o...
Full night 🥵 Call Girls Delhi New Friends Colony {9711199171} Sanya Reddy ✌️o...
 
RadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdfRadioAdProWritingCinderellabyButleri.pdf
RadioAdProWritingCinderellabyButleri.pdf
 
20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf20240419 - Measurecamp Amsterdam - SAM.pdf
20240419 - Measurecamp Amsterdam - SAM.pdf
 
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in Kishangarh
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in  KishangarhDelhi 99530 vip 56974 Genuine Escort Service Call Girls in  Kishangarh
Delhi 99530 vip 56974 Genuine Escort Service Call Girls in Kishangarh
 
Call Girls In Mahipalpur O9654467111 Escorts Service
Call Girls In Mahipalpur O9654467111  Escorts ServiceCall Girls In Mahipalpur O9654467111  Escorts Service
Call Girls In Mahipalpur O9654467111 Escorts Service
 
Spark3's new memory model/management
Spark3's new memory model/managementSpark3's new memory model/management
Spark3's new memory model/management
 
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
Indian Call Girls in Abu Dhabi O5286O24O8 Call Girls in Abu Dhabi By Independ...
 
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
Deep Generative Learning for All - The Gen AI Hype (Spring 2024)
 
Unveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data AnalystUnveiling Insights: The Role of a Data Analyst
Unveiling Insights: The Role of a Data Analyst
 
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.pptdokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
dokumen.tips_chapter-4-transient-heat-conduction-mehmet-kanoglu.ppt
 
VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...
VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...
VIP Call Girls Service Charbagh { Lucknow Call Girls Service 9548273370 } Boo...
 
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip CallDelhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
Delhi Call Girls Punjabi Bagh 9711199171 ☎✔👌✔ Whatsapp Hard And Sexy Vip Call
 
Schema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdfSchema on read is obsolete. Welcome metaprogramming..pdf
Schema on read is obsolete. Welcome metaprogramming..pdf
 
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
VIP Call Girls Service Miyapur Hyderabad Call +91-8250192130
 
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
VIP High Profile Call Girls Amravati Aarushi 8250192130 Independent Escort Se...
 
Industrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdfIndustrialised data - the key to AI success.pdf
Industrialised data - the key to AI success.pdf
 
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdfKantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
Kantar AI Summit- Under Embargo till Wednesday, 24th April 2024, 4 PM, IST.pdf
 
100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx100-Concepts-of-AI by Anupama Kate .pptx
100-Concepts-of-AI by Anupama Kate .pptx
 
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
꧁❤ Greater Noida Call Girls Delhi ❤꧂ 9711199171 ☎️ Hard And Sexy Vip Call
 
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
VIP High Class Call Girls Jamshedpur Anushka 8250192130 Independent Escort Se...
 

Stateful streaming data pipelines

  • 1. Stateful Streaming Data Pipelines with Apache Apex Chandni Singh PMC and Committer, Apache Apex Founder, Simplifi.it Timothy Farkas Committer, Apache Apex Founder, Simplifi.it
  • 2. Agenda ● Introduction to Apache Apex ● Managed State ● Spillable Data-structures ● Questions
  • 3. What is Apache Apex ● Distributed data processing engine ● Runs on Hadoop ● Real-time streaming ● Fault-tolerant
  • 4. Anatomy of An Apex Application ● Tuple: Discrete unit of information sent from one operator to another. ● Operator: Java code that performs an operation on tuples. The code runs in a yarn container on a yarn cluster. ● DAG: Operators can be connected to form an application. Tuple transfer between operators is 1-way, so the application forms a Directed Acyclic Graph. ● Window Marker: An Id that is associated with tuples and operators, and is used for fault-tolerance.
  • 5. Anatomy of An Apex Operator public class MyOperator implements Operator { private Map<String, String> inMemState = new HashMap<>(); // checkpointed in memory state private int myProperty; public final transient DefaultInputPort<String> inputPort = new DefaultInputPort<String>() { public void process(String event) { // Custom event processing logic } } public void setup(Context context) { // One time setup tasks to be performed when the operator first starts } public void beginWindow(long windowId) { // Next window has started } public void endWindow() { } public void teardown() { // Operator is shutting down. Any cleanup needs to be done here. } public void setMyProperty(int myProperty) { this.myProperty = myProperty } public int getMyProperty() { return myProperty} }
  • 6. Fault tolerance in Apex ● Apex inserts window markers with IDs in the data stream, which operators are notified of. ● It provides fault-tolerance by checkpointing the state of every operator in the pipeline every N windows. ● If an operator crashes, it restores the operator with the state corresponding to a checkpointed window. ● Committed window: In the simple case, when all operators are checkpointed at the same frequency, committed window is the latest window which has been checkpointed by all the operators in the DAG.
  • 7. What is the problem? ● Time to checkpoint ∝ size of operator state ● With increasing state, the operator will eventually crash. ● Even before the operator crashes, the platform may assume that the operator is unresponsive and instruct Yarn to kill it.
  • 8. Managed State - Introduction A reusable component that can be added to any operator to manage its key/value state. ● Checkpoints key/value state incrementally. ● Allows to set a threshold on the size of data in memory. Data that has been persisted, is off-loaded from memory when the threshold is reached. ● Keys can be partitioned in user-defined buckets which helps with operator partitioning and efficient off-loading from memory. ● Key/values are persisted on hdfs in a state that is optimized for querying. ● Purges stale data from disk.
  • 9. Managed State API ● Write to managed state managedState.put(1L, key, value) ● Read from managed state managedState.getSync(1L, key) managedState.getAsync(1L, key)
  • 10. Architecture For simplicity, in the following examples we will use window Ids for time buckets because window Ids roughly correspond to processing time.
  • 12. Writes to Managed State ● Key/Values are put in the bucket cache. ● At checkpoints, data from the bucket cache is moved to checkpoint cache which is written to WAL. ● When a window is committed, data in the WAL till the current committed window is transferred to key/value store which is the Bucket File System.
  • 13. Writes to Managed State - Continued
  • 14. Purging of Data Delete time-buckets older than 2 days. 2 days are approximately equivalent to 5760 windows.
  • 15. Fault-tolerance in Managed State Scenario 1: Operator failure
  • 16. Fault-tolerance in Managed State Scenario 2: Transferring data from WAL to Bucket File System
  • 17. Implementations of Managed State ManagedStateImpl ManagedTimeStateImpl ManagedTimeUnifiedStateImpl Buckets Users specify buckets Users specify buckets Users specify time properties which are used to create buckets. Example: bucketSpan = 30 minutes expireBefore = 60 minutes referenceInstant = now, then Number of buckets = 60/30 = 2 Data on Disk A bucket data is partitioned into time-buckets. Time-buckets are derived using processing time. A bucket data is partitioned into time-buckets. Time-buckets are derived using event time. In this implementation a bucket is already a time-bucket so it is not partitioned further on disk. Operator Partitioning A bucket belongs to a single partition. Multiple partitions cannot write to the same bucket. Same as ManagedStateImpl Multiple partitions can write to the same time-bucket. On the disk each partition’s data is segregated by the operator id.
  • 19. Why Spillable Data Structures? store.put(0L, new Slice(keyBytes), new Slice(valueBytes)); valueSlice = store.getSync(0L, new Slice(keyBytes)); ● More cognitive load to worry about the details of storing data. ● We are used to working with Maps, Lists, and Sets. ● But we can’t work with simple in memory data structures. ● We need to decouple data from how we serialize and deserialize it.
  • 20. Spillable Data Structures Architecture ● Spillable Data Structures are created by a factory ● Backend store is pluggable ● The factory has an Id Generator, which generates a unique Id (key prefix) for each Spillable Data Structure ● Serializer and deserializer a configured for each data structure individually
  • 21. public class MyOperator implements Operator { private SpillableStateStore store; private SpillableComplexComponent spillableComplexComponent; private Spillable.SpillableByteMap<String, String> mapString = null; public final transient DefaultInputPort<String> inputPort = new DefaultInputPort<String>() { public void process(String event) { /* Custom event processing logic */ } } public void setup(Context context) { if (spillableComplexComponent == null) { spillableComplexComponent = new SpillableComplexComponentImpl(store); mapString = spillableComplexComponent.newSpillableByteMap(0, new StringSerde(), new StringSerde()); } spillableComplexComponent.setup(context); } public void beginWindow(long windowId) { spillableComplexComponent.beginWindow(windowId); } public void endWindow() { spillableComplexComponent.endWindow(); } public void teardown() { spillableComplexComponent.teardown(); } // Some other checkpointed callbacks need to be overridden and called on spillableComplexComponent, but are omitted for shortness. public void setStore(SpillableStateStore store) { this.store = Preconditions.checkNotNull(store); } public SpillableStateStore getStore() { return store; }} Spillable Data Structures Usage
  • 22. Building a Map on top Of Managed State // Psuedo code public static class SpillableMap<K, V> implements Map<K, V> { private ManagedState store; private Serde<K> serdeKey; private Serde<V> serdeValue; public SpillableMap(ManagedState store, Serde<K> serdeKey, Serde<V> serdeValue) { this.store = store; this.serdeKey = serdeKey; this.serdeValue = serdeValue; } public V get(K key) { byte[] keyBytes = serdeKey.serialize(key) byte[] valueBytes = store.getSync(0L, new Slice(keyBytes)).toByteArray() return serdeValue.deserialize(valueBytes); } public void put(K key, V value) { /* code similar to above */ } }
  • 23. What If I Wanted To Store Multiple Maps? Key collisions for multiple maps
  • 24. Handling Multiple Maps (And Data-structures) Keys have a fixed bit-width prefix
  • 29. Caching Strategy Simple write and read through cache is kept in memory.
  • 30. Implementations For Apache Apex ● SpillableMap:https://github.com/apache/apex-malhar/blob/master/library/src/main/java/org/apache/ apex/malhar/lib/state/spillable/SpillableMapImpl.java ● SpillablArrayList:https://github.com/apache/apex-malhar/blob/master/library/src/main/java/org/apac he/apex/malhar/lib/state/spillable/SpillableArrayListImpl.java ● SpillableArrayListMultimap:https://github.com/apache/apex-malhar/blob/master/library/src/main/ja va/org/apache/apex/malhar/lib/state/spillable/SpillableArrayListMultimapImpl.java ● SpillableSetImpl:https://github.com/apache/apex-malhar/blob/master/library/src/main/java/org/apac he/apex/malhar/lib/state/spillable/SpillableSetImpl.java ● SpillableFactory:https://github.com/apache/apex-malhar/blob/master/library/src/main/java/org/apac he/apex/malhar/lib/state/spillable/SpillableComplexComponentImpl.java
  • 31. Spillable Data Structures In Action We use them at Simplifi.it to run a Data Aggregation Pipeline built on Apache Apex.