SlideShare a Scribd company logo
1 of 31
Download to read offline
Developing Streaming Applications with
Apache Apex
David Yan <davidyan@apache.org>
PMC Member, Apache Apex
Software Engineer, Google
Strata+Hadoop World, San Jose, CA
March 16, 2017
Agenda
● Technical Overview of Apache Apex as a streaming
platform
● Developing an Application Pipeline
● Developing an Operator, and the Operator Library
● Event Time Windowing
● Debugging, Scaling and Tuning
● Q & A
What is Apache Apex? Why?
● A platform for streaming applications
● Written in Java
● Uses YARN for resource management and HDFS for storage
● Has a comprehensive operator library.
● Scalable, fault tolerant, high throughput and low latency
● Provides low-level control for performance tuning
● Users include GE (Predix), Capital One, Royal Bank of Canada,
Pubmatic, SilverSpring Network, etc.
○ (more at https://apex.apache.org/powered-by-apex.html)
Technical
Overview
Application Development Model
● Directed Acyclic Graph (DAG)
● A stream is a sequence of data tuples
● An operator takes one or more input streams, performs computations, emits
one or more output streams
Filtered
Stream
Output
Stream
Tuple Tuple
FilteredStream
Enriched
Stream
Enriched
Stream
er
Operator
er
Operator
er
Operator
er
Operator
er
Operator
er
Operator
From Logical DAG to Actual Work
● CLI issues a launch command with
App Package
● YARN starts up Application Master
● Logical DAG is translated to
Physical DAG
Apex CLI
YARN
RM
NM NM NM NM
AM
1 2
3
4
5 6Input Filter Transform Output
From Logical DAG to Actual Work
(cont'd)
● AM requests worker containers
from YARN to run physical
operators
● Worker Containers send data
using a pub-sub mechanism
Apex CLI
YARN
RM
AM Worker WorkerWorker
Worker
Worker
6
4
1
3
25
1 2
3
4
5 6
HDFS
PubSub
overflow &
Checkpoints
Streaming Windows & Checkpointing
● Stream is divided into fixed time slices
called streaming windows
● Checkpoint is performed by Worker
Containers at streaming window
boundaries
● Worker Containers send heartbeats to
AM
● Recovery is incremental without resetting
full DAG
● Checkpoints are purged after the
corresponding window is committed
● AM is also checkpointed
BeginWindow nEndWindow n
BeginWindow n+1
EndWindow n+1
Time
......
Bookkeeping &
Checkpointing done
here
Writing
an Apex Application
Pipeline
Writing an Apex Application Pipeline
● Creating a project: Maven archetype
● Two APIs to specify an Apex pipeline
○ Compositional
○ Declarative
Compositional API
Input Parser Counter Output
CountsWordsLines
Kafka Database
Filter
Filtered
Declarative API
StreamFactory.fromKafka09(brokers, topic)
.flatMap(input -> Arrays.asList(input.split("s+")))
.filter(input -> filterWords.contains(input))
.window(new WindowOption.GlobalWindow(),
new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1))
.sumByKey(input -> new Tuple.PlainTuple<>(new KeyValPair<>(input, 1L)))
.map(input -> input.getValue())
.endWith(new JdbcOutput())
.populateDag(dag);
Input Parser Counter Output
CountsWordsLines
Kafka Database
Filter
Filtered
Operators
Operator API
Next
streaming
window
Next
streaming
window
Input Adapters - Starting of the pipeline. Interacts with external system to generate stream
Generic Operators - Processing part of pipeline
Output Adapters - Last operator in pipeline. Interacts with external system to finalize the processed stream
OutputPort::emit()
Operator Example (Stateless)
Operator Example (Stateful)
RDBMS
• JDBC
• MySQL
• Oracle
• MemSQL
NoSQL
• Cassandra, HBase
• Aerospike, Accumulo
• Couchbase/ CouchDB
• Redis, MongoDB
• Geode
Messaging
• Kafka
• JMS (ActiveMQ etc.)
• Kinesis, SQS
• Flume, NiFi
• MQTT
File Systems
• HDFS/ Hive
• Local File
• S3
Parsers
• XML
• JSON
• CSV
• Avro
• Parquet
Transformations
• Filters, Expression, Enrich
• Windowing, Aggregation
• Join
• Dedup
Analytics
• Dimensional Aggregations
(with state management for
historical data + query)
Protocols
• HTTP
• FTP
• WebSocket
• SMTP
Other
• Elastic Search
• Script (JavaScript, Python, R)
• Solr
• Twitter
Apex "Malhar" Operator Library
Queryable State
A set of operators in the library that supports real-time queries and updates of
operator state.
HashTag
Extractor
Top Counter
Twitter Feed
Input Operator
Unique
Counter
Query
Result
PubSub
Broker
Query
Input
Queryable State (cont'd)
Event Time
Windowing
What, Why, and How?
● Event Time Windows not to be confused with "streaming windows"
● Time of the event vs processing time and ingression time
● Data often arrives late and out of order
● Concepts outlined by Google's Millwheel and Apache Beam: Watermarks,
Allowed Lateness, Accumulation Modes, and Triggers
Example
StreamFactory.fromFolder("/tmp")
.flatMap(input -> Arrays.asList(input.split( "s+")))
.map(input -> new TimestampedTuple<>(System.currentTimeMillis(), input))
.window(new TimeWindows(Duration.standardMinutes(5)),
TriggerOption.AtWatermark()
.accumulatingFiredPanes()
.withEarlyFiringsAtEvery(Duration.standardSeconds(1)),
Duration.standardSeconds(15))
.sumByKey(input -> new TimestampedTuple<>(input.getTimestamp(), new KeyValPair<>(input.getValue(),
1L ))))
.map(new FormatAsTableRowFn()) // format for printing
.print()
.populateDag(dag);
Debugging, Scaling
& Tuning Your
Application
Debugging
● Logs
● Local Mode
● Remote JVM Attachment
● Tuple Recording
Scaling: Partitioning
Partitioning with Unifiers: NxM Partitioning:
0 1 2
0
1
a
1
b
1
c
U 2
Logical DAG
Physical DAG with operator 1 with 3 partitions
0 21 3
0
2
a
1
b 3
1
a
1
c
2
b
U U
0
2
a
1
b 3
1
a
1
c
2
b
U1
U
U2
Scaling: Partitioning (cont'd)
Parallel Partition: Cascading Unifiers:
0
1
a
1
b
U 2 3 4
0
1
a
1
b
U 4
2
a
3
a
2
b
3
b
1 2
1
1
1
1
U 2
1
1
1
1
U
1
2
U
2
U
3
Dynamic Partitioning
● Partitioning change while
application is running
○ Change number of partitions at
runtime based on stats
○ Supports re-distribution of state when
number of partitions change
○ API for custom scaler or partitioner
StatelessThroughputBasedPartitioner<MyOperator>
partitioner
= new StatelessThroughputBasedPartitioner<>();
partitioner.setCooldownMillis(45000);
partitioner.setMaximumEvents(30000);
partitioner.setMinimumEvents(10000);
dag.setAttribute(op, OperatorContext.STATS_LISTENERS,
Arrays.asList(new StatsListener[]{partitioner}));
dag.setAttribute(op, OperatorContext.PARTITIONER,
partitioner);
Locality
● Node-Local: The two operators should be placed in the same node
● Container-Local: Same worker container (saves serialization)
● Thread-Local: Same thread (simple function call)
// setting locality between op1 and op2 to be NODE_LOCAL
dag.addStream( "streamName", op1.output, op2.input)
.setLocality(DAG.Locality.NODE_LOCAL);
Recent Additions and Roadmap
● Apex runner in Apache Beam
● Iterative processing
● Integrated with Apache Samoa, opens up ML
● Integrated with Apache Calcite, allows SQL
---------------------------------------------
● Enhanced support for Batch Processing
● Support for Mesos and Kubernetes
● Encrypted Streams
● Python API
Q & A
Resources
● http://apex.apache.org/
● Learn more: http://apex.apache.org/docs.html
● GitHub Repositories
○ http://github.com/apache/apex-core
○ http://github.com/apache/apex-malhar
● Subscribe - http://apex.apache.org/community.html
● Download - http://apex.apache.org/downloads.html
● Follow @ApacheApex - https://twitter.com/apacheapex
● Meetups – http://www.meetup.com/topics/apache-apex/
● Slideshare: http://www.slideshare.net/ApacheApex/presentations
● https://www.youtube.com/results?search_query=apache+apex

More Related Content

What's hot

Capital One's Next Generation Decision in less than 2 ms
Capital One's Next Generation Decision in less than 2 msCapital One's Next Generation Decision in less than 2 ms
Capital One's Next Generation Decision in less than 2 msApache 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
 
Introduction to Apache Apex
Introduction to Apache ApexIntroduction to Apache Apex
Introduction to Apache ApexApache Apex
 
Java High Level Stream API
Java High Level Stream APIJava High Level Stream API
Java High Level Stream APIApache Apex
 
Fault Tolerance and Processing Semantics in Apache Apex
Fault Tolerance and Processing Semantics in Apache ApexFault Tolerance and Processing Semantics in Apache Apex
Fault Tolerance and Processing Semantics in Apache ApexApache Apex Organizer
 
Building your first aplication using Apache Apex
Building your first aplication using Apache ApexBuilding your first aplication using Apache Apex
Building your first aplication using Apache ApexYogi Devendra Vyavahare
 
Intro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
Intro to Apache Apex (next gen Hadoop) & comparison to Spark StreamingIntro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
Intro to Apache Apex (next gen Hadoop) & comparison to Spark StreamingApache Apex
 
Intro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
Intro to Apache Apex - Next Gen Native Hadoop Platform - HackacIntro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
Intro to Apache Apex - Next Gen Native Hadoop Platform - HackacApache Apex
 
Smart Partitioning with Apache Apex (Webinar)
Smart Partitioning with Apache Apex (Webinar)Smart Partitioning with Apache Apex (Webinar)
Smart Partitioning with Apache Apex (Webinar)Apache Apex
 
Stream Processing with Apache Apex
Stream Processing with Apache ApexStream Processing with Apache Apex
Stream Processing with Apache ApexPramod Immaneni
 
Introduction to Apache Apex and writing a big data streaming application
Introduction to Apache Apex and writing a big data streaming application  Introduction to Apache Apex and writing a big data streaming application
Introduction to Apache Apex and writing a big data streaming application Apache Apex
 
Introduction to Apache Apex - CoDS 2016
Introduction to Apache Apex - CoDS 2016Introduction to Apache Apex - CoDS 2016
Introduction to Apache Apex - CoDS 2016Bhupesh Chawda
 
Introduction to Apache Apex
Introduction to Apache ApexIntroduction to Apache Apex
Introduction to Apache ApexApache Apex
 
Apache Apex: Stream Processing Architecture and Applications
Apache Apex: Stream Processing Architecture and ApplicationsApache Apex: Stream Processing Architecture and Applications
Apache Apex: Stream Processing Architecture and ApplicationsThomas Weise
 
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 Apex Introduction with PubMatic
Apache Apex Introduction with PubMaticApache Apex Introduction with PubMatic
Apache Apex Introduction with PubMaticApache Apex
 
Ingestion and Dimensions Compute and Enrich using Apache Apex
Ingestion and Dimensions Compute and Enrich using Apache ApexIngestion and Dimensions Compute and Enrich using Apache Apex
Ingestion and Dimensions Compute and Enrich using Apache ApexApache Apex
 
Apache Apex Fault Tolerance and Processing Semantics
Apache Apex Fault Tolerance and Processing SemanticsApache Apex Fault Tolerance and Processing Semantics
Apache Apex Fault Tolerance and Processing SemanticsApache Apex
 
Stream data from Apache Kafka for processing with Apache Apex
Stream data from Apache Kafka for processing with Apache ApexStream data from Apache Kafka for processing with Apache Apex
Stream data from Apache Kafka for processing with Apache ApexApache Apex
 

What's hot (20)

Capital One's Next Generation Decision in less than 2 ms
Capital One's Next Generation Decision in less than 2 msCapital One's Next Generation Decision in less than 2 ms
Capital One's Next Generation Decision in less than 2 ms
 
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
 
Introduction to Apache Apex
Introduction to Apache ApexIntroduction to Apache Apex
Introduction to Apache Apex
 
Java High Level Stream API
Java High Level Stream APIJava High Level Stream API
Java High Level Stream API
 
Fault Tolerance and Processing Semantics in Apache Apex
Fault Tolerance and Processing Semantics in Apache ApexFault Tolerance and Processing Semantics in Apache Apex
Fault Tolerance and Processing Semantics in Apache Apex
 
Building your first aplication using Apache Apex
Building your first aplication using Apache ApexBuilding your first aplication using Apache Apex
Building your first aplication using Apache Apex
 
Intro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
Intro to Apache Apex (next gen Hadoop) & comparison to Spark StreamingIntro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
Intro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
 
Intro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
Intro to Apache Apex - Next Gen Native Hadoop Platform - HackacIntro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
Intro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
 
Smart Partitioning with Apache Apex (Webinar)
Smart Partitioning with Apache Apex (Webinar)Smart Partitioning with Apache Apex (Webinar)
Smart Partitioning with Apache Apex (Webinar)
 
Stream Processing with Apache Apex
Stream Processing with Apache ApexStream Processing with Apache Apex
Stream Processing with Apache Apex
 
Introduction to Apache Apex and writing a big data streaming application
Introduction to Apache Apex and writing a big data streaming application  Introduction to Apache Apex and writing a big data streaming application
Introduction to Apache Apex and writing a big data streaming application
 
Introduction to Apache Apex - CoDS 2016
Introduction to Apache Apex - CoDS 2016Introduction to Apache Apex - CoDS 2016
Introduction to Apache Apex - CoDS 2016
 
Introduction to Apache Apex
Introduction to Apache ApexIntroduction to Apache Apex
Introduction to Apache Apex
 
Apache Apex: Stream Processing Architecture and Applications
Apache Apex: Stream Processing Architecture and ApplicationsApache Apex: Stream Processing Architecture and Applications
Apache Apex: Stream Processing Architecture and Applications
 
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 Apex Introduction with PubMatic
Apache Apex Introduction with PubMaticApache Apex Introduction with PubMatic
Apache Apex Introduction with PubMatic
 
Apex as yarn application
Apex as yarn applicationApex as yarn application
Apex as yarn application
 
Ingestion and Dimensions Compute and Enrich using Apache Apex
Ingestion and Dimensions Compute and Enrich using Apache ApexIngestion and Dimensions Compute and Enrich using Apache Apex
Ingestion and Dimensions Compute and Enrich using Apache Apex
 
Apache Apex Fault Tolerance and Processing Semantics
Apache Apex Fault Tolerance and Processing SemanticsApache Apex Fault Tolerance and Processing Semantics
Apache Apex Fault Tolerance and Processing Semantics
 
Stream data from Apache Kafka for processing with Apache Apex
Stream data from Apache Kafka for processing with Apache ApexStream data from Apache Kafka for processing with Apache Apex
Stream data from Apache Kafka for processing with Apache Apex
 

Similar to Developing streaming applications with apache apex (strata + hadoop world)

Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseKostas Tzoumas
 
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 Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and FriendsApache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and FriendsStephan Ewen
 
RxJava applied [JavaDay Kyiv 2016]
RxJava applied [JavaDay Kyiv 2016]RxJava applied [JavaDay Kyiv 2016]
RxJava applied [JavaDay Kyiv 2016]Igor Lozynskyi
 
Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Stephan Ewen
 
SamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationSamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationYi Pan
 
Apache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache ApexApache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache ApexApache Apex
 
Nextcon samza preso july - final
Nextcon samza preso   july - finalNextcon samza preso   july - final
Nextcon samza preso july - finalYi Pan
 
Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkDataWorks Summit
 
Test strategies for data processing pipelines, v2.0
Test strategies for data processing pipelines, v2.0Test strategies for data processing pipelines, v2.0
Test strategies for data processing pipelines, v2.0Lars Albertsson
 
Distributed Real-Time Stream Processing: Why and How 2.0
Distributed Real-Time Stream Processing:  Why and How 2.0Distributed Real-Time Stream Processing:  Why and How 2.0
Distributed Real-Time Stream Processing: Why and How 2.0Petr Zapletal
 
Apache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupApache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupRobert Metzger
 
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
 
Apache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault ToleranceApache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault ToleranceSachin Aggarwal
 
Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Anyscale
 
Binary Studio Academy: Concurrency in C# 5.0
Binary Studio Academy: Concurrency in C# 5.0Binary Studio Academy: Concurrency in C# 5.0
Binary Studio Academy: Concurrency in C# 5.0Binary Studio
 
Stateful Distributed Stream Processing
Stateful Distributed Stream ProcessingStateful Distributed Stream Processing
Stateful Distributed Stream ProcessingGyula Fóra
 

Similar to Developing streaming applications with apache apex (strata + hadoop world) (20)

Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San Jose
 
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
 
Apache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and FriendsApache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and Friends
 
RxJava applied [JavaDay Kyiv 2016]
RxJava applied [JavaDay Kyiv 2016]RxJava applied [JavaDay Kyiv 2016]
RxJava applied [JavaDay Kyiv 2016]
 
Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)
 
SamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentationSamzaSQL QCon'16 presentation
SamzaSQL QCon'16 presentation
 
Apache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache ApexApache Big Data EU 2016: Building Streaming Applications with Apache Apex
Apache Big Data EU 2016: Building Streaming Applications with Apache Apex
 
Nextcon samza preso july - final
Nextcon samza preso   july - finalNextcon samza preso   july - final
Nextcon samza preso july - final
 
Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache Flink
 
Test strategies for data processing pipelines, v2.0
Test strategies for data processing pipelines, v2.0Test strategies for data processing pipelines, v2.0
Test strategies for data processing pipelines, v2.0
 
Distributed Real-Time Stream Processing: Why and How 2.0
Distributed Real-Time Stream Processing:  Why and How 2.0Distributed Real-Time Stream Processing:  Why and How 2.0
Distributed Real-Time Stream Processing: Why and How 2.0
 
Apache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupApache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya Meetup
 
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...
 
Apache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault ToleranceApache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault Tolerance
 
Google cloud Dataflow & Apache Flink
Google cloud Dataflow & Apache FlinkGoogle cloud Dataflow & Apache Flink
Google cloud Dataflow & Apache Flink
 
Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0
 
Spark streaming: Best Practices
Spark streaming: Best PracticesSpark streaming: Best Practices
Spark streaming: Best Practices
 
Binary Studio Academy: Concurrency in C# 5.0
Binary Studio Academy: Concurrency in C# 5.0Binary Studio Academy: Concurrency in C# 5.0
Binary Studio Academy: Concurrency in C# 5.0
 
Stateful Distributed Stream Processing
Stateful Distributed Stream ProcessingStateful Distributed Stream Processing
Stateful Distributed Stream Processing
 
So you think you can stream.pptx
So you think you can stream.pptxSo you think you can stream.pptx
So you think you can stream.pptx
 

More from Apache Apex

Hadoop Interacting with HDFS
Hadoop Interacting with HDFSHadoop Interacting with HDFS
Hadoop Interacting with HDFSApache Apex
 
Introduction to Real-Time Data Processing
Introduction to Real-Time Data ProcessingIntroduction to Real-Time Data Processing
Introduction to Real-Time Data ProcessingApache Apex
 
Introduction to Yarn
Introduction to YarnIntroduction to Yarn
Introduction to YarnApache Apex
 
Introduction to Map Reduce
Introduction to Map ReduceIntroduction to Map Reduce
Introduction to Map ReduceApache Apex
 
Intro to Big Data Hadoop
Intro to Big Data HadoopIntro to Big Data Hadoop
Intro to Big Data HadoopApache Apex
 
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data TransformationsKafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data TransformationsApache Apex
 
Building Your First Apache Apex (Next Gen Big Data/Hadoop) Application
Building Your First Apache Apex (Next Gen Big Data/Hadoop) ApplicationBuilding Your First Apache Apex (Next Gen Big Data/Hadoop) Application
Building Your First Apache Apex (Next Gen Big Data/Hadoop) ApplicationApache Apex
 
Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)
Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)
Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)Apache Apex
 
Ingesting Data from Kafka to JDBC with Transformation and Enrichment
Ingesting Data from Kafka to JDBC with Transformation and EnrichmentIngesting Data from Kafka to JDBC with Transformation and Enrichment
Ingesting Data from Kafka to JDBC with Transformation and EnrichmentApache Apex
 
Big Data Berlin v8.0 Stream Processing with Apache Apex
Big Data Berlin v8.0 Stream Processing with Apache Apex Big Data Berlin v8.0 Stream Processing with Apache Apex
Big Data Berlin v8.0 Stream Processing with Apache Apex Apache Apex
 
Apache Beam (incubating)
Apache Beam (incubating)Apache Beam (incubating)
Apache Beam (incubating)Apache Apex
 
Making sense of Apache Bigtop's role in ODPi and how it matters to Apache Apex
Making sense of Apache Bigtop's role in ODPi and how it matters to Apache ApexMaking sense of Apache Bigtop's role in ODPi and how it matters to Apache Apex
Making sense of Apache Bigtop's role in ODPi and how it matters to Apache ApexApache Apex
 
Apache Apex & Bigtop
Apache Apex & BigtopApache Apex & Bigtop
Apache Apex & BigtopApache Apex
 
Building Your First Apache Apex Application
Building Your First Apache Apex ApplicationBuilding Your First Apache Apex Application
Building Your First Apache Apex ApplicationApache Apex
 

More from Apache Apex (15)

Hadoop Interacting with HDFS
Hadoop Interacting with HDFSHadoop Interacting with HDFS
Hadoop Interacting with HDFS
 
Introduction to Real-Time Data Processing
Introduction to Real-Time Data ProcessingIntroduction to Real-Time Data Processing
Introduction to Real-Time Data Processing
 
Introduction to Yarn
Introduction to YarnIntroduction to Yarn
Introduction to Yarn
 
Introduction to Map Reduce
Introduction to Map ReduceIntroduction to Map Reduce
Introduction to Map Reduce
 
HDFS Internals
HDFS InternalsHDFS Internals
HDFS Internals
 
Intro to Big Data Hadoop
Intro to Big Data HadoopIntro to Big Data Hadoop
Intro to Big Data Hadoop
 
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data TransformationsKafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
 
Building Your First Apache Apex (Next Gen Big Data/Hadoop) Application
Building Your First Apache Apex (Next Gen Big Data/Hadoop) ApplicationBuilding Your First Apache Apex (Next Gen Big Data/Hadoop) Application
Building Your First Apache Apex (Next Gen Big Data/Hadoop) Application
 
Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)
Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)
Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)
 
Ingesting Data from Kafka to JDBC with Transformation and Enrichment
Ingesting Data from Kafka to JDBC with Transformation and EnrichmentIngesting Data from Kafka to JDBC with Transformation and Enrichment
Ingesting Data from Kafka to JDBC with Transformation and Enrichment
 
Big Data Berlin v8.0 Stream Processing with Apache Apex
Big Data Berlin v8.0 Stream Processing with Apache Apex Big Data Berlin v8.0 Stream Processing with Apache Apex
Big Data Berlin v8.0 Stream Processing with Apache Apex
 
Apache Beam (incubating)
Apache Beam (incubating)Apache Beam (incubating)
Apache Beam (incubating)
 
Making sense of Apache Bigtop's role in ODPi and how it matters to Apache Apex
Making sense of Apache Bigtop's role in ODPi and how it matters to Apache ApexMaking sense of Apache Bigtop's role in ODPi and how it matters to Apache Apex
Making sense of Apache Bigtop's role in ODPi and how it matters to Apache Apex
 
Apache Apex & Bigtop
Apache Apex & BigtopApache Apex & Bigtop
Apache Apex & Bigtop
 
Building Your First Apache Apex Application
Building Your First Apache Apex ApplicationBuilding Your First Apache Apex Application
Building Your First Apache Apex Application
 

Recently uploaded

+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...Health
 
%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrand%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrandmasabamasaba
 
Devoxx UK 2024 - Going serverless with Quarkus, GraalVM native images and AWS...
Devoxx UK 2024 - Going serverless with Quarkus, GraalVM native images and AWS...Devoxx UK 2024 - Going serverless with Quarkus, GraalVM native images and AWS...
Devoxx UK 2024 - Going serverless with Quarkus, GraalVM native images and AWS...Bert Jan Schrijver
 
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...masabamasaba
 
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024VictoriaMetrics
 
The title is not connected to what is inside
The title is not connected to what is insideThe title is not connected to what is inside
The title is not connected to what is insideshinachiaurasa2
 
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...Steffen Staab
 
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfonteinmasabamasaba
 
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...Jittipong Loespradit
 
AI & Machine Learning Presentation Template
AI & Machine Learning Presentation TemplateAI & Machine Learning Presentation Template
AI & Machine Learning Presentation TemplatePresentation.STUDIO
 
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyviewmasabamasaba
 
WSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go PlatformlessWSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go PlatformlessWSO2
 
%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain
%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain
%in Bahrain+277-882-255-28 abortion pills for sale in Bahrainmasabamasaba
 
%in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park %in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park masabamasaba
 
Software Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsSoftware Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsArshad QA
 
%in Soweto+277-882-255-28 abortion pills for sale in soweto
%in Soweto+277-882-255-28 abortion pills for sale in soweto%in Soweto+277-882-255-28 abortion pills for sale in soweto
%in Soweto+277-882-255-28 abortion pills for sale in sowetomasabamasaba
 
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...WSO2
 
Introducing Microsoft’s new Enterprise Work Management (EWM) Solution
Introducing Microsoft’s new Enterprise Work Management (EWM) SolutionIntroducing Microsoft’s new Enterprise Work Management (EWM) Solution
Introducing Microsoft’s new Enterprise Work Management (EWM) SolutionOnePlan Solutions
 
%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisa%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisamasabamasaba
 
Define the academic and professional writing..pdf
Define the academic and professional writing..pdfDefine the academic and professional writing..pdf
Define the academic and professional writing..pdfPearlKirahMaeRagusta1
 

Recently uploaded (20)

+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
+971565801893>>SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHAB...
 
%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrand%in Midrand+277-882-255-28 abortion pills for sale in midrand
%in Midrand+277-882-255-28 abortion pills for sale in midrand
 
Devoxx UK 2024 - Going serverless with Quarkus, GraalVM native images and AWS...
Devoxx UK 2024 - Going serverless with Quarkus, GraalVM native images and AWS...Devoxx UK 2024 - Going serverless with Quarkus, GraalVM native images and AWS...
Devoxx UK 2024 - Going serverless with Quarkus, GraalVM native images and AWS...
 
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
%+27788225528 love spells in Huntington Beach Psychic Readings, Attraction sp...
 
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
Large-scale Logging Made Easy: Meetup at Deutsche Bank 2024
 
The title is not connected to what is inside
The title is not connected to what is insideThe title is not connected to what is inside
The title is not connected to what is inside
 
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
Shapes for Sharing between Graph Data Spaces - and Epistemic Querying of RDF-...
 
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
%in kaalfontein+277-882-255-28 abortion pills for sale in kaalfontein
 
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
 
AI & Machine Learning Presentation Template
AI & Machine Learning Presentation TemplateAI & Machine Learning Presentation Template
AI & Machine Learning Presentation Template
 
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
 
WSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go PlatformlessWSO2CON2024 - It's time to go Platformless
WSO2CON2024 - It's time to go Platformless
 
%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain
%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain
%in Bahrain+277-882-255-28 abortion pills for sale in Bahrain
 
%in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park %in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park
 
Software Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsSoftware Quality Assurance Interview Questions
Software Quality Assurance Interview Questions
 
%in Soweto+277-882-255-28 abortion pills for sale in soweto
%in Soweto+277-882-255-28 abortion pills for sale in soweto%in Soweto+277-882-255-28 abortion pills for sale in soweto
%in Soweto+277-882-255-28 abortion pills for sale in soweto
 
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
 
Introducing Microsoft’s new Enterprise Work Management (EWM) Solution
Introducing Microsoft’s new Enterprise Work Management (EWM) SolutionIntroducing Microsoft’s new Enterprise Work Management (EWM) Solution
Introducing Microsoft’s new Enterprise Work Management (EWM) Solution
 
%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisa%in tembisa+277-882-255-28 abortion pills for sale in tembisa
%in tembisa+277-882-255-28 abortion pills for sale in tembisa
 
Define the academic and professional writing..pdf
Define the academic and professional writing..pdfDefine the academic and professional writing..pdf
Define the academic and professional writing..pdf
 

Developing streaming applications with apache apex (strata + hadoop world)

  • 1. Developing Streaming Applications with Apache Apex David Yan <davidyan@apache.org> PMC Member, Apache Apex Software Engineer, Google Strata+Hadoop World, San Jose, CA March 16, 2017
  • 2. Agenda ● Technical Overview of Apache Apex as a streaming platform ● Developing an Application Pipeline ● Developing an Operator, and the Operator Library ● Event Time Windowing ● Debugging, Scaling and Tuning ● Q & A
  • 3. What is Apache Apex? Why? ● A platform for streaming applications ● Written in Java ● Uses YARN for resource management and HDFS for storage ● Has a comprehensive operator library. ● Scalable, fault tolerant, high throughput and low latency ● Provides low-level control for performance tuning ● Users include GE (Predix), Capital One, Royal Bank of Canada, Pubmatic, SilverSpring Network, etc. ○ (more at https://apex.apache.org/powered-by-apex.html)
  • 5. Application Development Model ● Directed Acyclic Graph (DAG) ● A stream is a sequence of data tuples ● An operator takes one or more input streams, performs computations, emits one or more output streams Filtered Stream Output Stream Tuple Tuple FilteredStream Enriched Stream Enriched Stream er Operator er Operator er Operator er Operator er Operator er Operator
  • 6. From Logical DAG to Actual Work ● CLI issues a launch command with App Package ● YARN starts up Application Master ● Logical DAG is translated to Physical DAG Apex CLI YARN RM NM NM NM NM AM 1 2 3 4 5 6Input Filter Transform Output
  • 7. From Logical DAG to Actual Work (cont'd) ● AM requests worker containers from YARN to run physical operators ● Worker Containers send data using a pub-sub mechanism Apex CLI YARN RM AM Worker WorkerWorker Worker Worker 6 4 1 3 25 1 2 3 4 5 6 HDFS PubSub overflow & Checkpoints
  • 8. Streaming Windows & Checkpointing ● Stream is divided into fixed time slices called streaming windows ● Checkpoint is performed by Worker Containers at streaming window boundaries ● Worker Containers send heartbeats to AM ● Recovery is incremental without resetting full DAG ● Checkpoints are purged after the corresponding window is committed ● AM is also checkpointed BeginWindow nEndWindow n BeginWindow n+1 EndWindow n+1 Time ...... Bookkeeping & Checkpointing done here
  • 10. Writing an Apex Application Pipeline ● Creating a project: Maven archetype ● Two APIs to specify an Apex pipeline ○ Compositional ○ Declarative
  • 11. Compositional API Input Parser Counter Output CountsWordsLines Kafka Database Filter Filtered
  • 12. Declarative API StreamFactory.fromKafka09(brokers, topic) .flatMap(input -> Arrays.asList(input.split("s+"))) .filter(input -> filterWords.contains(input)) .window(new WindowOption.GlobalWindow(), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1)) .sumByKey(input -> new Tuple.PlainTuple<>(new KeyValPair<>(input, 1L))) .map(input -> input.getValue()) .endWith(new JdbcOutput()) .populateDag(dag); Input Parser Counter Output CountsWordsLines Kafka Database Filter Filtered
  • 14. Operator API Next streaming window Next streaming window Input Adapters - Starting of the pipeline. Interacts with external system to generate stream Generic Operators - Processing part of pipeline Output Adapters - Last operator in pipeline. Interacts with external system to finalize the processed stream OutputPort::emit()
  • 17. RDBMS • JDBC • MySQL • Oracle • MemSQL NoSQL • Cassandra, HBase • Aerospike, Accumulo • Couchbase/ CouchDB • Redis, MongoDB • Geode Messaging • Kafka • JMS (ActiveMQ etc.) • Kinesis, SQS • Flume, NiFi • MQTT File Systems • HDFS/ Hive • Local File • S3 Parsers • XML • JSON • CSV • Avro • Parquet Transformations • Filters, Expression, Enrich • Windowing, Aggregation • Join • Dedup Analytics • Dimensional Aggregations (with state management for historical data + query) Protocols • HTTP • FTP • WebSocket • SMTP Other • Elastic Search • Script (JavaScript, Python, R) • Solr • Twitter Apex "Malhar" Operator Library
  • 18. Queryable State A set of operators in the library that supports real-time queries and updates of operator state. HashTag Extractor Top Counter Twitter Feed Input Operator Unique Counter Query Result PubSub Broker Query Input
  • 21. What, Why, and How? ● Event Time Windows not to be confused with "streaming windows" ● Time of the event vs processing time and ingression time ● Data often arrives late and out of order ● Concepts outlined by Google's Millwheel and Apache Beam: Watermarks, Allowed Lateness, Accumulation Modes, and Triggers
  • 22. Example StreamFactory.fromFolder("/tmp") .flatMap(input -> Arrays.asList(input.split( "s+"))) .map(input -> new TimestampedTuple<>(System.currentTimeMillis(), input)) .window(new TimeWindows(Duration.standardMinutes(5)), TriggerOption.AtWatermark() .accumulatingFiredPanes() .withEarlyFiringsAtEvery(Duration.standardSeconds(1)), Duration.standardSeconds(15)) .sumByKey(input -> new TimestampedTuple<>(input.getTimestamp(), new KeyValPair<>(input.getValue(), 1L )))) .map(new FormatAsTableRowFn()) // format for printing .print() .populateDag(dag);
  • 23. Debugging, Scaling & Tuning Your Application
  • 24. Debugging ● Logs ● Local Mode ● Remote JVM Attachment ● Tuple Recording
  • 25. Scaling: Partitioning Partitioning with Unifiers: NxM Partitioning: 0 1 2 0 1 a 1 b 1 c U 2 Logical DAG Physical DAG with operator 1 with 3 partitions 0 21 3 0 2 a 1 b 3 1 a 1 c 2 b U U 0 2 a 1 b 3 1 a 1 c 2 b U1 U U2
  • 26. Scaling: Partitioning (cont'd) Parallel Partition: Cascading Unifiers: 0 1 a 1 b U 2 3 4 0 1 a 1 b U 4 2 a 3 a 2 b 3 b 1 2 1 1 1 1 U 2 1 1 1 1 U 1 2 U 2 U 3
  • 27. Dynamic Partitioning ● Partitioning change while application is running ○ Change number of partitions at runtime based on stats ○ Supports re-distribution of state when number of partitions change ○ API for custom scaler or partitioner StatelessThroughputBasedPartitioner<MyOperator> partitioner = new StatelessThroughputBasedPartitioner<>(); partitioner.setCooldownMillis(45000); partitioner.setMaximumEvents(30000); partitioner.setMinimumEvents(10000); dag.setAttribute(op, OperatorContext.STATS_LISTENERS, Arrays.asList(new StatsListener[]{partitioner})); dag.setAttribute(op, OperatorContext.PARTITIONER, partitioner);
  • 28. Locality ● Node-Local: The two operators should be placed in the same node ● Container-Local: Same worker container (saves serialization) ● Thread-Local: Same thread (simple function call) // setting locality between op1 and op2 to be NODE_LOCAL dag.addStream( "streamName", op1.output, op2.input) .setLocality(DAG.Locality.NODE_LOCAL);
  • 29. Recent Additions and Roadmap ● Apex runner in Apache Beam ● Iterative processing ● Integrated with Apache Samoa, opens up ML ● Integrated with Apache Calcite, allows SQL --------------------------------------------- ● Enhanced support for Batch Processing ● Support for Mesos and Kubernetes ● Encrypted Streams ● Python API
  • 30. Q & A
  • 31. Resources ● http://apex.apache.org/ ● Learn more: http://apex.apache.org/docs.html ● GitHub Repositories ○ http://github.com/apache/apex-core ○ http://github.com/apache/apex-malhar ● Subscribe - http://apex.apache.org/community.html ● Download - http://apex.apache.org/downloads.html ● Follow @ApacheApex - https://twitter.com/apacheapex ● Meetups – http://www.meetup.com/topics/apache-apex/ ● Slideshare: http://www.slideshare.net/ApacheApex/presentations ● https://www.youtube.com/results?search_query=apache+apex